├── .gitignore ├── README.md ├── build.gradle ├── data └── sampleData.txt ├── gradle └── wrapper │ └── gradle-wrapper.properties ├── gradlew ├── gradlew.bat ├── settings.gradle └── src └── main └── scala └── com └── aokolnychyi └── spark └── datasource ├── CustomDatasourceApp.scala ├── CustomDatasourceRelation.scala └── DefaultSource.scala /.gitignore: -------------------------------------------------------------------------------- 1 | # Created by .ignore support plugin (hsz.mobi) 2 | ### Gradle template 3 | .gradle 4 | build/ 5 | out/ 6 | output/ 7 | 8 | # Ignore Gradle GUI config 9 | gradle-app.setting 10 | 11 | # Avoid ignoring Gradle wrapper jar file (.jar files are usually ignored) 12 | !gradle-wrapper.jar 13 | 14 | # Cache of project 15 | .gradletasknamecache 16 | 17 | # # Work around https://youtrack.jetbrains.com/issue/IDEA-116898 18 | # gradle/wrapper/gradle-wrapper.properties 19 | ### Scala template 20 | *.class 21 | *.log 22 | 23 | # sbt specific 24 | .cache 25 | .history 26 | .lib/ 27 | dist/* 28 | target/ 29 | lib_managed/ 30 | src_managed/ 31 | project/boot/ 32 | project/plugins/project/ 33 | 34 | # Scala-IDE specific 35 | .scala_dependencies 36 | .worksheet 37 | ### Java template 38 | 39 | # Mobile Tools for Java (J2ME) 40 | .mtj.tmp/ 41 | 42 | # Package Files # 43 | *.jar 44 | *.war 45 | *.ear 46 | 47 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 48 | hs_err_pid* 49 | 50 | .idea/ -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Custom Spark Datasource 2 | 3 | This repository contains a sample Spark application that implements the Datasource API. 4 | For simplicity's sake, the implementation works with text files that have three columns separated by "$", 5 | which include information about name, surname, salary. 6 | 7 | ## Schema Identification 8 | 9 | 1. Create a class called `DefaultSource` that extends `RelationProvider` and `SchemaRelationProvider` traits. 10 | 11 | The `RelationProvider` trait is implemented by objects that produce relations for a specific kind of 12 | data source. Users may omit the fully qualified class name of a given data source. In this case, 13 | Spark SQL will append the class name `DefaultSource` to the path, 14 | allowing for less verbose invocation. For example, `org.apache.spark.sql.json` would resolve to 15 | the data source `org.apache.spark.sql.json.DefaultSource`. 16 | 17 | 2. Create a class that extends `BaseRelation`. 18 | 19 | `BaseRelation` represents a collection of tuples with a known schema. 20 | Simply speaking, it is used to infer/define schemas. 21 | 22 | ## Reading Data 23 | 24 | 1. Implement the `TableScan` trait in the custom relation class. This method should return all rows 25 | from the custom data source as an `RDD` of `Rows`. 26 | 27 | ## Writing Data 28 | 29 | 1. To support write calls, `DefaultSource` has to implement one additional trait called `CreatableRelationProvider`. 30 | 31 | ## Column Pruning 32 | 33 | 1. To implement the column pruning, the custom relation class has to implement the `PrunedScan`. It can help to optimize the column access. 34 | 35 | ## Filter Pushdown 36 | 37 | 1. To optimize filtering, the custom relation class can extend the `PrunedFilterScan` trait. -------------------------------------------------------------------------------- /build.gradle: -------------------------------------------------------------------------------- 1 | plugins { 2 | id 'idea' 3 | id 'scala' 4 | id 'com.github.johnrengelman.shadow' version '1.2.4' 5 | } 6 | 7 | group 'com.aokolnychyi' 8 | version '1.0-SNAPSHOT' 9 | 10 | sourceCompatibility = 1.8 11 | targetCompatibility = 1.8 12 | 13 | repositories { 14 | mavenCentral() 15 | jcenter() 16 | } 17 | 18 | ext { 19 | scalaVersion = '2.11.8' 20 | sparkVersion = '2.1.0' 21 | } 22 | 23 | dependencies { 24 | compile "org.apache.spark:spark-core_2.11:$sparkVersion" 25 | compile "org.apache.spark:spark-sql_2.11:$sparkVersion" 26 | compile "org.scala-lang:scala-library:$scalaVersion" 27 | } 28 | 29 | shadowJar { 30 | zip64 true 31 | dependencies { 32 | exclude(dependency("org.apache.spark:.*")) 33 | exclude(dependency("org.apache.hadoop:.*")) 34 | } 35 | } 36 | 37 | sourceSets { 38 | main { 39 | scala { 40 | srcDirs = ['src/main/scala', 'src/main/java'] 41 | } 42 | java { 43 | srcDirs = [] 44 | } 45 | } 46 | } -------------------------------------------------------------------------------- /data/sampleData.txt: -------------------------------------------------------------------------------- 1 | Albert$Weil$40000 2 | Jorn$Kastanien$50000 -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | #Thu Apr 13 18:32:42 CEST 2017 2 | distributionBase=GRADLE_USER_HOME 3 | distributionPath=wrapper/dists 4 | zipStoreBase=GRADLE_USER_HOME 5 | zipStorePath=wrapper/dists 6 | distributionUrl=https\://services.gradle.org/distributions/gradle-3.3-bin.zip 7 | -------------------------------------------------------------------------------- /gradlew: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env sh 2 | 3 | ############################################################################## 4 | ## 5 | ## Gradle start up script for UN*X 6 | ## 7 | ############################################################################## 8 | 9 | # Attempt to set APP_HOME 10 | # Resolve links: $0 may be a link 11 | PRG="$0" 12 | # Need this for relative symlinks. 13 | while [ -h "$PRG" ] ; do 14 | ls=`ls -ld "$PRG"` 15 | link=`expr "$ls" : '.*-> \(.*\)$'` 16 | if expr "$link" : '/.*' > /dev/null; then 17 | PRG="$link" 18 | else 19 | PRG=`dirname "$PRG"`"/$link" 20 | fi 21 | done 22 | SAVED="`pwd`" 23 | cd "`dirname \"$PRG\"`/" >/dev/null 24 | APP_HOME="`pwd -P`" 25 | cd "$SAVED" >/dev/null 26 | 27 | APP_NAME="Gradle" 28 | APP_BASE_NAME=`basename "$0"` 29 | 30 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 31 | DEFAULT_JVM_OPTS="" 32 | 33 | # Use the maximum available, or set MAX_FD != -1 to use that value. 34 | MAX_FD="maximum" 35 | 36 | warn ( ) { 37 | echo "$*" 38 | } 39 | 40 | die ( ) { 41 | echo 42 | echo "$*" 43 | echo 44 | exit 1 45 | } 46 | 47 | # OS specific support (must be 'true' or 'false'). 48 | cygwin=false 49 | msys=false 50 | darwin=false 51 | nonstop=false 52 | case "`uname`" in 53 | CYGWIN* ) 54 | cygwin=true 55 | ;; 56 | Darwin* ) 57 | darwin=true 58 | ;; 59 | MINGW* ) 60 | msys=true 61 | ;; 62 | NONSTOP* ) 63 | nonstop=true 64 | ;; 65 | esac 66 | 67 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar 68 | 69 | # Determine the Java command to use to start the JVM. 70 | if [ -n "$JAVA_HOME" ] ; then 71 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then 72 | # IBM's JDK on AIX uses strange locations for the executables 73 | JAVACMD="$JAVA_HOME/jre/sh/java" 74 | else 75 | JAVACMD="$JAVA_HOME/bin/java" 76 | fi 77 | if [ ! -x "$JAVACMD" ] ; then 78 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME 79 | 80 | Please set the JAVA_HOME variable in your environment to match the 81 | location of your Java installation." 82 | fi 83 | else 84 | JAVACMD="java" 85 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 86 | 87 | Please set the JAVA_HOME variable in your environment to match the 88 | location of your Java installation." 89 | fi 90 | 91 | # Increase the maximum file descriptors if we can. 92 | if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then 93 | MAX_FD_LIMIT=`ulimit -H -n` 94 | if [ $? -eq 0 ] ; then 95 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then 96 | MAX_FD="$MAX_FD_LIMIT" 97 | fi 98 | ulimit -n $MAX_FD 99 | if [ $? -ne 0 ] ; then 100 | warn "Could not set maximum file descriptor limit: $MAX_FD" 101 | fi 102 | else 103 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" 104 | fi 105 | fi 106 | 107 | # For Darwin, add options to specify how the application appears in the dock 108 | if $darwin; then 109 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" 110 | fi 111 | 112 | # For Cygwin, switch paths to Windows format before running java 113 | if $cygwin ; then 114 | APP_HOME=`cygpath --path --mixed "$APP_HOME"` 115 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` 116 | JAVACMD=`cygpath --unix "$JAVACMD"` 117 | 118 | # We build the pattern for arguments to be converted via cygpath 119 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` 120 | SEP="" 121 | for dir in $ROOTDIRSRAW ; do 122 | ROOTDIRS="$ROOTDIRS$SEP$dir" 123 | SEP="|" 124 | done 125 | OURCYGPATTERN="(^($ROOTDIRS))" 126 | # Add a user-defined pattern to the cygpath arguments 127 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then 128 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" 129 | fi 130 | # Now convert the arguments - kludge to limit ourselves to /bin/sh 131 | i=0 132 | for arg in "$@" ; do 133 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` 134 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option 135 | 136 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition 137 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` 138 | else 139 | eval `echo args$i`="\"$arg\"" 140 | fi 141 | i=$((i+1)) 142 | done 143 | case $i in 144 | (0) set -- ;; 145 | (1) set -- "$args0" ;; 146 | (2) set -- "$args0" "$args1" ;; 147 | (3) set -- "$args0" "$args1" "$args2" ;; 148 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;; 149 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; 150 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; 151 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; 152 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; 153 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; 154 | esac 155 | fi 156 | 157 | # Escape application args 158 | save ( ) { 159 | for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done 160 | echo " " 161 | } 162 | APP_ARGS=$(save "$@") 163 | 164 | # Collect all arguments for the java command, following the shell quoting and substitution rules 165 | eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" 166 | 167 | # by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong 168 | if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then 169 | cd "$(dirname "$0")" 170 | fi 171 | 172 | exec "$JAVACMD" "$@" 173 | -------------------------------------------------------------------------------- /gradlew.bat: -------------------------------------------------------------------------------- 1 | @if "%DEBUG%" == "" @echo off 2 | @rem ########################################################################## 3 | @rem 4 | @rem Gradle startup script for Windows 5 | @rem 6 | @rem ########################################################################## 7 | 8 | @rem Set local scope for the variables with windows NT shell 9 | if "%OS%"=="Windows_NT" setlocal 10 | 11 | set DIRNAME=%~dp0 12 | if "%DIRNAME%" == "" set DIRNAME=. 13 | set APP_BASE_NAME=%~n0 14 | set APP_HOME=%DIRNAME% 15 | 16 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 17 | set DEFAULT_JVM_OPTS= 18 | 19 | @rem Find java.exe 20 | if defined JAVA_HOME goto findJavaFromJavaHome 21 | 22 | set JAVA_EXE=java.exe 23 | %JAVA_EXE% -version >NUL 2>&1 24 | if "%ERRORLEVEL%" == "0" goto init 25 | 26 | echo. 27 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 28 | echo. 29 | echo Please set the JAVA_HOME variable in your environment to match the 30 | echo location of your Java installation. 31 | 32 | goto fail 33 | 34 | :findJavaFromJavaHome 35 | set JAVA_HOME=%JAVA_HOME:"=% 36 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe 37 | 38 | if exist "%JAVA_EXE%" goto init 39 | 40 | echo. 41 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 42 | echo. 43 | echo Please set the JAVA_HOME variable in your environment to match the 44 | echo location of your Java installation. 45 | 46 | goto fail 47 | 48 | :init 49 | @rem Get command-line arguments, handling Windows variants 50 | 51 | if not "%OS%" == "Windows_NT" goto win9xME_args 52 | 53 | :win9xME_args 54 | @rem Slurp the command line arguments. 55 | set CMD_LINE_ARGS= 56 | set _SKIP=2 57 | 58 | :win9xME_args_slurp 59 | if "x%~1" == "x" goto execute 60 | 61 | set CMD_LINE_ARGS=%* 62 | 63 | :execute 64 | @rem Setup the command line 65 | 66 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar 67 | 68 | @rem Execute Gradle 69 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% 70 | 71 | :end 72 | @rem End local scope for the variables with windows NT shell 73 | if "%ERRORLEVEL%"=="0" goto mainEnd 74 | 75 | :fail 76 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of 77 | rem the _cmd.exe /c_ return code! 78 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 79 | exit /b 1 80 | 81 | :mainEnd 82 | if "%OS%"=="Windows_NT" endlocal 83 | 84 | :omega 85 | -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | rootProject.name = 'spark-datasource-example' 2 | 3 | -------------------------------------------------------------------------------- /src/main/scala/com/aokolnychyi/spark/datasource/CustomDatasourceApp.scala: -------------------------------------------------------------------------------- 1 | package com.aokolnychyi.spark.datasource 2 | 3 | import org.apache.spark.SparkConf 4 | import org.apache.spark.sql.{SaveMode, SparkSession} 5 | 6 | object CustomDatasourceApp extends App { 7 | 8 | val conf = new SparkConf().setAppName("spark-custom-datasource") 9 | val spark = SparkSession.builder().config(conf).master("local").getOrCreate() 10 | 11 | val df = spark.read.format("com.aokolnychyi.spark.datasource").load("data/") 12 | 13 | // Step 1 (Schema verification) 14 | df.printSchema() 15 | // Step 2 (Read data) 16 | df.show() 17 | // Step 3 (Write data) 18 | df.write 19 | .options(Map("format" -> "customFormat")) 20 | .mode(SaveMode.Overwrite) 21 | .format("com.aokolnychyi.spark.datasource") 22 | .save("out/") 23 | // Step 4 (Column Pruning) 24 | df.createOrReplaceTempView("salaries") 25 | spark.sql("SELECT surname, salary FROM salaries").show() 26 | // Step 5 (Filter Pushdown) 27 | spark.sql("SELECT name, surname FROM salaries WHERE salary > 40000").show() 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/com/aokolnychyi/spark/datasource/CustomDatasourceRelation.scala: -------------------------------------------------------------------------------- 1 | package com.aokolnychyi.spark.datasource 2 | 3 | import org.apache.spark.rdd.RDD 4 | import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan, PrunedScan, TableScan} 5 | import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} 6 | import org.apache.spark.sql.{Row, SQLContext} 7 | 8 | // This class has to be serializable since it is shipped over the network. 9 | class CustomDatasourceRelation( 10 | override val sqlContext: SQLContext, 11 | path: String, 12 | userSchema: StructType) 13 | extends BaseRelation 14 | with TableScan with PrunedScan with PrunedFilteredScan with Serializable { 15 | 16 | override def schema: StructType = { 17 | if (userSchema != null) { 18 | // The user defined a schema, simply return it 19 | userSchema 20 | } else { 21 | // There is no user-defined schema. 22 | // You need to infer it on your own. E.g., read the header of CSV file. 23 | StructType( 24 | StructField("name", StringType, nullable = true) :: 25 | StructField("surname", StringType, nullable = true) :: 26 | StructField("salary", IntegerType, nullable = true) :: 27 | Nil 28 | ) 29 | } 30 | } 31 | 32 | override def buildScan(): RDD[Row] = { 33 | val initialRdd = sqlContext.sparkContext.wholeTextFiles(path).map(_._2) 34 | val schemaFields = schema.fields 35 | 36 | val rowsRdd = initialRdd.map(fileContent => { 37 | val lines = fileContent.split("\n") 38 | val data = lines.map(line => line.split("\\$").toSeq) 39 | 40 | val records = data.map(words => words.zipWithIndex.map { 41 | case (value, index) => 42 | 43 | val columnType = schemaFields(index).dataType 44 | val castValue = columnType match { 45 | case StringType => value 46 | case IntegerType => value.toInt 47 | } 48 | castValue 49 | }) 50 | records.map(record => Row.fromSeq(record)) 51 | }) 52 | 53 | rowsRdd.flatMap(row => row) 54 | } 55 | 56 | override def buildScan(requiredColumns: Array[String]): RDD[Row] = { 57 | println("Selecting only required columns...") 58 | // An example, does not provide any specific performance benefits 59 | val initialRdd = sqlContext.sparkContext.wholeTextFiles(path).map(_._2) 60 | val schemaFields = schema.fields 61 | 62 | val rowsRdd = initialRdd.map(fileContent => { 63 | val lines = fileContent.split("\n") 64 | val data = lines.map(line => line.split("\\$").toSeq) 65 | 66 | val records = data.map(words => words.zipWithIndex.map { 67 | case (value, index) => 68 | val field = schemaFields(index) 69 | if (requiredColumns.contains(field.name)) Some(cast(value, field.dataType)) else None 70 | }) 71 | 72 | records 73 | .map(record => record.filter(_.isDefined)) 74 | .map(record => Row.fromSeq(record)) 75 | }) 76 | 77 | rowsRdd.flatMap(row => row) 78 | } 79 | 80 | 81 | override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { 82 | // Nothing is actually pushed down, just iterate through all filters and print them 83 | println("Trying to push down filters...") 84 | filters foreach println 85 | buildScan(requiredColumns) 86 | } 87 | 88 | private def cast(value: String, dataType: DataType) = dataType match { 89 | case StringType => value 90 | case IntegerType => value.toInt 91 | } 92 | 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/com/aokolnychyi/spark/datasource/DefaultSource.scala: -------------------------------------------------------------------------------- 1 | package com.aokolnychyi.spark.datasource 2 | 3 | import org.apache.hadoop.fs.Path 4 | 5 | import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider, SchemaRelationProvider} 6 | import org.apache.spark.sql.types.StructType 7 | import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} 8 | 9 | class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { 10 | 11 | // Method that comes from RelationProvider. 12 | override def createRelation( 13 | sqlContext: SQLContext, 14 | parameters: Map[String, String]): BaseRelation = { 15 | 16 | createRelation(sqlContext, parameters, null) 17 | } 18 | 19 | // Method that comes from SchemaRelationProvider, which allows users to specify the schema. 20 | // In this case, we do not need to discover it on our own. 21 | override def createRelation( 22 | sqlContext: SQLContext, 23 | parameters: Map[String, String], 24 | schema: StructType): BaseRelation = { 25 | 26 | val pathParameter = parameters.get("path") 27 | pathParameter match { 28 | case Some(path) => new CustomDatasourceRelation(sqlContext, path, schema) 29 | case None => throw new IllegalArgumentException("The path parameter cannot be empty!") 30 | } 31 | } 32 | 33 | override def createRelation( 34 | sqlContext: SQLContext, 35 | mode: SaveMode, 36 | parameters: Map[String, String], 37 | data: DataFrame): BaseRelation = { 38 | 39 | val pathParameter = parameters.getOrElse("path", "./output/") 40 | val fsPath = new Path(pathParameter) 41 | val fs = fsPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) 42 | 43 | mode match { 44 | case SaveMode.Append => sys.error("Append mode is not supported by " + this.getClass.getCanonicalName); sys.exit(1) 45 | case SaveMode.Overwrite => fs.delete(fsPath, true) 46 | case SaveMode.ErrorIfExists if fs.exists(fsPath) => sys.error("Given path: " + pathParameter + " already exists!!"); sys.exit(1) 47 | case SaveMode.ErrorIfExists => sys.error("Given path: " + pathParameter + " already exists!!"); sys.exit(1) 48 | case SaveMode.Ignore => sys.exit() 49 | } 50 | 51 | val formatName = parameters.getOrElse("format", "customFormat") 52 | formatName match { 53 | case "customFormat" => saveAsCustomFormat(data, pathParameter, mode) 54 | case _ => throw new IllegalArgumentException(formatName + " is not supported!") 55 | } 56 | createRelation(sqlContext, parameters, data.schema) 57 | } 58 | 59 | private def saveAsCustomFormat(data: DataFrame, path: String, mode: SaveMode): Unit = { 60 | val customFormatRDD = data.rdd.map(row => { 61 | row.toSeq.map(value => value.toString).mkString(";") 62 | }) 63 | customFormatRDD.saveAsTextFile(path) 64 | } 65 | } 66 | --------------------------------------------------------------------------------