codope commented on code in PR #10225:
URL: https://github.com/apache/hudi/pull/10225#discussion_r1422874813
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala:
##########
@@ -303,8 +308,21 @@ object DefaultSource {
HoodieBootstrapMORRelation(sqlContext, userSchema, globPaths,
metaClient, parameters)
}
- case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
- MergeOnReadIncrementalRelation(sqlContext, parameters, metaClient,
userSchema)
+ case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, true) =>
+ if (fileFormatUtils.isDefined) {
+ new HoodieMergeOnReadIncrementalHadoopFsRelationFactory(
+ sqlContext, metaClient, parameters, userSchema, isBootstrap =
true).build()
+ } else {
+ MergeOnReadIncrementalRelation(sqlContext, parameters, metaClient,
userSchema)
+ }
+
+ case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, false) =>
Review Comment:
this case can be merged with the above case right?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCDCFileIndex.scala:
##########
@@ -28,11 +28,12 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Expression,
GenericInternalRow}
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache,
NoopCache, PartitionDirectory}
+import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import scala.jdk.CollectionConverters.{asScalaBufferConverter,
mapAsScalaMapConverter}
-class HoodieCDCFileIndex (override val spark: SparkSession,
+class HoodieCDCFileIndex(override val spark: SparkSession,
Review Comment:
nit: fix the indentation
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala:
##########
@@ -140,4 +142,6 @@ trait HoodieCatalystPlansUtils {
def failAnalysisForMIT(a: Attribute, cols: String): Unit = {}
def createMITJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType,
condition: Option[Expression], hint: String): LogicalPlan
+
+ def sameOutput(a: LogicalPlan, b: LogicalPlan): Boolean
Review Comment:
please add scaladoc
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -278,6 +276,8 @@ class
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
val baseFileReader = super.buildReaderWithPartitionValues(sparkSession,
dataSchema, partitionSchema, requiredSchema,
filters ++ requiredFilters, options, new Configuration(hadoopConf))
m.put(generateKey(dataSchema, requiredSchema), baseFileReader)
+ m.put(0, super.buildReaderWithPartitionValues(sparkSession, dataSchema,
partitionSchema, requiredSchema,
Review Comment:
What's the reason of key being hard-coded to 0 here? Also, is there any
chance that the key evaluated in line 278 above i.e. `generateKey(dataSchema,
requiredSchema)` can be 0. In that case, this line will override, which may not
be the expected behavior.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala:
##########
@@ -50,31 +50,32 @@ class TestClusteringProcedure extends
HoodieSparkProcedureTestBase {
| id int,
| name string,
| price double,
- | ts long
+ | ts long,
+ | partition long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
- | partitioned by(ts)
+ | partitioned by(partition)
Review Comment:
why is there a need to change the table schema for test? Moreover, the new
partition firld has the same type as the old one, so i'm not sure if it adds
any value for the test.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/NewFileFormatUtils.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.
+ */
+
+/*
+ * 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.spark.sql
+
+import org.apache.hudi.{HoodieCDCFileIndex, SparkAdapterSupport,
SparkHoodieTableFileIndex}
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute,
AttributeSet, Contains, EndsWith, EqualNullSafe, EqualTo, Expression,
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan,
LessThanOrEqual, Literal, NamedExpression, Not, Or, StartsWith}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan,
Project}
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.execution.datasources.parquet.{HoodieFormatTrait,
ParquetFileFormat}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+
+import scala.util.Try
+
+object NewFileFormatUtils extends SparkAdapterSupport {
Review Comment:
Consider renaming to FileFormatUtilsV2 or something like that. Similarly,
consider renaming `NewHoodieParquetFileFormat`. Maybe do it in a separate PR.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/NewFileFormatUtils.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.
+ */
+
+/*
+ * 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.spark.sql
+
+import org.apache.hudi.{HoodieCDCFileIndex, SparkAdapterSupport,
SparkHoodieTableFileIndex}
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute,
AttributeSet, Contains, EndsWith, EqualNullSafe, EqualTo, Expression,
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan,
LessThanOrEqual, Literal, NamedExpression, Not, Or, StartsWith}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan,
Project}
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.execution.datasources.parquet.{HoodieFormatTrait,
ParquetFileFormat}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+
+import scala.util.Try
+
+object NewFileFormatUtils extends SparkAdapterSupport {
+
+ def applyNewFileFormatChanges(scanOperation: LogicalPlan, logicalRelation:
LogicalPlan, fs: HadoopFsRelation): LogicalPlan = {
+ val ff = fs.fileFormat.asInstanceOf[ParquetFileFormat with
HoodieFormatTrait]
+ ff.isProjected = true
+ val tableSchema = fs.location match {
+ case index: HoodieCDCFileIndex => index.cdcRelation.schema
+ case index: SparkHoodieTableFileIndex => index.schema
+ }
+ val resolvedSchema = logicalRelation.resolve(tableSchema,
fs.sparkSession.sessionState.analyzer.resolver)
+ val unfilteredPlan = if (!fs.partitionSchema.fields.isEmpty &&
sparkAdapter.getCatalystPlanUtils.sameOutput(scanOperation, logicalRelation)) {
+ Project(resolvedSchema, scanOperation)
+ } else {
+ scanOperation
+ }
+ applyFiltersToPlan(unfilteredPlan, tableSchema, resolvedSchema,
ff.getRequiredFilters)
+ }
+
+ /**
+ * adapted from
https://github.com/apache/spark/blob/20df062d85e80422a55afae80ddbf2060f26516c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala
+ */
+ def applyFiltersToPlan(plan: LogicalPlan, tableSchema: StructType,
resolvedSchema: Seq[Attribute], filters:
Seq[org.apache.spark.sql.sources.Filter]): LogicalPlan = {
+
+ def filterToExpression(
+ filter: sources.Filter,
+ toRef: String => Option[NamedExpression]):
Option[Expression] = {
+ def zipAttributeAndValue(name: String, value: Any):
Option[(NamedExpression, Literal)] = {
+ zip(toRef(name), toLiteral(value))
+ }
+
+ def translate(filter: sources.Filter): Option[Expression] = filter match
{
+ case sources.And(left, right) =>
+ zip(translate(left), translate(right)).map(And.tupled)
+ case sources.Or(left, right) =>
+ zip(translate(left), translate(right)).map(Or.tupled)
+ case sources.Not(child) =>
+ translate(child).map(Not)
+ case sources.EqualTo(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(EqualTo.tupled)
+ case sources.EqualNullSafe(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled)
+ case sources.IsNull(attribute) =>
+ toRef(attribute).map(IsNull)
+ case sources.IsNotNull(attribute) =>
+ toRef(attribute).map(IsNotNull)
+ case sources.In(attribute, values) =>
+ val literals = values.toSeq.flatMap(toLiteral)
+ if (literals.length == values.length) {
+ toRef(attribute).map(In(_, literals))
+ } else {
+ None
+ }
+ case sources.GreaterThan(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(GreaterThan.tupled)
+ case sources.GreaterThanOrEqual(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled)
+ case sources.LessThan(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(LessThan.tupled)
+ case sources.LessThanOrEqual(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled)
+ case sources.StringContains(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(Contains.tupled)
+ case sources.StringStartsWith(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(StartsWith.tupled)
+ case sources.StringEndsWith(attribute, value) =>
+ zipAttributeAndValue(attribute, value).map(EndsWith.tupled)
+ /* Not supported in spark2. If needed, we will need to create separate
spark 2 and 3 implementations
Review Comment:
Should we then move this method to `SparkAdapter`?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCDCFileIndex.scala:
##########
@@ -78,4 +79,8 @@ class HoodieCDCFileIndex (override val spark: SparkSession,
new Path(fileGroupId.getPartitionPath, fileGroupId.getFileId).toString
}.toArray
}
+
+ override def getRequiredFilters: Seq[Filter] = {
+ Seq.empty
Review Comment:
Why empty for CDC?
--
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]