Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[VL] Enable timestamp in parquet write #6428

Merged
merged 3 commits into from
Jul 26, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,6 @@ object VeloxBackendSettings extends BackendSettingsApi {
val unsupportedTypes = fields.flatMap {
field =>
field.dataType match {
case _: TimestampType => Some("TimestampType")
case _: StructType => Some("StructType")
case _: ArrayType => Some("ArrayType")
case _: MapType => Some("MapType")
Expand Down Expand Up @@ -293,7 +292,7 @@ object VeloxBackendSettings extends BackendSettingsApi {
fields.map {
field =>
field.dataType match {
case _: TimestampType | _: StructType | _: ArrayType | _: MapType => return false
case _: StructType | _: ArrayType | _: MapType => return false
case _ =>
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,6 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit
val df = spark.read
.format("parquet")
.load(data_path)
.drop("timestamp")
.drop("array")
.drop("struct")
.drop("map")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -731,12 +731,14 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenParquetRebaseDatetimeV1Suite]
// Velox doesn't write file metadata into parquet file.
.excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
.excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
// jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some
.excludeByPrefix("SPARK-31159")
.excludeByPrefix("SPARK-35427")
enableSuite[GlutenParquetRebaseDatetimeV2Suite]
// Velox doesn't write file metadata into parquet file.
.excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
.excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
// jar path and ignore PARQUET_REBASE_MODE_IN_READ
.excludeByPrefix("SPARK-31159")
.excludeByPrefix("SPARK-35427")
Expand Down Expand Up @@ -767,6 +769,8 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenFileMetadataStructRowIndexSuite]
enableSuite[GlutenParquetV1AggregatePushDownSuite]
enableSuite[GlutenParquetV2AggregatePushDownSuite]
// TODO: Timestamp columns stats will lost if using int64 in parquet writer.
.exclude("aggregate push down - different data types")
enableSuite[GlutenOrcV1AggregatePushDownSuite]
.exclude("nested column: Count(nested sub-field) not push down")
enableSuite[GlutenOrcV2AggregatePushDownSuite]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -738,12 +738,14 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenParquetRebaseDatetimeV1Suite]
// Velox doesn't write file metadata into parquet file.
.excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
.excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
// jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some
.excludeByPrefix("SPARK-31159")
.excludeByPrefix("SPARK-35427")
enableSuite[GlutenParquetRebaseDatetimeV2Suite]
// Velox doesn't write file metadata into parquet file.
.excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
.excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
// jar path and ignore PARQUET_REBASE_MODE_IN_READ
.excludeByPrefix("SPARK-31159")
.excludeByPrefix("SPARK-35427")
Expand Down Expand Up @@ -773,6 +775,8 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenFileMetadataStructSuite]
enableSuite[GlutenParquetV1AggregatePushDownSuite]
enableSuite[GlutenParquetV2AggregatePushDownSuite]
// TODO: Timestamp columns stats will lost if using int64 in parquet writer.
.exclude("aggregate push down - different data types")
enableSuite[GlutenOrcV1AggregatePushDownSuite]
.exclude("nested column: Count(nested sub-field) not push down")
enableSuite[GlutenOrcV2AggregatePushDownSuite]
Expand Down
Loading