shubham19may commented on code in PR #14499:
URL: https://github.com/apache/iceberg/pull/14499#discussion_r2509751253
##########
spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java:
##########
@@ -232,6 +233,81 @@ protected WriteSupport<InternalRow>
getWriteSupport(Configuration configuration)
}
}
+ @Test
+ public void testTimestampMillisProducedBySparkIsReadCorrectly() throws
IOException {
+ String outputFilePath =
+ String.format("%s/%s", temp.toAbsolutePath(),
"parquet_timestamp_millis.parquet");
+ HadoopOutputFile outputFile =
+ HadoopOutputFile.fromPath(
+ new org.apache.hadoop.fs.Path(outputFilePath), new
Configuration());
+
+ Schema schema = new Schema(required(1, "event_time",
Types.TimestampType.withZone()));
+
+ StructType sparkSchema =
+ new StructType(
+ new StructField[] {
+ new StructField("event_time", DataTypes.TimestampType, false,
Metadata.empty())
+ });
+
+ List<InternalRow> originalRows =
Lists.newArrayList(RandomData.generateSpark(schema, 10, 0L));
+ List<InternalRow> rows = Lists.newArrayList();
+ for (InternalRow row : originalRows) {
+ long timestampMicros = row.getLong(0);
+ long timestampMillis = (timestampMicros / 1000) * 1000;
+ rows.add(
+ new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(
+ new Object[] {timestampMillis}));
+ }
+
+ try (ParquetWriter<InternalRow> writer =
+ new NativeSparkWriterBuilder(outputFile)
+ .set("org.apache.spark.sql.parquet.row.attributes",
sparkSchema.json())
+ .set("spark.sql.parquet.writeLegacyFormat", "false")
+ .set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS")
+ .set("spark.sql.parquet.fieldId.write.enabled", "true")
+ .build()) {
+ for (InternalRow row : rows) {
+ writer.write(row);
+ }
+ }
Review Comment:
@nandorKollar well, for
(a) ExampleParquetWriter : I fear, we have to manually construct Parquet
group objects and manually define the schema with TIMESTAMP_MILLIS logical
type, which is much more complex than our current approach. Moreover, Iceberg
never used ExampleParquetWriter, using it would require writing low-level
parquet code.
(b) AvroParquetWriter : Iceberg’s AvroSchemaUtil always uses timestampMicros
, never timestampMillis [check
here](https://github.com/apache/iceberg/blob/ae4f5c1f8893f199b633f3ccbc6ddd94170db596/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java#L45-L48).
While avro supports timestamp-millis, you cannot use it through. We have to
manually create the avro schema for it, and also handle the data conversion.
Again, it will bring more unnecessary code.
IMO, we should stick to our current approach.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]