yihua commented on code in PR #10167:
URL: https://github.com/apache/hudi/pull/10167#discussion_r1417907117
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -355,12 +364,50 @@ class
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
(baseFileReader, preMergeBaseFileReader, skeletonReader,
bootstrapBaseReader)
}
+}
+
+object HoodieFileGroupReaderBasedParquetFileFormat {
+ private val ROW_INDEX = "row_index"
+ private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX"
+ private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY =
"__file_source_generated_metadata_col"
+ private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col"
+ private val METADATA_COL_ATTR_KEY = "__metadata_col"
- protected def getRecordKeyRelatedFilters(filters: Seq[Filter],
recordKeyColumn: String): Seq[Filter] = {
+ def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn:
String): Seq[Filter] = {
filters.filter(f => f.references.exists(c =>
c.equalsIgnoreCase(recordKeyColumn)))
}
- protected def getLogFilesFromSlice(fileSlice: FileSlice):
List[HoodieLogFile] = {
+ def getLogFilesFromSlice(fileSlice: FileSlice): List[HoodieLogFile] = {
fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList
}
+
+ def getFieldMetadata(name: String, internalName: String): Metadata = {
+ new MetadataBuilder()
+ .putString(METADATA_COL_ATTR_KEY, name)
+ .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true)
+ .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, internalName)
+ .build()
+ }
+
+ def getAppliedRequiredSchema(requiredSchema: StructType,
+ shouldUseRecordPosition: Boolean,
+ recordPositionColumn: String): StructType = {
+ if (HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition) {
Review Comment:
Have a util method for whether to add the meta column?
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestIfRecordPositionColumnWorkForSpark3_5.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.hudi.common.table.read
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hudi.SparkAdapterSupport.sparkAdapter
+import org.apache.hudi.common.config.HoodieStorageConfig
+import org.apache.hudi.common.testutils.HoodieTestTable
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.hudi.util.CloseableInternalRowIterator
+import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils}
+import org.apache.spark.sql.catalyst.InternalRow
+import
org.apache.spark.sql.execution.datasources.parquet.{HoodieFileGroupReaderBasedParquetFileFormat,
ParquetFileFormat}
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField,
StructType}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.Test
+
+class TestIfRecordPositionColumnWorkForSpark3_5 extends
SparkClientFunctionalTestHarness {
+ private val HOOD_FORMAT = "org.apache.hudi"
+ private val MOR_TABLE_TYPE = "MERGE_ON_READ"
+ private val PARQUET_FORMAT = "parquet"
+ private val ROW_INDEX_COLUMN = "_tmp_metadata_row_index"
+ private val SPARK_MERGER = "org.apache.hudi.HoodieSparkRecordMerger"
+
+ @Test
+ def testConfirmIfRowIndexColumnWorks(): Unit = {
+ val _spark = spark
+ import _spark.implicits._
+
+ val userToCountryDF = Seq(
+ (1, "US", "1000"),
+ (2, "US", "1000"),
+ (3, "China", "1000"),
+ (4, "Singapore", "1000")
+ ).toDF("userid", "country", "ts")
+ val userToCountryTblPath = s"$basePath/user_to_country_table"
+
+ userToCountryDF.write.format(HOOD_FORMAT)
Review Comment:
directly use "hudi" instead of "org.apache.hudi"
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -355,12 +364,50 @@ class
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
(baseFileReader, preMergeBaseFileReader, skeletonReader,
bootstrapBaseReader)
}
+}
+
+object HoodieFileGroupReaderBasedParquetFileFormat {
+ private val ROW_INDEX = "row_index"
+ private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX"
+ private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY =
"__file_source_generated_metadata_col"
+ private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col"
+ private val METADATA_COL_ATTR_KEY = "__metadata_col"
- protected def getRecordKeyRelatedFilters(filters: Seq[Filter],
recordKeyColumn: String): Seq[Filter] = {
+ def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn:
String): Seq[Filter] = {
filters.filter(f => f.references.exists(c =>
c.equalsIgnoreCase(recordKeyColumn)))
}
- protected def getLogFilesFromSlice(fileSlice: FileSlice):
List[HoodieLogFile] = {
+ def getLogFilesFromSlice(fileSlice: FileSlice): List[HoodieLogFile] = {
fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList
}
+
+ def getFieldMetadata(name: String, internalName: String): Metadata = {
+ new MetadataBuilder()
+ .putString(METADATA_COL_ATTR_KEY, name)
+ .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true)
+ .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, internalName)
+ .build()
+ }
+
+ def getAppliedRequiredSchema(requiredSchema: StructType,
+ shouldUseRecordPosition: Boolean,
+ recordPositionColumn: String): StructType = {
+ if (HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition) {
+ val metadata = getFieldMetadata(recordPositionColumn,
ROW_INDEX_TEMPORARY_COLUMN_NAME)
+ val rowIndexField = StructField(recordPositionColumn, LongType, nullable
= false, metadata)
+ StructType(requiredSchema.fields :+ rowIndexField)
+ } else {
+ requiredSchema
+ }
+ }
+
+ def getAppliedFilters(requiredFilters: Seq[Filter],
+ recordKeyRelatedFilters: Seq[Filter],
+ shouldUseRecordPosition: Boolean): Seq[Filter] = {
+ if (!HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition) {
+ requiredFilters
+ } else {
+ requiredFilters ++ recordKeyRelatedFilters
+ }
+ }
Review Comment:
The condition seems wrong. When Spark version is below 3.5 and not using
positions, the record key filters can still be applied, correct?
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestIfRecordPositionColumnWorkForSpark3_5.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.hudi.common.table.read
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hudi.SparkAdapterSupport.sparkAdapter
+import org.apache.hudi.common.config.HoodieStorageConfig
+import org.apache.hudi.common.testutils.HoodieTestTable
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.hudi.util.CloseableInternalRowIterator
+import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils}
+import org.apache.spark.sql.catalyst.InternalRow
+import
org.apache.spark.sql.execution.datasources.parquet.{HoodieFileGroupReaderBasedParquetFileFormat,
ParquetFileFormat}
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField,
StructType}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.Test
+
+class TestIfRecordPositionColumnWorkForSpark3_5 extends
SparkClientFunctionalTestHarness {
Review Comment:
Rename to `TestSpark35RecordPositionMetadataColumn`
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -355,12 +364,50 @@ class
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
(baseFileReader, preMergeBaseFileReader, skeletonReader,
bootstrapBaseReader)
}
+}
+
+object HoodieFileGroupReaderBasedParquetFileFormat {
+ private val ROW_INDEX = "row_index"
+ private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX"
+ private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY =
"__file_source_generated_metadata_col"
+ private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col"
+ private val METADATA_COL_ATTR_KEY = "__metadata_col"
- protected def getRecordKeyRelatedFilters(filters: Seq[Filter],
recordKeyColumn: String): Seq[Filter] = {
+ def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn:
String): Seq[Filter] = {
filters.filter(f => f.references.exists(c =>
c.equalsIgnoreCase(recordKeyColumn)))
}
- protected def getLogFilesFromSlice(fileSlice: FileSlice):
List[HoodieLogFile] = {
+ def getLogFilesFromSlice(fileSlice: FileSlice): List[HoodieLogFile] = {
fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList
}
+
+ def getFieldMetadata(name: String, internalName: String): Metadata = {
+ new MetadataBuilder()
+ .putString(METADATA_COL_ATTR_KEY, name)
+ .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true)
+ .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, internalName)
+ .build()
+ }
+
+ def getAppliedRequiredSchema(requiredSchema: StructType,
+ shouldUseRecordPosition: Boolean,
+ recordPositionColumn: String): StructType = {
+ if (HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition) {
+ val metadata = getFieldMetadata(recordPositionColumn,
ROW_INDEX_TEMPORARY_COLUMN_NAME)
+ val rowIndexField = StructField(recordPositionColumn, LongType, nullable
= false, metadata)
+ StructType(requiredSchema.fields :+ rowIndexField)
+ } else {
+ requiredSchema
+ }
+ }
+
+ def getAppliedFilters(requiredFilters: Seq[Filter],
+ recordKeyRelatedFilters: Seq[Filter],
+ shouldUseRecordPosition: Boolean): Seq[Filter] = {
+ if (!HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition) {
Review Comment:
Similar here for the condition.
--
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]