From b2e92b4706c5ed3b141805933f29beb87e1b7371 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 11 Feb 2018 21:06:53 +0100 Subject: [PATCH 01/95] Test for reading json in UTF-16 with BOM --- .../test/resources/json-tests/utf16WithBOM.json | Bin 0 -> 170 bytes .../execution/datasources/json/JsonSuite.scala | 11 +++++++++++ .../execution/datasources/json/TestJsonData.scala | 4 ++++ 3 files changed, 15 insertions(+) create mode 100644 sql/core/src/test/resources/json-tests/utf16WithBOM.json diff --git a/sql/core/src/test/resources/json-tests/utf16WithBOM.json b/sql/core/src/test/resources/json-tests/utf16WithBOM.json new file mode 100644 index 0000000000000000000000000000000000000000..65e7e2f72948103b198017d2553f501efe850da0 GIT binary patch literal 170 zcmezWubM%LA&nuEp@^ZFp@hMYA(0`MAr&ZQ1;ow_89?z&po|iO4ub-a%mK<{s&fL0 if=oyOs;Fh)W#D4KXQ~TBK0_%(Isr3-fU5G5OauU86dZN{ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 8c8d41ebf115..49541b4951c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2063,4 +2063,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } } + + test("json in UTF-16 with BOM") { + val schema = new StructType().add("firstName", StringType).add("lastName", StringType) + val jsonDF = spark.read.schema(schema) + .option("mode", "DROPMALFORMED") + .json(testFile("json-tests/utf16WithBOM.json")) + + checkAnswer(jsonDF, Seq( + Row("Chris", "Baird"), Row("Doug", "Rood") + )) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index 13084ba4a7f0..0db33cca3c63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -22,6 +22,10 @@ import org.apache.spark.sql.{Dataset, Encoders, SparkSession} private[json] trait TestJsonData { protected def spark: SparkSession + def testFile(fileName: String): String = { + Thread.currentThread().getContextClassLoader.getResource(fileName).toString + } + def primitiveFieldAndType: Dataset[String] = spark.createDataset(spark.sparkContext.parallelize( """{"string":"this is a simple string.", From cb2f27ba73cb5838e2910c31ca204100bb4eebca Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 11 Feb 2018 21:48:35 +0100 Subject: [PATCH 02/95] Use user's charset or autodetect it if the charset is not specified --- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 9 +++++++-- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 3 +++ .../sql/execution/datasources/json/JsonDataSource.scala | 4 +++- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 025a388aacaa..532f49fb9656 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -39,8 +39,13 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def text(jsonFactory: JsonFactory, record: Text): JsonParser = { - jsonFactory.createParser(record.getBytes, 0, record.getLength) + def text(charset: Option[String])(jsonFactory: JsonFactory, record: Text): JsonParser = { + charset.map {cs => + val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) + jsonFactory.createParser(new InputStreamReader(bain, cs)) + }.getOrElse { + jsonFactory.createParser(record.getBytes, 0, record.getLength) + } } def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 652412b34478..99873cb1d8e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -85,6 +85,9 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) + /** Standard charset name. For example UTF-8, UTF-16 and UTF-32 */ + val charset = parameters.get("charset") + /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 77e7edc8e7a2..049f953b5f2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -122,8 +122,10 @@ object TextInputJsonDataSource extends JsonDataSource { schema: StructType): Iterator[InternalRow] = { val linesReader = new HadoopFileLinesReader(file, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) + val charset = parser.options.charset + val safeParser = new FailureSafeParser[Text]( - input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), + input => parser.parse(input, CreateJacksonParser.text(charset), textToUTF8String), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) From 0d45fd382bb90ebd7161d57a3da23820b4497f67 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 10:48:08 +0100 Subject: [PATCH 03/95] Added a type and a comment for charset --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 99873cb1d8e3..073f4c6e2fcb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -85,8 +85,9 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) - /** Standard charset name. For example UTF-8, UTF-16 and UTF-32 */ - val charset = parameters.get("charset") + /** Standard charset name. For example UTF-8, UTF-16 and UTF-32. + * If charset is not specified (None), it will be detected automatically. */ + val charset: Option[String] = parameters.get("charset") /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { From 1fb9b321a4fac0f41cfb9dd5f85b61feb6796227 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 11:00:27 +0100 Subject: [PATCH 04/95] Replacing the monadic chaining by matching because it is more readable --- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 532f49fb9656..acb1df005c02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -40,11 +40,12 @@ private[sql] object CreateJacksonParser extends Serializable { } def text(charset: Option[String])(jsonFactory: JsonFactory, record: Text): JsonParser = { - charset.map {cs => - val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) - jsonFactory.createParser(new InputStreamReader(bain, cs)) - }.getOrElse { - jsonFactory.createParser(record.getBytes, 0, record.getLength) + charset match { + case Some(cs) => + val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) + jsonFactory.createParser(new InputStreamReader(bain, cs)) + case _ => + jsonFactory.createParser(record.getBytes, 0, record.getLength) } } From c3b04ee68338ad4f93a5361a41db28b37f020907 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 11:44:19 +0100 Subject: [PATCH 05/95] Keeping the old method for backward compatibility --- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 6 +++++- .../sql/execution/datasources/json/JsonDataSource.scala | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index acb1df005c02..1a529bd8611b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -39,7 +39,7 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def text(charset: Option[String])(jsonFactory: JsonFactory, record: Text): JsonParser = { + def textInCharset(charset: Option[String])(jsonFactory: JsonFactory, record: Text): JsonParser = { charset match { case Some(cs) => val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) @@ -49,6 +49,10 @@ private[sql] object CreateJacksonParser extends Serializable { } } + def text(jsonFactory: JsonFactory, record: Text): JsonParser = { + textInCharset(None)(jsonFactory, record) + } + def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { jsonFactory.createParser(record) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 049f953b5f2e..aa2c594bdfd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -125,7 +125,7 @@ object TextInputJsonDataSource extends JsonDataSource { val charset = parser.options.charset val safeParser = new FailureSafeParser[Text]( - input => parser.parse(input, CreateJacksonParser.text(charset), textToUTF8String), + input => parser.parse(input, CreateJacksonParser.textInCharset(charset), textToUTF8String), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) From 93d38794dd261ee1bbe2497470ee43de1186ef3c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 11:54:52 +0100 Subject: [PATCH 06/95] testFile is moved into the test to make more local because it is used only in the test --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 6 +++++- .../spark/sql/execution/datasources/json/TestJsonData.scala | 4 ---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 49541b4951c5..0b5dc7addf6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2065,10 +2065,14 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("json in UTF-16 with BOM") { + val testFile = { + val fileName = "json-tests/utf16WithBOM.json" + Thread.currentThread().getContextClassLoader.getResource(fileName).toString + } val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("mode", "DROPMALFORMED") - .json(testFile("json-tests/utf16WithBOM.json")) + .json(testFile) checkAnswer(jsonDF, Seq( Row("Chris", "Baird"), Row("Doug", "Rood") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index 0db33cca3c63..13084ba4a7f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -22,10 +22,6 @@ import org.apache.spark.sql.{Dataset, Encoders, SparkSession} private[json] trait TestJsonData { protected def spark: SparkSession - def testFile(fileName: String): String = { - Thread.currentThread().getContextClassLoader.getResource(fileName).toString - } - def primitiveFieldAndType: Dataset[String] = spark.createDataset(spark.sparkContext.parallelize( """{"string":"this is a simple string.", From 15798a1ce61df29e9a32f960e755495e3d63f4e3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 12:15:25 +0100 Subject: [PATCH 07/95] Adding the charset as third parameter to the text method --- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 8 ++------ .../sql/execution/datasources/json/JsonDataSource.scala | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 1a529bd8611b..bb9592f14272 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -39,7 +39,7 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def textInCharset(charset: Option[String])(jsonFactory: JsonFactory, record: Text): JsonParser = { + def text(jsonFactory: JsonFactory, record: Text, charset: Option[String] = None): JsonParser = { charset match { case Some(cs) => val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) @@ -48,11 +48,7 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(record.getBytes, 0, record.getLength) } } - - def text(jsonFactory: JsonFactory, record: Text): JsonParser = { - textInCharset(None)(jsonFactory, record) - } - + def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { jsonFactory.createParser(record) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index aa2c594bdfd0..657916fc185c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -125,7 +125,7 @@ object TextInputJsonDataSource extends JsonDataSource { val charset = parser.options.charset val safeParser = new FailureSafeParser[Text]( - input => parser.parse(input, CreateJacksonParser.textInCharset(charset), textToUTF8String), + input => parser.parse[Text](input, CreateJacksonParser.text(_, _, charset), textToUTF8String), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) From cc05ce9af7c9f1d14bd10c1f46a60ce043c13fe1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 12:29:57 +0100 Subject: [PATCH 08/95] Removing whitespaces at the end of the line --- .../apache/spark/sql/catalyst/json/CreateJacksonParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index bb9592f14272..cf2e3f85dc4f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -48,7 +48,7 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(record.getBytes, 0, record.getLength) } } - + def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { jsonFactory.createParser(record) } From 74f2026e62389902ab7a4c418aa96a492fa14f6f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 13:29:28 +0100 Subject: [PATCH 09/95] Fix the comment in javadoc style --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 073f4c6e2fcb..c261778421c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -85,8 +85,10 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) - /** Standard charset name. For example UTF-8, UTF-16 and UTF-32. - * If charset is not specified (None), it will be detected automatically. */ + /** + * Standard charset name. For example UTF-8, UTF-16 and UTF-32. + * If charset is not specified (None), it will be detected automatically. + */ val charset: Option[String] = parameters.get("charset") /** Sets config options on a Jackson [[JsonFactory]]. */ From 4856b8e0b287b3ba3331865298f0603dde18459c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Feb 2018 13:32:48 +0100 Subject: [PATCH 10/95] Simplifying of the UTF-16 test --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 0b5dc7addf6d..c84ebcd3c9af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2065,10 +2065,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("json in UTF-16 with BOM") { - val testFile = { - val fileName = "json-tests/utf16WithBOM.json" - Thread.currentThread().getContextClassLoader.getResource(fileName).toString - } + val fileName = "json-tests/utf16WithBOM.json" + val testFile = Thread.currentThread().getContextClassLoader.getResource(fileName).toString val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("mode", "DROPMALFORMED") From 084f41fb6edd7c86aeb8643973119cb4b38a34fa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 15 Feb 2018 18:33:25 +0100 Subject: [PATCH 11/95] A hint to the exception how to set the charset explicitly --- .../apache/spark/sql/catalyst/json/JacksonParser.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 7f6956994f31..4f14e17ae618 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.json -import java.io.ByteArrayOutputStream +import java.io.{ByteArrayOutputStream, CharConversionException} import scala.collection.mutable.ArrayBuffer import scala.util.Try @@ -361,6 +361,14 @@ class JacksonParser( // For such records, all fields other than the field configured by // `columnNameOfCorruptRecord` are set to `null`. throw BadRecordException(() => recordLiteral(record), () => None, e) + case e: CharConversionException if options.charset.isEmpty => + val msg = e.getMessage + + """ + |Charset was detected automatically. You might want to set it explicitly via the charset option: + | .option("charset", "UTF-8") + |Example of supported charsets: UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE and etc. + """.stripMargin + throw new CharConversionException(msg) } } } From 31cd793a86e6a0e48e0150ffb8c36da2872c65ca Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 15 Feb 2018 19:00:55 +0100 Subject: [PATCH 12/95] Fix for scala style checks --- .../apache/spark/sql/catalyst/json/JacksonParser.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 4f14e17ae618..5ca823115e12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -364,10 +364,10 @@ class JacksonParser( case e: CharConversionException if options.charset.isEmpty => val msg = e.getMessage + """ - |Charset was detected automatically. You might want to set it explicitly via the charset option: - | .option("charset", "UTF-8") - |Example of supported charsets: UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE and etc. - """.stripMargin + |Charset was detected automatically. You might want to set it explicitly + |via the charset option like: .option("charset", "UTF-8") + |Example of supported charsets: """.stripMargin + + "UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" throw new CharConversionException(msg) } } From 6eacd186a954a3f724ee607826b17f432ead77e1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 15 Feb 2018 19:44:04 +0100 Subject: [PATCH 13/95] Run tests again --- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 5ca823115e12..b46b6f27c27f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -366,8 +366,8 @@ class JacksonParser( """ |Charset was detected automatically. You might want to set it explicitly |via the charset option like: .option("charset", "UTF-8") - |Example of supported charsets: """.stripMargin + - "UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" + |Example of supported charsets:""".stripMargin + + " UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" throw new CharConversionException(msg) } } From 3b4a509d0260cfab720a5471ccd937de55c56093 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 15 Feb 2018 20:06:06 +0100 Subject: [PATCH 14/95] Improving of the exception message --- .../spark/sql/catalyst/json/JacksonParser.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index b46b6f27c27f..5fe975000c90 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -362,12 +362,12 @@ class JacksonParser( // `columnNameOfCorruptRecord` are set to `null`. throw BadRecordException(() => recordLiteral(record), () => None, e) case e: CharConversionException if options.charset.isEmpty => - val msg = e.getMessage + - """ - |Charset was detected automatically. You might want to set it explicitly - |via the charset option like: .option("charset", "UTF-8") - |Example of supported charsets:""".stripMargin + - " UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" + val msg = + """Failed to parse a character. Charset was detected automatically. + |You might want to set it explicitly via the charset option like: + | .option("charset", "UTF-8") + |Example of supported charsets: """.stripMargin + + "UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" throw new CharConversionException(msg) } } From cd1124ef7e6329f4dcd6926064271cd24b5a150d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 15 Feb 2018 20:41:35 +0100 Subject: [PATCH 15/95] Appended the original message to the exception --- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 5fe975000c90..8ff165a1032d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -366,8 +366,9 @@ class JacksonParser( """Failed to parse a character. Charset was detected automatically. |You might want to set it explicitly via the charset option like: | .option("charset", "UTF-8") - |Example of supported charsets: """.stripMargin + - "UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE" + |Example of supported charsets: + | UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE + |""".stripMargin + e.getMessage throw new CharConversionException(msg) } } From ebf53904151582eef6d95780ca30b773404ae141 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 17 Feb 2018 21:36:28 +0100 Subject: [PATCH 16/95] Multi-line reading of json file in utf-32 --- .../resources/json-tests/utf32beWithBOM.json | Bin 0 -> 388 bytes .../datasources/json/JsonSuite.scala | 19 ++++++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/resources/json-tests/utf32beWithBOM.json diff --git a/sql/core/src/test/resources/json-tests/utf32beWithBOM.json b/sql/core/src/test/resources/json-tests/utf32beWithBOM.json new file mode 100644 index 0000000000000000000000000000000000000000..82e3ba8abcaa8b1e436db3d7b216abe7d9ecc26f GIT binary patch literal 388 zcmbu(u?mAg5QO2aeG0Mm2~ryC6t*FSpi#jfBwC5DZt@STa99N`KK2gw=EiLOn%Nx@ zVmC7rJkjBe4@TTj;)NR8yZF2^TWCMbQ- Date: Sat, 17 Feb 2018 22:43:23 +0100 Subject: [PATCH 17/95] Autodetect charset of jsons in the multiline mode --- .../catalyst/json/CreateJacksonParser.scala | 12 +++++-- .../datasources/json/JsonDataSource.scala | 34 ++++++++++-------- .../resources/json-tests/utf32beWithBOM.json | Bin 388 -> 172 bytes .../datasources/json/JsonSuite.scala | 4 +-- 4 files changed, 31 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index cf2e3f85dc4f..16a9513d6239 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -49,7 +49,15 @@ private[sql] object CreateJacksonParser extends Serializable { } } - def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { - jsonFactory.createParser(record) + def inputStream(jsonFactory: JsonFactory, + is: InputStream, + charset: Option[String] + ): JsonParser = { + charset match { + case Some(cs) => + jsonFactory.createParser(new InputStreamReader(is, cs)) + case _ => + jsonFactory.createParser(is) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 657916fc185c..6d8a25840661 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -148,6 +148,15 @@ object MultiLineJsonDataSource extends JsonDataSource { parsedOptions: JSONOptions): StructType = { val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths) val sampled: RDD[PortableDataStream] = JsonUtils.sample(json, parsedOptions) + def createParser(jsonFactory: JsonFactory, record: PortableDataStream): JsonParser = { + val path = new Path(record.getPath()) + CreateJacksonParser.inputStream( + jsonFactory, + CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path), + parsedOptions.charset + ) + } + JsonInferSchema.infer(sampled, parsedOptions, createParser) } @@ -170,33 +179,30 @@ object MultiLineJsonDataSource extends JsonDataSource { .values } - private def createParser(jsonFactory: JsonFactory, record: PortableDataStream): JsonParser = { - val path = new Path(record.getPath()) - CreateJacksonParser.inputStream( - jsonFactory, - CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path)) - } - override def readFile( conf: Configuration, file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { + def inputStream = { + CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))) + } def partitionedFileString(ignored: Any): UTF8String = { - Utils.tryWithResource { - CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))) - } { inputStream => - UTF8String.fromBytes(ByteStreams.toByteArray(inputStream)) + Utils.tryWithResource(inputStream) { is => + UTF8String.fromBytes(ByteStreams.toByteArray(is)) } } + val charset = parser.options.charset val safeParser = new FailureSafeParser[InputStream]( - input => parser.parse(input, CreateJacksonParser.inputStream, partitionedFileString), + input => parser.parse[InputStream](input, + CreateJacksonParser.inputStream(_, _, charset), + partitionedFileString + ), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) - safeParser.parse( - CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath)))) + safeParser.parse(inputStream) } } diff --git a/sql/core/src/test/resources/json-tests/utf32beWithBOM.json b/sql/core/src/test/resources/json-tests/utf32beWithBOM.json index 82e3ba8abcaa8b1e436db3d7b216abe7d9ecc26f..6c7733c577872845ed506ecb97ec70d68300198d 100644 GIT binary patch delta 17 YcmZo+Uc<=3!0_+?L=pFi=?RQn05X{cg#Z8m delta 117 zcmZ3(*upHr!0_)s0|P@e5OV>s0tioJapy#EJd$~JP-U$pU62KU?0g_D1>*FH0rs*% TPyvu&3J@y+aV-#ojEn^Uc4Q55 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f4e9f0a71add..76f8ce1fc36b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2087,8 +2087,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .option("multiline", "true") .json(testFile(fileName)) - checkAnswer(jsonDF, Seq( - Row("Chris", "Baird"), Row("Doug", "Rood") - )) + checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } } From ef5e6c6ec607239864375053a6e921acd3deae96 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 17 Feb 2018 22:54:21 +0100 Subject: [PATCH 18/95] Test for reading a json in UTF-16LE in the multiline mode by using user's charset --- .../src/test/resources/json-tests/utf16LE.json | Bin 0 -> 98 bytes .../sql/execution/datasources/json/JsonSuite.scala | 11 +++++++++++ 2 files changed, 11 insertions(+) create mode 100644 sql/core/src/test/resources/json-tests/utf16LE.json diff --git a/sql/core/src/test/resources/json-tests/utf16LE.json b/sql/core/src/test/resources/json-tests/utf16LE.json new file mode 100644 index 0000000000000000000000000000000000000000..ce4117fd299dfcbc7089e7c0530098bfcaf5a27e GIT binary patch literal 98 zcmbi20w;GhFpeJpqLd9J2PYe#WR62N(?$cl`!==KvkHk Roq(bsb5ek+xfp7J7y!-s4k`cu literal 0 HcmV?d00001 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 76f8ce1fc36b..a1ea9033d7a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2089,4 +2089,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } + + test("Use user's charset in reading of multi-line json in UTF-16LE") { + val fileName = "json-tests/utf16LE.json" + val schema = new StructType().add("firstName", StringType).add("lastName", StringType) + val jsonDF = spark.read.schema(schema) + .option("multiline", "true") + .option("charset", "UTF-16LE") + .json(testFile(fileName)) + + checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) + } } From f9b6ad141c7a1b9668fe0a2e4bdf6bdbdc54b98e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Feb 2018 04:08:37 +0100 Subject: [PATCH 19/95] Fix test: rename the test file - utf32be -> utf32BE --- .../{utf32beWithBOM.json => utf32BEWithBOM.json} | Bin 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/core/src/test/resources/json-tests/{utf32beWithBOM.json => utf32BEWithBOM.json} (100%) diff --git a/sql/core/src/test/resources/json-tests/utf32beWithBOM.json b/sql/core/src/test/resources/json-tests/utf32BEWithBOM.json similarity index 100% rename from sql/core/src/test/resources/json-tests/utf32beWithBOM.json rename to sql/core/src/test/resources/json-tests/utf32BEWithBOM.json From 3b7714c8bbe31475b4797e4303ded6c59634921a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Feb 2018 22:03:47 +0100 Subject: [PATCH 20/95] Fix code style --- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 16a9513d6239..969f2d84c94e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -49,10 +49,10 @@ private[sql] object CreateJacksonParser extends Serializable { } } - def inputStream(jsonFactory: JsonFactory, - is: InputStream, - charset: Option[String] - ): JsonParser = { + def inputStream( + jsonFactory: JsonFactory, + is: InputStream, + charset: Option[String]): JsonParser = { charset match { case Some(cs) => jsonFactory.createParser(new InputStreamReader(is, cs)) From edb9167903c9e7667f6a536f139561ed3aadb6e6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Feb 2018 22:09:28 +0100 Subject: [PATCH 21/95] Appending the create verb to the method for readability --- .../sql/execution/datasources/json/JsonDataSource.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 6d8a25840661..ec39141557d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -184,11 +184,11 @@ object MultiLineJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - def inputStream = { + def createInputStream() = { CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))) } def partitionedFileString(ignored: Any): UTF8String = { - Utils.tryWithResource(inputStream) { is => + Utils.tryWithResource(createInputStream()) { is => UTF8String.fromBytes(ByteStreams.toByteArray(is)) } } @@ -203,6 +203,6 @@ object MultiLineJsonDataSource extends JsonDataSource { schema, parser.options.columnNameOfCorruptRecord) - safeParser.parse(inputStream) + safeParser.parse(createInputStream()) } } From 5ba2881c252f40f7c736232cd01c1421ba4b811c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Feb 2018 22:20:04 +0100 Subject: [PATCH 22/95] Making the createParser as a separate private method --- .../catalyst/json/CreateJacksonParser.scala | 2 +- .../datasources/json/JsonDataSource.scala | 26 ++++++++++++------- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 969f2d84c94e..df393906557f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -52,7 +52,7 @@ private[sql] object CreateJacksonParser extends Serializable { def inputStream( jsonFactory: JsonFactory, is: InputStream, - charset: Option[String]): JsonParser = { + charset: Option[String] = None): JsonParser = { charset match { case Some(cs) => jsonFactory.createParser(new InputStreamReader(is, cs)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index ec39141557d9..912ac64c55dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -148,16 +148,12 @@ object MultiLineJsonDataSource extends JsonDataSource { parsedOptions: JSONOptions): StructType = { val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths) val sampled: RDD[PortableDataStream] = JsonUtils.sample(json, parsedOptions) - def createParser(jsonFactory: JsonFactory, record: PortableDataStream): JsonParser = { - val path = new Path(record.getPath()) - CreateJacksonParser.inputStream( - jsonFactory, - CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path), - parsedOptions.charset - ) - } - JsonInferSchema.infer(sampled, parsedOptions, createParser) + JsonInferSchema.infer[PortableDataStream]( + sampled, + parsedOptions, + createParser(_, _, parsedOptions.charset) + ) } private def createBaseRdd( @@ -179,6 +175,18 @@ object MultiLineJsonDataSource extends JsonDataSource { .values } + private def createParser( + jsonFactory: JsonFactory, + record: PortableDataStream, + charset: Option[String] = None): JsonParser = { + val path = new Path(record.getPath()) + CreateJacksonParser.inputStream( + jsonFactory, + CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path), + charset + ) + } + override def readFile( conf: Configuration, file: PartitionedFile, From 1509e103f8b86393b5442d516ee283a16b7fa7e7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Feb 2018 22:27:01 +0100 Subject: [PATCH 23/95] Fix code style --- .../spark/sql/execution/datasources/json/JsonDataSource.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 912ac64c55dc..913b15c09b09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -203,7 +203,8 @@ object MultiLineJsonDataSource extends JsonDataSource { val charset = parser.options.charset val safeParser = new FailureSafeParser[InputStream]( - input => parser.parse[InputStream](input, + input => parser.parse[InputStream]( + input, CreateJacksonParser.inputStream(_, _, charset), partitionedFileString ), From e3184b35e504ce46b82ee18babd3395b7d1fc34d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 19 Feb 2018 22:17:43 +0100 Subject: [PATCH 24/95] Checks the charset option is supported --- python/pyspark/sql/tests.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 480815d27333..95ca9cd66529 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -654,6 +654,12 @@ def test_multiLine_json(self): multiLine=True) self.assertEqual(people1.collect(), people_array.collect()) + def test_charset_json(self): + people1 = self.spark.read.option("charset", "UTF-8").json("python/test_support/sql/people.json") + people_array = self.spark.read.json("python/test_support/sql/people_array.json", + multiLine=True, charset="UTF-8") + self.assertEqual(people1.collect(), people_array.collect()) + def test_multiline_csv(self): ages_newlines = self.spark.read.csv( "python/test_support/sql/ages_newlines.csv", multiLine=True) From 87d259c7d190716a89016c85b7450d471b3481bf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 19 Feb 2018 22:19:02 +0100 Subject: [PATCH 25/95] Support charset as a parameter of the json method --- python/pyspark/sql/readwriter.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index facc16bc5310..2add53d482ce 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -176,7 +176,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, - multiLine=None, allowUnquotedControlChars=None): + multiLine=None, allowUnquotedControlChars=None, charset=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -237,6 +237,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. + :param charset: standard charset name, for example UTF-8, UTF-16 and UTF-32 If None is + set, the charset of input json will be detected automatically. >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes From 76c1d08af25f8f4717314d6ba1409476d63b2ffd Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 19 Feb 2018 23:16:31 +0100 Subject: [PATCH 26/95] Test for charset different from utf-8 --- python/pyspark/sql/tests.py | 9 +++++---- python/test_support/sql/people_array_utf16le.json | Bin 0 -> 182 bytes 2 files changed, 5 insertions(+), 4 deletions(-) create mode 100644 python/test_support/sql/people_array_utf16le.json diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 95ca9cd66529..fc019f2d1ebe 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -655,10 +655,11 @@ def test_multiLine_json(self): self.assertEqual(people1.collect(), people_array.collect()) def test_charset_json(self): - people1 = self.spark.read.option("charset", "UTF-8").json("python/test_support/sql/people.json") - people_array = self.spark.read.json("python/test_support/sql/people_array.json", - multiLine=True, charset="UTF-8") - self.assertEqual(people1.collect(), people_array.collect()) + people_array = self.spark.read\ + .json("python/test_support/sql/people_array_utf16le.json", + multiLine=True, charset="UTF-16LE") + expected = [Row(age=30, name=u'Andy'), Row(age=19, name=u'Justin')] + self.assertEqual(people_array.collect(), expected) def test_multiline_csv(self): ages_newlines = self.spark.read.csv( diff --git a/python/test_support/sql/people_array_utf16le.json b/python/test_support/sql/people_array_utf16le.json new file mode 100644 index 0000000000000000000000000000000000000000..9c657fa30ac9c651076ff8aa3676baa400b121fb GIT binary patch literal 182 zcma!M;9^h!!fGfDVk Date: Tue, 20 Feb 2018 13:39:34 +0100 Subject: [PATCH 27/95] Description of the charset option of the json method --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 0139913aaa4e..cd271bbdf618 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -366,6 +366,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * `java.text.SimpleDateFormat`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • + *
  • `charset` (by default it is not set): allows to forcibly set one of standard basic + * or extended charsets for input jsons. For example UTF-8, UTF-16BE, UTF-32. If the charset + * is not specified (by default), the charset is detected automatically.
  • * * * @since 2.0.0 From f2f8ae72e024f39efaed8f93da11a7ebb0ef6870 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 21 Feb 2018 13:48:41 +0100 Subject: [PATCH 28/95] Minor changes in comments: added . at the end of a sentence --- python/pyspark/sql/readwriter.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 2add53d482ce..5a45708f16fd 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -237,7 +237,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. - :param charset: standard charset name, for example UTF-8, UTF-16 and UTF-32 If None is + :param charset: standard charset name, for example UTF-8, UTF-16 and UTF-32. If None is set, the charset of input json will be detected automatically. >>> df1 = spark.read.json('python/test_support/sql/people.json') From b451a03f900aa76365e44fe10419cd8345feae09 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 21 Feb 2018 20:47:58 +0100 Subject: [PATCH 29/95] Added a test for wrong charset name --- .../sql/execution/datasources/json/JsonSuite.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index a1ea9033d7a3..0e6624f39ebb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2100,4 +2100,18 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } + + test("Unsupported charset name") { + val invalidCharset = "UTF-128" + val exception = intercept[SparkException] { + spark.read + .option("charset", invalidCharset) + .json(testFile("json-tests/utf16LE.json")) + .count() + } + val causedBy = exception.getCause + + assert(causedBy.isInstanceOf[java.io.UnsupportedEncodingException]) + assert(causedBy.getMessage.contains(invalidCharset)) + } } From c13c15946b077800d6d68fb77f0f4692cc9f3a17 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 21 Feb 2018 21:05:03 +0100 Subject: [PATCH 30/95] Testing that charset in any case is acceptable --- .../sql/execution/datasources/json/JsonSuite.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 0e6624f39ebb..f674ab352e98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2114,4 +2114,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(causedBy.isInstanceOf[java.io.UnsupportedEncodingException]) assert(causedBy.getMessage.contains(invalidCharset)) } + + test("checking that the charset option is case agnostic") { + val fileName = "json-tests/utf16LE.json" + val schema = new StructType().add("firstName", StringType).add("lastName", StringType) + val jsonDF = spark.read.schema(schema) + .option("multiline", "true") + .option("charset", "uTf-16lE") + .json(testFile(fileName)) + + checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) + } } From 1cb3ac055bbdc6da720564a105b71f2eea3f5b55 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 21 Feb 2018 21:07:19 +0100 Subject: [PATCH 31/95] Test: user specified wrong (but supported) charset --- .../execution/datasources/json/JsonSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f674ab352e98..17ba95a09458 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2125,4 +2125,21 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } + + + test("specified charset is not matched to actual charset") { + val fileName = "json-tests/utf16LE.json" + val schema = new StructType().add("firstName", StringType).add("lastName", StringType) + val exception = intercept[SparkException] { + spark.read.schema(schema) + .option("mode", "FAILFAST") + .option("multiline", "true") + .option("charset", "UTF-16BE") + .json(testFile(fileName)) + .count() + } + val errMsg = exception.getMessage + + assert(errMsg.contains("Malformed records are detected in record parsing")) + } } From 108e8e783d1575ea3dc406d372215002a871b02a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 25 Feb 2018 21:09:16 +0100 Subject: [PATCH 32/95] Set charset as an option --- python/pyspark/sql/readwriter.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 5a45708f16fd..64f5507730a8 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -256,7 +256,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, - allowUnquotedControlChars=allowUnquotedControlChars) + allowUnquotedControlChars=allowUnquotedControlChars, charset=charset) if isinstance(path, basestring): path = [path] if type(path) == list: From 0d20cc699552e185b472202268407bd77aab3169 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Feb 2018 13:52:55 +0100 Subject: [PATCH 33/95] Test: saving to json in UTF-32BE --- .../execution/datasources/json/JsonSuite.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 17ba95a09458..57e75c675acf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2142,4 +2142,22 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(errMsg.contains("Malformed records are detected in record parsing")) } + + test("save json in UTF-32BE") { + val charset = "UTF-32BE" + withTempPath { path => + val df = spark.createDataset(Seq(("Dog", 42))) + df.write + .option("charset", charset) + .format("json").mode("overwrite") + .save(path.getCanonicalPath) + val jsonFiles = new File(path.getCanonicalPath).listFiles() + .filter(_.isFile).filter(_.getName.endsWith("json")) + val written = jsonFiles.map { file => + scala.io.Source.fromFile(file, charset).mkString + }.mkString.trim.replaceAll(" ", "") + + assert(written == """{"_1":"Dog","_2":42}""") + } + } } From 54baf9fb0fe34fe3c5f2457a80149c7f0b3d2211 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Feb 2018 14:16:58 +0100 Subject: [PATCH 34/95] Taking user's charset for saved json --- .../execution/datasources/json/JsonFileFormat.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 0862c746fffa..8d422dd95bff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.json +import java.nio.charset.{Charset, StandardCharsets} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} @@ -151,7 +153,16 @@ private[json] class JsonOutputWriter( context: TaskAttemptContext) extends OutputWriter with Logging { - private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path)) + private val charset = options.charset match { + case Some(charsetName) => Charset.forName(charsetName) + case _ => StandardCharsets.UTF_8 + } + + private val writer = CodecStreams.createOutputStreamWriter( + context, + new Path(path), + charset + ) // create the Generator without separator inserted between 2 records private[this] val gen = new JacksonGenerator(dataSchema, writer, options) From 1d50d945587832d1b29cfbd8b92d989728df2ef8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Feb 2018 14:39:51 +0100 Subject: [PATCH 35/95] Test: output charset is UTF-8 by default --- .../datasources/json/JsonSuite.scala | 51 +++++++++++++++++-- 1 file changed, 46 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 57e75c675acf..0f8261d0cd8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2143,6 +2143,22 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(errMsg.contains("Malformed records are detected in record parsing")) } + def readJsonFiles(path: String, charset: String): String = { + val jsonFiles = new File(path) + .listFiles() + .filter(_.isFile) + .filter(_.getName.endsWith("json")) + val content = jsonFiles.map { file => + scala.io.Source.fromFile(file, charset).mkString + } + val result = content + .mkString + .trim + .replaceAll(" ", "") + + result + } + test("save json in UTF-32BE") { val charset = "UTF-32BE" withTempPath { path => @@ -2151,13 +2167,38 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .option("charset", charset) .format("json").mode("overwrite") .save(path.getCanonicalPath) - val jsonFiles = new File(path.getCanonicalPath).listFiles() - .filter(_.isFile).filter(_.getName.endsWith("json")) - val written = jsonFiles.map { file => - scala.io.Source.fromFile(file, charset).mkString - }.mkString.trim.replaceAll(" ", "") + val written = readJsonFiles(path.getCanonicalPath, charset) + + assert(written == """{"_1":"Dog","_2":42}""") + } + } + + test("save json in default charset - UTF-8") { + withTempPath { path => + val df = spark.createDataset(Seq(("Dog", 42))) + df.write + .format("json").mode("overwrite") + .save(path.getCanonicalPath) + val written = readJsonFiles(path.getCanonicalPath, "UTF-8") assert(written == """{"_1":"Dog","_2":42}""") } } + + test("wrong output charset") { + val charset = "UTF-128" + val exception = intercept[SparkException] { + withTempPath { path => + val df = spark.createDataset(Seq((0))) + df.write + .option("charset", charset) + .format("json").mode("overwrite") + .save(path.getCanonicalPath) + } + } + val causedBy = exception.getCause.getCause.getCause + + assert(causedBy.isInstanceOf[java.nio.charset.UnsupportedCharsetException]) + assert(causedBy.getMessage == charset) + } } From bb537981d7fb2fb6fe359ebe70ffc0bc4f483a03 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 4 Mar 2018 15:05:35 +0100 Subject: [PATCH 36/95] Changing the readJsonFiles method for readability --- .../datasources/json/JsonSuite.scala | 30 ++++++++++++------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 0f8261d0cd8d..8ad8914e31c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2143,20 +2143,24 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(errMsg.contains("Malformed records are detected in record parsing")) } - def readJsonFiles(path: String, charset: String): String = { - val jsonFiles = new File(path) + def checkCharset( + expectedCharset: String, + pathToJsonFiles: String, + expectedContent: String + ): Unit = { + val jsonFiles = new File(pathToJsonFiles) .listFiles() .filter(_.isFile) .filter(_.getName.endsWith("json")) - val content = jsonFiles.map { file => - scala.io.Source.fromFile(file, charset).mkString + val jsonContent = jsonFiles.map { file => + scala.io.Source.fromFile(file, expectedCharset).mkString } - val result = content + val cleanedContent = jsonContent .mkString .trim .replaceAll(" ", "") - result + assert(cleanedContent == expectedContent) } test("save json in UTF-32BE") { @@ -2167,9 +2171,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .option("charset", charset) .format("json").mode("overwrite") .save(path.getCanonicalPath) - val written = readJsonFiles(path.getCanonicalPath, charset) - assert(written == """{"_1":"Dog","_2":42}""") + checkCharset( + expectedCharset = charset, + pathToJsonFiles = path.getCanonicalPath, + expectedContent = """{"_1":"Dog","_2":42}""" + ) } } @@ -2179,9 +2186,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { df.write .format("json").mode("overwrite") .save(path.getCanonicalPath) - val written = readJsonFiles(path.getCanonicalPath, "UTF-8") - assert(written == """{"_1":"Dog","_2":42}""") + checkCharset( + expectedCharset = "UTF-8", + pathToJsonFiles = path.getCanonicalPath, + expectedContent = """{"_1":"Dog","_2":42}""" + ) } } From 961b48225b450f9053681405f594911896e3a7ff Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 4 Mar 2018 16:08:45 +0100 Subject: [PATCH 37/95] The test checks that json written by Spark can be read back --- .../datasources/json/JsonSuite.scala | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 8ad8914e31c3..0b18a6948035 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2072,6 +2072,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val fileName = "json-tests/utf16WithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) + // The mode filters null rows produced because new line delimiter + // for UTF-8 is used by default. .option("mode", "DROPMALFORMED") .json(testFile(fileName)) @@ -2211,4 +2213,28 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(causedBy.isInstanceOf[java.nio.charset.UnsupportedCharsetException]) assert(causedBy.getMessage == charset) } + + test("read written json in UTF-16") { + val charset = "UTF-16" + case class Rec(f1: String, f2: Int) + withTempPath { path => + val ds = spark.createDataset(Seq( + ("a", 1), ("b", 2), ("c", 3)) + ).repartition(2) + ds.write + .option("charset", charset) + .format("json").mode("overwrite") + .save(path.getCanonicalPath) + val savedDf = spark + .read + .schema(ds.schema) + .option("charset", charset) + // Wrong (nulls) rows are produced because new line delimiter + // for UTF-8 is used by default. + .option("mode", "DROPMALFORMED") + .json(path.getCanonicalPath) + + checkAnswer(savedDf.toDF(), ds.toDF()) + } + } } From a794988407b6fd28364f5d993a6a52ac0b85ec5f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 24 Feb 2018 21:11:00 +0100 Subject: [PATCH 38/95] Adding the delimiter option encoded in base64 --- .../apache/spark/sql/catalyst/json/JSONOptions.scala | 11 +++++++++++ .../execution/datasources/HadoopFileLinesReader.scala | 10 ++++++++-- .../execution/datasources/json/JsonDataSource.scala | 2 +- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 652412b34478..08b41bbfa4c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.json import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} +import org.apache.commons.codec.binary.Base64 import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging @@ -85,6 +86,16 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) + /** + * Standard charset name. For example UTF-8, UTF-16 and UTF-32. + * If charset is not specified (None), it will be detected automatically. + */ + val charset: Option[String] = parameters.get("charset") + + val delimiter: Option[Array[Byte]] = { + parameters.get("delimiter").map(Base64.decodeBase64(_)) + } + /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 83cf26c63a17..6835aaba2761 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -32,7 +32,10 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl * in that file. */ class HadoopFileLinesReader( - file: PartitionedFile, conf: Configuration) extends Iterator[Text] with Closeable { + file: PartitionedFile, + conf: Configuration, + recordDelimiter: Option[Array[Byte]] = None + ) extends Iterator[Text] with Closeable { private val iterator = { val fileSplit = new FileSplit( new Path(new URI(file.filePath)), @@ -42,7 +45,10 @@ class HadoopFileLinesReader( Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - val reader = new LineRecordReader() + val reader = recordDelimiter match { + case Some(delimiter) => new LineRecordReader(delimiter) + case _ => new LineRecordReader() + } reader.initialize(fileSplit, hadoopAttemptContext) new RecordReaderIterator(reader) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 77e7edc8e7a2..c8efaf089e23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -120,7 +120,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf) + val linesReader = new HadoopFileLinesReader(file, conf, parser.options.delimiter) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), From dccdaa2e97cb4e2f6f8ea7e03320cdb05a43668c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 24 Feb 2018 21:59:46 +0100 Subject: [PATCH 39/95] Separator encoded as a sequence of bytes in hex --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 9 ++++++--- .../sql/execution/datasources/json/JsonDataSource.scala | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 08b41bbfa4c2..977482dfa5c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -92,9 +92,12 @@ private[sql] class JSONOptions( */ val charset: Option[String] = parameters.get("charset") - val delimiter: Option[Array[Byte]] = { - parameters.get("delimiter").map(Base64.decodeBase64(_)) - } + // A separator of json records + val sep: Option[Array[Byte]] = parameters.get("sep").map( + _.replaceAll("[^0-9A-Fa-f]", "") + .sliding(2, 2) + .toArray.map(Integer.parseInt(_, 16).toByte) + ) /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index c8efaf089e23..ff838972266a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -120,7 +120,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf, parser.options.delimiter) + val linesReader = new HadoopFileLinesReader(file, conf,parser.options.sep) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), From d0abab7e4b74dd42e06972f9484bc712b8f11c63 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 24 Feb 2018 22:06:08 +0100 Subject: [PATCH 40/95] Refactoring: removed unused imports and renaming a parameter --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 1 - .../sql/execution/datasources/HadoopFileLinesReader.scala | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 977482dfa5c9..37ed598d9ec7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.json import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} -import org.apache.commons.codec.binary.Base64 import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 6835aaba2761..9253388e5e8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl class HadoopFileLinesReader( file: PartitionedFile, conf: Configuration, - recordDelimiter: Option[Array[Byte]] = None + lineSeparator: Option[Array[Byte]] = None ) extends Iterator[Text] with Closeable { private val iterator = { val fileSplit = new FileSplit( @@ -45,8 +45,8 @@ class HadoopFileLinesReader( Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - val reader = recordDelimiter match { - case Some(delimiter) => new LineRecordReader(delimiter) + val reader = lineSeparator match { + case Some(sep) => new LineRecordReader(sep) case _ => new LineRecordReader() } reader.initialize(fileSplit, hadoopAttemptContext) From 674179601b4c82e315eb1156df0f3f5035e91154 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 4 Mar 2018 18:24:42 +0100 Subject: [PATCH 41/95] The sep option is renamed to recordSeparator. The supported format is sequence of bytes in hex like x0d 0a --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 7 +++++-- .../sql/execution/datasources/json/JsonDataSource.scala | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 37ed598d9ec7..d59c5c1e9253 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -91,8 +91,11 @@ private[sql] class JSONOptions( */ val charset: Option[String] = parameters.get("charset") - // A separator of json records - val sep: Option[Array[Byte]] = parameters.get("sep").map( + /** + * A sequence of bytes between two consecutive json records. Supported formats: + * - sequence of bytes in hex format (starts from x): x0a 0d + */ + val recordSeparator: Option[Array[Byte]] = parameters.get("recordSeparator").map( _.replaceAll("[^0-9A-Fa-f]", "") .sliding(2, 2) .toArray.map(Integer.parseInt(_, 16).toByte) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index ff838972266a..e003af4dcfac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -120,7 +120,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf,parser.options.sep) + val linesReader = new HadoopFileLinesReader(file, conf, parser.options.recordSeparator) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), From e4faae155cb5b0761da9ac72a12f67cdde6b2e6b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 13:40:21 +0100 Subject: [PATCH 42/95] Renaming recordSeparator to recordDelimiter --- .../scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala | 2 +- .../spark/sql/execution/datasources/json/JsonDataSource.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index d59c5c1e9253..f9f6c6dccbd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -95,7 +95,7 @@ private[sql] class JSONOptions( * A sequence of bytes between two consecutive json records. Supported formats: * - sequence of bytes in hex format (starts from x): x0a 0d */ - val recordSeparator: Option[Array[Byte]] = parameters.get("recordSeparator").map( + val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").map( _.replaceAll("[^0-9A-Fa-f]", "") .sliding(2, 2) .toArray.map(Integer.parseInt(_, 16).toByte) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index e003af4dcfac..4a6041df7dc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -120,7 +120,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf, parser.options.recordSeparator) + val linesReader = new HadoopFileLinesReader(file, conf, parser.options.recordDelimiter) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), From 01f4ef584a2cc1ce460359f260ebbe22808d034e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 14:17:59 +0100 Subject: [PATCH 43/95] Comments for the recordDelimiter option --- .../spark/sql/catalyst/json/JSONOptions.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index f9f6c6dccbd5..5c4dcecdfc15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -92,14 +92,20 @@ private[sql] class JSONOptions( val charset: Option[String] = parameters.get("charset") /** - * A sequence of bytes between two consecutive json records. Supported formats: - * - sequence of bytes in hex format (starts from x): x0a 0d + * A sequence of bytes between two consecutive json records. Format of the option is: + * selector (1 char) + delimiter body (any length) + * The following selectors are supported: + * - 'x' + sequence of bytes in hexadecimal format. For example: "x0a 0d". + * Hex pairs can be separated by any chars different from 0-9,A-F,a-f + * - '\' - reserved for a sequence of control chars like "\r\n" + * - '/' - reserved for a sequence of visible chars like "/===" */ - val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").map( - _.replaceAll("[^0-9A-Fa-f]", "") - .sliding(2, 2) - .toArray.map(Integer.parseInt(_, 16).toByte) - ) + val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").collect { + case hexs if hexs.startsWith("x") => + hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray + .map(Integer.parseInt(_, 16).toByte) + case d => throw new NotImplementedError(d) + } /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { From 24cedb9d809b026fa36b01fb2b425918b43857df Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 15:36:31 +0100 Subject: [PATCH 44/95] Support other formats of recordDelimiter --- .../spark/sql/catalyst/json/JSONOptions.scala | 5 ++-- .../datasources/json/JsonSuite.scala | 27 +++++++++++++++++++ 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 5c4dcecdfc15..770225ec0acf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -98,13 +98,14 @@ private[sql] class JSONOptions( * - 'x' + sequence of bytes in hexadecimal format. For example: "x0a 0d". * Hex pairs can be separated by any chars different from 0-9,A-F,a-f * - '\' - reserved for a sequence of control chars like "\r\n" - * - '/' - reserved for a sequence of visible chars like "/===" + * and unicode escape like "\u000D\u000A" */ val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").collect { case hexs if hexs.startsWith("x") => hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray .map(Integer.parseInt(_, 16).toByte) - case d => throw new NotImplementedError(d) + case delim => delim.getBytes(charset.getOrElse( + throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } /** Sets config options on a Jackson [[JsonFactory]]. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 8c8d41ebf115..911a68e392e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2063,4 +2063,31 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } } + + def readWrittenJson(delimiter: (String, Int)): Unit = { + val (recordDelimiter, index) = delimiter + test(s"read written json in UTF-16BE with delimiter $index") { + val charset = "UTF-16BE" + case class Rec(f1: String, f2: Int) + withTempPath { path => + val ds = spark.createDataset(Seq( + ("a", 1), ("b", 2), ("c", 3)) + ).repartition(1) + ds.write + .option("charset", charset) + .format("json").mode("overwrite") + .save(path.getCanonicalPath) + val savedDf = spark + .read + .schema(ds.schema) + .option("charset", charset) + .option("recordDelimiter", recordDelimiter) + .json(path.getCanonicalPath) + + checkAnswer(savedDf.toDF(), ds.toDF()) + } + } + } + + List("x00 0a", "\n", "\u000a").zipWithIndex.foreach(readWrittenJson(_)) } From d40dda22587deaf79cfad3b20ccf6854554fc11d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 17:30:26 +0100 Subject: [PATCH 45/95] Checking different charsets and record delimiters --- .../execution/datasources/json/JsonSuite.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 911a68e392e4..4019026f5ac3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2064,10 +2064,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - def readWrittenJson(delimiter: (String, Int)): Unit = { - val (recordDelimiter, index) = delimiter - test(s"read written json in UTF-16BE with delimiter $index") { - val charset = "UTF-16BE" + def readSparkJson(charset: String, delimiter: String, runId: Int): Unit = { + test(s"checks Spark is able to read json written by Spark itself #{$runId}") { case class Rec(f1: String, f2: Int) withTempPath { path => val ds = spark.createDataset(Seq( @@ -2081,7 +2079,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .read .schema(ds.schema) .option("charset", charset) - .option("recordDelimiter", recordDelimiter) + .option("recordDelimiter", delimiter) .json(path.getCanonicalPath) checkAnswer(savedDf.toDF(), ds.toDF()) @@ -2089,5 +2087,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - List("x00 0a", "\n", "\u000a").zipWithIndex.foreach(readWrittenJson(_)) + List( + ("\n", "UTF-8"), + ("x00 0a", "UTF-16BE"), + ("\n", "UTF-16LE"), + ("\u000a", "UTF-32BE"), + ("x0a 00 00 00", "UTF-32LE") + ).zipWithIndex.foreach{case ((d, c), i) => readSparkJson(c, d, i)} } From ad6496c6d9415bcd49630272b5d6c327ffcb1378 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 17:39:07 +0100 Subject: [PATCH 46/95] Renaming test's method to make it more readable --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 4019026f5ac3..19bbda496b75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2063,8 +2063,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } } - - def readSparkJson(charset: String, delimiter: String, runId: Int): Unit = { + + def checkReadWrittenJson(charset: String, delimiter: String, runId: Int): Unit = { test(s"checks Spark is able to read json written by Spark itself #{$runId}") { case class Rec(f1: String, f2: Int) withTempPath { path => @@ -2093,5 +2093,5 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("\n", "UTF-16LE"), ("\u000a", "UTF-32BE"), ("x0a 00 00 00", "UTF-32LE") - ).zipWithIndex.foreach{case ((d, c), i) => readSparkJson(c, d, i)} + ).zipWithIndex.foreach{case ((d, c), i) => checkReadWrittenJson(c, d, i)} } From 358863d91bf0c0d9761aa13698eb7f8532e5fc90 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 18:20:38 +0100 Subject: [PATCH 47/95] Test of reading json in different charsets and delimiters --- .../datasources/json/JsonSuite.scala | 47 +++++++++++++++++-- 1 file changed, 44 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 19bbda496b75..9031ad10cb75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.json -import java.io.{File, StringWriter} +import java.io.{File, FileOutputStream, StringWriter} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util.Locale @@ -2065,8 +2065,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } def checkReadWrittenJson(charset: String, delimiter: String, runId: Int): Unit = { - test(s"checks Spark is able to read json written by Spark itself #{$runId}") { - case class Rec(f1: String, f2: Int) + test(s"checks Spark is able to read json written by Spark itself #${runId}") { withTempPath { path => val ds = spark.createDataset(Seq( ("a", 1), ("b", 2), ("c", 3)) @@ -2094,4 +2093,46 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("\u000a", "UTF-32BE"), ("x0a 00 00 00", "UTF-32LE") ).zipWithIndex.foreach{case ((d, c), i) => checkReadWrittenJson(c, d, i)} + + def checkReadJson(charset: String, delimiter: String, runId: Int): Unit = { + test(s"checks reading json in ${charset} #${runId}") { + val delimInBytes = { + if (delimiter.startsWith("x")) { + delimiter.replaceAll("[^0-9A-Fa-f]", "") + .sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) + } else { + delimiter.getBytes(charset) + } + } + case class Rec(f1: String, f2: Int) { + def json = s"""{"f1":"${f1}", "f2":$f2}""" + def bytes = json.getBytes(charset) + def row = Row(f1, f2) + } + val schema = new StructType().add("f1", StringType).add("f2", IntegerType) + withTempPath { path => + val records = List(Rec("a", 1), Rec("b", 2)) + val data = records.map(_.bytes).reduce((a1, a2) => a1 ++ delimInBytes ++ a2) + val os = new FileOutputStream(path) + os.write(data) + os.close() + val savedDf = spark + .read + .schema(schema) + .option("charset", charset) + .option("recordDelimiter", delimiter) + .json(path.getCanonicalPath) + checkAnswer(savedDf, records.map(_.row)) + } + } + } + + List( + ("sep", "UTF-8"), + ("x00 0a 00 0d", "UTF-16BE"), + ("\r\n", "UTF-16LE"), + ("\u000d\u000a", "UTF-32BE"), + ("===", "UTF-32LE"), + ("куку", "CP1251") + ).zipWithIndex.foreach{case ((d, c), i) => checkReadJson(c, d, i)} } From 7e5be5e2b4cf7f77914a0d91e74ea31ab8c272d0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 21:25:47 +0100 Subject: [PATCH 48/95] Fix inferring of csv schema for any charsets --- .../catalyst/json/CreateJacksonParser.scala | 10 ++++++ .../spark/sql/catalyst/json/JSONOptions.scala | 10 ++++++ .../datasources/HadoopFileLinesReader.scala | 6 ++-- .../datasources/json/JsonDataSource.scala | 19 +++++++--- .../datasources/text/TextFileFormat.scala | 12 +++++-- .../datasources/text/TextOptions.scala | 4 +++ .../datasources/json/JsonSuite.scala | 35 +++++++++++++------ 7 files changed, 74 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 025a388aacaa..f6295dac8aa8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -22,6 +22,7 @@ import java.io.{ByteArrayInputStream, InputStream, InputStreamReader} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} import org.apache.hadoop.io.Text +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.unsafe.types.UTF8String private[sql] object CreateJacksonParser extends Serializable { @@ -46,4 +47,13 @@ private[sql] object CreateJacksonParser extends Serializable { def inputStream(jsonFactory: JsonFactory, record: InputStream): JsonParser = { jsonFactory.createParser(record) } + + def internalRow( + jsonFactory: JsonFactory, + row: InternalRow, + charset: Option[String] = None + ): JsonParser = { + val is = new ByteArrayInputStream(row.getBinary(0)) + inputStream(jsonFactory, is, charset) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 770225ec0acf..e6e56871b7c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -99,6 +99,10 @@ private[sql] class JSONOptions( * Hex pairs can be separated by any chars different from 0-9,A-F,a-f * - '\' - reserved for a sequence of control chars like "\r\n" * and unicode escape like "\u000D\u000A" + * + * Note: the option defines a delimiter for the json reader only, the json writer + * uses '\n' as the delimiter of output records (it is converted to sequence of + * bytes according to charset) */ val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").collect { case hexs if hexs.startsWith("x") => @@ -119,4 +123,10 @@ private[sql] class JSONOptions( allowBackslashEscapingAnyCharacter) factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars) } + + def getTextOptions: Map[String, String] = { + recordDelimiter.map{ bytes => + "recordDelimiter" -> bytes.map("%02x".format(_)).mkString + }.toMap + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 9253388e5e8f..0e5a7462770b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl class HadoopFileLinesReader( file: PartitionedFile, conf: Configuration, - lineSeparator: Option[Array[Byte]] = None + recordDelimiter: Option[Array[Byte]] = None ) extends Iterator[Text] with Closeable { private val iterator = { val fileSplit = new FileSplit( @@ -45,8 +45,8 @@ class HadoopFileLinesReader( Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - val reader = lineSeparator match { - case Some(sep) => new LineRecordReader(sep) + val reader = recordDelimiter match { + case Some(delim) => new LineRecordReader(delim) case _ => new LineRecordReader() } reader.initialize(fileSplit, hadoopAttemptContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 4a6041df7dc1..f9321cbc5c21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -33,6 +33,7 @@ import org.apache.spark.input.{PortableDataStream, StreamInputFormat} import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{AnalysisException, Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat @@ -92,25 +93,33 @@ object TextInputJsonDataSource extends JsonDataSource { sparkSession: SparkSession, inputPaths: Seq[FileStatus], parsedOptions: JSONOptions): StructType = { - val json: Dataset[String] = createBaseDataset(sparkSession, inputPaths) + val json: Dataset[String] = createBaseDataset(sparkSession, inputPaths, parsedOptions) inferFromDataset(json, parsedOptions) } def inferFromDataset(json: Dataset[String], parsedOptions: JSONOptions): StructType = { val sampled: Dataset[String] = JsonUtils.sample(json, parsedOptions) - val rdd: RDD[UTF8String] = sampled.queryExecution.toRdd.map(_.getUTF8String(0)) - JsonInferSchema.infer(rdd, parsedOptions, CreateJacksonParser.utf8String) + val rdd: RDD[InternalRow] = sampled.queryExecution.toRdd + + JsonInferSchema.infer[InternalRow]( + rdd, + parsedOptions, + CreateJacksonParser.internalRow(_, _, parsedOptions.charset) + ) } private def createBaseDataset( sparkSession: SparkSession, - inputPaths: Seq[FileStatus]): Dataset[String] = { + inputPaths: Seq[FileStatus], + parsedOptions: JSONOptions + ): Dataset[String] = { val paths = inputPaths.map(_.getPath.toString) sparkSession.baseRelationToDataFrame( DataSource.apply( sparkSession, paths = paths, - className = classOf[TextFileFormat].getName + className = classOf[TextFileFormat].getName, + options = parsedOptions.getTextOptions ).resolveRelation(checkFilesExist = false)) .select("value").as(Encoders.STRING) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index c661e9bd3b94..c7af107e296d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -113,18 +113,24 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - readToUnsafeMem(broadcastedHadoopConf, requiredSchema, textOptions.wholeText) + readToUnsafeMem( + broadcastedHadoopConf, + requiredSchema, + textOptions.wholeText, + textOptions.recordDelimiter + ) } private def readToUnsafeMem( conf: Broadcast[SerializableConfiguration], requiredSchema: StructType, - wholeTextMode: Boolean): (PartitionedFile) => Iterator[UnsafeRow] = { + wholeTextMode: Boolean, + recordDelimiter: Option[Array[Byte]]): (PartitionedFile) => Iterator[UnsafeRow] = { (file: PartitionedFile) => { val confValue = conf.value.value val reader = if (!wholeTextMode) { - new HadoopFileLinesReader(file, confValue) + new HadoopFileLinesReader(file, confValue, recordDelimiter) } else { new HadoopFileWholeTextReader(file, confValue) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index 2a661561ab51..eea8e04ad171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -39,9 +39,13 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti */ val wholeText = parameters.getOrElse(WHOLETEXT, "false").toBoolean + val recordDelimiter: Option[Array[Byte]] = parameters.get(RECORDDELIMITER).map { hex => + hex.sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) + } } private[text] object TextOptions { val COMPRESSION = "compression" val WHOLETEXT = "wholetext" + val RECORDDELIMITER = "recordDelimiter" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 9031ad10cb75..2d681d1be0aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2094,7 +2094,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("x0a 00 00 00", "UTF-32LE") ).zipWithIndex.foreach{case ((d, c), i) => checkReadWrittenJson(c, d, i)} - def checkReadJson(charset: String, delimiter: String, runId: Int): Unit = { + def checkReadJson( + charset: String, + delimiter: String, + inferSchema: Boolean, + runId: Int + ): Unit = { test(s"checks reading json in ${charset} #${runId}") { val delimInBytes = { if (delimiter.startsWith("x")) { @@ -2116,9 +2121,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val os = new FileOutputStream(path) os.write(data) os.close() - val savedDf = spark - .read - .schema(schema) + val reader = if (inferSchema) { + spark.read + } else { + spark.read.schema(schema) + } + val savedDf = reader .option("charset", charset) .option("recordDelimiter", delimiter) .json(path.getCanonicalPath) @@ -2128,11 +2136,16 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } List( - ("sep", "UTF-8"), - ("x00 0a 00 0d", "UTF-16BE"), - ("\r\n", "UTF-16LE"), - ("\u000d\u000a", "UTF-32BE"), - ("===", "UTF-32LE"), - ("куку", "CP1251") - ).zipWithIndex.foreach{case ((d, c), i) => checkReadJson(c, d, i)} + ("sep", "UTF-8", false), + ("x00 0a 00 0d", "UTF-16BE", false), + ("x00 0a 00 0d", "UTF-16BE", true), + ("\r\n", "UTF-16LE", false), + ("\r\n", "UTF-16LE", true), + ("\u000d\u000a", "UTF-32BE", false), + ("\u000a\u000d", "UTF-32BE", true), + ("===", "UTF-32LE", false), + ("$^+", "UTF-32LE", true), + ("куку", "CP1251", false), + ("куку", "CP1251", true) + ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} } From d138d2d4e7b6e0c3e46d73939ff06a875128d59d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 18 Mar 2018 22:02:44 +0100 Subject: [PATCH 49/95] Fix errors of scalastyle check --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 2d681d1be0aa..4d708cf52248 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2145,7 +2145,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("\u000a\u000d", "UTF-32BE", true), ("===", "UTF-32LE", false), ("$^+", "UTF-32LE", true), - ("куку", "CP1251", false), - ("куку", "CP1251", true) + ("xEA.F3.EA.F3", "CP1251", false), + ("xEA.F3.EA.F3", "CP1251", true) ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} } From c26ef5d3d2a3970c80c973eec696805929bd7725 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 12:20:34 +0100 Subject: [PATCH 50/95] Reserving format for regular expressions and concatenated json --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index e6e56871b7c9..b4deab202026 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -99,6 +99,8 @@ private[sql] class JSONOptions( * Hex pairs can be separated by any chars different from 0-9,A-F,a-f * - '\' - reserved for a sequence of control chars like "\r\n" * and unicode escape like "\u000D\u000A" + * - 'r' - specifies a regular expression + * - 'none' - json records are not divided by any delimiter * * Note: the option defines a delimiter for the json reader only, the json writer * uses '\n' as the delimiter of output records (it is converted to sequence of @@ -108,6 +110,8 @@ private[sql] class JSONOptions( case hexs if hexs.startsWith("x") => hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray .map(Integer.parseInt(_, 16).toByte) + case reserved if reserved.startsWith("r") || reserved.startsWith("none") => + throw new NotImplementedError(s"the $reserved selector has not supported yet") case delim => delim.getBytes(charset.getOrElse( throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } From 5f0b0694f142bd69127c8991d83a24f528316b2b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 21:18:21 +0100 Subject: [PATCH 51/95] Fix recordDelimiter tests --- .../catalyst/json/CreateJacksonParser.scala | 4 +- .../spark/sql/catalyst/json/JSONOptions.scala | 6 +-- .../datasources/json/JsonSuite.scala | 49 +++---------------- 3 files changed, 12 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index f6295dac8aa8..a27b73a5e827 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -53,7 +53,9 @@ private[sql] object CreateJacksonParser extends Serializable { row: InternalRow, charset: Option[String] = None ): JsonParser = { + require(charset == Some("UTF-8")) val is = new ByteArrayInputStream(row.getBinary(0)) - inputStream(jsonFactory, is, charset) + + inputStream(jsonFactory, is) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index b4deab202026..e0e78453d972 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -85,11 +85,7 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) - /** - * Standard charset name. For example UTF-8, UTF-16 and UTF-32. - * If charset is not specified (None), it will be detected automatically. - */ - val charset: Option[String] = parameters.get("charset") + val charset: Option[String] = Some("UTF-8") /** * A sequence of bytes between two consecutive json records. Format of the option is: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 4d708cf52248..e0322c0da818 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2063,36 +2063,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } } - - def checkReadWrittenJson(charset: String, delimiter: String, runId: Int): Unit = { - test(s"checks Spark is able to read json written by Spark itself #${runId}") { - withTempPath { path => - val ds = spark.createDataset(Seq( - ("a", 1), ("b", 2), ("c", 3)) - ).repartition(1) - ds.write - .option("charset", charset) - .format("json").mode("overwrite") - .save(path.getCanonicalPath) - val savedDf = spark - .read - .schema(ds.schema) - .option("charset", charset) - .option("recordDelimiter", delimiter) - .json(path.getCanonicalPath) - - checkAnswer(savedDf.toDF(), ds.toDF()) - } - } - } - - List( - ("\n", "UTF-8"), - ("x00 0a", "UTF-16BE"), - ("\n", "UTF-16LE"), - ("\u000a", "UTF-32BE"), - ("x0a 00 00 00", "UTF-32LE") - ).zipWithIndex.foreach{case ((d, c), i) => checkReadWrittenJson(c, d, i)} def checkReadJson( charset: String, @@ -2127,7 +2097,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader - .option("charset", charset) .option("recordDelimiter", delimiter) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) @@ -2137,15 +2106,13 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { List( ("sep", "UTF-8", false), - ("x00 0a 00 0d", "UTF-16BE", false), - ("x00 0a 00 0d", "UTF-16BE", true), - ("\r\n", "UTF-16LE", false), - ("\r\n", "UTF-16LE", true), - ("\u000d\u000a", "UTF-32BE", false), - ("\u000a\u000d", "UTF-32BE", true), - ("===", "UTF-32LE", false), - ("$^+", "UTF-32LE", true), - ("xEA.F3.EA.F3", "CP1251", false), - ("xEA.F3.EA.F3", "CP1251", true) + ("x00 0a 00 0d", "UTF-8", true), + ("xEA.F3.EA.F3", "UTF-8", false), + ("\r\n", "UTF-8", false), + ("\r\n", "UTF-8", true), + ("\u000d\u000a", "UTF-8", false), + ("\u000a\u000d", "UTF-8", true), + ("===", "UTF-8", false), + ("$^+", "UTF-8", true) ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} } From ef8248f862949becdb3d370ac94a1cfc1f7c3068 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 21:34:56 +0100 Subject: [PATCH 52/95] Additional cases are added to the delimiter test --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e0322c0da818..a2ffc5b60e9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2104,7 +2104,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } + // scalastyle:off nonascii List( + ("|", "UTF-8", false), + ("^", "UTF-8", true), + ("::", "UTF-8", true), + ("!!!@3", "UTF-8", false), + (0x1E.toChar.toString, "UTF-8", true), + ("ì•„", "UTF-8", false), + ("куку", "UTF-8", true), ("sep", "UTF-8", false), ("x00 0a 00 0d", "UTF-8", true), ("xEA.F3.EA.F3", "UTF-8", false), @@ -2115,4 +2123,5 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("===", "UTF-8", false), ("$^+", "UTF-8", true) ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} + // scalastyle:on nonascii } From 2efac082ea4e40b89b4d01274851c0dcdd49eb44 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 22:01:56 +0100 Subject: [PATCH 53/95] Renaming recordDelimiter to lineSeparator --- .../spark/sql/catalyst/json/JSONOptions.scala | 17 ++++++++++------- .../sql/catalyst/json/JacksonGenerator.scala | 7 ++++++- .../datasources/HadoopFileLinesReader.scala | 4 ++-- .../datasources/json/JsonDataSource.scala | 2 +- .../datasources/text/TextFileFormat.scala | 6 +++--- .../datasources/text/TextOptions.scala | 4 ++-- .../execution/datasources/json/JsonSuite.scala | 2 +- 7 files changed, 25 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index e0e78453d972..daf92e1cf353 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -89,28 +89,31 @@ private[sql] class JSONOptions( /** * A sequence of bytes between two consecutive json records. Format of the option is: - * selector (1 char) + delimiter body (any length) + * selector (1 char) + delimiter body (any length) | sequence of chars * The following selectors are supported: * - 'x' + sequence of bytes in hexadecimal format. For example: "x0a 0d". * Hex pairs can be separated by any chars different from 0-9,A-F,a-f * - '\' - reserved for a sequence of control chars like "\r\n" * and unicode escape like "\u000D\u000A" - * - 'r' - specifies a regular expression - * - 'none' - json records are not divided by any delimiter + * - 'r' and '/' - reserved for future use * * Note: the option defines a delimiter for the json reader only, the json writer * uses '\n' as the delimiter of output records (it is converted to sequence of * bytes according to charset) */ - val recordDelimiter: Option[Array[Byte]] = parameters.get("recordDelimiter").collect { + val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").collect { case hexs if hexs.startsWith("x") => hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray .map(Integer.parseInt(_, 16).toByte) - case reserved if reserved.startsWith("r") || reserved.startsWith("none") => + case reserved if reserved.startsWith("r") || reserved.startsWith("/") => throw new NotImplementedError(s"the $reserved selector has not supported yet") case delim => delim.getBytes(charset.getOrElse( throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } + val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator + + // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. + val lineSeparatorInWrite: String = parameters.get("lineSepInWrite").getOrElse("\n") /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { @@ -125,8 +128,8 @@ private[sql] class JSONOptions( } def getTextOptions: Map[String, String] = { - recordDelimiter.map{ bytes => - "recordDelimiter" -> bytes.map("%02x".format(_)).mkString + lineSeparatorInRead.map{ bytes => + "lineSep" -> bytes.map("%02x".format(_)).mkString }.toMap } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index eb06e4f304f0..a94a2fe7881d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -74,6 +74,8 @@ private[sql] class JacksonGenerator( private val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) + private val lineSeparator: String = options.lineSeparatorInWrite + private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => (row: SpecializedGetters, ordinal: Int) => @@ -251,5 +253,8 @@ private[sql] class JacksonGenerator( mapType = dataType.asInstanceOf[MapType])) } - def writeLineEnding(): Unit = gen.writeRaw('\n') + def writeLineEnding(): Unit = { + // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. + gen.writeRaw(lineSeparator) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 0e5a7462770b..e7ad6c32e024 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl class HadoopFileLinesReader( file: PartitionedFile, conf: Configuration, - recordDelimiter: Option[Array[Byte]] = None + lineSeparator: Option[Array[Byte]] = None ) extends Iterator[Text] with Closeable { private val iterator = { val fileSplit = new FileSplit( @@ -45,7 +45,7 @@ class HadoopFileLinesReader( Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - val reader = recordDelimiter match { + val reader = lineSeparator match { case Some(delim) => new LineRecordReader(delim) case _ => new LineRecordReader() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index f9321cbc5c21..b2a4503e50a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -129,7 +129,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf, parser.options.recordDelimiter) + val linesReader = new HadoopFileLinesReader(file, conf, parser.options.lineSeparatorInRead) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index c7af107e296d..e606da213d0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -117,7 +117,7 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { broadcastedHadoopConf, requiredSchema, textOptions.wholeText, - textOptions.recordDelimiter + textOptions.lineSeparator ) } @@ -125,12 +125,12 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { conf: Broadcast[SerializableConfiguration], requiredSchema: StructType, wholeTextMode: Boolean, - recordDelimiter: Option[Array[Byte]]): (PartitionedFile) => Iterator[UnsafeRow] = { + lineSeparator: Option[Array[Byte]]): (PartitionedFile) => Iterator[UnsafeRow] = { (file: PartitionedFile) => { val confValue = conf.value.value val reader = if (!wholeTextMode) { - new HadoopFileLinesReader(file, confValue, recordDelimiter) + new HadoopFileLinesReader(file, confValue, lineSeparator) } else { new HadoopFileWholeTextReader(file, confValue) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index eea8e04ad171..a3cebd7cb62c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -39,7 +39,7 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti */ val wholeText = parameters.getOrElse(WHOLETEXT, "false").toBoolean - val recordDelimiter: Option[Array[Byte]] = parameters.get(RECORDDELIMITER).map { hex => + val lineSeparator: Option[Array[Byte]] = parameters.get(LINESEP).map { hex => hex.sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) } } @@ -47,5 +47,5 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti private[text] object TextOptions { val COMPRESSION = "compression" val WHOLETEXT = "wholetext" - val RECORDDELIMITER = "recordDelimiter" + val LINESEP = "lineSep" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index a2ffc5b60e9a..72e6941fe7af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2097,7 +2097,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader - .option("recordDelimiter", delimiter) + .option("lineSep", delimiter) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) } From b2020fa99584d03e1754a4a1b5991dce4875f448 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 22:38:33 +0100 Subject: [PATCH 54/95] Adding HyukjinKwon changes --- python/pyspark/sql/readwriter.py | 16 ++++--- python/pyspark/sql/streaming.py | 4 +- python/pyspark/sql/tests.py | 17 +++++++ .../apache/spark/sql/DataFrameReader.scala | 3 ++ .../apache/spark/sql/DataFrameWriter.scala | 3 ++ .../sql/streaming/DataStreamReader.scala | 3 ++ .../datasources/json/JsonSuite.scala | 45 +++++++++++++++---- 7 files changed, 75 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index facc16bc5310..ea06e4b71e90 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -176,7 +176,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, - multiLine=None, allowUnquotedControlChars=None): + multiLine=None, allowUnquotedControlChars=None, lineSep=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -237,7 +237,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. - + :param lineSep: defines the line separator that should be used for parsing. If None is + set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes [('age', 'bigint'), ('name', 'string')] @@ -254,7 +255,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, - allowUnquotedControlChars=allowUnquotedControlChars) + allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -744,7 +745,8 @@ def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options) self._jwrite.saveAsTable(name) @since(1.4) - def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None): + def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None, + lineSep=None): """Saves the content of the :class:`DataFrame` in JSON format (`JSON Lines text format or newline-delimited JSON `_) at the specified path. @@ -768,12 +770,14 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm formats follow the formats at ``java.text.SimpleDateFormat``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. - + :param lineSep: defines the line separator that should be used for writing. If None is + set, it uses the default value, ``\\n``. >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) self._set_opts( - compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat) + compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat, + lineSep=lineSep) self._jwrite.json(path) @since(1.4) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index e8966c20a8f4..a0b6180ef14b 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -407,7 +407,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, - multiLine=None, allowUnquotedControlChars=None): + multiLine=None, allowUnquotedControlChars=None, lineSep=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`. @@ -484,7 +484,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, - allowUnquotedControlChars=allowUnquotedControlChars) + allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep) if isinstance(path, basestring): return self._df(self._jreader.json(path)) else: diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 480815d27333..5045cc20d682 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -648,6 +648,23 @@ def test_non_existed_udaf(self): self.assertRaisesRegexp(AnalysisException, "Can not load class non_existed_udaf", lambda: spark.udf.registerJavaUDAF("udaf1", "non_existed_udaf")) + def test_linesep_json(self): + df = self.spark.read.json("python/test_support/sql/people.json", lineSep=",") + expected = [Row(_corrupt_record=None, name=u'Michael'), + Row(_corrupt_record=u' "age":30}\n{"name":"Justin"', name=None), + Row(_corrupt_record=u' "age":19}\n', name=None)] + self.assertEqual(df.collect(), expected) + + tpath = tempfile.mkdtemp() + shutil.rmtree(tpath) + try: + df = self.spark.read.json("python/test_support/sql/people.json") + df.write.json(tpath, lineSep="!!") + readback = self.spark.read.json(tpath, lineSep="!!") + self.assertEqual(readback.collect(), df.collect()) + finally: + shutil.rmtree(tpath) + def test_multiLine_json(self): people1 = self.spark.read.json("python/test_support/sql/people.json") people_array = self.spark.read.json("python/test_support/sql/people_array.json", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 0139913aaa4e..4a0dcc80107b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -367,6 +367,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • * + *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator + * that should be used for parsing.
  • + * * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index ed7a9100cc7f..28ad15a470b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -519,6 +519,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type. * + *
  • `lineSep` (default `\n`): defines the line separator that should + * be used for writing.
  • + * * * @since 1.4.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index c393dcdfdd7e..b23fe3f03a96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -269,6 +269,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • * + *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator + * that should be used for parsing.
  • + * * * @since 2.0.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 72e6941fe7af..7aaa623a0bdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.json import java.io.{File, FileOutputStream, StringWriter} import java.nio.charset.StandardCharsets +import java.nio.file.Files import java.sql.{Date, Timestamp} import java.util.Locale @@ -27,7 +28,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{functions => F, _} import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} @@ -2066,17 +2067,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { def checkReadJson( charset: String, - delimiter: String, + lineSep: String, inferSchema: Boolean, runId: Int ): Unit = { test(s"checks reading json in ${charset} #${runId}") { val delimInBytes = { - if (delimiter.startsWith("x")) { - delimiter.replaceAll("[^0-9A-Fa-f]", "") + if (lineSep.startsWith("x")) { + lineSep.replaceAll("[^0-9A-Fa-f]", "") .sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) } else { - delimiter.getBytes(charset) + lineSep.getBytes(charset) } } case class Rec(f1: String, f2: Int) { @@ -2097,7 +2098,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader - .option("lineSep", delimiter) + .option("lineSep", lineSep) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) } @@ -2114,8 +2115,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("ì•„", "UTF-8", false), ("куку", "UTF-8", true), ("sep", "UTF-8", false), - ("x00 0a 00 0d", "UTF-8", true), - ("xEA.F3.EA.F3", "UTF-8", false), + ("x0a 0d", "UTF-8", true), + ("x54.45", "UTF-8", false), ("\r\n", "UTF-8", false), ("\r\n", "UTF-8", true), ("\u000d\u000a", "UTF-8", false), @@ -2124,4 +2125,32 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("$^+", "UTF-8", true) ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} // scalastyle:on nonascii + + def testLineSeparator(lineSep: String): Unit = { + test(s"SPARK-21289: Support line separator - lineSep: '$lineSep'") { + // Write + withTempPath { path => + Seq("a", "b", "c").toDF("value").coalesce(1) + .write.option("lineSepInWrite", lineSep).json(path.getAbsolutePath) + val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) + assert( + readBack === s"""{"value":"a"}$lineSep{"value":"b"}$lineSep{"value":"c"}$lineSep""") + } + + // Roundtrip + withTempPath { path => + val df = Seq("a", "b", "c").toDF() + df.write.option("lineSepInWrite", lineSep).json(path.getAbsolutePath) + val readBack = spark.read.option("lineSep", lineSep).json(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + Seq("|", "^", "::", "!!!@3", 0x1E.toChar.toString, "ì•„").foreach { lineSep => + testLineSeparator(lineSep) + } + // scalastyle:on nonascii } From f99c1e16f2ad90c2a94e8c4b206b5b740506e136 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 22 Mar 2018 23:23:21 +0100 Subject: [PATCH 55/95] Revert lineSepInWrite back to lineSep --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 4 +++- .../spark/sql/execution/datasources/json/JsonSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index daf92e1cf353..a447911067da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -113,7 +113,9 @@ private[sql] class JSONOptions( val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. - val lineSeparatorInWrite: String = parameters.get("lineSepInWrite").getOrElse("\n") + val lineSeparatorInWrite: String = { + lineSeparator.map(new String(_, charset.getOrElse("UTF-8"))).getOrElse("\n") + } /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 7aaa623a0bdf..8ad31231d51c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2131,7 +2131,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Write withTempPath { path => Seq("a", "b", "c").toDF("value").coalesce(1) - .write.option("lineSepInWrite", lineSep).json(path.getAbsolutePath) + .write.option("lineSep", lineSep).json(path.getAbsolutePath) val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) assert( @@ -2141,7 +2141,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Roundtrip withTempPath { path => val df = Seq("a", "b", "c").toDF() - df.write.option("lineSepInWrite", lineSep).json(path.getAbsolutePath) + df.write.option("lineSep", lineSep).json(path.getAbsolutePath) val readBack = spark.read.option("lineSep", lineSep).json(path.getAbsolutePath) checkAnswer(df, readBack) } From 77112ef5b12d4738914c78b46c25d058e6201b61 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Mar 2018 00:07:22 +0100 Subject: [PATCH 56/95] Fix passing of the lineSeparator to HadoopFileLinesReader --- .../spark/sql/execution/datasources/json/JsonDataSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index b2a4503e50a3..c09a1617690b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -129,7 +129,7 @@ object TextInputJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - val linesReader = new HadoopFileLinesReader(file, conf, parser.options.lineSeparatorInRead) + val linesReader = new HadoopFileLinesReader(file, parser.options.lineSeparatorInRead, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) val safeParser = new FailureSafeParser[Text]( input => parser.parse(input, CreateJacksonParser.text, textToUTF8String), From d632706bf14c7a7c2688237e6dc552ca5aa9c98a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Mar 2018 09:39:35 +0100 Subject: [PATCH 57/95] Fix python style checking --- python/pyspark/sql/readwriter.py | 2 ++ python/pyspark/sql/streaming.py | 2 ++ python/pyspark/sql/tests.py | 12 ++++++------ 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index d3e2c7471293..ce428b228338 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -239,6 +239,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, including tab and line feed characters) or not. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. + >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes [('age', 'bigint'), ('name', 'string')] @@ -774,6 +775,7 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param lineSep: defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. + >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 886049964d49..490df4accf87 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -470,6 +470,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. + :param lineSep: defines the line separator that should be used for parsing. If None is + set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 09b6e170f3af..505fc056369f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -670,6 +670,12 @@ def test_linesep_text(self): finally: shutil.rmtree(tpath) + def test_multiline_json(self): + people1 = self.spark.read.json("python/test_support/sql/people.json") + people_array = self.spark.read.json("python/test_support/sql/people_array.json", + multiLine=True) + self.assertEqual(people1.collect(), people_array.collect()) + def test_linesep_json(self): df = self.spark.read.json("python/test_support/sql/people.json", lineSep=",") expected = [Row(_corrupt_record=None, name=u'Michael'), @@ -687,12 +693,6 @@ def test_linesep_json(self): finally: shutil.rmtree(tpath) - def test_multiline_json(self): - people1 = self.spark.read.json("python/test_support/sql/people.json") - people_array = self.spark.read.json("python/test_support/sql/people_array.json", - multiLine=True) - self.assertEqual(people1.collect(), people_array.collect()) - def test_multiline_csv(self): ages_newlines = self.spark.read.csv( "python/test_support/sql/ages_newlines.csv", multiLine=True) From bbff40206e6871ea9ab035e7a8876f495bdf3d90 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 23 Mar 2018 11:58:55 +0100 Subject: [PATCH 58/95] Fix text source tests and javadoc comments --- .../apache/spark/sql/catalyst/json/JSONOptions.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameReader.scala | 1 - .../scala/org/apache/spark/sql/DataFrameWriter.scala | 1 - .../sql/execution/datasources/text/TextOptions.scala | 12 ++++++++++-- .../spark/sql/streaming/DataStreamReader.scala | 1 - 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index a447911067da..c908c7254b0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -131,7 +131,7 @@ private[sql] class JSONOptions( def getTextOptions: Map[String, String] = { lineSeparatorInRead.map{ bytes => - "lineSep" -> bytes.map("%02x".format(_)).mkString + "lineSep" -> bytes.map("x%02x".format(_)).mkString }.toMap } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index af60db0aefbf..ae3ba1690f69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -366,7 +366,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * `java.text.SimpleDateFormat`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • - * *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
  • * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index d15cc90f8e3f..bbc063148a72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -518,7 +518,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • - * *
  • `lineSep` (default `\n`): defines the line separator that should * be used for writing.
  • * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index a1b5d22b65d9..386512c612ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -41,8 +41,16 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti */ val wholeText = parameters.getOrElse(WHOLETEXT, "false").toBoolean - val lineSeparator: Option[Array[Byte]] = parameters.get(LINE_SEPARATOR).map { hex => - hex.sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) + val charset: Option[String] = Some("UTF-8") + + val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").collect { + case hexs if hexs.startsWith("x") => + hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray + .map(Integer.parseInt(_, 16).toByte) + case reserved if reserved.startsWith("r") || reserved.startsWith("/") => + throw new NotImplementedError(s"the $reserved selector has not supported yet") + case delim => delim.getBytes(charset.getOrElse( + throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } // Note that the option 'lineSep' uses a different default value in read and write. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index ffec904f8a41..ae93965bc50e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -268,7 +268,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * `java.text.SimpleDateFormat`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • - * *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
  • * From ab8210c994e900e38767a92f28bbbac447dfbbf7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 27 Mar 2018 21:50:28 +0200 Subject: [PATCH 59/95] Getting UTF-8 as default charset for lineSep --- .../apache/spark/sql/catalyst/json/CreateJacksonParser.scala | 3 +-- .../scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 7f0c22e6e14a..7da6223b1d1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -67,9 +67,8 @@ private[sql] object CreateJacksonParser extends Serializable { row: InternalRow, charset: Option[String] = None ): JsonParser = { - require(charset == Some("UTF-8")) val is = new ByteArrayInputStream(row.getBinary(0)) - inputStream(jsonFactory, is) + inputStream(jsonFactory, is, charset) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 4b326fbf12ec..f065b323f363 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -111,8 +111,7 @@ private[sql] class JSONOptions( .map(Integer.parseInt(_, 16).toByte) case reserved if reserved.startsWith("r") || reserved.startsWith("/") => throw new NotImplementedError(s"the $reserved selector has not supported yet") - case delim => delim.getBytes(charset.getOrElse( - throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) + case delim => delim.getBytes(charset.getOrElse("UTF-8")) } val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator From 7c6f11552d6e62b9a8f9a79fa36739f95c015ffe Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 27 Mar 2018 21:58:51 +0200 Subject: [PATCH 60/95] Set charset different from UTF-8 in the test --- .../datasources/json/JsonSuite.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index d0bbc900bcbc..e6a7a7988ba7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2282,21 +2282,21 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:off nonascii List( ("|", "UTF-8", false), - ("^", "UTF-8", true), - ("::", "UTF-8", true), - ("!!!@3", "UTF-8", false), + ("^", "UTF-16BE", true), + ("::", "UTF-16", true), + ("!!!@3", "UTF-32LE", false), (0x1E.toChar.toString, "UTF-8", true), - ("ì•„", "UTF-8", false), - ("куку", "UTF-8", true), + ("ì•„", "UTF-32BE", false), + ("куку", "CP1251", true), ("sep", "UTF-8", false), - ("x0a 0d", "UTF-8", true), + ("x0a 0d", "UTF-32", true), ("x54.45", "UTF-8", false), - ("\r\n", "UTF-8", false), - ("\r\n", "UTF-8", true), - ("\u000d\u000a", "UTF-8", false), + ("\r\n", "UTF-16LE", false), + ("\r\n", "UTF-16BE", true), + ("\u000d\u000a", "UTF-32BE", false), ("\u000a\u000d", "UTF-8", true), - ("===", "UTF-8", false), - ("$^+", "UTF-8", true) + ("===", "UTF-16", false), + ("$^+", "UTF-32LE", true) ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} // scalastyle:on nonascii From f553b070def71c9ce209db93f16709a127c1b20e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 27 Mar 2018 22:11:46 +0200 Subject: [PATCH 61/95] Fix for the charset test: charset wasn't specified --- .../apache/spark/sql/execution/datasources/json/JsonSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e6a7a7988ba7..b6acaeead650 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2272,6 +2272,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader + .option("charset", charset) .option("lineSep", lineSep) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) From d6a07a101853060e40e89333e7f868bde11d39eb Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 27 Mar 2018 22:36:38 +0200 Subject: [PATCH 62/95] Removing line leaved after merge --- python/pyspark/sql/tests.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 6ebd5be9b5f3..c11e915a54ea 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -699,7 +699,6 @@ def test_linesep_json(self): self.assertEqual(readback.collect(), df.collect()) finally: shutil.rmtree(tpath) ->>>>>>> json-line-sep def test_multiline_csv(self): ages_newlines = self.spark.read.csv( From cb12ea3f167dd6fd3a60baacfe98b49394b465b3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 28 Mar 2018 18:25:20 +0200 Subject: [PATCH 63/95] Removing flexible format for lineSep --- .../spark/sql/catalyst/json/JSONOptions.scala | 34 ++++--------------- .../datasources/text/TextOptions.scala | 9 ++--- .../datasources/json/JsonSuite.scala | 2 -- 3 files changed, 9 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index f065b323f363..3e4936eab1c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -92,33 +92,15 @@ private[sql] class JSONOptions( val charset: Option[String] = parameters.get("charset") /** - * A sequence of bytes between two consecutive json records. Format of the option is: - * selector (1 char) + delimiter body (any length) | sequence of chars - * The following selectors are supported: - * - 'x' + sequence of bytes in hexadecimal format. For example: "x0a 0d". - * Hex pairs can be separated by any chars different from 0-9,A-F,a-f - * - '\' - reserved for a sequence of control chars like "\r\n" - * and unicode escape like "\u000D\u000A" - * - 'r' and '/' - reserved for future use - * - * Note: the option defines a delimiter for the json reader only, the json writer - * uses '\n' as the delimiter of output records (it is converted to sequence of - * bytes according to charset) + * A sequence of bytes between two consecutive json records. */ - val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").collect { - case hexs if hexs.startsWith("x") => - hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray - .map(Integer.parseInt(_, 16).toByte) - case reserved if reserved.startsWith("r") || reserved.startsWith("/") => - throw new NotImplementedError(s"the $reserved selector has not supported yet") - case delim => delim.getBytes(charset.getOrElse("UTF-8")) - } - val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator + val lineSeparator: Option[String] = parameters.get("lineSep") - // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. - val lineSeparatorInWrite: String = { - lineSeparator.map(new String(_, charset.getOrElse("UTF-8"))).getOrElse("\n") + val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => + lineSep.getBytes(charset.getOrElse("UTF-8")) } + // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. + val lineSeparatorInWrite: String = lineSeparator.getOrElse("\n") /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { @@ -133,8 +115,6 @@ private[sql] class JSONOptions( } def getTextOptions: Map[String, String] = { - lineSeparatorInRead.map{ bytes => - "lineSep" -> bytes.map("x%02x".format(_)).mkString - }.toMap + Map[String, String]() ++ charset.map("charset" -> _) ++ lineSeparator.map("lineSep" -> _) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index 386512c612ec..0433347de148 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -43,13 +43,8 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti val charset: Option[String] = Some("UTF-8") - val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").collect { - case hexs if hexs.startsWith("x") => - hexs.replaceAll("[^0-9A-Fa-f]", "").sliding(2, 2).toArray - .map(Integer.parseInt(_, 16).toByte) - case reserved if reserved.startsWith("r") || reserved.startsWith("/") => - throw new NotImplementedError(s"the $reserved selector has not supported yet") - case delim => delim.getBytes(charset.getOrElse( + val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").map { lineSep => + lineSep.getBytes(charset.getOrElse( throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index b6acaeead650..10e50470d970 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2290,8 +2290,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("ì•„", "UTF-32BE", false), ("куку", "CP1251", true), ("sep", "UTF-8", false), - ("x0a 0d", "UTF-32", true), - ("x54.45", "UTF-8", false), ("\r\n", "UTF-16LE", false), ("\r\n", "UTF-16BE", true), ("\u000d\u000a", "UTF-32BE", false), From eb2965b67bd55f9e1b6daa18b35d8c39800f13ba Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 28 Mar 2018 18:31:26 +0200 Subject: [PATCH 64/95] Adding ticket number to test titles --- .../datasources/json/JsonSuite.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 10e50470d970..d2a8c1079d9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2069,7 +2069,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Thread.currentThread().getContextClassLoader.getResource(fileName).toString } - test("json in UTF-16 with BOM") { + test("SPARK-23723: json in UTF-16 with BOM") { val fileName = "json-tests/utf16WithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) @@ -2083,7 +2083,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { )) } - test("multi-line json in UTF-32BE with BOM") { + test("SPARK-23723: multi-line json in UTF-32BE with BOM") { val fileName = "json-tests/utf32BEWithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) @@ -2093,7 +2093,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } - test("Use user's charset in reading of multi-line json in UTF-16LE") { + test("SPARK-23723: Use user's charset in reading of multi-line json in UTF-16LE") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) @@ -2104,7 +2104,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } - test("Unsupported charset name") { + test("SPARK-23723: Unsupported charset name") { val invalidCharset = "UTF-128" val exception = intercept[SparkException] { spark.read @@ -2118,7 +2118,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(causedBy.getMessage.contains(invalidCharset)) } - test("checking that the charset option is case agnostic") { + test("SPARK-23723: checking that the charset option is case agnostic") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) @@ -2130,7 +2130,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } - test("specified charset is not matched to actual charset") { + test("SPARK-23723: specified charset is not matched to actual charset") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val exception = intercept[SparkException] { @@ -2166,7 +2166,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(cleanedContent == expectedContent) } - test("save json in UTF-32BE") { + test("SPARK-23723: save json in UTF-32BE") { val charset = "UTF-32BE" withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) @@ -2183,7 +2183,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("save json in default charset - UTF-8") { + test("SPARK-23723: save json in default charset - UTF-8") { withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) df.write @@ -2198,7 +2198,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("wrong output charset") { + test("SPARK-23723: wrong output charset") { val charset = "UTF-128" val exception = intercept[SparkException] { withTempPath { path => @@ -2215,7 +2215,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(causedBy.getMessage == charset) } - test("read written json in UTF-16") { + test("SPARK-23723: read written json in UTF-16") { val charset = "UTF-16" case class Rec(f1: String, f2: Int) withTempPath { path => @@ -2245,7 +2245,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { inferSchema: Boolean, runId: Int ): Unit = { - test(s"checks reading json in ${charset} #${runId}") { + test(s"SPARK-23724: checks reading json in ${charset} #${runId}") { val delimInBytes = { if (lineSep.startsWith("x")) { lineSep.replaceAll("[^0-9A-Fa-f]", "") @@ -2300,7 +2300,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:on nonascii def testLineSeparator(lineSep: String): Unit = { - test(s"SPARK-21289: Support line separator - lineSep: '$lineSep'") { + test(s"SPARK-23724: Support line separator - lineSep: '$lineSep'") { // Write withTempPath { path => Seq("a", "b", "c").toDF("value").coalesce(1) From 7a4865cb64223ca7161f452b619570275a32aa6e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 28 Mar 2018 22:34:04 +0200 Subject: [PATCH 65/95] Making comments more precise --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index d2a8c1079d9c..f05adbd98ad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2073,8 +2073,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val fileName = "json-tests/utf16WithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) - // The mode filters null rows produced because new line delimiter - // for UTF-8 is used by default. + // This option will be replaced by .option("lineSep", "x00 0a") + // as soon as lineSep allows to specify sequence of bytes in hexadecimal format. .option("mode", "DROPMALFORMED") .json(testFile(fileName)) @@ -2230,8 +2230,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .read .schema(ds.schema) .option("charset", charset) - // Wrong (nulls) rows are produced because new line delimiter - // for UTF-8 is used by default. + // This option will be replaced by .option("lineSep", "x00 0a") + // as soon as lineSep allows to specify sequence of bytes in hexadecimal format. .option("mode", "DROPMALFORMED") .json(path.getCanonicalPath) From dbeb0c1cee5ec5b4cba37f2b04e1efe51cb63917 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 28 Mar 2018 23:36:01 +0200 Subject: [PATCH 66/95] lineSep must be specified if charset is different from UTF-8 --- .../spark/sql/catalyst/json/JSONOptions.scala | 20 +++++++--- .../datasources/json/JsonSuite.scala | 40 +++++++++++-------- 2 files changed, 38 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 3e4936eab1c4..249990c5b891 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -86,20 +86,28 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) /** - * Standard charset name. For example UTF-8, UTF-16 and UTF-32. - * If charset is not specified (None), it will be detected automatically. + * A sequence of bytes between two consecutive json records. */ - val charset: Option[String] = parameters.get("charset") + val lineSeparator: Option[String] = parameters.get("lineSep") /** - * A sequence of bytes between two consecutive json records. + * Standard charset name. For example UTF-8, UTF-16 and UTF-32. + * If charset is not specified (None), it will be detected automatically. */ - val lineSeparator: Option[String] = parameters.get("lineSep") + val charset: Option[String] = parameters.get("charset").map { cs => + if (cs != "UTF-8" && lineSeparator.isEmpty) { + throw new IllegalArgumentException( + s"""Please, set the 'lineSep' option for the given charset $cs. + |Example: .option("lineSep", "|^|") + |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin + ) + } + cs + } val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => lineSep.getBytes(charset.getOrElse("UTF-8")) } - // Note that JSON uses writer with UTF-8 charset. This string will be written out as UTF-8. val lineSeparatorInWrite: String = lineSeparator.getOrElse("\n") /** Sets config options on a Jackson [[JsonFactory]]. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f05adbd98ad3..72bbd3540839 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2098,7 +2098,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") - .option("charset", "UTF-16LE") + .options(Map("charset" -> "UTF-16LE", "lineSep" -> "\n")) .json(testFile(fileName)) checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) @@ -2106,16 +2106,14 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: Unsupported charset name") { val invalidCharset = "UTF-128" - val exception = intercept[SparkException] { + val exception = intercept[java.io.UnsupportedEncodingException] { spark.read - .option("charset", invalidCharset) + .options(Map("charset" -> invalidCharset, "lineSep" -> "\n")) .json(testFile("json-tests/utf16LE.json")) .count() } - val causedBy = exception.getCause - assert(causedBy.isInstanceOf[java.io.UnsupportedEncodingException]) - assert(causedBy.getMessage.contains(invalidCharset)) + assert(exception.getMessage.contains(invalidCharset)) } test("SPARK-23723: checking that the charset option is case agnostic") { @@ -2123,7 +2121,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") - .option("charset", "uTf-16lE") + .options(Map("charset" -> "uTf-16lE", "lineSep" -> "\n")) .json(testFile(fileName)) checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) @@ -2137,7 +2135,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) .option("mode", "FAILFAST") .option("multiline", "true") - .option("charset", "UTF-16BE") + .options(Map("charset" -> "UTF-16BE", "lineSep" -> "\n")) .json(testFile(fileName)) .count() } @@ -2171,7 +2169,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) df.write - .option("charset", charset) + .options(Map("charset" -> charset, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) @@ -2200,19 +2198,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: wrong output charset") { val charset = "UTF-128" - val exception = intercept[SparkException] { + val exception = intercept[java.io.UnsupportedEncodingException] { withTempPath { path => val df = spark.createDataset(Seq((0))) df.write - .option("charset", charset) + .options(Map("charset" -> charset, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) } } - val causedBy = exception.getCause.getCause.getCause - assert(causedBy.isInstanceOf[java.nio.charset.UnsupportedCharsetException]) - assert(causedBy.getMessage == charset) + assert(exception.getMessage == charset) } test("SPARK-23723: read written json in UTF-16") { @@ -2223,13 +2219,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("a", 1), ("b", 2), ("c", 3)) ).repartition(2) ds.write - .option("charset", charset) + .options(Map("charset" -> charset, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) val savedDf = spark .read .schema(ds.schema) - .option("charset", charset) // This option will be replaced by .option("lineSep", "x00 0a") // as soon as lineSep allows to specify sequence of bytes in hexadecimal format. .option("mode", "DROPMALFORMED") @@ -2326,4 +2321,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { testLineSeparator(lineSep) } // scalastyle:on nonascii + + test("SPARK-23724: lineSep should be set if charset if different from UTF-8") { + val exception = intercept[IllegalArgumentException] { + spark.read + .options(Map("charset" -> "UTF-16LE")) + .json(testFile("json-tests/utf16LE.json")) + .count() + } + + assert(exception.getMessage.contains( + """Please, set the 'lineSep' option for the given charset UTF-16LE""" + )) + } } From ac67020c7f83dd17ca4b8b48d174c8e9fe9a37f9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 29 Mar 2018 16:03:18 +0200 Subject: [PATCH 67/95] Support encoding as a synonym for the charset option --- python/pyspark/sql/readwriter.py | 14 +++--- python/pyspark/sql/tests.py | 4 +- .../spark/sql/catalyst/json/JSONOptions.scala | 19 +++---- .../apache/spark/sql/DataFrameReader.scala | 6 +-- .../apache/spark/sql/DataFrameWriter.scala | 4 +- .../datasources/json/JsonSuite.scala | 49 ++++++++++--------- 6 files changed, 51 insertions(+), 45 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 5fac5e691070..5d579fb45a5c 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -176,7 +176,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, - multiLine=None, allowUnquotedControlChars=None, charset=None, lineSep=None): + multiLine=None, allowUnquotedControlChars=None, encoding=None, lineSep=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -237,8 +237,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. - :param charset: standard charset name, for example UTF-8, UTF-16 and UTF-32. If None is - set, the charset of input json will be detected automatically. + :param encoding: standard charset name, for example UTF-8, UTF-16 and UTF-32. If None is + set, the encoding of input json will be detected automatically. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. @@ -258,7 +258,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, - allowUnquotedControlChars=allowUnquotedControlChars, charset=charset, lineSep=lineSep) + allowUnquotedControlChars=allowUnquotedControlChars, encoding=encoding, lineSep=lineSep) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -751,7 +751,7 @@ def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options) @since(1.4) def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None, - lineSep=None): + encoding=None, lineSep=None): """Saves the content of the :class:`DataFrame` in JSON format (`JSON Lines text format or newline-delimited JSON `_) at the specified path. @@ -775,6 +775,8 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm formats follow the formats at ``java.text.SimpleDateFormat``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + :param encoding: specifies encoding (charset) of saved json files. If None is set, + the default UTF-8 charset will be used. :param lineSep: defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. @@ -783,7 +785,7 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm self.mode(mode) self._set_opts( compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat, - lineSep=lineSep) + encoding=encoding, lineSep=lineSep) self._jwrite.json(path) @since(1.4) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c11e915a54ea..88975cbeb6a6 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -676,10 +676,10 @@ def test_multiline_json(self): multiLine=True) self.assertEqual(people1.collect(), people_array.collect()) - def test_charset_json(self): + def test_encoding_json(self): people_array = self.spark.read\ .json("python/test_support/sql/people_array_utf16le.json", - multiLine=True, charset="UTF-16LE") + multiLine=True, encoding="UTF-16LE") expected = [Row(age=30, name=u'Andy'), Row(age=19, name=u'Justin')] self.assertEqual(people_array.collect(), expected) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 249990c5b891..9eefb68679f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -94,15 +94,16 @@ private[sql] class JSONOptions( * Standard charset name. For example UTF-8, UTF-16 and UTF-32. * If charset is not specified (None), it will be detected automatically. */ - val charset: Option[String] = parameters.get("charset").map { cs => - if (cs != "UTF-8" && lineSeparator.isEmpty) { - throw new IllegalArgumentException( - s"""Please, set the 'lineSep' option for the given charset $cs. - |Example: .option("lineSep", "|^|") - |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin - ) - } - cs + val charset: Option[String] = parameters.get("charset") + .orElse(parameters.get("encoding")).map { cs => + if (multiLine == false && cs != "UTF-8" && lineSeparator.isEmpty) { + throw new IllegalArgumentException( + s"""Please, set the 'lineSep' option for the given charset $cs. + |Example: .option("lineSep", "|^|") + |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin + ) + } + cs } val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 23667dfc39c3..d479aead71fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -366,9 +366,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * `java.text.SimpleDateFormat`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • - *
  • `charset` (by default it is not set): allows to forcibly set one of standard basic - * or extended charsets for input jsons. For example UTF-8, UTF-16BE, UTF-32. If the charset - * is not specified (by default), the charset is detected automatically.
  • + *
  • `encoding` (by default it is not set): allows to forcibly set one of standard basic + * or extended charsets for input jsons. For example UTF-8, UTF-16BE, UTF-32. If the encoding + * is not specified (by default), it will be detected automatically.
  • *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
  • * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index bbc063148a72..398829043c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -518,7 +518,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • - *
  • `lineSep` (default `\n`): defines the line separator that should + *
  • `encoding` (by default it's not set): specifies encoding (charset) of saved json + * files. If it is not set, the UTF-8 charset will be used.
  • + *
  • lineSep` (default `\n`): defines the line separator that should * be used for writing.
  • * * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 72bbd3540839..16eb17368d7c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2093,12 +2093,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } - test("SPARK-23723: Use user's charset in reading of multi-line json in UTF-16LE") { + test("SPARK-23723: Use user's encoding in reading of multi-line json in UTF-16LE") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") - .options(Map("charset" -> "UTF-16LE", "lineSep" -> "\n")) + .options(Map("encoding" -> "UTF-16LE")) .json(testFile(fileName)) checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) @@ -2121,7 +2121,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") - .options(Map("charset" -> "uTf-16lE", "lineSep" -> "\n")) + .options(Map("charset" -> "uTf-16lE")) .json(testFile(fileName)) checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) @@ -2135,7 +2135,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) .option("mode", "FAILFAST") .option("multiline", "true") - .options(Map("charset" -> "UTF-16BE", "lineSep" -> "\n")) + .options(Map("charset" -> "UTF-16BE")) .json(testFile(fileName)) .count() } @@ -2165,16 +2165,16 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23723: save json in UTF-32BE") { - val charset = "UTF-32BE" + val encoding = "UTF-32BE" withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) df.write - .options(Map("charset" -> charset, "lineSep" -> "\n")) + .options(Map("charset" -> encoding, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) checkCharset( - expectedCharset = charset, + expectedCharset = encoding, pathToJsonFiles = path.getCanonicalPath, expectedContent = """{"_1":"Dog","_2":42}""" ) @@ -2235,8 +2235,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } def checkReadJson( - charset: String, lineSep: String, + charsetOption: String, + charset: String, inferSchema: Boolean, runId: Int ): Unit = { @@ -2267,7 +2268,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader - .option("charset", charset) + .option(charsetOption, charset) .option("lineSep", lineSep) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) @@ -2277,21 +2278,21 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:off nonascii List( - ("|", "UTF-8", false), - ("^", "UTF-16BE", true), - ("::", "UTF-16", true), - ("!!!@3", "UTF-32LE", false), - (0x1E.toChar.toString, "UTF-8", true), - ("ì•„", "UTF-32BE", false), - ("куку", "CP1251", true), - ("sep", "UTF-8", false), - ("\r\n", "UTF-16LE", false), - ("\r\n", "UTF-16BE", true), - ("\u000d\u000a", "UTF-32BE", false), - ("\u000a\u000d", "UTF-8", true), - ("===", "UTF-16", false), - ("$^+", "UTF-32LE", true) - ).zipWithIndex.foreach{case ((d, c, s), i) => checkReadJson(c, d, s, i)} + ("|", "encoding", "UTF-8", false), + ("^", "charset", "UTF-16BE", true), + ("::", "encoding", "UTF-16", true), + ("!!!@3", "charset", "UTF-32LE", false), + (0x1E.toChar.toString, "encoding", "UTF-8", true), + ("ì•„", "encoding", "UTF-32BE", false), + ("куку", "charset", "CP1251", true), + ("sep", "charset", "UTF-8", false), + ("\r\n", "encoding", "UTF-16LE", false), + ("\r\n", "charset", "UTF-16BE", true), + ("\u000d\u000a", "encoding", "UTF-32BE", false), + ("\u000a\u000d", "charset", "UTF-8", true), + ("===", "encoding", "UTF-16", false), + ("$^+", "charset", "UTF-32LE", true) + ).zipWithIndex.foreach{case ((d, o, c, s), i) => checkReadJson(d, o, c, s, i)} // scalastyle:on nonascii def testLineSeparator(lineSep: String): Unit = { From 75f7bb6602c86c6cf1580700a16d705df46731f8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 29 Mar 2018 16:36:53 +0200 Subject: [PATCH 68/95] Fix missing require and specifying field of internal row explicitly --- .../apache/spark/sql/catalyst/json/CreateJacksonParser.scala | 3 ++- .../spark/sql/execution/datasources/json/JsonDataSource.scala | 2 +- .../spark/sql/execution/datasources/text/TextOptions.scala | 1 + 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 7da6223b1d1a..7f1eb658f1d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -65,9 +65,10 @@ private[sql] object CreateJacksonParser extends Serializable { def internalRow( jsonFactory: JsonFactory, row: InternalRow, + field: Int, charset: Option[String] = None ): JsonParser = { - val is = new ByteArrayInputStream(row.getBinary(0)) + val is = new ByteArrayInputStream(row.getBinary(field)) inputStream(jsonFactory, is, charset) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index f3bc7822ceab..6faa64e2a5ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -105,7 +105,7 @@ object TextInputJsonDataSource extends JsonDataSource { JsonInferSchema.infer[InternalRow]( rdd, parsedOptions, - CreateJacksonParser.internalRow(_, _, parsedOptions.charset) + CreateJacksonParser.internalRow(_, _, 0, parsedOptions.charset) ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index 8e424d869215..a523f5aee63e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -44,6 +44,7 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti val charset: Option[String] = Some("UTF-8") val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").map { lineSep => + require(lineSep.nonEmpty, s"'$LINE_SEPARATOR' cannot be an empty string.") lineSep.getBytes(charset.getOrElse( throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) } From d93dcdc9d76edb290eb33d116967a99af8fe21dd Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 29 Mar 2018 17:53:37 +0200 Subject: [PATCH 69/95] Making the doc generator happy --- .../main/scala/org/apache/spark/sql/DataFrameWriter.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 398829043c9f..e183fa6f9542 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -518,10 +518,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • - *
  • `encoding` (by default it's not set): specifies encoding (charset) of saved json + *
  • `encoding` (by default it is not set): specifies encoding (charset) of saved json * files. If it is not set, the UTF-8 charset will be used.
  • - *
  • lineSep` (default `\n`): defines the line separator that should - * be used for writing.
  • + *
  • `lineSep` (default `\n`): defines the line separator that should be used for writing.
  • * * * @since 1.4.0 @@ -591,8 +590,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • - *
  • `lineSep` (default `\n`): defines the line separator that should - * be used for writing.
  • + *
  • `lineSep` (default `\n`): defines the line separator that should be used for writing.
  • * * * @since 1.6.0 From 65b4b73ed99fdc4039b890e1640353d2954700ce Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 31 Mar 2018 15:25:17 +0200 Subject: [PATCH 70/95] Making the encoding name as the primary name --- .../spark/sql/catalyst/json/JSONOptions.scala | 23 +++---- .../sql/catalyst/json/JacksonParser.scala | 2 +- .../datasources/json/JsonDataSource.scala | 8 +-- .../datasources/json/JsonFileFormat.scala | 2 +- .../datasources/text/TextOptions.scala | 11 +-- .../datasources/json/JsonSuite.scala | 68 +++++++++---------- 6 files changed, 55 insertions(+), 59 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index e79a98fb1efb..85971036caee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.json -import java.nio.charset.StandardCharsets import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} @@ -92,23 +91,22 @@ private[sql] class JSONOptions( val lineSeparator: Option[String] = parameters.get("lineSep") /** - * Standard charset name. For example UTF-8, UTF-16 and UTF-32. - * If charset is not specified (None), it will be detected automatically. + * Standard charset name. For example UTF-8, UTF-16LE and UTF-32BE. + * If the encoding is not specified (None), it will be detected automatically. */ - val charset: Option[String] = parameters.get("charset") - .orElse(parameters.get("encoding")).map { cs => - if (multiLine == false && cs != "UTF-8" && lineSeparator.isEmpty) { + val encoding: Option[String] = parameters.get("encoding") + .orElse(parameters.get("charset")).map { enc => + if (multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) { throw new IllegalArgumentException( - s"""Please, set the 'lineSep' option for the given charset $cs. + s"""Please, set the 'lineSep' option for the given encoding $enc. |Example: .option("lineSep", "|^|") - |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin - ) + |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin) } - cs + enc } val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => - lineSep.getBytes(charset.getOrElse("UTF-8")) + lineSep.getBytes(encoding.getOrElse("UTF-8")) } val lineSeparatorInWrite: String = lineSeparator.getOrElse("\n") @@ -125,6 +123,7 @@ private[sql] class JSONOptions( } def getTextOptions: Map[String, String] = { - Map[String, String]() ++ charset.map("charset" -> _) ++ lineSeparator.map("lineSep" -> _) + Map[String, String]() ++ + encoding.map("encoding" -> _) ++ lineSeparator.map("lineSep" -> _) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 8ff165a1032d..c6518a6badf8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -361,7 +361,7 @@ class JacksonParser( // For such records, all fields other than the field configured by // `columnNameOfCorruptRecord` are set to `null`. throw BadRecordException(() => recordLiteral(record), () => None, e) - case e: CharConversionException if options.charset.isEmpty => + case e: CharConversionException if options.encoding.isEmpty => val msg = """Failed to parse a character. Charset was detected automatically. |You might want to set it explicitly via the charset option like: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 6faa64e2a5ec..4f4f3ceb0af0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -105,7 +105,7 @@ object TextInputJsonDataSource extends JsonDataSource { JsonInferSchema.infer[InternalRow]( rdd, parsedOptions, - CreateJacksonParser.internalRow(_, _, 0, parsedOptions.charset) + CreateJacksonParser.internalRow(_, _, 0, parsedOptions.encoding) ) } @@ -132,7 +132,7 @@ object TextInputJsonDataSource extends JsonDataSource { schema: StructType): Iterator[InternalRow] = { val linesReader = new HadoopFileLinesReader(file, parser.options.lineSeparatorInRead, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) - val charset = parser.options.charset + val charset = parser.options.encoding val safeParser = new FailureSafeParser[Text]( input => parser.parse[Text](input, CreateJacksonParser.text(_, _, charset), textToUTF8String), @@ -162,7 +162,7 @@ object MultiLineJsonDataSource extends JsonDataSource { JsonInferSchema.infer[PortableDataStream]( sampled, parsedOptions, - createParser(_, _, parsedOptions.charset) + createParser(_, _, parsedOptions.encoding) ) } @@ -210,7 +210,7 @@ object MultiLineJsonDataSource extends JsonDataSource { UTF8String.fromBytes(ByteStreams.toByteArray(is)) } } - val charset = parser.options.charset + val charset = parser.options.encoding val safeParser = new FailureSafeParser[InputStream]( input => parser.parse[InputStream]( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 8d422dd95bff..3e3be7c64a2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -153,7 +153,7 @@ private[json] class JsonOutputWriter( context: TaskAttemptContext) extends OutputWriter with Logging { - private val charset = options.charset match { + private val charset = options.encoding match { case Some(charsetName) => Charset.forName(charsetName) case _ => StandardCharsets.UTF_8 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index a523f5aee63e..53b95f01c461 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -41,22 +41,23 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti */ val wholeText = parameters.getOrElse(WHOLETEXT, "false").toBoolean - val charset: Option[String] = Some("UTF-8") + val encoding: Option[String] = parameters.get(ENCODING) - val lineSeparator: Option[Array[Byte]] = parameters.get("lineSep").map { lineSep => + val lineSeparator: Option[Array[Byte]] = parameters.get(LINE_SEPARATOR).map { lineSep => require(lineSep.nonEmpty, s"'$LINE_SEPARATOR' cannot be an empty string.") - lineSep.getBytes(charset.getOrElse( - throw new IllegalArgumentException("Please, set the charset option for the delimiter"))) + + lineSep.getBytes(encoding.getOrElse("UTF-8")) } // Note that the option 'lineSep' uses a different default value in read and write. val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator val lineSeparatorInWrite: Array[Byte] = - lineSeparatorInRead.getOrElse("\n".getBytes(StandardCharsets.UTF_8)) + lineSeparatorInRead.getOrElse("\n".getBytes("UTF-8")) } private[datasources] object TextOptions { val COMPRESSION = "compression" val WHOLETEXT = "wholetext" + val ENCODING = "encoding" val LINE_SEPARATOR = "lineSep" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 0b954eb24b7d..f3d2c53b70ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2104,11 +2104,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } - test("SPARK-23723: Unsupported charset name") { + test("SPARK-23723: Unsupported encoding name") { val invalidCharset = "UTF-128" val exception = intercept[java.io.UnsupportedEncodingException] { spark.read - .options(Map("charset" -> invalidCharset, "lineSep" -> "\n")) + .options(Map("encoding" -> invalidCharset, "lineSep" -> "\n")) .json(testFile("json-tests/utf16LE.json")) .count() } @@ -2116,26 +2116,26 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(exception.getMessage.contains(invalidCharset)) } - test("SPARK-23723: checking that the charset option is case agnostic") { + test("SPARK-23723: checking that the encoding option is case agnostic") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") - .options(Map("charset" -> "uTf-16lE")) + .options(Map("encoding" -> "uTf-16lE")) .json(testFile(fileName)) checkAnswer(jsonDF, Seq(Row("Chris", "Baird"))) } - test("SPARK-23723: specified charset is not matched to actual charset") { + test("SPARK-23723: specified encoding is not matched to actual encoding") { val fileName = "json-tests/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val exception = intercept[SparkException] { spark.read.schema(schema) .option("mode", "FAILFAST") .option("multiline", "true") - .options(Map("charset" -> "UTF-16BE")) + .options(Map("encoding" -> "UTF-16BE")) .json(testFile(fileName)) .count() } @@ -2169,7 +2169,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) df.write - .options(Map("charset" -> encoding, "lineSep" -> "\n")) + .options(Map("encoding" -> encoding, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) @@ -2181,7 +2181,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("SPARK-23723: save json in default charset - UTF-8") { + test("SPARK-23723: save json in default encoding - UTF-8") { withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) df.write @@ -2196,38 +2196,34 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("SPARK-23723: wrong output charset") { - val charset = "UTF-128" + test("SPARK-23723: wrong output encoding") { + val encoding = "UTF-128" val exception = intercept[java.io.UnsupportedEncodingException] { withTempPath { path => val df = spark.createDataset(Seq((0))) df.write - .options(Map("charset" -> charset, "lineSep" -> "\n")) + .options(Map("encoding" -> encoding, "lineSep" -> "\n")) .format("json").mode("overwrite") .save(path.getCanonicalPath) } } - assert(exception.getMessage == charset) + assert(exception.getMessage == encoding) } - test("SPARK-23723: read written json in UTF-16") { - val charset = "UTF-16" - case class Rec(f1: String, f2: Int) + test("SPARK-23723: read written json in UTF-16LE") { + val options = Map("encoding" -> "UTF-16LE", "lineSep" -> "\n") withTempPath { path => val ds = spark.createDataset(Seq( ("a", 1), ("b", 2), ("c", 3)) ).repartition(2) ds.write - .options(Map("charset" -> charset, "lineSep" -> "\n")) + .options(options) .format("json").mode("overwrite") .save(path.getCanonicalPath) val savedDf = spark .read - .schema(ds.schema) - // This option will be replaced by .option("lineSep", "x00 0a") - // as soon as lineSep allows to specify sequence of bytes in hexadecimal format. - .option("mode", "DROPMALFORMED") + .options(options) .json(path.getCanonicalPath) checkAnswer(savedDf.toDF(), ds.toDF()) @@ -2236,23 +2232,23 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { def checkReadJson( lineSep: String, - charsetOption: String, - charset: String, + encodingOption: String, + encoding: String, inferSchema: Boolean, runId: Int ): Unit = { - test(s"SPARK-23724: checks reading json in ${charset} #${runId}") { + test(s"SPARK-23724: checks reading json in ${encoding} #${runId}") { val delimInBytes = { if (lineSep.startsWith("x")) { lineSep.replaceAll("[^0-9A-Fa-f]", "") .sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) } else { - lineSep.getBytes(charset) + lineSep.getBytes(encoding) } } case class Rec(f1: String, f2: Int) { def json = s"""{"f1":"${f1}", "f2":$f2}""" - def bytes = json.getBytes(charset) + def bytes = json.getBytes(encoding) def row = Row(f1, f2) } val schema = new StructType().add("f1", StringType).add("f2", IntegerType) @@ -2268,7 +2264,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val savedDf = reader - .option(charsetOption, charset) + .option(encodingOption, encoding) .option("lineSep", lineSep) .json(path.getCanonicalPath) checkAnswer(savedDf, records.map(_.row)) @@ -2281,17 +2277,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ("|", "encoding", "UTF-8", false), ("^", "charset", "UTF-16BE", true), ("::", "encoding", "UTF-16", true), - ("!!!@3", "charset", "UTF-32LE", false), - (0x1E.toChar.toString, "encoding", "UTF-8", true), + ("!!!@3", "encoding", "UTF-32LE", false), + (0x1E.toChar.toString, "charset", "UTF-8", true), ("ì•„", "encoding", "UTF-32BE", false), - ("куку", "charset", "CP1251", true), - ("sep", "charset", "UTF-8", false), + ("куку", "encoding", "CP1251", true), + ("sep", "encoding", "UTF-8", false), ("\r\n", "encoding", "UTF-16LE", false), - ("\r\n", "charset", "UTF-16BE", true), + ("\r\n", "encoding", "UTF-16BE", true), ("\u000d\u000a", "encoding", "UTF-32BE", false), - ("\u000a\u000d", "charset", "UTF-8", true), + ("\u000a\u000d", "encoding", "UTF-8", true), ("===", "encoding", "UTF-16", false), - ("$^+", "charset", "UTF-32LE", true) + ("$^+", "encoding", "UTF-32LE", true) ).zipWithIndex.foreach{case ((d, o, c, s), i) => checkReadJson(d, o, c, s, i)} // scalastyle:on nonascii @@ -2323,16 +2319,16 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } // scalastyle:on nonascii - test("SPARK-23724: lineSep should be set if charset if different from UTF-8") { + test("SPARK-23724: lineSep should be set if encoding if different from UTF-8") { val exception = intercept[IllegalArgumentException] { spark.read - .options(Map("charset" -> "UTF-16LE")) + .options(Map("encoding" -> "UTF-16LE")) .json(testFile("json-tests/utf16LE.json")) .count() } assert(exception.getMessage.contains( - """Please, set the 'lineSep' option for the given charset UTF-16LE""" + """Please, set the 'lineSep' option for the given encoding UTF-16LE""" )) } From 6b52419e85af80aabafc16cb441f6935651cc9af Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 31 Mar 2018 16:24:58 +0200 Subject: [PATCH 71/95] Blacklisting UTF-16 and UTF-32 in per-line mode --- .../spark/sql/catalyst/json/JSONOptions.scala | 16 ++++++++------ .../resources/json-tests/utf16WithBOM.json | Bin 170 -> 200 bytes .../datasources/json/JsonSuite.scala | 20 +++++++++--------- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 85971036caee..1b3ee27c5310 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -96,12 +96,16 @@ private[sql] class JSONOptions( */ val encoding: Option[String] = parameters.get("encoding") .orElse(parameters.get("charset")).map { enc => - if (multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) { - throw new IllegalArgumentException( - s"""Please, set the 'lineSep' option for the given encoding $enc. - |Example: .option("lineSep", "|^|") - |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin) - } + val blacklist = List("UTF-16", "UTF-32") + require(!(multiLine == false && blacklist.contains(enc)), + s"""The ${enc} encoding must not be included in the blacklist: + | ${blacklist.mkString(", ")}""".stripMargin) + + val forcingLineSep = !(multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) + require(forcingLineSep, + s"""The lineSep option must be specified for the $enc encoding. + |Example: .option("lineSep", "|^|") + |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin) enc } diff --git a/sql/core/src/test/resources/json-tests/utf16WithBOM.json b/sql/core/src/test/resources/json-tests/utf16WithBOM.json index 65e7e2f72948103b198017d2553f501efe850da0..cf4d29328b860ffe8288edea437222c6d432a100 100644 GIT binary patch delta 50 mcmZ3*c!H7t|G#JkUIs1(1qhrNE;~`qj8_L$ESN18EC2u(^a$br delta 17 YcmX@XxQdbK|G$X>LCm}iTodzy0Xm)q)Bpeg diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f3d2c53b70ae..862310c1a791 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2073,9 +2073,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val fileName = "json-tests/utf16WithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) - // This option will be replaced by .option("lineSep", "x00 0a") - // as soon as lineSep allows to specify sequence of bytes in hexadecimal format. - .option("mode", "DROPMALFORMED") + .option("multiline", "true") + .option("encoding", "UTF-16") .json(testFile(fileName)) checkAnswer(jsonDF, Seq( @@ -2276,18 +2275,18 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { List( ("|", "encoding", "UTF-8", false), ("^", "charset", "UTF-16BE", true), - ("::", "encoding", "UTF-16", true), + ("::", "encoding", "ISO-8859-1", true), ("!!!@3", "encoding", "UTF-32LE", false), (0x1E.toChar.toString, "charset", "UTF-8", true), ("ì•„", "encoding", "UTF-32BE", false), ("куку", "encoding", "CP1251", true), - ("sep", "encoding", "UTF-8", false), + ("sep", "encoding", "utf-8", false), ("\r\n", "encoding", "UTF-16LE", false), - ("\r\n", "encoding", "UTF-16BE", true), + ("\r\n", "encoding", "utf-16be", true), ("\u000d\u000a", "encoding", "UTF-32BE", false), ("\u000a\u000d", "encoding", "UTF-8", true), - ("===", "encoding", "UTF-16", false), - ("$^+", "encoding", "UTF-32LE", true) + ("===", "encoding", "US-ASCII", false), + ("$^+", "encoding", "utf-32le", true) ).zipWithIndex.foreach{case ((d, o, c, s), i) => checkReadJson(d, o, c, s, i)} // scalastyle:on nonascii @@ -2320,15 +2319,16 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:on nonascii test("SPARK-23724: lineSep should be set if encoding if different from UTF-8") { + val encoding = "UTF-16LE" val exception = intercept[IllegalArgumentException] { spark.read - .options(Map("encoding" -> "UTF-16LE")) + .options(Map("encoding" -> encoding)) .json(testFile("json-tests/utf16LE.json")) .count() } assert(exception.getMessage.contains( - """Please, set the 'lineSep' option for the given encoding UTF-16LE""" + s"""The lineSep option must be specified for the $encoding encoding""" )) } From 6116bace010b63008a7c21408a1bb43f9fa28670 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 31 Mar 2018 17:06:24 +0200 Subject: [PATCH 72/95] Changes after code review --- .../spark/sql/catalyst/json/JSONOptions.scala | 5 +- .../datasources/json/JsonFileFormat.scala | 7 +- .../datasources/json/JsonSuite.scala | 105 +++++++++++------- 3 files changed, 68 insertions(+), 49 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 1b3ee27c5310..6ec634db780b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -96,8 +96,9 @@ private[sql] class JSONOptions( */ val encoding: Option[String] = parameters.get("encoding") .orElse(parameters.get("charset")).map { enc => - val blacklist = List("UTF-16", "UTF-32") - require(!(multiLine == false && blacklist.contains(enc)), + val blacklist = List("UTF16", "UTF32") + val isBlacklisted = blacklist.contains(enc.toUpperCase.replaceAll("-|_", "")) + require(multiLine || !isBlacklisted, s"""The ${enc} encoding must not be included in the blacklist: | ${blacklist.mkString(", ")}""".stripMargin) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 3e3be7c64a2b..59df73f37def 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -153,16 +153,13 @@ private[json] class JsonOutputWriter( context: TaskAttemptContext) extends OutputWriter with Logging { - private val charset = options.encoding match { + private val encoding = options.encoding match { case Some(charsetName) => Charset.forName(charsetName) case _ => StandardCharsets.UTF_8 } private val writer = CodecStreams.createOutputStreamWriter( - context, - new Path(path), - charset - ) + context, new Path(path), encoding) // create the Generator without separator inserted between 2 records private[this] val gen = new JacksonGenerator(dataSchema, writer, options) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 862310c1a791..2b49279059ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2065,6 +2065,69 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } + def testLineSeparator(lineSep: String): Unit = { + test(s"SPARK-21289: Support line separator - lineSep: '$lineSep'") { + // Read + val data = + s""" + | {"f": + |"a", "f0": 1}$lineSep{"f": + | + |"c", "f0": 2}$lineSep{"f": "d", "f0": 3} + """.stripMargin + val dataWithTrailingLineSep = s"$data$lineSep" + + Seq(data, dataWithTrailingLineSep).foreach { lines => + withTempPath { path => + Files.write(path.toPath, lines.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.option("lineSep", lineSep).json(path.getAbsolutePath) + val expectedSchema = + StructType(StructField("f", StringType) :: StructField("f0", LongType) :: Nil) + checkAnswer(df, Seq(("a", 1), ("c", 2), ("d", 3)).toDF()) + assert(df.schema === expectedSchema) + } + } + + // Write + withTempPath { path => + Seq("a", "b", "c").toDF("value").coalesce(1) + .write.option("lineSep", lineSep).json(path.getAbsolutePath) + val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) + assert( + readBack === s"""{"value":"a"}$lineSep{"value":"b"}$lineSep{"value":"c"}$lineSep""") + } + + // Roundtrip + withTempPath { path => + val df = Seq("a", "b", "c").toDF() + df.write.option("lineSep", lineSep).json(path.getAbsolutePath) + val readBack = spark.read.option("lineSep", lineSep).json(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + Seq("|", "^", "::", "!!!@3", 0x1E.toChar.toString, "ì•„").foreach { lineSep => + testLineSeparator(lineSep) + } + // scalastyle:on nonascii + + test("""SPARK-21289: Support line separator - default value \r, \r\n and \n""") { + val data = + "{\"f\": \"a\", \"f0\": 1}\r{\"f\": \"c\", \"f0\": 2}\r\n{\"f\": \"d\", \"f0\": 3}\n" + + withTempPath { path => + Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.json(path.getAbsolutePath) + val expectedSchema = + StructType(StructField("f", StringType) :: StructField("f0", LongType) :: Nil) + checkAnswer(df, Seq(("a", 1), ("c", 2), ("d", 3)).toDF()) + assert(df.schema === expectedSchema) + } + } + def testFile(fileName: String): String = { Thread.currentThread().getContextClassLoader.getResource(fileName).toString } @@ -2290,34 +2353,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ).zipWithIndex.foreach{case ((d, o, c, s), i) => checkReadJson(d, o, c, s, i)} // scalastyle:on nonascii - def testLineSeparator(lineSep: String): Unit = { - test(s"SPARK-23724: Support line separator - lineSep: '$lineSep'") { - // Write - withTempPath { path => - Seq("a", "b", "c").toDF("value").coalesce(1) - .write.option("lineSep", lineSep).json(path.getAbsolutePath) - val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head - val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) - assert( - readBack === s"""{"value":"a"}$lineSep{"value":"b"}$lineSep{"value":"c"}$lineSep""") - } - - // Roundtrip - withTempPath { path => - val df = Seq("a", "b", "c").toDF() - df.write.option("lineSep", lineSep).json(path.getAbsolutePath) - val readBack = spark.read.option("lineSep", lineSep).json(path.getAbsolutePath) - checkAnswer(df, readBack) - } - } - } - - // scalastyle:off nonascii - Seq("|", "^", "::", "!!!@3", 0x1E.toChar.toString, "ì•„").foreach { lineSep => - testLineSeparator(lineSep) - } - // scalastyle:on nonascii - test("SPARK-23724: lineSep should be set if encoding if different from UTF-8") { val encoding = "UTF-16LE" val exception = intercept[IllegalArgumentException] { @@ -2331,18 +2366,4 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { s"""The lineSep option must be specified for the $encoding encoding""" )) } - - test("""SPARK-21289: Support line separator - default value \r, \r\n and \n""") { - val data = - "{\"f\": \"a\", \"f0\": 1}\r{\"f\": \"c\", \"f0\": 2}\r\n{\"f\": \"d\", \"f0\": 3}\n" - - withTempPath { path => - Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) - val df = spark.read.json(path.getAbsolutePath) - val expectedSchema = - StructType(StructField("f", StringType) :: StructField("f0", LongType) :: Nil) - checkAnswer(df, Seq(("a", 1), ("c", 2), ("d", 3)).toDF()) - assert(df.schema === expectedSchema) - } - } } From 53834005ba22c0c3e7be883949ab01a5bf1a0b9a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 31 Mar 2018 17:23:49 +0200 Subject: [PATCH 73/95] Renaming charset to encoding --- .../catalyst/json/CreateJacksonParser.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 7f1eb658f1d3..dbffb31543e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -40,11 +40,11 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def text(jsonFactory: JsonFactory, record: Text, charset: Option[String] = None): JsonParser = { - charset match { - case Some(cs) => + def text(jsonFactory: JsonFactory, record: Text, encoding: Option[String] = None): JsonParser = { + encoding match { + case Some(enc) => val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) - jsonFactory.createParser(new InputStreamReader(bain, cs)) + jsonFactory.createParser(new InputStreamReader(bain, enc)) case _ => jsonFactory.createParser(record.getBytes, 0, record.getLength) } @@ -53,10 +53,10 @@ private[sql] object CreateJacksonParser extends Serializable { def inputStream( jsonFactory: JsonFactory, is: InputStream, - charset: Option[String] = None): JsonParser = { - charset match { - case Some(cs) => - jsonFactory.createParser(new InputStreamReader(is, cs)) + encoding: Option[String] = None): JsonParser = { + encoding match { + case Some(enc) => + jsonFactory.createParser(new InputStreamReader(is, enc)) case _ => jsonFactory.createParser(is) } @@ -66,10 +66,10 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory: JsonFactory, row: InternalRow, field: Int, - charset: Option[String] = None + encoding: Option[String] = None ): JsonParser = { val is = new ByteArrayInputStream(row.getBinary(field)) - inputStream(jsonFactory, is, charset) + inputStream(jsonFactory, is, encoding) } } From 1aeae3c50fb5ffb1b02c4bc4a9ad0fec9008dec7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Apr 2018 22:46:44 +0200 Subject: [PATCH 74/95] Changes requested by HyukjinKwon in the review --- python/pyspark/sql/readwriter.py | 4 +- .../catalyst/json/CreateJacksonParser.scala | 24 ++++------ .../datasources/json/JsonDataSource.scala | 12 ++--- .../datasources/json/JsonSuite.scala | 45 ++++++++----------- 4 files changed, 35 insertions(+), 50 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 5d579fb45a5c..daa2c7a5b79d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -237,8 +237,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. - :param encoding: standard charset name, for example UTF-8, UTF-16 and UTF-32. If None is - set, the encoding of input json will be detected automatically. + :param encoding: standard encoding (charset) name, for example UTF-8, UTF-16LE and UTF-32BE. + If None is set, the encoding of input JSON will be detected automatically. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index dbffb31543e1..d06d3874c590 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -40,34 +40,26 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def text(jsonFactory: JsonFactory, record: Text, encoding: Option[String] = None): JsonParser = { + def text(jsonFactory: JsonFactory, record: Text, encoding: Option[String]): JsonParser = { encoding match { case Some(enc) => val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) jsonFactory.createParser(new InputStreamReader(bain, enc)) - case _ => + case None => jsonFactory.createParser(record.getBytes, 0, record.getLength) } } - def inputStream( - jsonFactory: JsonFactory, - is: InputStream, - encoding: Option[String] = None): JsonParser = { + def inputStream(jsonFactory: JsonFactory, is: InputStream, + encoding: Option[String]): JsonParser = { encoding match { - case Some(enc) => - jsonFactory.createParser(new InputStreamReader(is, enc)) - case _ => - jsonFactory.createParser(is) + case Some(enc) => jsonFactory.createParser(new InputStreamReader(is, enc)) + case None => jsonFactory.createParser(is) } } - def internalRow( - jsonFactory: JsonFactory, - row: InternalRow, - field: Int, - encoding: Option[String] = None - ): JsonParser = { + def internalRow(jsonFactory: JsonFactory, row: InternalRow, field: Int, + encoding: Option[String]): JsonParser = { val is = new ByteArrayInputStream(row.getBinary(field)) inputStream(jsonFactory, is, encoding) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 4f4f3ceb0af0..cb1a43dc02e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -202,12 +202,11 @@ object MultiLineJsonDataSource extends JsonDataSource { file: PartitionedFile, parser: JacksonParser, schema: StructType): Iterator[InternalRow] = { - def createInputStream() = { - CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))) - } def partitionedFileString(ignored: Any): UTF8String = { - Utils.tryWithResource(createInputStream()) { is => - UTF8String.fromBytes(ByteStreams.toByteArray(is)) + Utils.tryWithResource { + CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))) + } { inputStream => + UTF8String.fromBytes(ByteStreams.toByteArray(inputStream)) } } val charset = parser.options.encoding @@ -222,6 +221,7 @@ object MultiLineJsonDataSource extends JsonDataSource { schema, parser.options.columnNameOfCorruptRecord) - safeParser.parse(createInputStream()) + safeParser.parse( + CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath)))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 2b49279059ae..f070258faa07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2206,8 +2206,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(errMsg.contains("Malformed records are detected in record parsing")) } - def checkCharset( - expectedCharset: String, + def checkEncoding( + expectedEncoding: String, pathToJsonFiles: String, expectedContent: String ): Unit = { @@ -2215,15 +2215,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .listFiles() .filter(_.isFile) .filter(_.getName.endsWith("json")) - val jsonContent = jsonFiles.map { file => - scala.io.Source.fromFile(file, expectedCharset).mkString - } - val cleanedContent = jsonContent - .mkString - .trim - .replaceAll(" ", "") + val actualContent = jsonFiles.map { file => + new String(Files.readAllBytes(file.toPath), expectedEncoding) + }.mkString.trim.replaceAll(" ", "") - assert(cleanedContent == expectedContent) + assert(actualContent == expectedContent) } test("SPARK-23723: save json in UTF-32BE") { @@ -2235,8 +2231,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .format("json").mode("overwrite") .save(path.getCanonicalPath) - checkCharset( - expectedCharset = encoding, + checkEncoding( + expectedEncoding = encoding, pathToJsonFiles = path.getCanonicalPath, expectedContent = """{"_1":"Dog","_2":42}""" ) @@ -2250,8 +2246,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .format("json").mode("overwrite") .save(path.getCanonicalPath) - checkCharset( - expectedCharset = "UTF-8", + checkEncoding( + expectedEncoding = "UTF-8", pathToJsonFiles = path.getCanonicalPath, expectedContent = """{"_1":"Dog","_2":42}""" ) @@ -2283,12 +2279,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .options(options) .format("json").mode("overwrite") .save(path.getCanonicalPath) - val savedDf = spark + val readBack = spark .read .options(options) .json(path.getCanonicalPath) - checkAnswer(savedDf.toDF(), ds.toDF()) + checkAnswer(readBack.toDF(), ds.toDF()) } } @@ -2300,7 +2296,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { runId: Int ): Unit = { test(s"SPARK-23724: checks reading json in ${encoding} #${runId}") { - val delimInBytes = { + val lineSepInBytes = { if (lineSep.startsWith("x")) { lineSep.replaceAll("[^0-9A-Fa-f]", "") .sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) @@ -2308,15 +2304,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { lineSep.getBytes(encoding) } } - case class Rec(f1: String, f2: Int) { - def json = s"""{"f1":"${f1}", "f2":$f2}""" - def bytes = json.getBytes(encoding) - def row = Row(f1, f2) - } val schema = new StructType().add("f1", StringType).add("f2", IntegerType) withTempPath { path => - val records = List(Rec("a", 1), Rec("b", 2)) - val data = records.map(_.bytes).reduce((a1, a2) => a1 ++ delimInBytes ++ a2) + val records = List(("a", 1), ("b", 2)) + val data = records + .map(rec => s"""{"f1":"${rec._1}", "f2":${rec._2}}""".getBytes(encoding)) + .reduce((a1, a2) => a1 ++ lineSepInBytes ++ a2) val os = new FileOutputStream(path) os.write(data) os.close() @@ -2325,11 +2318,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } else { spark.read.schema(schema) } - val savedDf = reader + val readBack = reader .option(encodingOption, encoding) .option("lineSep", lineSep) .json(path.getCanonicalPath) - checkAnswer(savedDf, records.map(_.row)) + checkAnswer(readBack, records.map(rec => Row(rec._1, rec._2))) } } } From 7e208912a370af0466dfcadbff0787f2b3248dd9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Apr 2018 23:10:09 +0200 Subject: [PATCH 75/95] Adding tests for SPARK-23094 --- .../datasources/json/JsonSuite.scala | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f070258faa07..b873040637a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2359,4 +2359,42 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { s"""The lineSep option must be specified for the $encoding encoding""" )) } + + private val badJson = "\u0000\u0000\u0000A\u0001AAA" + + test("SPARK-23094: invalid json with leading nulls - from file (multiLine=true)") { + import testImplicits._ + withTempDir { tempDir => + val path = tempDir.getAbsolutePath + Seq(badJson + """{"a":1}""").toDS().write.mode("overwrite").text(path) + val expected = s"""${badJson}{"a":1}\n""" + val schema = new StructType().add("a", IntegerType).add("_corrupt_record", StringType) + val df = spark.read.format("json") + .option("multiLine", true) + .option("encoding", "UTF-8") + .schema(schema).load(path) + checkAnswer(df, Row(null, expected)) + } + } + + test("SPARK-23094: invalid json with leading nulls - from file (multiLine=false)") { + import testImplicits._ + withTempDir { tempDir => + val path = tempDir.getAbsolutePath + Seq(badJson, """{"a":1}""").toDS().write.mode("overwrite").text(path) + val schema = new StructType().add("a", IntegerType).add("_corrupt_record", StringType) + val df = spark.read.format("json") + .option("multiLine", false) + .option("encoding", "UTF-8") + .schema(schema).load(path) + checkAnswer(df, Seq(Row(1, null), Row(null, badJson))) + } + } + + test("SPARK-23094: invalid json with leading nulls - from dataset") { + import testImplicits._ + checkAnswer( + spark.read.option("encoding", "UTF-8").json(Seq(badJson).toDS()), + Row(badJson)) + } } From 0d3ed3cc2ea3158cec82e31f44936fd387943638 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 5 Apr 2018 17:38:04 +0200 Subject: [PATCH 76/95] Fix comments --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 6ec634db780b..dde2d3aef66b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -86,12 +86,12 @@ private[sql] class JSONOptions( val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) /** - * A sequence of bytes between two consecutive json records. + * A string between two consecutive JSON records. */ val lineSeparator: Option[String] = parameters.get("lineSep") /** - * Standard charset name. For example UTF-8, UTF-16LE and UTF-32BE. + * Standard encoding (charset) name. For example UTF-8, UTF-16LE and UTF-32BE. * If the encoding is not specified (None), it will be detected automatically. */ val encoding: Option[String] = parameters.get("encoding") From 5d5c29582fdce06f71a8c79697a6b5bf89f011d8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Apr 2018 13:41:14 +0200 Subject: [PATCH 77/95] Matching by encoding per each line is eliminated --- .../catalyst/json/CreateJacksonParser.scala | 39 +++++++++++-------- .../datasources/json/JsonDataSource.scala | 36 ++++++++--------- 2 files changed, 39 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index d06d3874c590..7966afa7f87c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -40,28 +40,33 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(bain, "UTF-8")) } - def text(jsonFactory: JsonFactory, record: Text, encoding: Option[String]): JsonParser = { - encoding match { - case Some(enc) => - val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) - jsonFactory.createParser(new InputStreamReader(bain, enc)) - case None => - jsonFactory.createParser(record.getBytes, 0, record.getLength) - } + def text(jsonFactory: JsonFactory, record: Text): JsonParser = { + jsonFactory.createParser(record.getBytes, 0, record.getLength) } - def inputStream(jsonFactory: JsonFactory, is: InputStream, - encoding: Option[String]): JsonParser = { - encoding match { - case Some(enc) => jsonFactory.createParser(new InputStreamReader(is, enc)) - case None => jsonFactory.createParser(is) - } + def text(enc: String, jsonFactory: JsonFactory, record: Text): JsonParser = { + val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) + jsonFactory.createParser(new InputStreamReader(bain, enc)) } - def internalRow(jsonFactory: JsonFactory, row: InternalRow, field: Int, - encoding: Option[String]): JsonParser = { + def inputStream(jsonFactory: JsonFactory, is: InputStream): JsonParser = { + jsonFactory.createParser(is) + } + + def inputStream(enc: String, jsonFactory: JsonFactory, is: InputStream): JsonParser = { + jsonFactory.createParser(new InputStreamReader(is, enc)) + } + + def internalRow(jsonFactory: JsonFactory, row: InternalRow, field: Int): JsonParser = { + val is = new ByteArrayInputStream(row.getBinary(field)) + + inputStream(jsonFactory, is) + } + + def internalRow(enc: String, jsonFactory: JsonFactory, + row: InternalRow, field: Int): JsonParser = { val is = new ByteArrayInputStream(row.getBinary(field)) - inputStream(jsonFactory, is, encoding) + inputStream(enc, jsonFactory, is) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index cb1a43dc02e4..acfc525f8eb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -101,12 +101,11 @@ object TextInputJsonDataSource extends JsonDataSource { def inferFromDataset(json: Dataset[String], parsedOptions: JSONOptions): StructType = { val sampled: Dataset[String] = JsonUtils.sample(json, parsedOptions) val rdd: RDD[InternalRow] = sampled.queryExecution.toRdd + val rowParser = parsedOptions.encoding.map { enc => + CreateJacksonParser.internalRow(enc, _: JsonFactory, _: InternalRow, 0) + }.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow, 0)) - JsonInferSchema.infer[InternalRow]( - rdd, - parsedOptions, - CreateJacksonParser.internalRow(_, _, 0, parsedOptions.encoding) - ) + JsonInferSchema.infer(rdd, parsedOptions, rowParser) } private def createBaseDataset( @@ -132,10 +131,12 @@ object TextInputJsonDataSource extends JsonDataSource { schema: StructType): Iterator[InternalRow] = { val linesReader = new HadoopFileLinesReader(file, parser.options.lineSeparatorInRead, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) - val charset = parser.options.encoding + val textParser = parser.options.encoding + .map(enc => CreateJacksonParser.text(enc, _: JsonFactory, _: Text)) + .getOrElse(CreateJacksonParser.text(_: JsonFactory, _: Text)) val safeParser = new FailureSafeParser[Text]( - input => parser.parse[Text](input, CreateJacksonParser.text(_, _, charset), textToUTF8String), + input => parser.parse(input, textParser, textToUTF8String), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) @@ -188,13 +189,12 @@ object MultiLineJsonDataSource extends JsonDataSource { private def createParser( jsonFactory: JsonFactory, record: PortableDataStream, - charset: Option[String] = None): JsonParser = { + encoding: Option[String] = None): JsonParser = { val path = new Path(record.getPath()) - CreateJacksonParser.inputStream( - jsonFactory, - CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path), - charset - ) + val is = CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path) + + encoding.map(enc => CreateJacksonParser.inputStream(enc, jsonFactory, is)) + .getOrElse(CreateJacksonParser.inputStream(jsonFactory, is)) } override def readFile( @@ -209,14 +209,12 @@ object MultiLineJsonDataSource extends JsonDataSource { UTF8String.fromBytes(ByteStreams.toByteArray(inputStream)) } } - val charset = parser.options.encoding + val streamParser = parser.options.encoding + .map(enc => CreateJacksonParser.inputStream(enc, _: JsonFactory, _: InputStream)) + .getOrElse(CreateJacksonParser.inputStream(_: JsonFactory, _: InputStream)) val safeParser = new FailureSafeParser[InputStream]( - input => parser.parse[InputStream]( - input, - CreateJacksonParser.inputStream(_, _, charset), - partitionedFileString - ), + input => parser.parse[InputStream](input, streamParser, partitionedFileString), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) From e7be77d52a1c13a8817eba086f25454c06981e6f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Apr 2018 14:06:29 +0200 Subject: [PATCH 78/95] Addressing Hyukjin's review comments --- .../spark/sql/catalyst/json/JSONOptions.scala | 5 ---- .../sql/catalyst/json/JacksonParser.scala | 10 +++---- .../datasources/json/JsonDataSource.scala | 12 +++++---- .../datasources/json/JsonFileFormat.scala | 2 +- .../datasources/json/JsonSuite.scala | 26 +++++++------------ 5 files changed, 22 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index dde2d3aef66b..a230bde0fe6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -126,9 +126,4 @@ private[sql] class JSONOptions( allowBackslashEscapingAnyCharacter) factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars) } - - def getTextOptions: Map[String, String] = { - Map[String, String]() ++ - encoding.map("encoding" -> _) ++ lineSeparator.map("lineSep" -> _) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index c6518a6badf8..2c30e939263b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -363,11 +363,11 @@ class JacksonParser( throw BadRecordException(() => recordLiteral(record), () => None, e) case e: CharConversionException if options.encoding.isEmpty => val msg = - """Failed to parse a character. Charset was detected automatically. - |You might want to set it explicitly via the charset option like: - | .option("charset", "UTF-8") - |Example of supported charsets: - | UTF-8, UTF-16, UTF-16BE, UTF-16LE, UTF-32, UTF-32BE, UTF-32LE + """Failed to parse a character. Encoding was detected automatically. + |You might want to set it explicitly via the encoding option like: + | .option("encoding", "UTF-8") + |Example of supported encodings: + | UTF-8, UTF-16BE, UTF-16LE, UTF-32BE, UTF-32LE |""".stripMargin + e.getMessage throw new CharConversionException(msg) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index acfc525f8eb0..7848b401a51e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -111,15 +111,18 @@ object TextInputJsonDataSource extends JsonDataSource { private def createBaseDataset( sparkSession: SparkSession, inputPaths: Seq[FileStatus], - parsedOptions: JSONOptions - ): Dataset[String] = { + parsedOptions: JSONOptions): Dataset[String] = { val paths = inputPaths.map(_.getPath.toString) + val textOptions = Map.empty[String, String] ++ + parsedOptions.encoding.map("encoding" -> _) ++ + parsedOptions.lineSeparator.map("lineSep" -> _) + sparkSession.baseRelationToDataFrame( DataSource.apply( sparkSession, paths = paths, className = classOf[TextFileFormat].getName, - options = parsedOptions.getTextOptions + options = textOptions ).resolveRelation(checkFilesExist = false)) .select("value").as(Encoders.STRING) } @@ -163,8 +166,7 @@ object MultiLineJsonDataSource extends JsonDataSource { JsonInferSchema.infer[PortableDataStream]( sampled, parsedOptions, - createParser(_, _, parsedOptions.encoding) - ) + createParser(_, _, parsedOptions.encoding)) } private def createBaseRdd( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 59df73f37def..3b04510d2969 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -155,7 +155,7 @@ private[json] class JsonOutputWriter( private val encoding = options.encoding match { case Some(charsetName) => Charset.forName(charsetName) - case _ => StandardCharsets.UTF_8 + case None => StandardCharsets.UTF_8 } private val writer = CodecStreams.createOutputStreamWriter( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index b873040637a0..393f584c1178 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2070,9 +2070,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Read val data = s""" - | {"f": - |"a", "f0": 1}$lineSep{"f": - | + | {"f": + |"a", "f0": 1}$lineSep{"f": + | |"c", "f0": 2}$lineSep{"f": "d", "f0": 3} """.stripMargin val dataWithTrailingLineSep = s"$data$lineSep" @@ -2140,9 +2140,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .option("encoding", "UTF-16") .json(testFile(fileName)) - checkAnswer(jsonDF, Seq( - Row("Chris", "Baird"), Row("Doug", "Rood") - )) + checkAnswer(jsonDF, Seq(Row("Chris", "Baird"), Row("Doug", "Rood"))) } test("SPARK-23723: multi-line json in UTF-32BE with BOM") { @@ -2207,10 +2205,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } def checkEncoding( - expectedEncoding: String, - pathToJsonFiles: String, - expectedContent: String - ): Unit = { + expectedEncoding: String, + pathToJsonFiles: String, + expectedContent: String): Unit = { val jsonFiles = new File(pathToJsonFiles) .listFiles() .filter(_.isFile) @@ -2288,13 +2285,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - def checkReadJson( - lineSep: String, - encodingOption: String, - encoding: String, - inferSchema: Boolean, - runId: Int - ): Unit = { + def checkReadJson(lineSep: String, encodingOption: String, encoding: String, + inferSchema: Boolean, runId: Int): Unit = { test(s"SPARK-23724: checks reading json in ${encoding} #${runId}") { val lineSepInBytes = { if (lineSep.startsWith("x")) { From 6bd841a4765a2d781d2a9ab24f8ef6253f47558b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Apr 2018 14:22:02 +0200 Subject: [PATCH 79/95] Fixes regarding to coding style --- .../spark/sql/execution/datasources/json/JsonSuite.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 393f584c1178..3e807e3500bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2231,8 +2231,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkEncoding( expectedEncoding = encoding, pathToJsonFiles = path.getCanonicalPath, - expectedContent = """{"_1":"Dog","_2":42}""" - ) + expectedContent = """{"_1":"Dog","_2":42}""") } } @@ -2246,8 +2245,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkEncoding( expectedEncoding = "UTF-8", pathToJsonFiles = path.getCanonicalPath, - expectedContent = """{"_1":"Dog","_2":42}""" - ) + expectedContent = """{"_1":"Dog","_2":42}""") } } @@ -2348,8 +2346,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } assert(exception.getMessage.contains( - s"""The lineSep option must be specified for the $encoding encoding""" - )) + s"""The lineSep option must be specified for the $encoding encoding""")) } private val badJson = "\u0000\u0000\u0000A\u0001AAA" From 6a626794ba8c48ab82c9b7a5a7b9ccea58c3fa60 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Apr 2018 14:35:56 +0200 Subject: [PATCH 80/95] Making lineSep as opt string --- .../sql/execution/datasources/text/TextOptions.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index 53b95f01c461..4284d50c438b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -43,14 +43,16 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti val encoding: Option[String] = parameters.get(ENCODING) - val lineSeparator: Option[Array[Byte]] = parameters.get(LINE_SEPARATOR).map { lineSep => + val lineSeparator: Option[String] = parameters.get(LINE_SEPARATOR).map { lineSep => require(lineSep.nonEmpty, s"'$LINE_SEPARATOR' cannot be an empty string.") - lineSep.getBytes(encoding.getOrElse("UTF-8")) + lineSep } // Note that the option 'lineSep' uses a different default value in read and write. - val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator + val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => + lineSep.getBytes(encoding.getOrElse("UTF-8")) + } val lineSeparatorInWrite: Array[Byte] = lineSeparatorInRead.getOrElse("\n".getBytes("UTF-8")) } From 3b30ce036fbd2a8d6b9b2cf40a418624ecccda25 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Apr 2018 15:09:28 +0200 Subject: [PATCH 81/95] Removing option name in a test --- .../datasources/json/JsonSuite.scala | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 3e807e3500bc..a772b1762c1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2283,9 +2283,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - def checkReadJson(lineSep: String, encodingOption: String, encoding: String, - inferSchema: Boolean, runId: Int): Unit = { - test(s"SPARK-23724: checks reading json in ${encoding} #${runId}") { + def checkReadJson(lineSep: String, encoding: String, inferSchema: Boolean, id: Int): Unit = { + test(s"SPARK-23724: checks reading json in ${encoding} #${id}") { val lineSepInBytes = { if (lineSep.startsWith("x")) { lineSep.replaceAll("[^0-9A-Fa-f]", "") @@ -2309,7 +2308,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema) } val readBack = reader - .option(encodingOption, encoding) + .option("encoding", encoding) .option("lineSep", lineSep) .json(path.getCanonicalPath) checkAnswer(readBack, records.map(rec => Row(rec._1, rec._2))) @@ -2319,21 +2318,21 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:off nonascii List( - ("|", "encoding", "UTF-8", false), - ("^", "charset", "UTF-16BE", true), - ("::", "encoding", "ISO-8859-1", true), - ("!!!@3", "encoding", "UTF-32LE", false), - (0x1E.toChar.toString, "charset", "UTF-8", true), - ("ì•„", "encoding", "UTF-32BE", false), - ("куку", "encoding", "CP1251", true), - ("sep", "encoding", "utf-8", false), - ("\r\n", "encoding", "UTF-16LE", false), - ("\r\n", "encoding", "utf-16be", true), - ("\u000d\u000a", "encoding", "UTF-32BE", false), - ("\u000a\u000d", "encoding", "UTF-8", true), - ("===", "encoding", "US-ASCII", false), - ("$^+", "encoding", "utf-32le", true) - ).zipWithIndex.foreach{case ((d, o, c, s), i) => checkReadJson(d, o, c, s, i)} + (0, "|", "UTF-8", false), + (1, "^", "UTF-16BE", true), + (2, "::", "ISO-8859-1", true), + (3, "!!!@3", "UTF-32LE", false), + (4, 0x1E.toChar.toString, "UTF-8", true), + (5, "ì•„", "UTF-32BE", false), + (6, "куку", "CP1251", true), + (7, "sep", "utf-8", false), + (8, "\r\n", "UTF-16LE", false), + (9, "\r\n", "utf-16be", true), + (10, "\u000d\u000a", "UTF-32BE", false), + (11, "\u000a\u000d", "UTF-8", true), + (12, "===", "US-ASCII", false), + (13, "$^+", "utf-32le", true) + ).foreach{case (i, d, c, s) => checkReadJson(d, c, s, i)} // scalastyle:on nonascii test("SPARK-23724: lineSep should be set if encoding if different from UTF-8") { From af71324bf7504be9070e0835f4d2ca0f366e6ac5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 8 Apr 2018 21:22:26 +0200 Subject: [PATCH 82/95] Addressing HyukjinKwon's review comments --- python/pyspark/sql/readwriter.py | 2 +- .../spark/sql/catalyst/json/JSONOptions.scala | 18 ++++++++++++++---- .../org/apache/spark/sql/DataFrameReader.scala | 2 +- .../datasources/json/JsonDataSource.scala | 2 +- .../execution/datasources/json/JsonSuite.scala | 9 +++------ 5 files changed, 20 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index daa2c7a5b79d..12a62ba55470 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -238,7 +238,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, characters (ASCII characters with value less than 32, including tab and line feed characters) or not. :param encoding: standard encoding (charset) name, for example UTF-8, UTF-16LE and UTF-32BE. - If None is set, the encoding of input JSON will be detected automatically. + If None is set, the encoding of input JSON will be detected automatically. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index a230bde0fe6a..ea6e9565a814 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.json +import java.nio.charset.Charset import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} @@ -88,7 +89,10 @@ private[sql] class JSONOptions( /** * A string between two consecutive JSON records. */ - val lineSeparator: Option[String] = parameters.get("lineSep") + val lineSeparator: Option[String] = parameters.get("lineSep").map { sep => + require(sep.nonEmpty, "'lineSep' cannot be an empty string.") + sep + } /** * Standard encoding (charset) name. For example UTF-8, UTF-16LE and UTF-32BE. @@ -96,10 +100,16 @@ private[sql] class JSONOptions( */ val encoding: Option[String] = parameters.get("encoding") .orElse(parameters.get("charset")).map { enc => - val blacklist = List("UTF16", "UTF32") - val isBlacklisted = blacklist.contains(enc.toUpperCase.replaceAll("-|_", "")) + // The following encodings are not supported in per-line mode (multiline is false) + // because they cause some problems in reading files with BOM which is supposed to + // present in the files with such encodings. After splitting input files by lines, + // only the first lines will have the BOM which leads to impossibility for reading + // the rest lines. Besides of that, the lineSep option must have the BOM in such + // encodings which can never present between lines. + val blacklist = Seq(Charset.forName("UTF-16"), Charset.forName("UTF-32")) + val isBlacklisted = blacklist.contains(Charset.forName(enc)) require(multiLine || !isBlacklisted, - s"""The ${enc} encoding must not be included in the blacklist: + s"""The ${enc} encoding must not be included in the blacklist when multiLine is disabled: | ${blacklist.mkString(", ")}""".stripMargin) val forcingLineSep = !(multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index d479aead71fe..c8d5798a3be0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -367,7 +367,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `encoding` (by default it is not set): allows to forcibly set one of standard basic - * or extended charsets for input jsons. For example UTF-8, UTF-16BE, UTF-32. If the encoding + * or extended encoding for the JSON files. For example UTF-8, UTF-16BE, UTF-32LE. If the encoding * is not specified (by default), it will be detected automatically.
  • *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
  • diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 7848b401a51e..572091bdd7ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -191,7 +191,7 @@ object MultiLineJsonDataSource extends JsonDataSource { private def createParser( jsonFactory: JsonFactory, record: PortableDataStream, - encoding: Option[String] = None): JsonParser = { + encoding: Option[String]): JsonParser = { val path = new Path(record.getPath()) val is = CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index a772b1762c1b..a9a7ee599334 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.json import java.io.{File, FileOutputStream, StringWriter} -import java.nio.charset.StandardCharsets +import java.nio.charset.{StandardCharsets, UnsupportedCharsetException} import java.nio.file.Files import java.sql.{Date, Timestamp} import java.util.Locale @@ -2166,7 +2166,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: Unsupported encoding name") { val invalidCharset = "UTF-128" - val exception = intercept[java.io.UnsupportedEncodingException] { + val exception = intercept[UnsupportedCharsetException] { spark.read .options(Map("encoding" -> invalidCharset, "lineSep" -> "\n")) .json(testFile("json-tests/utf16LE.json")) @@ -2251,7 +2251,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: wrong output encoding") { val encoding = "UTF-128" - val exception = intercept[java.io.UnsupportedEncodingException] { + val exception = intercept[UnsupportedCharsetException] { withTempPath { path => val df = spark.createDataset(Seq((0))) df.write @@ -2351,7 +2351,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { private val badJson = "\u0000\u0000\u0000A\u0001AAA" test("SPARK-23094: invalid json with leading nulls - from file (multiLine=true)") { - import testImplicits._ withTempDir { tempDir => val path = tempDir.getAbsolutePath Seq(badJson + """{"a":1}""").toDS().write.mode("overwrite").text(path) @@ -2366,7 +2365,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23094: invalid json with leading nulls - from file (multiLine=false)") { - import testImplicits._ withTempDir { tempDir => val path = tempDir.getAbsolutePath Seq(badJson, """{"a":1}""").toDS().write.mode("overwrite").text(path) @@ -2380,7 +2378,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23094: invalid json with leading nulls - from dataset") { - import testImplicits._ checkAnswer( spark.read.option("encoding", "UTF-8").json(Seq(badJson).toDS()), Row(badJson)) From b817184d35d0e2589682f1dcd88b9f29b2063f5b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 8 Apr 2018 21:56:58 +0200 Subject: [PATCH 83/95] Making Scala style checker and compiler happy --- .../apache/spark/sql/execution/datasources/json/JsonSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e699cad8a316..1626fb89609c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.json import java.io.{File, FileOutputStream, StringWriter} import java.nio.charset.{StandardCharsets, UnsupportedCharsetException} -import java.nio.file.Files +import java.nio.file.{Files, Paths, StandardOpenOption} import java.sql.{Date, Timestamp} import java.util.Locale From 36253f4bcf5caaf1945fe526deeca61551e13da4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 14 Apr 2018 09:56:03 +0200 Subject: [PATCH 84/95] Adressing Hyukjin Kwon's review comments --- python/pyspark/sql/readwriter.py | 3 ++- .../spark/sql/catalyst/json/CreateJacksonParser.scala | 9 ++++----- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 8 +++----- .../apache/spark/sql/catalyst/json/JacksonParser.scala | 7 ++----- .../scala/org/apache/spark/sql/DataFrameReader.scala | 4 ++-- .../sql/execution/datasources/json/JsonDataSource.scala | 4 ++-- 6 files changed, 15 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 12a62ba55470..13a81fd54f7f 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -238,7 +238,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, characters (ASCII characters with value less than 32, including tab and line feed characters) or not. :param encoding: standard encoding (charset) name, for example UTF-8, UTF-16LE and UTF-32BE. - If None is set, the encoding of input JSON will be detected automatically. + If None is set, the encoding of input JSON will be detected automatically + when the multiLine option is set to ``true``. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 7966afa7f87c..917b376de577 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -57,15 +57,14 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(new InputStreamReader(is, enc)) } - def internalRow(jsonFactory: JsonFactory, row: InternalRow, field: Int): JsonParser = { - val is = new ByteArrayInputStream(row.getBinary(field)) + def internalRow(jsonFactory: JsonFactory, row: InternalRow): JsonParser = { + val is = new ByteArrayInputStream(row.getBinary(0)) inputStream(jsonFactory, is) } - def internalRow(enc: String, jsonFactory: JsonFactory, - row: InternalRow, field: Int): JsonParser = { - val is = new ByteArrayInputStream(row.getBinary(field)) + def internalRow(enc: String, jsonFactory: JsonFactory, row: InternalRow): JsonParser = { + val is = new ByteArrayInputStream(row.getBinary(0)) inputStream(enc, jsonFactory, is) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index ea6e9565a814..6a92531b0532 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -96,7 +96,8 @@ private[sql] class JSONOptions( /** * Standard encoding (charset) name. For example UTF-8, UTF-16LE and UTF-32BE. - * If the encoding is not specified (None), it will be detected automatically. + * If the encoding is not specified (None), it will be detected automatically + * when the multiLine option is set to `true`. */ val encoding: Option[String] = parameters.get("encoding") .orElse(parameters.get("charset")).map { enc => @@ -113,10 +114,7 @@ private[sql] class JSONOptions( | ${blacklist.mkString(", ")}""".stripMargin) val forcingLineSep = !(multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) - require(forcingLineSep, - s"""The lineSep option must be specified for the $enc encoding. - |Example: .option("lineSep", "|^|") - |Note: lineSep can be detected automatically for UTF-8 only.""".stripMargin) + require(forcingLineSep, s"The lineSep option must be specified for the $enc encoding") enc } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 2c30e939263b..e3caa9948dfe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -363,11 +363,8 @@ class JacksonParser( throw BadRecordException(() => recordLiteral(record), () => None, e) case e: CharConversionException if options.encoding.isEmpty => val msg = - """Failed to parse a character. Encoding was detected automatically. - |You might want to set it explicitly via the encoding option like: - | .option("encoding", "UTF-8") - |Example of supported encodings: - | UTF-8, UTF-16BE, UTF-16LE, UTF-32BE, UTF-32LE + """JSON parser cannot handle a character in its input. + |Specifying encoding as an input option explicitly might help to resolve the issue. |""".stripMargin + e.getMessage throw new CharConversionException(msg) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index c8d5798a3be0..fb44873f14d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -367,8 +367,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `encoding` (by default it is not set): allows to forcibly set one of standard basic - * or extended encoding for the JSON files. For example UTF-8, UTF-16BE, UTF-32LE. If the encoding - * is not specified (by default), it will be detected automatically.
  • + * or extended encoding for the JSON files. For example UTF-16BE, UTF-32LE. If the encoding + * is not specified and `multiLine` is set to `true`, it will be detected automatically. *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
  • * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 572091bdd7ed..a2c3427bda84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -102,8 +102,8 @@ object TextInputJsonDataSource extends JsonDataSource { val sampled: Dataset[String] = JsonUtils.sample(json, parsedOptions) val rdd: RDD[InternalRow] = sampled.queryExecution.toRdd val rowParser = parsedOptions.encoding.map { enc => - CreateJacksonParser.internalRow(enc, _: JsonFactory, _: InternalRow, 0) - }.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow, 0)) + CreateJacksonParser.internalRow(enc, _: JsonFactory, _: InternalRow) + }.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow)) JsonInferSchema.infer(rdd, parsedOptions, rowParser) } From aa69559d2a7f7d215638dfa7a1a4f9e2b2cf63ab Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 15:14:21 +0200 Subject: [PATCH 85/95] Adding benchmarks for json reads --- .../datasources/json/JsonBenchmarks.scala | 130 ++++++++++++++++++ 1 file changed, 130 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala new file mode 100644 index 000000000000..267e0f400aeb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.json + +import java.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.{Benchmark, Utils} + +/** + * Benchmark to measure JSON read/write performance. + * To run this: + * spark-submit --class --jars + */ +object JSONBenchmarks { + val conf = new SparkConf() + + val spark = SparkSession.builder + .master("local[1]") + .appName("benchmark-json-datasource") + .config(conf) + .getOrCreate() + import spark.implicits._ + + def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } + + + def schemaInferring(rowsNum: Int): Unit = { + val benchmark = new Benchmark("JSON schema inferring", rowsNum) + + withTempPath { path => + // scalastyle:off + benchmark.out.println("Preparing data for benchmarking ...") + // scalastyle:on + + spark.sparkContext.range(0, rowsNum, 1) + .map(_ => "a") + .toDF("fieldA") + .write + .option("encoding", "UTF-8") + .json(path.getAbsolutePath) + + benchmark.addCase("No encoding", 3) { _ => + spark.read.json(path.getAbsolutePath) + } + + benchmark.addCase("UTF-8 is set", 3) { _ => + spark.read + .option("encoding", "UTF-8") + .json(path.getAbsolutePath) + } + + /* + Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz + + JSON schema inferring: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + -------------------------------------------------------------------------------------------- + No encoding 41193 / 41484 2.4 411.9 1.0X + UTF-8 is set 109807 / 110674 0.9 1098.1 0.4X + */ + benchmark.run() + } + } + + def perlineParsing(rowsNum: Int): Unit = { + val benchmark = new Benchmark("JSON per-line parsing", rowsNum) + + withTempPath { path => + // scalastyle:off + benchmark.out.println("Preparing data for benchmarking ...") + // scalastyle:on + + spark.sparkContext.range(0, rowsNum, 1) + .map(_ => "a") + .toDF("fieldA") + .write.json(path.getAbsolutePath) + val schema = new StructType().add("fieldA", StringType) + + benchmark.addCase("No encoding", 3) { _ => + spark.read + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.addCase("UTF-8 is set", 3) { _ => + spark.read + .option("encoding", "UTF-8") + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + /* + Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz + + JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + No encoding 28953 / 29274 3.5 289.5 1.0X + UTF-8 is set 96040 / 96888 1.0 960.4 0.3X + */ + benchmark.run() + } + } + + def main(args: Array[String]): Unit = { + schemaInferring(100 * 1000 * 1000) + perlineParsing(100 * 1000 * 1000) + } +} \ No newline at end of file From c35d5d1176a6297467f6a1d80a8c78dd50f39be3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 15:25:58 +0200 Subject: [PATCH 86/95] Making Scala style checker happy --- .../sql/execution/datasources/json/JsonBenchmarks.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 267e0f400aeb..3daf62166d88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -114,10 +114,10 @@ object JSONBenchmarks { /* Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz - JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - No encoding 28953 / 29274 3.5 289.5 1.0X - UTF-8 is set 96040 / 96888 1.0 960.4 0.3X + JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + -------------------------------------------------------------------------------------------- + No encoding 28953 / 29274 3.5 289.5 1.0X + UTF-8 is set 96040 / 96888 1.0 960.4 0.3X */ benchmark.run() } From 58fc5c66f7856af4fd196d8194ab42be083324d0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 18:09:03 +0200 Subject: [PATCH 87/95] Eliminate unneeded wrapping by ByteArrayInputStream per-line during schema inferring --- .../apache/spark/sql/catalyst/json/CreateJacksonParser.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 917b376de577..1e3a8c91db7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -46,6 +46,7 @@ private[sql] object CreateJacksonParser extends Serializable { def text(enc: String, jsonFactory: JsonFactory, record: Text): JsonParser = { val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) + jsonFactory.createParser(new InputStreamReader(bain, enc)) } @@ -58,9 +59,9 @@ private[sql] object CreateJacksonParser extends Serializable { } def internalRow(jsonFactory: JsonFactory, row: InternalRow): JsonParser = { - val is = new ByteArrayInputStream(row.getBinary(0)) + val ba = row.getBinary(0) - inputStream(jsonFactory, is) + jsonFactory.createParser(ba, 0, ba.length) } def internalRow(enc: String, jsonFactory: JsonFactory, row: InternalRow): JsonParser = { From 63b589402c71ba706414f9ce23464dca2339151c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 19:53:34 +0200 Subject: [PATCH 88/95] Adding benchmarks for wide lines --- .../datasources/json/JsonBenchmarks.scala | 52 ++++++++++++++++++- 1 file changed, 51 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 3daf62166d88..9762e225bc59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.execution.datasources.json import java.io.File import org.apache.spark.SparkConf + import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.util.{Benchmark, Utils} /** @@ -123,8 +124,57 @@ object JSONBenchmarks { } } + def perlineParsingOfWideColumn(rowsNum: Int): Unit = { + val benchmark = new Benchmark("JSON parsing of wide lines", rowsNum) + + withTempPath { path => + // scalastyle:off + benchmark.out.println("Preparing data for benchmarking ...") + // scalastyle:on + + spark.sparkContext.range(0, rowsNum, 1) + .map { i => + val s = "abcdef0123456789ABCDEF" * 20 + s"""{"a":"$s","b": $i,"c":"$s","d":$i,"e":"$s","f":$i,"x":"$s","y":$i,"z":"$s"}""" + } + .toDF().write.text(path.getAbsolutePath) + val schema = new StructType() + .add("a", StringType).add("b", LongType) + .add("c", StringType).add("d", LongType) + .add("e", StringType).add("f", LongType) + .add("x", StringType).add("y", LongType) + .add("z", StringType) + + benchmark.addCase("No encoding", 3) { _ => + spark.read + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.addCase("UTF-8 is set", 3) { _ => + spark.read + .option("encoding", "UTF-8") + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + /* + Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz + + JSON parsing of wide lines: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + No encoding 45543 / 45660 0.2 4554.3 1.0X + UTF-8 is set 65737 / 65957 0.2 6573.7 0.7X + */ + benchmark.run() + } + } + def main(args: Array[String]): Unit = { schemaInferring(100 * 1000 * 1000) perlineParsing(100 * 1000 * 1000) + perlineParsingOfWideColumn(10 * 1000 * 1000) } } \ No newline at end of file From 1ace082bffbfa532c18302c1c767992d2c0f25f9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 19:54:25 +0200 Subject: [PATCH 89/95] Making comments shorter --- .../sql/execution/datasources/json/JsonBenchmarks.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 9762e225bc59..65ea367d4d35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -163,10 +163,10 @@ object JSONBenchmarks { /* Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz - JSON parsing of wide lines: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - No encoding 45543 / 45660 0.2 4554.3 1.0X - UTF-8 is set 65737 / 65957 0.2 6573.7 0.7X + JSON parsing of wide lines: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + -------------------------------------------------------------------------------------------- + No encoding 45543 / 45660 0.2 4554.3 1.0X + UTF-8 is set 65737 / 65957 0.2 6573.7 0.7X */ benchmark.run() } From 6c0df038138c2d2f5bcf8579728b673a2c6a5fdc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 19:56:33 +0200 Subject: [PATCH 90/95] Removing empty line between spark's imports --- .../spark/sql/execution/datasources/json/JsonBenchmarks.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 65ea367d4d35..3235ffe22ba7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.datasources.json import java.io.File import org.apache.spark.SparkConf - import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.util.{Benchmark, Utils} From b4c0d383efc63a71a6850cc5f2cdb403d41a3610 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 23:37:32 +0200 Subject: [PATCH 91/95] Creating a stream decoder with specific buffer size --- .../catalyst/json/CreateJacksonParser.scala | 22 ++++++++++++++----- .../datasources/json/JsonDataSource.scala | 5 ++--- .../datasources/json/JsonBenchmarks.scala | 12 +++++----- 3 files changed, 25 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 1e3a8c91db7e..54e8db5584f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -18,9 +18,12 @@ package org.apache.spark.sql.catalyst.json import java.io.{ByteArrayInputStream, InputStream, InputStreamReader} +import java.nio.channels.Channels +import java.nio.charset.Charset import com.fasterxml.jackson.core.{JsonFactory, JsonParser} import org.apache.hadoop.io.Text +import sun.nio.cs.StreamDecoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.unsafe.types.UTF8String @@ -44,10 +47,18 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(record.getBytes, 0, record.getLength) } - def text(enc: String, jsonFactory: JsonFactory, record: Text): JsonParser = { - val bain = new ByteArrayInputStream(record.getBytes, 0, record.getLength) + def getStreamDecoder(enc: String, in: Array[Byte], length: Int): StreamDecoder = { + val bais = new ByteArrayInputStream(in, 0, length) + val byteChannel = Channels.newChannel(bais) + val decodingBufferSize = Math.min(length, 8192) + val decoder = Charset.forName(enc).newDecoder() + + StreamDecoder.forDecoder(byteChannel, decoder, decodingBufferSize) + } - jsonFactory.createParser(new InputStreamReader(bain, enc)) + def text(enc: String, jsonFactory: JsonFactory, record: Text): JsonParser = { + val sd = getStreamDecoder(enc, record.getBytes, record.getLength) + jsonFactory.createParser(sd) } def inputStream(jsonFactory: JsonFactory, is: InputStream): JsonParser = { @@ -65,8 +76,9 @@ private[sql] object CreateJacksonParser extends Serializable { } def internalRow(enc: String, jsonFactory: JsonFactory, row: InternalRow): JsonParser = { - val is = new ByteArrayInputStream(row.getBinary(0)) + val binary = row.getBinary(0) + val sd = getStreamDecoder(enc, binary, binary.length) - inputStream(enc, jsonFactory, is) + jsonFactory.createParser(sd) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index a2c3427bda84..ae645e3909f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -31,12 +31,11 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.spark.TaskContext import org.apache.spark.input.{PortableDataStream, StreamInputFormat} import org.apache.spark.rdd.{BinaryFileRDD, RDD} -import org.apache.spark.sql.{AnalysisException, Dataset, Encoders, SparkSession} +import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.text.{TextFileFormat, TextOptions} +import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 3235ffe22ba7..112a529f1835 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -75,8 +75,8 @@ object JSONBenchmarks { JSON schema inferring: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------- - No encoding 41193 / 41484 2.4 411.9 1.0X - UTF-8 is set 109807 / 110674 0.9 1098.1 0.4X + No encoding 38902 / 39282 2.6 389.0 1.0X + UTF-8 is set 56959 / 57261 1.8 569.6 0.7X */ benchmark.run() } @@ -116,8 +116,8 @@ object JSONBenchmarks { JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------- - No encoding 28953 / 29274 3.5 289.5 1.0X - UTF-8 is set 96040 / 96888 1.0 960.4 0.3X + No encoding 25947 / 26188 3.9 259.5 1.0X + UTF-8 is set 46319 / 46417 2.2 463.2 0.6X */ benchmark.run() } @@ -172,8 +172,8 @@ object JSONBenchmarks { } def main(args: Array[String]): Unit = { - schemaInferring(100 * 1000 * 1000) - perlineParsing(100 * 1000 * 1000) + // schemaInferring(100 * 1000 * 1000) + // perlineParsing(100 * 1000 * 1000) perlineParsingOfWideColumn(10 * 1000 * 1000) } } \ No newline at end of file From f2a259ffb0a322daf4aca5a127ec990071d5f935 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Apr 2018 23:38:10 +0200 Subject: [PATCH 92/95] Enable all JSON benchmarks --- .../spark/sql/execution/datasources/json/JsonBenchmarks.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 112a529f1835..0852bd314907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -172,8 +172,8 @@ object JSONBenchmarks { } def main(args: Array[String]): Unit = { - // schemaInferring(100 * 1000 * 1000) - // perlineParsing(100 * 1000 * 1000) + schemaInferring(100 * 1000 * 1000) + perlineParsing(100 * 1000 * 1000) perlineParsingOfWideColumn(10 * 1000 * 1000) } } \ No newline at end of file From 482b79969b9e0cc475e63b415051b32423facef4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Apr 2018 21:51:05 +0200 Subject: [PATCH 93/95] Addressing Hyukjin Kwon's review comments --- .../datasources/json/JsonDataSource.scala | 27 ++++++++++--------- .../datasources/text/TextOptions.scala | 6 ++--- .../datasources/json/JsonBenchmarks.scala | 2 +- 3 files changed, 19 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index ae645e3909f2..983a5f0dcade 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -161,11 +161,11 @@ object MultiLineJsonDataSource extends JsonDataSource { parsedOptions: JSONOptions): StructType = { val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths) val sampled: RDD[PortableDataStream] = JsonUtils.sample(json, parsedOptions) + val parser = parsedOptions.encoding + .map(enc => createParser(enc, _: JsonFactory, _: PortableDataStream)) + .getOrElse(createParser(_: JsonFactory, _: PortableDataStream)) - JsonInferSchema.infer[PortableDataStream]( - sampled, - parsedOptions, - createParser(_, _, parsedOptions.encoding)) + JsonInferSchema.infer[PortableDataStream](sampled, parsedOptions, parser) } private def createBaseRdd( @@ -187,15 +187,18 @@ object MultiLineJsonDataSource extends JsonDataSource { .values } - private def createParser( - jsonFactory: JsonFactory, - record: PortableDataStream, - encoding: Option[String]): JsonParser = { - val path = new Path(record.getPath()) - val is = CodecStreams.createInputStreamWithCloseResource(record.getConfiguration, path) + private def dataToInputStream(dataStream: PortableDataStream): InputStream = { + val path = new Path(dataStream.getPath()) + CodecStreams.createInputStreamWithCloseResource(dataStream.getConfiguration, path) + } + + private def createParser(jsonFactory: JsonFactory, stream: PortableDataStream): JsonParser = { + CreateJacksonParser.inputStream(jsonFactory, dataToInputStream(stream)) + } - encoding.map(enc => CreateJacksonParser.inputStream(enc, jsonFactory, is)) - .getOrElse(CreateJacksonParser.inputStream(jsonFactory, is)) + private def createParser(enc: String, jsonFactory: JsonFactory, + stream: PortableDataStream): JsonParser = { + CreateJacksonParser.inputStream(enc, jsonFactory, dataToInputStream(stream)) } override def readFile( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index 4284d50c438b..e4e201995faa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.text -import java.nio.charset.StandardCharsets +import java.nio.charset.{Charset, StandardCharsets} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CompressionCodecs} @@ -51,10 +51,10 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti // Note that the option 'lineSep' uses a different default value in read and write. val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => - lineSep.getBytes(encoding.getOrElse("UTF-8")) + lineSep.getBytes(encoding.map(Charset.forName(_)).getOrElse(StandardCharsets.UTF_8)) } val lineSeparatorInWrite: Array[Byte] = - lineSeparatorInRead.getOrElse("\n".getBytes("UTF-8")) + lineSeparatorInRead.getOrElse("\n".getBytes(StandardCharsets.UTF_8)) } private[datasources] object TextOptions { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 0852bd314907..9406500a1021 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -176,4 +176,4 @@ object JSONBenchmarks { perlineParsing(100 * 1000 * 1000) perlineParsingOfWideColumn(10 * 1000 * 1000) } -} \ No newline at end of file +} From a0ab98b0c11aa2744b8ee5108b49e3c7cf55608c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 23 Apr 2018 22:35:48 +0200 Subject: [PATCH 94/95] Addressing Wenchen Fan's review comments --- .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 6 ++++-- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 4 +++- .../sql/execution/datasources/json/JsonBenchmarks.scala | 2 +- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 6a92531b0532..bfd2ad7ce119 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.json -import java.nio.charset.Charset +import java.nio.charset.{Charset, StandardCharsets} import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} @@ -113,8 +113,10 @@ private[sql] class JSONOptions( s"""The ${enc} encoding must not be included in the blacklist when multiLine is disabled: | ${blacklist.mkString(", ")}""".stripMargin) - val forcingLineSep = !(multiLine == false && enc != "UTF-8" && lineSeparator.isEmpty) + val forcingLineSep = !(multiLine == false && + Charset.forName(enc) != StandardCharsets.UTF_8 && lineSeparator.isEmpty) require(forcingLineSep, s"The lineSep option must be specified for the $enc encoding") + enc } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index e3caa9948dfe..a5a4a13eb608 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -366,7 +366,9 @@ class JacksonParser( """JSON parser cannot handle a character in its input. |Specifying encoding as an input option explicitly might help to resolve the issue. |""".stripMargin + e.getMessage - throw new CharConversionException(msg) + val wrappedCharException = new CharConversionException(msg) + wrappedCharException.initCause(e) + throw BadRecordException(() => recordLiteral(record), () => None, wrappedCharException) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 0852bd314907..9406500a1021 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -176,4 +176,4 @@ object JSONBenchmarks { perlineParsing(100 * 1000 * 1000) perlineParsingOfWideColumn(10 * 1000 * 1000) } -} \ No newline at end of file +} From d3d28aa852dc90acc898df5b7a4e38135b0daf10 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 28 Apr 2018 21:25:08 +0200 Subject: [PATCH 95/95] Addressing Hyukjin Kwon's review comments --- python/pyspark/sql/readwriter.py | 5 +- .../catalyst/json/CreateJacksonParser.scala | 12 ++- .../spark/sql/catalyst/json/JSONOptions.scala | 4 +- .../{json-tests => test-data}/utf16LE.json | Bin .../utf16WithBOM.json | Bin .../utf32BEWithBOM.json | Bin .../datasources/json/JsonBenchmarks.scala | 14 +-- .../datasources/json/JsonSuite.scala | 82 ++++++++---------- 8 files changed, 57 insertions(+), 60 deletions(-) rename sql/core/src/test/resources/{json-tests => test-data}/utf16LE.json (100%) rename sql/core/src/test/resources/{json-tests => test-data}/utf16WithBOM.json (100%) rename sql/core/src/test/resources/{json-tests => test-data}/utf32BEWithBOM.json (100%) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 1fd6e2ac7d1d..6811fa6b3b15 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -238,8 +238,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not. - :param encoding: standard encoding (charset) name, for example UTF-8, UTF-16LE and UTF-32BE. - If None is set, the encoding of input JSON will be detected automatically + :param encoding: allows to forcibly set one of standard basic or extended encoding for + the JSON files. For example UTF-16BE, UTF-32LE. If None is set, + the encoding of input JSON will be detected automatically when the multiLine option is set to ``true``. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala index 54e8db5584f1..3e8e6db1dbd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/CreateJacksonParser.scala @@ -47,7 +47,17 @@ private[sql] object CreateJacksonParser extends Serializable { jsonFactory.createParser(record.getBytes, 0, record.getLength) } - def getStreamDecoder(enc: String, in: Array[Byte], length: Int): StreamDecoder = { + // Jackson parsers can be ranked according to their performance: + // 1. Array based with actual encoding UTF-8 in the array. This is the fastest parser + // but it doesn't allow to set encoding explicitly. Actual encoding is detected automatically + // by checking leading bytes of the array. + // 2. InputStream based with actual encoding UTF-8 in the stream. Encoding is detected + // automatically by analyzing first bytes of the input stream. + // 3. Reader based parser. This is the slowest parser used here but it allows to create + // a reader with specific encoding. + // The method creates a reader for an array with given encoding and sets size of internal + // decoding buffer according to size of input array. + private def getStreamDecoder(enc: String, in: Array[Byte], length: Int): StreamDecoder = { val bais = new ByteArrayInputStream(in, 0, length) val byteChannel = Channels.newChannel(bais) val decodingBufferSize = Math.min(length, 8192) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index bfd2ad7ce119..5f130af606e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -113,9 +113,9 @@ private[sql] class JSONOptions( s"""The ${enc} encoding must not be included in the blacklist when multiLine is disabled: | ${blacklist.mkString(", ")}""".stripMargin) - val forcingLineSep = !(multiLine == false && + val isLineSepRequired = !(multiLine == false && Charset.forName(enc) != StandardCharsets.UTF_8 && lineSeparator.isEmpty) - require(forcingLineSep, s"The lineSep option must be specified for the $enc encoding") + require(isLineSepRequired, s"The lineSep option must be specified for the $enc encoding") enc } diff --git a/sql/core/src/test/resources/json-tests/utf16LE.json b/sql/core/src/test/resources/test-data/utf16LE.json similarity index 100% rename from sql/core/src/test/resources/json-tests/utf16LE.json rename to sql/core/src/test/resources/test-data/utf16LE.json diff --git a/sql/core/src/test/resources/json-tests/utf16WithBOM.json b/sql/core/src/test/resources/test-data/utf16WithBOM.json similarity index 100% rename from sql/core/src/test/resources/json-tests/utf16WithBOM.json rename to sql/core/src/test/resources/test-data/utf16WithBOM.json diff --git a/sql/core/src/test/resources/json-tests/utf32BEWithBOM.json b/sql/core/src/test/resources/test-data/utf32BEWithBOM.json similarity index 100% rename from sql/core/src/test/resources/json-tests/utf32BEWithBOM.json rename to sql/core/src/test/resources/test-data/utf32BEWithBOM.json diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index 9406500a1021..85cf054e51f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.util.{Benchmark, Utils} /** - * Benchmark to measure JSON read/write performance. + * The benchmarks aims to measure performance of JSON parsing when encoding is set and isn't. * To run this: * spark-submit --class --jars */ @@ -49,9 +49,9 @@ object JSONBenchmarks { val benchmark = new Benchmark("JSON schema inferring", rowsNum) withTempPath { path => - // scalastyle:off + // scalastyle:off println benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on + // scalastyle:on println spark.sparkContext.range(0, rowsNum, 1) .map(_ => "a") @@ -86,9 +86,9 @@ object JSONBenchmarks { val benchmark = new Benchmark("JSON per-line parsing", rowsNum) withTempPath { path => - // scalastyle:off + // scalastyle:off println benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on + // scalastyle:on println spark.sparkContext.range(0, rowsNum, 1) .map(_ => "a") @@ -127,9 +127,9 @@ object JSONBenchmarks { val benchmark = new Benchmark("JSON parsing of wide lines", rowsNum) withTempPath { path => - // scalastyle:off + // scalastyle:off println benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on + // scalastyle:on println spark.sparkContext.range(0, rowsNum, 1) .map { i => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 989c7834d4b9..0db688fec9a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -48,6 +48,10 @@ class TestFileFilter extends PathFilter { class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { import testImplicits._ + def testFile(fileName: String): String = { + Thread.currentThread().getContextClassLoader.getResource(fileName).toString + } + test("Type promotion") { def checkTypePromotion(expected: Any, actual: Any) { assert(expected.getClass == actual.getClass, @@ -2168,12 +2172,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(sampled.count() == ds.count()) } - def testFile(fileName: String): String = { - Thread.currentThread().getContextClassLoader.getResource(fileName).toString - } - test("SPARK-23723: json in UTF-16 with BOM") { - val fileName = "json-tests/utf16WithBOM.json" + val fileName = "test-data/utf16WithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") @@ -2184,7 +2184,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23723: multi-line json in UTF-32BE with BOM") { - val fileName = "json-tests/utf32BEWithBOM.json" + val fileName = "test-data/utf32BEWithBOM.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") @@ -2194,7 +2194,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23723: Use user's encoding in reading of multi-line json in UTF-16LE") { - val fileName = "json-tests/utf16LE.json" + val fileName = "test-data/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") @@ -2209,7 +2209,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val exception = intercept[UnsupportedCharsetException] { spark.read .options(Map("encoding" -> invalidCharset, "lineSep" -> "\n")) - .json(testFile("json-tests/utf16LE.json")) + .json(testFile("test-data/utf16LE.json")) .count() } @@ -2217,7 +2217,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-23723: checking that the encoding option is case agnostic") { - val fileName = "json-tests/utf16LE.json" + val fileName = "test-data/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val jsonDF = spark.read.schema(schema) .option("multiline", "true") @@ -2229,7 +2229,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: specified encoding is not matched to actual encoding") { - val fileName = "json-tests/utf16LE.json" + val fileName = "test-data/utf16LE.json" val schema = new StructType().add("firstName", StringType).add("lastName", StringType) val exception = intercept[SparkException] { spark.read.schema(schema) @@ -2244,9 +2244,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(errMsg.contains("Malformed records are detected in record parsing")) } - def checkEncoding( - expectedEncoding: String, - pathToJsonFiles: String, + def checkEncoding(expectedEncoding: String, pathToJsonFiles: String, expectedContent: String): Unit = { val jsonFiles = new File(pathToJsonFiles) .listFiles() @@ -2254,7 +2252,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .filter(_.getName.endsWith("json")) val actualContent = jsonFiles.map { file => new String(Files.readAllBytes(file.toPath), expectedEncoding) - }.mkString.trim.replaceAll(" ", "") + }.mkString.trim assert(actualContent == expectedContent) } @@ -2265,8 +2263,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val df = spark.createDataset(Seq(("Dog", 42))) df.write .options(Map("encoding" -> encoding, "lineSep" -> "\n")) - .format("json").mode("overwrite") - .save(path.getCanonicalPath) + .json(path.getCanonicalPath) checkEncoding( expectedEncoding = encoding, @@ -2278,9 +2275,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-23723: save json in default encoding - UTF-8") { withTempPath { path => val df = spark.createDataset(Seq(("Dog", 42))) - df.write - .format("json").mode("overwrite") - .save(path.getCanonicalPath) + df.write.json(path.getCanonicalPath) checkEncoding( expectedEncoding = "UTF-8", @@ -2296,24 +2291,19 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val df = spark.createDataset(Seq((0))) df.write .options(Map("encoding" -> encoding, "lineSep" -> "\n")) - .format("json").mode("overwrite") - .save(path.getCanonicalPath) + .json(path.getCanonicalPath) } } assert(exception.getMessage == encoding) } - test("SPARK-23723: read written json in UTF-16LE") { + test("SPARK-23723: read back json in UTF-16LE") { val options = Map("encoding" -> "UTF-16LE", "lineSep" -> "\n") withTempPath { path => - val ds = spark.createDataset(Seq( - ("a", 1), ("b", 2), ("c", 3)) - ).repartition(2) - ds.write - .options(options) - .format("json").mode("overwrite") - .save(path.getCanonicalPath) + val ds = spark.createDataset(Seq(("a", 1), ("b", 2), ("c", 3))).repartition(2) + ds.write.options(options).json(path.getCanonicalPath) + val readBack = spark .read .options(options) @@ -2325,20 +2315,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { def checkReadJson(lineSep: String, encoding: String, inferSchema: Boolean, id: Int): Unit = { test(s"SPARK-23724: checks reading json in ${encoding} #${id}") { - val lineSepInBytes = { - if (lineSep.startsWith("x")) { - lineSep.replaceAll("[^0-9A-Fa-f]", "") - .sliding(2, 2).toArray.map(Integer.parseInt(_, 16).toByte) - } else { - lineSep.getBytes(encoding) - } - } val schema = new StructType().add("f1", StringType).add("f2", IntegerType) withTempPath { path => val records = List(("a", 1), ("b", 2)) val data = records .map(rec => s"""{"f1":"${rec._1}", "f2":${rec._2}}""".getBytes(encoding)) - .reduce((a1, a2) => a1 ++ lineSepInBytes ++ a2) + .reduce((a1, a2) => a1 ++ lineSep.getBytes(encoding) ++ a2) val os = new FileOutputStream(path) os.write(data) os.close() @@ -2372,7 +2354,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { (11, "\u000a\u000d", "UTF-8", true), (12, "===", "US-ASCII", false), (13, "$^+", "utf-32le", true) - ).foreach{case (i, d, c, s) => checkReadJson(d, c, s, i)} + ).foreach { + case (testNum, sep, encoding, inferSchema) => checkReadJson(sep, encoding, inferSchema, testNum) + } // scalastyle:on nonascii test("SPARK-23724: lineSep should be set if encoding if different from UTF-8") { @@ -2380,7 +2364,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val exception = intercept[IllegalArgumentException] { spark.read .options(Map("encoding" -> encoding)) - .json(testFile("json-tests/utf16LE.json")) + .json(testFile("test-data/utf16LE.json")) .count() } @@ -2390,13 +2374,14 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { private val badJson = "\u0000\u0000\u0000A\u0001AAA" - test("SPARK-23094: invalid json with leading nulls - from file (multiLine=true)") { - withTempDir { tempDir => + test("SPARK-23094: permissively read JSON file with leading nulls when multiLine is enabled") { + withTempPath { tempDir => val path = tempDir.getAbsolutePath - Seq(badJson + """{"a":1}""").toDS().write.mode("overwrite").text(path) + Seq(badJson + """{"a":1}""").toDS().write.text(path) val expected = s"""${badJson}{"a":1}\n""" val schema = new StructType().add("a", IntegerType).add("_corrupt_record", StringType) val df = spark.read.format("json") + .option("mode", "PERMISSIVE") .option("multiLine", true) .option("encoding", "UTF-8") .schema(schema).load(path) @@ -2404,12 +2389,13 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("SPARK-23094: invalid json with leading nulls - from file (multiLine=false)") { - withTempDir { tempDir => + test("SPARK-23094: permissively read JSON file with leading nulls when multiLine is disabled") { + withTempPath { tempDir => val path = tempDir.getAbsolutePath - Seq(badJson, """{"a":1}""").toDS().write.mode("overwrite").text(path) + Seq(badJson, """{"a":1}""").toDS().write.text(path) val schema = new StructType().add("a", IntegerType).add("_corrupt_record", StringType) val df = spark.read.format("json") + .option("mode", "PERMISSIVE") .option("multiLine", false) .option("encoding", "UTF-8") .schema(schema).load(path) @@ -2417,9 +2403,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("SPARK-23094: invalid json with leading nulls - from dataset") { + test("SPARK-23094: permissively parse a dataset contains JSON with leading nulls") { checkAnswer( - spark.read.option("encoding", "UTF-8").json(Seq(badJson).toDS()), + spark.read.option("mode", "PERMISSIVE").option("encoding", "UTF-8").json(Seq(badJson).toDS()), Row(badJson)) } }