Skip to content

Commit 3ec9b05

Browse files
awdavidsonHyukjinKwon
authored andcommitted
[SPARK-40819][SQL][3.3] Timestamp nanos behaviour regression
As per HyukjinKwon request on #38312 to backport fix into 3.3 ### What changes were proposed in this pull request? Handle `TimeUnit.NANOS` for parquet `Timestamps` addressing a regression in behaviour since 3.2 ### Why are the changes needed? Since version 3.2 reading parquet files that contain attributes with type `TIMESTAMP(NANOS,true)` is not possible as ParquetSchemaConverter returns ``` Caused by: org.apache.spark.sql.AnalysisException: Illegal Parquet type: INT64 (TIMESTAMP(NANOS,true)) ``` https://issues.apache.org/jira/browse/SPARK-34661 introduced a change matching on the `LogicalTypeAnnotation` which only covers Timestamp cases for `TimeUnit.MILLIS` and `TimeUnit.MICROS` meaning `TimeUnit.NANOS` would return `illegalType()` Prior to 3.2 the matching used the `originalType` which for `TIMESTAMP(NANOS,true)` return `null` and therefore resulted to a `LongType`, the change proposed is too consider `TimeUnit.NANOS` and return `LongType` making behaviour the same as before. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit test covering this scenario. Internally deployed to read parquet files that contain `TIMESTAMP(NANOS,true)` Closes #39904 from awdavidson/ts-nanos-fix-3.3. Lead-authored-by: alfreddavidson <[email protected]> Co-authored-by: awdavidson <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
1 parent 51ed6ba commit 3ec9b05

File tree

7 files changed

+104
-12
lines changed

7 files changed

+104
-12
lines changed

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

+9
Original file line numberDiff line numberDiff line change
@@ -3459,6 +3459,13 @@ object SQLConf {
34593459
.booleanConf
34603460
.createWithDefault(false)
34613461

3462+
val LEGACY_PARQUET_NANOS_AS_LONG = buildConf("spark.sql.legacy.parquet.nanosAsLong")
3463+
.internal()
3464+
.doc("When true, the Parquet's nanos precision timestamps are converted to SQL long values.")
3465+
.version("3.2.3")
3466+
.booleanConf
3467+
.createWithDefault(false)
3468+
34623469
val PARQUET_INT96_REBASE_MODE_IN_WRITE =
34633470
buildConf("spark.sql.parquet.int96RebaseModeInWrite")
34643471
.internal()
@@ -4525,6 +4532,8 @@ class SQLConf extends Serializable with Logging {
45254532

45264533
def ignoreMissingParquetFieldId: Boolean = getConf(SQLConf.IGNORE_MISSING_PARQUET_FIELD_ID)
45274534

4535+
def legacyParquetNanosAsLong: Boolean = getConf(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG)
4536+
45284537
def useV1Command: Boolean = getConf(SQLConf.LEGACY_USE_V1_COMMAND)
45294538

45304539
def histogramNumericPropagateInputType: Boolean =

sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java

+2
Original file line numberDiff line numberDiff line change
@@ -149,6 +149,7 @@ protected void initialize(String path, List<String> columns) throws IOException
149149
config.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key() , false);
150150
config.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), false);
151151
config.setBoolean(SQLConf.CASE_SENSITIVE().key(), false);
152+
config.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG().key(), false);
152153

153154
this.file = new Path(path);
154155
long length = this.file.getFileSystem(config).getFileStatus(this.file).getLen();
@@ -199,6 +200,7 @@ protected void initialize(
199200
config.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key() , false);
200201
config.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), false);
201202
config.setBoolean(SQLConf.CASE_SENSITIVE().key(), false);
203+
config.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG().key(), false);
202204
this.parquetColumn = new ParquetToSparkSchemaConverter(config)
203205
.convertParquetColumn(requestedSchema, Option.empty());
204206
this.sparkSchema = (StructType) parquetColumn.sparkType();

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

