From 36fe0b37608330b20c5509336167cfbc3a8b7751 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Mon, 30 May 2022 23:24:47 +0900 Subject: [PATCH 01/19] Added documentation Signed-off-by: Chang Yong Lik --- docs/how-to-guides/feathr-job-configuration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/how-to-guides/feathr-job-configuration.md b/docs/how-to-guides/feathr-job-configuration.md index e63781d7c..f6db178d9 100644 --- a/docs/how-to-guides/feathr-job-configuration.md +++ b/docs/how-to-guides/feathr-job-configuration.md @@ -12,3 +12,4 @@ Since Feathr uses Spark as the underlying execution engine, there's a way to ove | ------------------------- | ------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------- | | spark.feathr.inputFormat | None | Specify the input format if the file cannot be tell automatically. By default, Feathr will read files by parsing the file extension name; However the file/folder name doesn't have extension name, this configuration can be set to tell Feathr which format it should use to read the data. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to read delta lake. | 0.2.1 | | spark.feathr.outputFormat | None | Specify the output format. "avro" is the default behavior if this value is not set. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to write delta lake. | 0.2.1 | +| spark.feathr.inputFormat.csvOptions.sep | None | Specify the delimiter. For example, "," for commas or "\t" for tabs. | TODO: Check version | \ No newline at end of file From b5c4e12ff719c10eaf20b86021dac93488cade67 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Thu, 23 Jun 2022 13:08:14 +0900 Subject: [PATCH 02/19] Added delimiter to CSVLoader Signed-off-by: Chang Yong Lik (cherry picked from commit bc71fad93c08f6d06e40f7e289456c6a1b4d45e0) --- .../source/dataloader/CsvDataLoader.scala | 7 +++++-- src/test/resources/anchor1-source.tsv | 8 ++++++++ .../source/dataloader/TestCsvDataLoader.scala | 19 +++++++++++++++++++ 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 src/test/resources/anchor1-source.tsv diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index 4e5b53012..79ff24438 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -32,17 +32,20 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat * @return an dataframe */ override def loadDataFrame(): DataFrame = { + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + try { log.debug(s"Loading CSV path :${path}") val absolutePath = new File(path).getPath log.debug(s"Got absolute CSV path: ${absolutePath}, loading..") - ss.read.format("csv").option("header", "true").load(absolutePath) + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(absolutePath) } catch { case _: Throwable => log.debug(s"Loading CSV failed, retry with class loader..") val absolutePath = getClass.getClassLoader.getResource(path).getPath log.debug(s"Got absolution CSV path from class loader: ${absolutePath}, loading.. ") - ss.read.format("csv").option("header", "true").load(absolutePath) + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(absolutePath) } } diff --git a/src/test/resources/anchor1-source.tsv b/src/test/resources/anchor1-source.tsv new file mode 100644 index 000000000..176ed963d --- /dev/null +++ b/src/test/resources/anchor1-source.tsv @@ -0,0 +1,8 @@ +mId alpha beta gamma omega +1 apple 10 10 0.1 +2 orange 10 3 0.1 +3 banana 10 2 0.9 +4 apple 10 1 0.7 +5 apple 11 11 1.0 +7 banana 2 10 81.27 +9 banana 4 4 0.4 \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala index f89782661..5919a6ee3 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala @@ -46,4 +46,23 @@ class TestCsvDataLoader extends TestFeathr { val expectedSchema = Schema.createRecord(expectedFields) assertEquals(schema.getFields, expectedSchema.getFields) } + + @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter") + def testLoadDataFrameWithCsvDelimiterOption() : Unit = { + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val dataLoader = new CsvDataLoader(ss, "anchor1-source.tsv") + + val df = dataLoader.loadDataFrame() + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } } From 802be28509748dca502bfd10183ddb88b6d796f5 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Thu, 23 Jun 2022 16:27:53 +0900 Subject: [PATCH 03/19] Added delimiter to BatchDataLoader, FileFormat and SourceUtils Signed-off-by: Chang Yong Lik --- .../offline/source/dataloader/BatchDataLoader.scala | 4 +++- .../feathr/offline/source/dataloader/CsvDataLoader.scala | 1 - .../offline/source/dataloader/hdfs/FileFormat.scala | 8 ++++++-- .../com/linkedin/feathr/offline/util/SourceUtils.scala | 4 +++- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index e80b786d6..bde9c995f 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -61,6 +61,8 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation val inputSplitSize = sparkConf.get("spark.feathr.input.split.size", "") val dataIOParametersWithSplitSize = Map(SparkIOUtils.SPLIT_SIZE -> inputSplitSize) ++ dataIOParameters log.info(s"Loading ${location.getPath} as DataFrame, using parameters ${dataIOParametersWithSplitSize}") + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") try { if (location.getPath.startsWith("jdbc")){ JdbcUtils.loadDataFrame(ss, location.getPath) @@ -69,7 +71,7 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation } } catch { case _: Throwable => - ss.read.format("csv").option("header", "true").load(location.getPath) + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(location.getPath) } } } diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index 79ff24438..c3d2127f5 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -34,7 +34,6 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat override def loadDataFrame(): DataFrame = { val sqlContext = ss.sqlContext val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - try { log.debug(s"Loading CSV path :${path}") val absolutePath = new File(path).getPath diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index ea42b4a1f..ee61f4505 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -50,9 +50,11 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") format match { case AVRO => new AvroJsonDataLoader(ss, path).loadDataFrame() - case CSV => ss.read.format("csv").option("header", "true").load(path) + case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) case PARQUET => new ParquetDataLoader(ss, path).loadDataFrame() case _ => ??? } @@ -82,9 +84,11 @@ object FileFormat { } def loadHdfsDataFrame(format: String, existingHdfsPaths: Seq[String]): DataFrame = { + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") val df = format match { case CSV => - ss.read.format("csv").option("header", "true").load(existingHdfsPaths: _*) + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(existingHdfsPaths: _*) case AVRO => ss.read.format(AVRO_DATASOURCE).load(existingHdfsPaths: _*) case ORC => diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index 4b2da6a1b..427771d6b 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -656,6 +656,8 @@ private[offline] object SourceUtils { def loadObservationAsDF(ss: SparkSession, conf: Configuration, inputData: InputData, failOnMissing: Boolean = true): DataFrame = { // TODO: Split isLocal case into Test Packages val format = FileFormat.getType(inputData.inputPath) + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") log.info(s"loading ${inputData.inputPath} input Path as Format: ${format}") format match { case FileFormat.PATHLIST => { @@ -674,7 +676,7 @@ private[offline] object SourceUtils { JdbcUtils.loadDataFrame(ss, inputData.inputPath) } case FileFormat.CSV => { - ss.read.format("csv").option("header", "true").load(inputData.inputPath) + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(inputData.inputPath) } case _ => { if (ss.sparkContext.isLocal){ From 542952db501d36578438c38eaa8312731a6efc2d Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Fri, 24 Jun 2022 00:41:18 +0900 Subject: [PATCH 04/19] Added test case for BatchDataLoader Signed-off-by: Chang Yong Lik --- .../offline/source/dataloader/TestDataLoaderFactory.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala index 4808c975d..3e70cfc1a 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala @@ -14,6 +14,10 @@ class TestDataLoaderFactory extends TestFeathr { val localDataLoaderFactory = new LocalDataLoaderFactory(ss) val csvLoader = localDataLoaderFactory.create("anchor1-source.csv") assertTrue(csvLoader.isInstanceOf[CsvDataLoader]) + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val csvLoaderWithDelimiter = localDataLoaderFactory.create("anchor1-source.tsv") + assertTrue(csvLoaderWithDelimiter.isInstanceOf[BatchDataLoader]) val avroJsonLoader = localDataLoaderFactory.create("anchor5-source.avro.json") assertTrue(avroJsonLoader.isInstanceOf[AvroJsonDataLoader]) val jsonWithSchemaLoader = localDataLoaderFactory.create("simple-obs2") // the mock data folder exists. From 6229f33f04e70746778971fee21d443223997300 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Fri, 24 Jun 2022 12:41:28 +0900 Subject: [PATCH 05/19] Added test case for FileFormat Signed-off-by: Chang Yong Lik --- .../dataloader/hdfs/TestFileFormat.scala | 74 +++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala new file mode 100644 index 000000000..efee93698 --- /dev/null +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala @@ -0,0 +1,74 @@ +package com.linkedin.feathr.offline.source.dataloader.hdfs + +import com.linkedin.feathr.offline.TestFeathr +import org.apache.spark.sql.Row +import org.testng.Assert.assertEquals +import org.testng.annotations.Test + +import java.io.File + + +/** + * unit tests for [[FileFormat]] + */ +class TestFileFormat extends TestFeathr { + + @Test(description = "test loading dataframe with FileFormat") + def testLoadDataFrame() : Unit = { + val path = "anchor1-source.csv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + + @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter") + def testLoadDataFrameWithCsvDelimiterOption() : Unit = { + val path = "anchor1-source.tsv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val df = FileFormat.loadDataFrame(ss, absolutePath, "CSV") + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + + @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter") + def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { + val path = "anchor1-source.tsv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + +} From aa43f7204b07fe05617d91792aa0fc51ebf47798 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Fri, 24 Jun 2022 13:08:33 +0900 Subject: [PATCH 06/19] Added test case for BatchDataLoader Signed-off-by: Chang Yong Lik --- .../dataloader/TestBatchDataLoader.scala | 49 +++++++++++++++++++ .../source/dataloader/TestCsvDataLoader.scala | 1 - .../dataloader/hdfs/TestFileFormat.scala | 3 -- 3 files changed, 49 insertions(+), 4 deletions(-) create mode 100644 src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala new file mode 100644 index 000000000..4f387e0f5 --- /dev/null +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala @@ -0,0 +1,49 @@ +package com.linkedin.feathr.offline.source.dataloader + +import com.linkedin.feathr.offline.TestFeathr +import com.linkedin.feathr.offline.config.location.SimplePath +import org.apache.spark.sql.Row +import org.testng.Assert.assertEquals +import org.testng.annotations.Test + +class TestBatchDataLoader extends TestFeathr { + + @Test(description = "test loading dataframe with BatchDataLoader") + def testBatchDataLoader() : Unit = { + val path = "anchor1-source.csv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val batchDataLoader = new BatchDataLoader(ss, SimplePath(absolutePath), List()) + val df = batchDataLoader.loadDataFrame() + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + + @Test(description = "test loading dataframe with BatchDataLoader by specifying delimiter") + def testBatchDataLoaderWithCsvDelimiterOption() : Unit = { + val path = "anchor1-source.tsv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val batchDataLoader = new BatchDataLoader(ss, SimplePath(absolutePath), List()) + val df = batchDataLoader.loadDataFrame() + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + +} diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala index 5919a6ee3..ab1d5295b 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala @@ -29,7 +29,6 @@ class TestCsvDataLoader extends TestFeathr { assertEquals(df.collect(), expectedRows) } - @Test(description = "test loading Avro schema with CsvDataLoader") def testLoadSchema() : Unit = { val dataLoader = new CsvDataLoader(ss, "anchor1-source.csv") diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala index efee93698..8fb1baf4a 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala @@ -5,9 +5,6 @@ import org.apache.spark.sql.Row import org.testng.Assert.assertEquals import org.testng.annotations.Test -import java.io.File - - /** * unit tests for [[FileFormat]] */ From d3142e9acc890f8a880dc36eb6cccbb7669347e6 Mon Sep 17 00:00:00 2001 From: Chang Yong Lik Date: Fri, 24 Jun 2022 13:51:55 +0900 Subject: [PATCH 07/19] Added test case and fixed indent Signed-off-by: Chang Yong Lik --- .../source/dataloader/hdfs/FileFormat.scala | 1 + .../dataloader/TestBatchDataLoader.scala | 3 + .../dataloader/TestDataLoaderFactory.scala | 2 +- .../feathr/offline/util/TestSourceUtils.scala | 58 +++++++++++++++++++ 4 files changed, 63 insertions(+), 1 deletion(-) create mode 100644 src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index 488127e40..e18641c09 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -38,6 +38,7 @@ object FileFormat { val p = path.toLowerCase() p match { case p if p.endsWith(".csv") => CSV + case p if p.endsWith(".tsv") => CSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC case p if p.endsWith(".avro.json") => AVRO_JSON diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala index 4f387e0f5..9efc5b436 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala @@ -6,6 +6,9 @@ import org.apache.spark.sql.Row import org.testng.Assert.assertEquals import org.testng.annotations.Test +/** + * unit tests for [[BatchDataLoader]] + */ class TestBatchDataLoader extends TestFeathr { @Test(description = "test loading dataframe with BatchDataLoader") diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala index 2f704f3ba..7a0b2ea76 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala @@ -5,7 +5,7 @@ import org.testng.Assert.assertTrue import org.testng.annotations.Test /** - * unit tests for DataLoaderFactory + * unit tests for [[DataLoaderFactory]] */ class TestDataLoaderFactory extends TestFeathr { diff --git a/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala new file mode 100644 index 000000000..a9453dce2 --- /dev/null +++ b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala @@ -0,0 +1,58 @@ +package com.linkedin.feathr.offline.util + +import com.linkedin.feathr.offline.TestFeathr +import com.linkedin.feathr.offline.client.InputData +import com.linkedin.feathr.offline.source.SourceFormatType +import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.Row +import org.testng.Assert.assertEquals +import org.testng.annotations.Test + +/** + * unit tests for [[SourceUtils]] + */ +class TestSourceUtils extends TestFeathr { + + @Test(description = "test loading dataframe with SourceUtils") + def testSourceUtils() : Unit = { + val conf: Configuration = new Configuration() + val path = "anchor1-source.csv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val inputData = InputData(absolutePath, SourceFormatType.FIXED_PATH) + val df = SourceUtils.loadObservationAsDF(ss, conf, inputData, List()) + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + + @Test(description = "test loading dataframe with SourceUtils by specifying delimiter") + def testSourceUtilsWithCsvDelimiterOption() : Unit = { + val conf: Configuration = new Configuration() + val path = "anchor1-source.tsv" + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val inputData = InputData(absolutePath, SourceFormatType.FIXED_PATH) + val df = SourceUtils.loadObservationAsDF(ss, conf, inputData, List()) + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } + +} From 54f55430d03a96d1b7a75130e40c3dc063b95e5a Mon Sep 17 00:00:00 2001 From: changyonglik Date: Sun, 10 Jul 2022 19:47:35 +0900 Subject: [PATCH 08/19] Passing failure Signed-off-by: changyonglik --- .../source/dataloader/hdfs/FileFormat.scala | 11 +++-- .../dataloader/hdfs/TestFileFormat.scala | 41 ++++++++++--------- 2 files changed, 28 insertions(+), 24 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index e18641c09..c6d8289ba 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -52,8 +52,11 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { - val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + + val csvDelimiterOption = ss.sparkContext.getConf.get("spark.feathr.inputFormat.csvOptions.sep", ",") + + println(s"$csvDelimiterOption") + format match { case AVRO => new AvroJsonDataLoader(ss, path).loadDataFrame() case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) @@ -78,8 +81,8 @@ object FileFormat { case p if p.endsWith(".avro") => AVRO case p if p.startsWith("jdbc:") => JDBC case _ => - // if we cannot tell the file format from the file extensions, we should read from `spark.feathr.inputFormat` to get the format that's sepcified by user. - dataIOParameters.getOrElse(DATA_FORMAT, ss.conf.get("spark.feathr.inputFormat", AVRO)).toUpperCase + // if we cannot tell the file format from the file extensions, we should read from `spark.feathr.inputFormat` to get the format that's sepcified by user. + dataIOParameters.getOrElse(DATA_FORMAT, ss.conf.get("spark.feathr.inputFormat", AVRO)).toUpperCase } diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala index 8fb1baf4a..2c167b11d 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala @@ -1,7 +1,7 @@ package com.linkedin.feathr.offline.source.dataloader.hdfs import com.linkedin.feathr.offline.TestFeathr -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Row, SparkSession} import org.testng.Assert.assertEquals import org.testng.annotations.Test @@ -33,6 +33,7 @@ class TestFileFormat extends TestFeathr { val path = "anchor1-source.tsv" val absolutePath = getClass.getClassLoader.getResource(path).getPath val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") val df = FileFormat.loadDataFrame(ss, absolutePath, "CSV") @@ -48,24 +49,24 @@ class TestFileFormat extends TestFeathr { assertEquals(df.collect(), expectedRows) } - @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter") - def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { - val path = "anchor1-source.tsv" - val absolutePath = getClass.getClassLoader.getResource(path).getPath - val sqlContext = ss.sqlContext - sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") - val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) - - val expectedRows = Array( - Row("1", "apple", "10", "10", "0.1"), - Row("2", "orange", "10", "3", "0.1"), - Row("3", "banana", "10", "2", "0.9"), - Row("4", "apple", "10", "1", "0.7"), - Row("5", "apple", "11", "11", "1.0"), - Row("7", "banana", "2", "10", "81.27"), - Row("9", "banana", "4", "4", "0.4") - ) - assertEquals(df.collect(), expectedRows) - } + // @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter HDFS") + // def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { + // val path = "anchor1-source.tsv" + // val absolutePath = getClass.getClassLoader.getResource(path).getPath + // val sqlContext = ss.sqlContext + // sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + // val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) + // + // val expectedRows = Array( + // Row("1", "apple", "10", "10", "0.1"), + // Row("2", "orange", "10", "3", "0.1"), + // Row("3", "banana", "10", "2", "0.9"), + // Row("4", "apple", "10", "1", "0.7"), + // Row("5", "apple", "11", "11", "1.0"), + // Row("7", "banana", "2", "10", "81.27"), + // Row("9", "banana", "4", "4", "0.4") + // ) + // assertEquals(df.collect(), expectedRows) + // } } From 68571576961b7c4fc7aeb48cbdad8bf6cf4603af Mon Sep 17 00:00:00 2001 From: changyonglik Date: Mon, 11 Jul 2022 21:43:36 +0900 Subject: [PATCH 09/19] Removed unused imports from BatchDataLoader Signed-off-by: changyonglik --- .../source/dataloader/BatchDataLoader.scala | 2 - .../source/dataloader/hdfs/FileFormat.scala | 12 ++++-- .../dataloader/TestDataLoaderFactory.scala | 1 + .../dataloader/hdfs/TestFileFormat.scala | 41 +++++++++---------- 4 files changed, 30 insertions(+), 26 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index 504c2c990..7e533dd2b 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -4,8 +4,6 @@ import com.linkedin.feathr.common.exception.{ErrorLabel, FeathrInputDataExceptio import com.linkedin.feathr.offline.config.location.InputLocation import com.linkedin.feathr.offline.generation.SparkIOUtils import com.linkedin.feathr.offline.job.DataSourceUtils.getSchemaFromAvroDataFile -import com.linkedin.feathr.offline.source.dataloader.jdbc.JdbcUtils -import com.linkedin.feathr.offline.source.dataloader.DataLoaderHandler import org.apache.avro.Schema import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapred.JobConf diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index c6d8289ba..efd777653 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -13,6 +13,7 @@ object FileFormat { // The each of the following are corresponding to one File format // Please update when new Format is supported val CSV = "CSV" + val TSV = "TSV" // This type is used for local test scenario of AVRO data source val AVRO_JSON = "AVRO_JSON" val AVRO = "AVRO" @@ -38,7 +39,7 @@ object FileFormat { val p = path.toLowerCase() p match { case p if p.endsWith(".csv") => CSV - case p if p.endsWith(".tsv") => CSV + case p if p.endsWith(".tsv") => TSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC case p if p.endsWith(".avro.json") => AVRO_JSON @@ -52,14 +53,15 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { - - val csvDelimiterOption = ss.sparkContext.getConf.get("spark.feathr.inputFormat.csvOptions.sep", ",") + val sqlContext = ss.sqlContext + val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") println(s"$csvDelimiterOption") format match { case AVRO => new AvroJsonDataLoader(ss, path).loadDataFrame() case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) + case TSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) case PARQUET => new ParquetDataLoader(ss, path).loadDataFrame() case _ => ??? } @@ -75,6 +77,7 @@ object FileFormat { val p = existingHdfsPaths.head.toLowerCase() p match { case p if p.endsWith(".csv") => CSV + case p if p.endsWith(".tsv") => TSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC case p if p.endsWith(".avro.json") => AVRO_JSON @@ -91,9 +94,12 @@ object FileFormat { def loadHdfsDataFrame(format: String, existingHdfsPaths: Seq[String]): DataFrame = { val sqlContext = ss.sqlContext val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + val df = format match { case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(existingHdfsPaths: _*) + case TSV => + ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(existingHdfsPaths: _*) case AVRO => ss.read.format(AVRO_DATASOURCE).load(existingHdfsPaths: _*) case ORC => diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala index 7a0b2ea76..c95871705 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala @@ -18,6 +18,7 @@ class TestDataLoaderFactory extends TestFeathr { sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") val csvLoaderWithDelimiter = localDataLoaderFactory.create("anchor1-source.tsv") assertTrue(csvLoaderWithDelimiter.isInstanceOf[BatchDataLoader]) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") val avroJsonLoader = localDataLoaderFactory.create("anchor5-source.avro.json") assertTrue(avroJsonLoader.isInstanceOf[AvroJsonDataLoader]) val jsonWithSchemaLoader = localDataLoaderFactory.create("simple-obs2") // the mock data folder exists. diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala index 2c167b11d..c5d82234d 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala @@ -33,9 +33,8 @@ class TestFileFormat extends TestFeathr { val path = "anchor1-source.tsv" val absolutePath = getClass.getClassLoader.getResource(path).getPath val sqlContext = ss.sqlContext - sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") - val df = FileFormat.loadDataFrame(ss, absolutePath, "CSV") + val df = FileFormat.loadDataFrame(ss, absolutePath, "TSV") val expectedRows = Array( Row("1", "apple", "10", "10", "0.1"), @@ -49,24 +48,24 @@ class TestFileFormat extends TestFeathr { assertEquals(df.collect(), expectedRows) } - // @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter HDFS") - // def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { - // val path = "anchor1-source.tsv" - // val absolutePath = getClass.getClassLoader.getResource(path).getPath - // val sqlContext = ss.sqlContext - // sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") - // val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) - // - // val expectedRows = Array( - // Row("1", "apple", "10", "10", "0.1"), - // Row("2", "orange", "10", "3", "0.1"), - // Row("3", "banana", "10", "2", "0.9"), - // Row("4", "apple", "10", "1", "0.7"), - // Row("5", "apple", "11", "11", "1.0"), - // Row("7", "banana", "2", "10", "81.27"), - // Row("9", "banana", "4", "4", "0.4") - // ) - // assertEquals(df.collect(), expectedRows) - // } + @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter for HDFS") + def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { + val path = "anchor1-source.tsv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val df = FileFormat.loadHdfsDataFrame("TSV", Seq(absolutePath)) + + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + } } From ebc43eebd518510c16e49f808369111760d2b0bb Mon Sep 17 00:00:00 2001 From: changyonglik Date: Sat, 23 Jul 2022 00:46:33 +0900 Subject: [PATCH 10/19] Fixed test failures Signed-off-by: changyonglik --- .../source/dataloader/BatchDataLoader.scala | 51 ++++--- .../source/dataloader/CsvDataLoader.scala | 20 ++- .../source/dataloader/hdfs/FileFormat.scala | 34 +++-- .../feathr/offline/util/SourceUtils.scala | 129 ++++++++++-------- .../dataloader/TestBatchDataLoader.scala | 9 +- .../source/dataloader/TestCsvDataLoader.scala | 3 +- .../dataloader/TestDataLoaderFactory.scala | 4 +- .../dataloader/hdfs/TestFileFormat.scala | 48 +++---- .../feathr/offline/util/TestSourceUtils.scala | 3 +- 9 files changed, 185 insertions(+), 116 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index 7e533dd2b..90f4059ac 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -4,6 +4,8 @@ import com.linkedin.feathr.common.exception.{ErrorLabel, FeathrInputDataExceptio import com.linkedin.feathr.offline.config.location.InputLocation import com.linkedin.feathr.offline.generation.SparkIOUtils import com.linkedin.feathr.offline.job.DataSourceUtils.getSchemaFromAvroDataFile +import com.linkedin.feathr.offline.source.dataloader.jdbc.JdbcUtils +import com.linkedin.feathr.offline.source.dataloader.DataLoaderHandler import org.apache.avro.Schema import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapred.JobConf @@ -49,6 +51,15 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation loadDataFrame(Map(), new JobConf(ss.sparkContext.hadoopConfiguration)) } + /** + * Convert string to special characters + * @return a String + */ + def escape(raw: String): String = { + import scala.reflect.runtime.universe._ + Literal(Constant(raw)).toString.replaceAll("\"", "") + } + /** * load the source data as dataframe. * @param dataIOParameters extra parameters @@ -62,28 +73,32 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation val dataPath = location.getPath log.info(s"Loading ${location} as DataFrame, using parameters ${dataIOParametersWithSplitSize}") - + + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - + // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep + val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + try { - import scala.util.control.Breaks._ + import scala.util.control.Breaks._ - var dfOpt: Option[DataFrame] = None - breakable { - for(dataLoaderHandler <- dataLoaderHandlers) { - println(s"Applying dataLoaderHandler ${dataLoaderHandler}") - if (dataLoaderHandler.validatePath(dataPath)) { - dfOpt = Some(dataLoaderHandler.createDataFrame(dataPath, dataIOParametersWithSplitSize, jobConf)) - break - } + var dfOpt: Option[DataFrame] = None + breakable { + for(dataLoaderHandler <- dataLoaderHandlers) { + println(s"Applying dataLoaderHandler ${dataLoaderHandler}") + if (dataLoaderHandler.validatePath(dataPath)) { + dfOpt = Some(dataLoaderHandler.createDataFrame(dataPath, dataIOParametersWithSplitSize, jobConf)) + break } } - val df = dfOpt match { - case Some(df) => df - case _ => location.loadDf(ss, dataIOParametersWithSplitSize) - } - df + } + val df = dfOpt match { + case Some(df) => df + case _ => location.loadDf(ss, dataIOParametersWithSplitSize) + } + df } catch { case feathrException: FeathrInputDataException => println(feathrException.toString) @@ -93,4 +108,4 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(dataPath) } } -} +} \ No newline at end of file diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index c3d2127f5..bd0d5265b 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -27,13 +27,29 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat doLoadCsvDocumentLikeAvro()._2 } + /** + * Convert string to special characters + * @return a String + */ + def escape(raw: String): String = { + import scala.reflect.runtime.universe._ + Literal(Constant(raw)).toString.replaceAll("\"", "") + } + + /** * load the source data as dataframe. * @return an dataframe */ override def loadDataFrame(): DataFrame = { + + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep + val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + try { log.debug(s"Loading CSV path :${path}") val absolutePath = new File(path).getPath @@ -103,4 +119,4 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat private def getArbitraryRecordName(x: AnyRef): String = { "AnonRecord_" + Integer.toHexString(x.hashCode) } -} +} \ No newline at end of file diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index efd777653..9e432843e 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -13,7 +13,6 @@ object FileFormat { // The each of the following are corresponding to one File format // Please update when new Format is supported val CSV = "CSV" - val TSV = "TSV" // This type is used for local test scenario of AVRO data source val AVRO_JSON = "AVRO_JSON" val AVRO = "AVRO" @@ -30,6 +29,15 @@ object FileFormat { val DATA_FORMAT = "data.format" + /** + * Convert string to special characters + * @return a String + */ + def escape(raw: String): String = { + import scala.reflect.runtime.universe._ + Literal(Constant(raw)).toString.replaceAll("\"", "") + } + /** * To define if the file is JDBC, Single File or Path list (default) * @param path @@ -39,7 +47,6 @@ object FileFormat { val p = path.toLowerCase() p match { case p if p.endsWith(".csv") => CSV - case p if p.endsWith(".tsv") => TSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC case p if p.endsWith(".avro.json") => AVRO_JSON @@ -53,15 +60,17 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { - val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - println(s"$csvDelimiterOption") + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep + val sqlContext = ss.sqlContext + // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep + val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption format match { case AVRO => new AvroJsonDataLoader(ss, path).loadDataFrame() case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) - case TSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(path) case PARQUET => new ParquetDataLoader(ss, path).loadDataFrame() case _ => ??? } @@ -77,7 +86,7 @@ object FileFormat { val p = existingHdfsPaths.head.toLowerCase() p match { case p if p.endsWith(".csv") => CSV - case p if p.endsWith(".tsv") => TSV + case p if p.endsWith(".tsv") => CSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC case p if p.endsWith(".avro.json") => AVRO_JSON @@ -92,14 +101,17 @@ object FileFormat { } def loadHdfsDataFrame(format: String, existingHdfsPaths: Seq[String]): DataFrame = { + + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep + val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption val df = format match { case CSV => ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(existingHdfsPaths: _*) - case TSV => - ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(existingHdfsPaths: _*) case AVRO => ss.read.format(AVRO_DATASOURCE).load(existingHdfsPaths: _*) case ORC => @@ -116,4 +128,4 @@ object FileFormat { } df } -} +} \ No newline at end of file diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index 47f9c55d1..3b411c583 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -65,6 +65,15 @@ private[offline] object SourceUtils { val FEATURE_MP_DEF_CONFIG_SUFFIX = ".conf" val firstRecordName = "topLevelRecord" + /** + * Convert string to special characters + * @return a String + */ + def escape(raw: String): String = { + import scala.reflect.runtime.universe._ + Literal(Constant(raw)).toString.replaceAll("\"", "") + } + /** * get AVRO datum type of a dataset we should use to load, * it is determined by the expect datatype from a set of anchor transformers @@ -96,13 +105,13 @@ private[offline] object SourceUtils { } def getPathList( - sourceFormatType: SourceFormatType, - sourcePath: String, - ss: SparkSession, - dateParam: Option[DateParam], - dataLoaderHandlers: List[DataLoaderHandler], - targetDate: Option[String] = None, - failOnMissing: Boolean = true): Seq[String] = { + sourceFormatType: SourceFormatType, + sourcePath: String, + ss: SparkSession, + dateParam: Option[DateParam], + dataLoaderHandlers: List[DataLoaderHandler], + targetDate: Option[String] = None, + failOnMissing: Boolean = true): Seq[String] = { sourceFormatType match { case SourceFormatType.FIXED_PATH => Seq(HdfsUtils.getLatestPath(sourcePath, ss.sparkContext.hadoopConfiguration)) case SourceFormatType.TIME_PATH => @@ -357,35 +366,35 @@ private[offline] object SourceUtils { val inputSplitSize = sparkConf.get("spark.feathr.input.split.size", "") var dataIOParameters = Map(SparkIOUtils.SPLIT_SIZE -> inputSplitSize) - val fileName = new File(factDataSourcePath).getName - if (fileName.endsWith("daily") || fileName.endsWith("hourly")) { // when source is pure HDFS with time partitions - // HDFS path with time partitions should have the following format: - // [source directory]/[daily/hourly]/YYYY/MM/dd/hh/ - // In Feathr configuration, only [source directory]/[daily/hourly] needs to be given. - // The rest section is constructed using `HdfsUtils.getPaths`. - val isDaily = fileName.endsWith("daily") // TODO: better handling for extracting "daily" or "hourly" - val (factDataStartTime, factDataEndTime) = getFactDataTimeRange(obsDataStartTime, obsDataEndTime, window, isDaily, timeDelayMapOpt) - // getPaths is left-inclusive - val hdfsPaths = if (isDaily) { - HdfsUtils.getPaths(factDataSourcePath, factDataStartTime, factDataEndTime.plusDays(1), ChronoUnit.DAYS) - } else { - HdfsUtils.getPaths(factDataSourcePath, factDataStartTime, factDataEndTime.plusHours(1), ChronoUnit.HOURS) - } - val existingHdfsPaths = hdfsPaths.filter(HdfsUtils.exists(_)) - if (existingHdfsPaths.isEmpty) { - throw new FeathrInputDataException( - ErrorLabel.FEATHR_USER_ERROR, - s"Trying to load feature data in HDFS. No available date partition exist in HDFS for path. " + - s"$factDataSourcePath between $factDataStartTime and $factDataEndTime. Please make sure there is needed " + - s"data for that time range.") - } - log.info(s"Loading HDFS path ${existingHdfsPaths} as union DataFrame for sliding window aggregation, using parameters ${dataIOParameters}") - SparkIOUtils.createUnionDataFrame(existingHdfsPaths, dataIOParameters, new JobConf(), dataLoaderHandlers) + val fileName = new File(factDataSourcePath).getName + if (fileName.endsWith("daily") || fileName.endsWith("hourly")) { // when source is pure HDFS with time partitions + // HDFS path with time partitions should have the following format: + // [source directory]/[daily/hourly]/YYYY/MM/dd/hh/ + // In Feathr configuration, only [source directory]/[daily/hourly] needs to be given. + // The rest section is constructed using `HdfsUtils.getPaths`. + val isDaily = fileName.endsWith("daily") // TODO: better handling for extracting "daily" or "hourly" + val (factDataStartTime, factDataEndTime) = getFactDataTimeRange(obsDataStartTime, obsDataEndTime, window, isDaily, timeDelayMapOpt) + // getPaths is left-inclusive + val hdfsPaths = if (isDaily) { + HdfsUtils.getPaths(factDataSourcePath, factDataStartTime, factDataEndTime.plusDays(1), ChronoUnit.DAYS) } else { - // Load a single folder - log.info(s"Loading HDFS path ${factDataSourcePath} as DataFrame for sliding window aggregation, using parameters ${dataIOParameters}") - SparkIOUtils.createDataFrame(SimplePath(factDataSourcePath), dataIOParameters, new JobConf(), dataLoaderHandlers) + HdfsUtils.getPaths(factDataSourcePath, factDataStartTime, factDataEndTime.plusHours(1), ChronoUnit.HOURS) + } + val existingHdfsPaths = hdfsPaths.filter(HdfsUtils.exists(_)) + if (existingHdfsPaths.isEmpty) { + throw new FeathrInputDataException( + ErrorLabel.FEATHR_USER_ERROR, + s"Trying to load feature data in HDFS. No available date partition exist in HDFS for path. " + + s"$factDataSourcePath between $factDataStartTime and $factDataEndTime. Please make sure there is needed " + + s"data for that time range.") } + log.info(s"Loading HDFS path ${existingHdfsPaths} as union DataFrame for sliding window aggregation, using parameters ${dataIOParameters}") + SparkIOUtils.createUnionDataFrame(existingHdfsPaths, dataIOParameters, new JobConf(), dataLoaderHandlers) + } else { + // Load a single folder + log.info(s"Loading HDFS path ${factDataSourcePath} as DataFrame for sliding window aggregation, using parameters ${dataIOParameters}") + SparkIOUtils.createDataFrame(SimplePath(factDataSourcePath), dataIOParameters, new JobConf(), dataLoaderHandlers) + } } /** @@ -404,13 +413,13 @@ private[offline] object SourceUtils { * */ private[feathr] def getFactDataTimeRange( - obsDataStartTime: LocalDateTime, - obsDataEndTime: LocalDateTime, - window: Duration, - isDaily: Boolean, - timeDelayMap: Map[String, Duration]): (LocalDateTime, LocalDateTime) = { + obsDataStartTime: LocalDateTime, + obsDataEndTime: LocalDateTime, + window: Duration, + isDaily: Boolean, + timeDelayMap: Map[String, Duration]): (LocalDateTime, LocalDateTime) = { - val minTimeDelay = timeDelayMap.values.size match { + val minTimeDelay = timeDelayMap.values.size match { case 0 => Duration.ZERO case _ => timeDelayMap.values.min } @@ -629,7 +638,7 @@ private[offline] object SourceUtils { * @return */ def loadAsUnionDataFrame(ss: SparkSession, inputPath: Seq[String], - dataLoaderHandlers: List[DataLoaderHandler]): DataFrame = { + dataLoaderHandlers: List[DataLoaderHandler]): DataFrame = { val sparkConf = ss.sparkContext.getConf val inputSplitSize = sparkConf.get("spark.feathr.input.split.size", "") val dataIOParameters = Map(SparkIOUtils.SPLIT_SIZE -> inputSplitSize) @@ -645,7 +654,7 @@ private[offline] object SourceUtils { * @return */ def loadAsDataFrame(ss: SparkSession, location: InputLocation, - dataLoaderHandlers: List[DataLoaderHandler]): DataFrame = { + dataLoaderHandlers: List[DataLoaderHandler]): DataFrame = { val sparkConf = ss.sparkContext.getConf val inputSplitSize = sparkConf.get("spark.feathr.input.split.size", "") val dataIOParameters = Map(SparkIOUtils.SPLIT_SIZE -> inputSplitSize) @@ -661,22 +670,28 @@ private[offline] object SourceUtils { * @return */ def loadObservationAsDF(ss: SparkSession, conf: Configuration, inputData: InputData, dataLoaderHandlers: List[DataLoaderHandler], - failOnMissing: Boolean = true): DataFrame = { + failOnMissing: Boolean = true): DataFrame = { // TODO: Split isLocal case into Test Packages val format = FileFormat.getType(inputData.inputPath) - val sqlContext = ss.sqlContext - val csvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") log.info(s"loading ${inputData.inputPath} input Path as Format: ${format}") + + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep + val sqlContext = ss.sqlContext + // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep + val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") + // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + format match { case FileFormat.PATHLIST => { val pathList = getPathList(sourceFormatType=inputData.sourceType, - sourcePath=inputData.inputPath, - ss=ss, - dateParam=inputData.dateParam, - targetDate=None, - failOnMissing=failOnMissing, - dataLoaderHandlers=dataLoaderHandlers - ) + sourcePath=inputData.inputPath, + ss=ss, + dateParam=inputData.dateParam, + targetDate=None, + failOnMissing=failOnMissing, + dataLoaderHandlers=dataLoaderHandlers + ) if (ss.sparkContext.isLocal) { // for test try { loadAsUnionDataFrame(ss, pathList, dataLoaderHandlers) @@ -924,10 +939,10 @@ private[offline] object SourceUtils { * @return a sequence of (dataframe of one day, the date of this dataframe) */ private[feathr] def loadTimeSeriesAvroJson( - ss: SparkSession, - paths: Seq[String], - basePath: String, - isSeparateAvroJson: Boolean = false): Seq[(DataFrame, Interval)] = { + ss: SparkSession, + paths: Seq[String], + basePath: String, + isSeparateAvroJson: Boolean = false): Seq[(DataFrame, Interval)] = { if (ss.sparkContext.isLocal && !paths.head.startsWith(HDFS_PREFIX)) { // If runs locally and source is not HDFS, this is for local test API // Assume feature data file always named as "data.avro.json", this is similar to hadoop names output as part-00000.avro @@ -1048,4 +1063,4 @@ private[offline] object SourceUtils { }) } } -} +} \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala index 9efc5b436..83e46c1d0 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestBatchDataLoader.scala @@ -11,6 +11,11 @@ import org.testng.annotations.Test */ class TestBatchDataLoader extends TestFeathr { + def escape(raw: String): String = { + import scala.reflect.runtime.universe._ + Literal(Constant(raw)).toString + } + @Test(description = "test loading dataframe with BatchDataLoader") def testBatchDataLoader() : Unit = { val path = "anchor1-source.csv" @@ -35,6 +40,7 @@ class TestBatchDataLoader extends TestFeathr { val absolutePath = getClass.getClassLoader.getResource(path).getPath val sqlContext = ss.sqlContext sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + println(s"Postset config testBatchDataLoaderWithCsvDelimiterOption: ${ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep")}") val batchDataLoader = new BatchDataLoader(ss, SimplePath(absolutePath), List()) val df = batchDataLoader.loadDataFrame() val expectedRows = Array( @@ -47,6 +53,7 @@ class TestBatchDataLoader extends TestFeathr { Row("9", "banana", "4", "4", "0.4") ) assertEquals(df.collect(), expectedRows) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") } -} +} \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala index ab1d5295b..ef838f0cb 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestCsvDataLoader.scala @@ -63,5 +63,6 @@ class TestCsvDataLoader extends TestFeathr { Row("9", "banana", "4", "4", "0.4") ) assertEquals(df.collect(), expectedRows) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") } -} +} \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala index c95871705..640957179 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/TestDataLoaderFactory.scala @@ -5,7 +5,7 @@ import org.testng.Assert.assertTrue import org.testng.annotations.Test /** - * unit tests for [[DataLoaderFactory]] + * unit tests for DataLoaderFactory */ class TestDataLoaderFactory extends TestFeathr { @@ -24,4 +24,4 @@ class TestDataLoaderFactory extends TestFeathr { val jsonWithSchemaLoader = localDataLoaderFactory.create("simple-obs2") // the mock data folder exists. assertTrue(jsonWithSchemaLoader.isInstanceOf[JsonWithSchemaDataLoader]) } -} +} \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala index c5d82234d..e690be3f9 100644 --- a/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala +++ b/src/test/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/TestFileFormat.scala @@ -1,7 +1,7 @@ package com.linkedin.feathr.offline.source.dataloader.hdfs import com.linkedin.feathr.offline.TestFeathr -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.Row import org.testng.Assert.assertEquals import org.testng.annotations.Test @@ -14,7 +14,7 @@ class TestFileFormat extends TestFeathr { def testLoadDataFrame() : Unit = { val path = "anchor1-source.csv" val absolutePath = getClass.getClassLoader.getResource(path).getPath - val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) + val df = FileFormat.loadDataFrame(ss, absolutePath, "CSV") val expectedRows = Array( Row("1", "apple", "10", "10", "0.1"), @@ -28,13 +28,13 @@ class TestFileFormat extends TestFeathr { assertEquals(df.collect(), expectedRows) } - @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter") + @Test(description = "test loading dataframe with FileFormat by specifying delimiter") def testLoadDataFrameWithCsvDelimiterOption() : Unit = { val path = "anchor1-source.tsv" val absolutePath = getClass.getClassLoader.getResource(path).getPath val sqlContext = ss.sqlContext sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") - val df = FileFormat.loadDataFrame(ss, absolutePath, "TSV") + val df = FileFormat.loadDataFrame(ss, absolutePath, "CSV") val expectedRows = Array( Row("1", "apple", "10", "10", "0.1"), @@ -46,26 +46,28 @@ class TestFileFormat extends TestFeathr { Row("9", "banana", "4", "4", "0.4") ) assertEquals(df.collect(), expectedRows) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") } - @Test(description = "test loading dataframe with CsvDataLoader by specifying delimiter for HDFS") - def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { - val path = "anchor1-source.tsv" - val absolutePath = getClass.getClassLoader.getResource(path).getPath - val sqlContext = ss.sqlContext - sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") - val df = FileFormat.loadHdfsDataFrame("TSV", Seq(absolutePath)) + @Test(description = "test loading dataframe with FileFormat by specifying delimiter for HDFS") + def testLoadDataFrameWithCsvDelimiterOptionHDFS() : Unit = { + val path = "anchor1-source.tsv" + val absolutePath = getClass.getClassLoader.getResource(path).getPath + val sqlContext = ss.sqlContext + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "\t") + val df = FileFormat.loadHdfsDataFrame("CSV", Seq(absolutePath)) - val expectedRows = Array( - Row("1", "apple", "10", "10", "0.1"), - Row("2", "orange", "10", "3", "0.1"), - Row("3", "banana", "10", "2", "0.9"), - Row("4", "apple", "10", "1", "0.7"), - Row("5", "apple", "11", "11", "1.0"), - Row("7", "banana", "2", "10", "81.27"), - Row("9", "banana", "4", "4", "0.4") - ) - assertEquals(df.collect(), expectedRows) - } + val expectedRows = Array( + Row("1", "apple", "10", "10", "0.1"), + Row("2", "orange", "10", "3", "0.1"), + Row("3", "banana", "10", "2", "0.9"), + Row("4", "apple", "10", "1", "0.7"), + Row("5", "apple", "11", "11", "1.0"), + Row("7", "banana", "2", "10", "81.27"), + Row("9", "banana", "4", "4", "0.4") + ) + assertEquals(df.collect(), expectedRows) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") + } -} +} \ No newline at end of file diff --git a/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala index a9453dce2..0ce2df9a6 100644 --- a/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala +++ b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala @@ -53,6 +53,7 @@ class TestSourceUtils extends TestFeathr { Row("9", "banana", "4", "4", "0.4") ) assertEquals(df.collect(), expectedRows) + sqlContext.setConf("spark.feathr.inputFormat.csvOptions.sep", "") } -} +} \ No newline at end of file From 4b4ffe925fa8aec6fab57fc4cb449f6c95de0b25 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Wed, 27 Jul 2022 12:20:21 +0900 Subject: [PATCH 11/19] Added release version Signed-off-by: changyonglik --- docs/how-to-guides/feathr-job-configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/how-to-guides/feathr-job-configuration.md b/docs/how-to-guides/feathr-job-configuration.md index 5967642a9..35b95a425 100644 --- a/docs/how-to-guides/feathr-job-configuration.md +++ b/docs/how-to-guides/feathr-job-configuration.md @@ -12,4 +12,4 @@ Since Feathr uses Spark as the underlying execution engine, there's a way to ove | ------------------------- | ------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------- | | spark.feathr.inputFormat | None | Specify the input format if the file cannot be tell automatically. By default, Feathr will read files by parsing the file extension name; However the file/folder name doesn't have extension name, this configuration can be set to tell Feathr which format it should use to read the data. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to read delta lake. | 0.2.1 | | spark.feathr.outputFormat | None | Specify the output format. "avro" is the default behavior if this value is not set. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to write delta lake. | 0.2.1 | -| spark.feathr.inputFormat.csvOptions.sep | None | Specify the delimiter. For example, "," for commas or "\t" for tabs. | TODO: Check version | \ No newline at end of file +| spark.feathr.inputFormat.csvOptions.sep | None | Specify the delimiter. For example, "," for commas or "\t" for tabs. | 0.6.0 | From 5e282258eedb63f2332571c100dd3558fe94687d Mon Sep 17 00:00:00 2001 From: changyonglik Date: Mon, 1 Aug 2022 23:29:56 +0900 Subject: [PATCH 12/19] Removed tailing space Signed-off-by: changyonglik --- .../com/linkedin/feathr/offline/util/TestSourceUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala index 0ce2df9a6..939239ba6 100644 --- a/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala +++ b/src/test/scala/com/linkedin/feathr/offline/util/TestSourceUtils.scala @@ -33,7 +33,7 @@ class TestSourceUtils extends TestFeathr { assertEquals(df.collect(), expectedRows) } - @Test(description = "test loading dataframe with SourceUtils by specifying delimiter") + @Test(description = "test loading dataframe with SourceUtils by specifying delimiter") def testSourceUtilsWithCsvDelimiterOption() : Unit = { val conf: Configuration = new Configuration() val path = "anchor1-source.tsv" From 0ffa608edfafbdf4f304c3ecbef6a2d366b0a542 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Tue, 2 Aug 2022 23:17:28 +0900 Subject: [PATCH 13/19] Removed wildcard imports Signed-off-by: changyonglik --- .../feathr/offline/source/dataloader/BatchDataLoader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index 90f4059ac..a8d01e878 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -56,7 +56,7 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation * @return a String */ def escape(raw: String): String = { - import scala.reflect.runtime.universe._ + import scala.reflect.runtime.universe.{Literal, Constant} Literal(Constant(raw)).toString.replaceAll("\"", "") } From 3ec3fdf2860e32c56cde018f469e1b336ba8f9e3 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Tue, 2 Aug 2022 23:35:22 +0900 Subject: [PATCH 14/19] Paraphrased comments and docstring Signed-off-by: changyonglik --- .../feathr/offline/source/dataloader/BatchDataLoader.scala | 3 +-- .../feathr/offline/source/dataloader/CsvDataLoader.scala | 5 ++--- .../feathr/offline/source/dataloader/hdfs/FileFormat.scala | 6 +++--- .../com/linkedin/feathr/offline/util/SourceUtils.scala | 5 ++--- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index a8d01e878..c798127b7 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -52,8 +52,7 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation } /** - * Convert string to special characters - * @return a String + * Convert delimiter to an escape character (e.g. " " -> "\t") */ def escape(raw: String): String = { import scala.reflect.runtime.universe.{Literal, Constant} diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index bd0d5265b..d7d56e305 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -28,11 +28,10 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat } /** - * Convert string to special characters - * @return a String + * Convert delimiter to an escape character (e.g. " " -> "\t") */ def escape(raw: String): String = { - import scala.reflect.runtime.universe._ + import scala.reflect.runtime.universe.{Literal, Constant} Literal(Constant(raw)).toString.replaceAll("\"", "") } diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index 9e432843e..bf8575105 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -30,11 +30,10 @@ object FileFormat { val DATA_FORMAT = "data.format" /** - * Convert string to special characters - * @return a String + * Convert delimiter to an escape character (e.g. " " -> "\t") */ def escape(raw: String): String = { - import scala.reflect.runtime.universe._ + import scala.reflect.runtime.universe.{Literal, Constant} Literal(Constant(raw)).toString.replaceAll("\"", "") } @@ -86,6 +85,7 @@ object FileFormat { val p = existingHdfsPaths.head.toLowerCase() p match { case p if p.endsWith(".csv") => CSV + // Tab-separated Format will be treated as CSV (Enum) here but with tab as the delimiter case p if p.endsWith(".tsv") => CSV case p if p.endsWith(".parquet") => PARQUET case p if p.endsWith(".orc") => ORC diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index 3b411c583..41c124968 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -66,11 +66,10 @@ private[offline] object SourceUtils { val firstRecordName = "topLevelRecord" /** - * Convert string to special characters - * @return a String + * Convert delimiter to an escape character (e.g. " " -> "\t") */ def escape(raw: String): String = { - import scala.reflect.runtime.universe._ + import scala.reflect.runtime.universe.{Literal, Constant} Literal(Constant(raw)).toString.replaceAll("\"", "") } From 52eb35f94e32d714fb24eaeecc029b9aedd06ded Mon Sep 17 00:00:00 2001 From: changyonglik Date: Wed, 3 Aug 2022 00:15:22 +0900 Subject: [PATCH 15/19] Added DelimiterUtils Signed-off-by: changyonglik --- .../offline/source/dataloader/BatchDataLoader.scala | 10 +--------- .../feathr/offline/util/DelimiterUtils.scala | 13 +++++++++++++ 2 files changed, 14 insertions(+), 9 deletions(-) create mode 100644 src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index c798127b7..a813032f9 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -4,8 +4,8 @@ import com.linkedin.feathr.common.exception.{ErrorLabel, FeathrInputDataExceptio import com.linkedin.feathr.offline.config.location.InputLocation import com.linkedin.feathr.offline.generation.SparkIOUtils import com.linkedin.feathr.offline.job.DataSourceUtils.getSchemaFromAvroDataFile -import com.linkedin.feathr.offline.source.dataloader.jdbc.JdbcUtils import com.linkedin.feathr.offline.source.dataloader.DataLoaderHandler +import com.linkedin.feathr.offline.util.DelimiterUtils.escape import org.apache.avro.Schema import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapred.JobConf @@ -51,14 +51,6 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation loadDataFrame(Map(), new JobConf(ss.sparkContext.hadoopConfiguration)) } - /** - * Convert delimiter to an escape character (e.g. " " -> "\t") - */ - def escape(raw: String): String = { - import scala.reflect.runtime.universe.{Literal, Constant} - Literal(Constant(raw)).toString.replaceAll("\"", "") - } - /** * load the source data as dataframe. * @param dataIOParameters extra parameters diff --git a/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala new file mode 100644 index 000000000..ce9e2cacd --- /dev/null +++ b/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala @@ -0,0 +1,13 @@ +package com.linkedin.feathr.offline.util + +object DelimiterUtils { + + /** + * Convert delimiter to an escape character (e.g. " " -> "\t") + */ + def escape(raw: String): String = { + import scala.reflect.runtime.universe.{Literal, Constant} + Literal(Constant(raw)).toString.replaceAll("\"", "") + } + +} From 2d41211b25e83327401eac9a2847989c9cb7b17e Mon Sep 17 00:00:00 2001 From: changyonglik Date: Thu, 4 Aug 2022 21:37:46 +0900 Subject: [PATCH 16/19] Refactored utils Signed-off-by: changyonglik --- .../source/dataloader/BatchDataLoader.scala | 10 +++----- .../source/dataloader/CsvDataLoader.scala | 19 +++----------- .../source/dataloader/hdfs/FileFormat.scala | 25 ++++--------------- .../feathr/offline/util/DelimiterUtils.scala | 9 ++++++- .../feathr/offline/util/SourceUtils.scala | 17 +++---------- 5 files changed, 23 insertions(+), 57 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index a813032f9..93004a99f 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -5,7 +5,7 @@ import com.linkedin.feathr.offline.config.location.InputLocation import com.linkedin.feathr.offline.generation.SparkIOUtils import com.linkedin.feathr.offline.job.DataSourceUtils.getSchemaFromAvroDataFile import com.linkedin.feathr.offline.source.dataloader.DataLoaderHandler -import com.linkedin.feathr.offline.util.DelimiterUtils.escape +import com.linkedin.feathr.offline.util.DelimiterUtils.checkDelimiterOption import org.apache.avro.Schema import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapred.JobConf @@ -65,12 +65,8 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation log.info(s"Loading ${location} as DataFrame, using parameters ${dataIOParametersWithSplitSize}") - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep - val sqlContext = ss.sqlContext - // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep - val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption - val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) try { import scala.util.control.Breaks._ diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index d7d56e305..8ea87262a 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -10,6 +10,8 @@ import org.apache.spark.sql.{DataFrame, SparkSession} import java.io.File import scala.collection.JavaConverters._ import scala.collection.convert.wrapAll._ +import com.linkedin.feathr.offline.util.DelimiterUtils.checkDelimiterOption + import scala.io.Source /** @@ -27,27 +29,14 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat doLoadCsvDocumentLikeAvro()._2 } - /** - * Convert delimiter to an escape character (e.g. " " -> "\t") - */ - def escape(raw: String): String = { - import scala.reflect.runtime.universe.{Literal, Constant} - Literal(Constant(raw)).toString.replaceAll("\"", "") - } - - /** * load the source data as dataframe. * @return an dataframe */ override def loadDataFrame(): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep - val sqlContext = ss.sqlContext - // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep - val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption - val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) try { log.debug(s"Loading CSV path :${path}") diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index bf8575105..89e2066d7 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -5,6 +5,7 @@ import com.linkedin.feathr.offline.source.dataloader._ import com.linkedin.feathr.offline.source.dataloader.jdbc.JdbcUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.DataFrame +import com.linkedin.feathr.offline.util.DelimiterUtils.checkDelimiterOption object FileFormat { @@ -29,14 +30,6 @@ object FileFormat { val DATA_FORMAT = "data.format" - /** - * Convert delimiter to an escape character (e.g. " " -> "\t") - */ - def escape(raw: String): String = { - import scala.reflect.runtime.universe.{Literal, Constant} - Literal(Constant(raw)).toString.replaceAll("\"", "") - } - /** * To define if the file is JDBC, Single File or Path list (default) * @param path @@ -60,12 +53,8 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep - val sqlContext = ss.sqlContext - // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep - val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption - val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) format match { case AVRO => new AvroJsonDataLoader(ss, path).loadDataFrame() @@ -102,12 +91,8 @@ object FileFormat { def loadHdfsDataFrame(format: String, existingHdfsPaths: Seq[String]): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep - val sqlContext = ss.sqlContext - // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep - val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption - val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) val df = format match { case CSV => diff --git a/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala index ce9e2cacd..bdf247acf 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/DelimiterUtils.scala @@ -1,4 +1,5 @@ package com.linkedin.feathr.offline.util +import scala.reflect.runtime.universe.{Literal, Constant} object DelimiterUtils { @@ -6,8 +7,14 @@ object DelimiterUtils { * Convert delimiter to an escape character (e.g. " " -> "\t") */ def escape(raw: String): String = { - import scala.reflect.runtime.universe.{Literal, Constant} Literal(Constant(raw)).toString.replaceAll("\"", "") } + /** + * If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption + */ + def checkDelimiterOption(csvDelimiterOption: String): String = { + if (escape(csvDelimiterOption).trim.isEmpty) "," else csvDelimiterOption + } + } diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index 41c124968..9729bc1e9 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -51,6 +51,7 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.reflect.ClassTag import scala.util.Try +import com.linkedin.feathr.offline.util.DelimiterUtils.checkDelimiterOption /** * Load "raw" not-yet-featurized data from HDFS data sets @@ -65,14 +66,6 @@ private[offline] object SourceUtils { val FEATURE_MP_DEF_CONFIG_SUFFIX = ".conf" val firstRecordName = "topLevelRecord" - /** - * Convert delimiter to an escape character (e.g. " " -> "\t") - */ - def escape(raw: String): String = { - import scala.reflect.runtime.universe.{Literal, Constant} - Literal(Constant(raw)).toString.replaceAll("\"", "") - } - /** * get AVRO datum type of a dataset we should use to load, * it is determined by the expect datatype from a set of anchor transformers @@ -674,12 +667,8 @@ private[offline] object SourceUtils { val format = FileFormat.getType(inputData.inputPath) log.info(s"loading ${inputData.inputPath} input Path as Format: ${format}") - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep - val sqlContext = ss.sqlContext - // Get rawCsvDelimiterOption from spark.feathr.inputFormat.csvOptions.sep - val rawCsvDelimiterOption = sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",") - // If rawCsvDelimiterOption is not properly set, defaults to "," as the delimiter else csvDelimiterOption - val csvDelimiterOption = if (escape(rawCsvDelimiterOption).trim.isEmpty) "," else rawCsvDelimiterOption + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) format match { case FileFormat.PATHLIST => { From 0b01cc6db70b096c9cca6ced228c98bc2d4b3672 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Thu, 4 Aug 2022 22:18:02 +0900 Subject: [PATCH 17/19] Updated wiki to support both tsv and csv Signed-off-by: changyonglik --- docs/how-to-guides/feathr-job-configuration.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/how-to-guides/feathr-job-configuration.md b/docs/how-to-guides/feathr-job-configuration.md index 35b95a425..c254ba6e1 100644 --- a/docs/how-to-guides/feathr-job-configuration.md +++ b/docs/how-to-guides/feathr-job-configuration.md @@ -8,8 +8,8 @@ parent: How-to Guides Since Feathr uses Spark as the underlying execution engine, there's a way to override Spark configuration by `FeathrClient.get_offline_features()` with `execution_configurations` parameters. The complete list of the available spark configuration is located in [Spark Configuration](https://spark.apache.org/docs/latest/configuration.html) (though not all of those are honored for cloud hosted Spark platforms such as Databricks), and there are a few Feathr specific ones that are documented here: -| Property Name | Default | Meaning | Since Version | -| ------------------------- | ------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------- | +| Property Name | Default | Meaning | Since Version | +| ------------------------- | ------- |---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| ------------- | | spark.feathr.inputFormat | None | Specify the input format if the file cannot be tell automatically. By default, Feathr will read files by parsing the file extension name; However the file/folder name doesn't have extension name, this configuration can be set to tell Feathr which format it should use to read the data. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to read delta lake. | 0.2.1 | -| spark.feathr.outputFormat | None | Specify the output format. "avro" is the default behavior if this value is not set. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to write delta lake. | 0.2.1 | -| spark.feathr.inputFormat.csvOptions.sep | None | Specify the delimiter. For example, "," for commas or "\t" for tabs. | 0.6.0 | +| spark.feathr.outputFormat | None | Specify the output format. "avro" is the default behavior if this value is not set. Currently can only be set for Spark built-in short names, including `json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`. For more details, see ["Manually Specifying Options"](https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#manually-specifying-options). Additionally, `delta` is also supported if users want to write delta lake. | 0.2.1 | +| spark.feathr.inputFormat.csvOptions.sep | None | Specify the delimiter. For example, "," for commas or "\t" for tabs. (Supports both csv and tsv) | 0.6.0 | From b4e55fb582226a8ce90b29f3352c22e051299fc3 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Thu, 4 Aug 2022 22:28:32 +0900 Subject: [PATCH 18/19] Fixed spelling error Signed-off-by: changyonglik --- .../feathr/offline/source/dataloader/BatchDataLoader.scala | 2 +- .../feathr/offline/source/dataloader/CsvDataLoader.scala | 2 +- .../feathr/offline/source/dataloader/hdfs/FileFormat.scala | 4 ++-- .../scala/com/linkedin/feathr/offline/util/SourceUtils.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala index 93004a99f..53f5efe82 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/BatchDataLoader.scala @@ -65,7 +65,7 @@ private[offline] class BatchDataLoader(ss: SparkSession, location: InputLocation log.info(s"Loading ${location} as DataFrame, using parameters ${dataIOParametersWithSplitSize}") - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSV) val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) try { diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala index 8ea87262a..c726113a7 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/CsvDataLoader.scala @@ -35,7 +35,7 @@ private[offline] class CsvDataLoader(ss: SparkSession, path: String) extends Dat */ override def loadDataFrame(): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSV) val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) try { diff --git a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala index 89e2066d7..8061e2618 100644 --- a/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala +++ b/src/main/scala/com/linkedin/feathr/offline/source/dataloader/hdfs/FileFormat.scala @@ -53,7 +53,7 @@ object FileFormat { // TODO: Complete a general loadDataFrame and replace current adhoc load data frame code def loadDataFrame(ss: SparkSession, path: String, format: String = CSV): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSV) val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) format match { @@ -91,7 +91,7 @@ object FileFormat { def loadHdfsDataFrame(format: String, existingHdfsPaths: Seq[String]): DataFrame = { - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSV) val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) val df = format match { diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index 9729bc1e9..02af43a40 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -667,7 +667,7 @@ private[offline] object SourceUtils { val format = FileFormat.getType(inputData.inputPath) log.info(s"loading ${inputData.inputPath} input Path as Format: ${format}") - // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSv) + // Get csvDelimiterOption set with spark.feathr.inputFormat.csvOptions.sep and check if it is set properly (Only for CSV and TSV) val csvDelimiterOption = checkDelimiterOption(ss.sqlContext.getConf("spark.feathr.inputFormat.csvOptions.sep", ",")) format match { From 53801a31f3f239d7a87da58a7ea9345034a9d0b2 Mon Sep 17 00:00:00 2001 From: changyonglik Date: Tue, 16 Aug 2022 22:15:47 +0800 Subject: [PATCH 19/19] trigger GitHub actions