Skip to content

Commit 168eb99

Browse files
committed
A SQL config for bypassing parser in the case of empty schema
1 parent 359c4fc commit 168eb99

File tree

3 files changed

+29
-13
lines changed

3 files changed

+29
-13
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1449,6 +1449,14 @@ object SQLConf {
14491449
.intConf
14501450
.checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
14511451
.createWithDefault(Deflater.DEFAULT_COMPRESSION)
1452+
1453+
val BYPASS_PARSER_FOR_EMPTY_SCHEMA = buildConf("spark.sql.bypassParserForEmptySchema")
1454+
.doc("If required schema passed to a text datasource is empty, the parameter controls " +
1455+
"invocation of underlying parser. For example, if it is set to false, uniVocity parser " +
1456+
"is invoke by CSV datasource or Jackson parser by JSON datasource. By default, it is set " +
1457+
"to true which means the parsers is not invoked for empty required schema.")
1458+
.booleanConf
1459+
.createWithDefault(true)
14521460
}
14531461

14541462
/**
@@ -1839,6 +1847,8 @@ class SQLConf extends Serializable with Logging {
18391847

18401848
def avroDeflateLevel: Int = getConf(SQLConf.AVRO_DEFLATE_LEVEL)
18411849

1850+
def bypassParserForEmptySchema: Boolean = getConf(SQLConf.BYPASS_PARSER_FOR_EMPTY_SCHEMA)
1851+
18421852
/** ********************** SQLConf functionality methods ************ */
18431853

18441854
/** Set Spark SQL configuration properties. */

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,11 @@
1818
package org.apache.spark.sql.execution.datasources
1919

2020
import org.apache.spark.SparkException
21+
2122
import org.apache.spark.sql.catalyst.InternalRow
2223
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
2324
import org.apache.spark.sql.catalyst.util._
25+
import org.apache.spark.sql.internal.SQLConf
2426
import org.apache.spark.sql.types.StructType
2527
import org.apache.spark.unsafe.types.UTF8String
2628

@@ -57,7 +59,9 @@ class FailureSafeParser[IN](
5759
}
5860
}
5961

60-
private val skipParsing = optimizeEmptySchema && schema.isEmpty
62+
private val skipParsing = {
63+
SQLConf.get.bypassParserForEmptySchema && optimizeEmptySchema && schema.isEmpty
64+
}
6165
def parse(input: IN): Iterator[InternalRow] = {
6266
try {
6367
if (skipParsing) {

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -2225,19 +2225,21 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
22252225

22262226

22272227
test("SPARK-23723: specified encoding is not matched to actual encoding") {
2228-
val fileName = "test-data/utf16LE.json"
2229-
val schema = new StructType().add("firstName", StringType).add("lastName", StringType)
2230-
val exception = intercept[SparkException] {
2231-
spark.read.schema(schema)
2232-
.option("mode", "FAILFAST")
2233-
.option("multiline", "true")
2234-
.options(Map("encoding" -> "UTF-16BE"))
2235-
.json(testFile(fileName))
2236-
.collect()
2237-
}
2238-
val errMsg = exception.getMessage
2228+
withSQLConf(SQLConf.BYPASS_PARSER_FOR_EMPTY_SCHEMA.key -> "false") {
2229+
val fileName = "test-data/utf16LE.json"
2230+
val schema = new StructType().add("firstName", StringType).add("lastName", StringType)
2231+
val exception = intercept[SparkException] {
2232+
spark.read.schema(schema)
2233+
.option("mode", "FAILFAST")
2234+
.option("multiline", "true")
2235+
.options(Map("encoding" -> "UTF-16BE"))
2236+
.json(testFile(fileName))
2237+
.count()
2238+
}
2239+
val errMsg = exception.getMessage
22392240

2240-
assert(errMsg.contains("Malformed records are detected in record parsing"))
2241+
assert(errMsg.contains("Malformed records are detected in record parsing"))
2242+
}
22412243
}
22422244

22432245
def checkEncoding(expectedEncoding: String, pathToJsonFiles: String,

0 commit comments

Comments
 (0)