+12-2
Original file line numberDiff line numberDiff line change
@@ -123,6 +123,10 @@ class ParquetFileFormat
123123
SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key,
124124
sparkSession.sessionState.conf.parquetFieldIdWriteEnabled.toString)
125125

126+
conf.set(
127+
SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key,
128+
sparkSession.sessionState.conf.legacyParquetNanosAsLong.toString)
129+
126130
// Sets compression scheme
127131
conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName)
128132

@@ -239,6 +243,9 @@ class ParquetFileFormat
239243
hadoopConf.setBoolean(
240244
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
241245
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
246+
hadoopConf.setBoolean(
247+
SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key,
248+
sparkSession.sessionState.conf.legacyParquetNanosAsLong)
242249

243250
val broadcastedHadoopConf =
244251
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
@@ -440,7 +447,8 @@ object ParquetFileFormat extends Logging {
440447

441448
val converter = new ParquetToSparkSchemaConverter(
442449
sparkSession.sessionState.conf.isParquetBinaryAsString,
443-
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
450+
sparkSession.sessionState.conf.isParquetINT96AsTimestamp,
451+
nanosAsLong = sparkSession.sessionState.conf.legacyParquetNanosAsLong)
444452

445453
val seen = mutable.HashSet[String]()
446454
val finalSchemas: Seq[StructType] = footers.flatMap { footer =>
@@ -536,12 +544,14 @@ object ParquetFileFormat extends Logging {
536544
sparkSession: SparkSession): Option[StructType] = {
537545
val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString
538546
val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp
547+
val nanosAsLong = sparkSession.sessionState.conf.legacyParquetNanosAsLong
539548

540549
val reader = (files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) => {
541550
// Converter used to convert Parquet `MessageType` to Spark SQL `StructType`
542551
val converter = new ParquetToSparkSchemaConverter(
543552
assumeBinaryIsString = assumeBinaryIsString,
544-
assumeInt96IsTimestamp = assumeInt96IsTimestamp)
553+
assumeInt96IsTimestamp = assumeInt96IsTimestamp,
554+
nanosAsLong = nanosAsLong)
545555

546556
readParquetFootersInParallel(conf, files, ignoreCorruptFiles)
547557
.map(ParquetFileFormat.readSchemaFromFooter(_, converter))

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

+12-3
Original file line numberDiff line numberDiff line change
@@ -49,21 +49,25 @@ import org.apache.spark.util.Utils
4949
* [[TimestampType]] fields.
5050
* @param caseSensitive Whether use case sensitive analysis when comparing Spark catalyst read
5151
* schema with Parquet schema
52+
* @param nanosAsLong Whether timestamps with nanos are converted to long.
5253
*/
5354
class ParquetToSparkSchemaConverter(
5455
assumeBinaryIsString: Boolean = SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get,
5556
assumeInt96IsTimestamp: Boolean = SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get,
56-
caseSensitive: Boolean = SQLConf.CASE_SENSITIVE.defaultValue.get) {
57+
caseSensitive: Boolean = SQLConf.CASE_SENSITIVE.defaultValue.get,
58+
nanosAsLong: Boolean = SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValue.get) {
5759

5860
def this(conf: SQLConf) = this(
5961
assumeBinaryIsString = conf.isParquetBinaryAsString,
6062
assumeInt96IsTimestamp = conf.isParquetINT96AsTimestamp,
61-
caseSensitive = conf.caseSensitiveAnalysis)
63+
caseSensitive = conf.caseSensitiveAnalysis,
64+
nanosAsLong = conf.legacyParquetNanosAsLong)
6265

6366
def this(conf: Configuration) = this(
6467
assumeBinaryIsString = conf.get(SQLConf.PARQUET_BINARY_AS_STRING.key).toBoolean,
6568
assumeInt96IsTimestamp = conf.get(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key).toBoolean,
66-
caseSensitive = conf.get(SQLConf.CASE_SENSITIVE.key).toBoolean)
69+
caseSensitive = conf.get(SQLConf.CASE_SENSITIVE.key).toBoolean,
70+
conf.get(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key).toBoolean)
6771

6872

6973
/**
@@ -257,6 +261,11 @@ class ParquetToSparkSchemaConverter(
257261
// SPARK-38829: Remove TimestampNTZ type support in Parquet for Spark 3.3
258262
if (Utils.isTesting) TimestampNTZType else TimestampType
259263
}
264+
// SPARK-40819: NANOS are not supported as a Timestamp, convert to LongType without
265+
// timezone awareness to address behaviour regression introduced by SPARK-34661
266+
case timestamp: TimestampLogicalTypeAnnotation
267+
if timestamp.getUnit == TimeUnit.NANOS && nanosAsLong =>
268+
LongType
260269
case _ => illegalType()
261270
}
262271

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala

+4
Original file line numberDiff line numberDiff line change
@@ -86,6 +86,10 @@ case class ParquetScan(
8686
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
8787
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
8888

89+
hadoopConf.setBoolean(
90+
SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key,
91+
sparkSession.sessionState.conf.legacyParquetNanosAsLong)
92+
8993
val broadcastedConf = sparkSession.sparkContext.broadcast(
9094
new SerializableConfiguration(hadoopConf))
9195
val sqlConf = sparkSession.sessionState.conf
Binary file not shown.

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala

+65-7
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import org.apache.spark.SparkException
3030
import org.apache.spark.sql.catalyst.ScalaReflection
3131
import org.apache.spark.sql.execution.QueryExecutionException
3232
import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException
33+
import org.apache.spark.sql.functions.desc
3334
import org.apache.spark.sql.internal.SQLConf
3435
import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType._
3536
import org.apache.spark.sql.test.SharedSparkSession
@@ -46,15 +47,17 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
4647
binaryAsString: Boolean,
4748
int96AsTimestamp: Boolean,
4849
writeLegacyParquetFormat: Boolean,
49-
expectedParquetColumn: Option[ParquetColumn] = None): Unit = {
50+
expectedParquetColumn: Option[ParquetColumn] = None,
51+
nanosAsLong: Boolean = false): Unit = {
5052
testSchema(
5153
testName,
5254
StructType.fromAttributes(ScalaReflection.attributesFor[T]),
5355
messageType,
5456
binaryAsString,
5557
int96AsTimestamp,
5658
writeLegacyParquetFormat,
57-
expectedParquetColumn = expectedParquetColumn)
59+
expectedParquetColumn = expectedParquetColumn,
60+
nanosAsLong = nanosAsLong)
5861
}
5962

6063
protected def testParquetToCatalyst(
@@ -65,11 +68,13 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
6568
int96AsTimestamp: Boolean,
6669
caseSensitive: Boolean = false,
6770
sparkReadSchema: Option[StructType] = None,
68-
expectedParquetColumn: Option[ParquetColumn] = None): Unit = {
71+
expectedParquetColumn: Option[ParquetColumn] = None,
72+
nanosAsLong: Boolean = false): Unit = {
6973
val converter = new ParquetToSparkSchemaConverter(
7074
assumeBinaryIsString = binaryAsString,
7175
assumeInt96IsTimestamp = int96AsTimestamp,
72-
caseSensitive = caseSensitive)
76+
caseSensitive = caseSensitive,
77+
nanosAsLong = nanosAsLong)
7378

7479
test(s"sql <= parquet: $testName") {
7580
val actualParquetColumn = converter.convertParquetColumn(
@@ -117,7 +122,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
117122
writeLegacyParquetFormat: Boolean,
118123
outputTimestampType: SQLConf.ParquetOutputTimestampType.Value =
119124
SQLConf.ParquetOutputTimestampType.INT96,
120-
expectedParquetColumn: Option[ParquetColumn] = None): Unit = {
125+
expectedParquetColumn: Option[ParquetColumn] = None,
126+
nanosAsLong: Boolean = false): Unit = {
121127

122128
testCatalystToParquet(
123129
testName,
@@ -132,7 +138,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
132138
parquetSchema,
133139
binaryAsString,
134140
int96AsTimestamp,
135-
expectedParquetColumn = expectedParquetColumn)
141+
expectedParquetColumn = expectedParquetColumn,
142+
nanosAsLong = nanosAsLong)
136143
}
137144

138145
protected def compareParquetColumn(actual: ParquetColumn, expected: ParquetColumn): Unit = {
@@ -147,7 +154,14 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
147154
val expectedDesc = expected.descriptor.get
148155
assert(actualDesc.getMaxRepetitionLevel == expectedDesc.getMaxRepetitionLevel)
149156
assert(actualDesc.getMaxRepetitionLevel == expectedDesc.getMaxRepetitionLevel)
150-
assert(actualDesc.getPrimitiveType === expectedDesc.getPrimitiveType)
157+
158+
actualDesc.getPrimitiveType.getLogicalTypeAnnotation match {
159+
case timestamp: LogicalTypeAnnotation.TimestampLogicalTypeAnnotation
160+
if timestamp.getUnit == LogicalTypeAnnotation.TimeUnit.NANOS =>
161+
assert(actual.sparkType == expected.sparkType)
162+
case _ =>
163+
assert(actualDesc.getPrimitiveType === expectedDesc.getPrimitiveType)
164+
}
151165
}
152166

153167
assert(actual.repetitionLevel == expected.repetitionLevel, "repetition level mismatch: " +
@@ -195,6 +209,32 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
195209
}
196210

197211
class ParquetSchemaInferenceSuite extends ParquetSchemaTest {
212+
testSchemaInference[Tuple1[Long]](
213+
"timestamp nanos",
214+
"""
215+
|message root {
216+
| required int64 _1 (TIMESTAMP(NANOS,true));
217+
|}
218+
""".stripMargin,
219+
binaryAsString = false,
220+
int96AsTimestamp = true,
221+
writeLegacyParquetFormat = true,
222+
expectedParquetColumn = Some(
223+
ParquetColumn(
224+
sparkType = StructType.fromAttributes(
225+
ScalaReflection.attributesFor[Tuple1[Long]]),
226+
descriptor = None,
227+
repetitionLevel = 0,
228+
definitionLevel = 0,
229+
required = false,
230+
path = Seq(),
231+
children = Seq(
232+
primitiveParquetColumn(LongType, PrimitiveTypeName.INT64, Repetition.REQUIRED,
233+
0, 0, Seq("_1"), logicalTypeAnnotation = Some(LogicalTypeAnnotation.intType(64, false)))
234+
))),
235+
nanosAsLong = true
236+
)
237+
198238
testSchemaInference[(Boolean, Int, Long, Float, Double, Array[Byte])](
199239
"basic types",
200240
"""
@@ -1005,6 +1045,24 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
10051045
}
10061046
}
10071047

1048+
test("SPARK-40819: parquet file with TIMESTAMP(NANOS, true) (with nanosAsLong=true)") {
1049+
val tsAttribute = "birthday"
1050+
withSQLConf(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key -> "true") {
1051+
val testDataPath = testFile("test-data/timestamp-nanos.parquet")
1052+
val data = spark.read.parquet(testDataPath).select(tsAttribute)
1053+
assert(data.schema.fields.head.dataType == LongType)
1054+
assert(data.orderBy(desc(tsAttribute)).take(1).head.getAs[Long](0) == 1668537129123534758L)
1055+
}
1056+
}
1057+
1058+
test("SPARK-40819: parquet file with TIMESTAMP(NANOS, true) (with default nanosAsLong=false)") {
1059+
val testDataPath = testFile("test-data/timestamp-nanos.parquet")
1060+
val e = intercept[SparkException] {
1061+
spark.read.parquet(testDataPath).collect()
1062+
}
1063+
assert(e.getMessage.contains("Illegal Parquet type: INT64 (TIMESTAMP(NANOS,true))"))
1064+
}
1065+
10081066
// =======================================================
10091067
// Tests for converting Parquet LIST to Catalyst ArrayType
10101068
// =======================================================

0 commit comments

Comments
 (0)