codope commented on code in PR #9819:
URL: https://github.com/apache/hudi/pull/9819#discussion_r1357025981


##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/KeySpec.java:
##########
@@ -0,0 +1,36 @@
+/*
+ *
+ *  * 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.log;
+
+import java.util.List;
+
+public interface KeySpec {

Review Comment:
   javadoc for the public interface



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/FullKeySpec.java:
##########
@@ -0,0 +1,40 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review Comment:
   Please fix this license comment block (double asterisk) here and other new 
classes



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/FileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.execution.datasources.parquet
+
+import kotlin.NotImplementedError
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.engine.HoodieReaderContext
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{FileSlice, HoodieBaseFile, HoodieLogFile, 
HoodieRecord}
+import org.apache.hudi.common.table.read.HoodieFileGroupReader
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.{
+  HoodieBaseRelation, HoodieSparkUtils, HoodieTableSchema, HoodieTableState, 
MergeOnReadSnapshotRelation,
+  PartitionFileSliceMapping, SparkAdapterSupport, 
SparkFileFormatInternalRowReaderContext}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.util
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+class FileGroupReaderBasedParquetFileFormat(tableState: 
Broadcast[HoodieTableState],
+                                            tableSchema: 
Broadcast[HoodieTableSchema],

Review Comment:
   Do we really need to broadcast? There will be (de)serialization overhead. 
Check 
https://github.com/apache/hudi/commit/f4832d9903fb7c59891ae501265496c961a3ec41 
which removed broadcasting from `NewHoodieParquetFileFormat`.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/FileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.execution.datasources.parquet
+
+import kotlin.NotImplementedError
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.engine.HoodieReaderContext
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{FileSlice, HoodieBaseFile, HoodieLogFile, 
HoodieRecord}
+import org.apache.hudi.common.table.read.HoodieFileGroupReader
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.{
+  HoodieBaseRelation, HoodieSparkUtils, HoodieTableSchema, HoodieTableState, 
MergeOnReadSnapshotRelation,
+  PartitionFileSliceMapping, SparkAdapterSupport, 
SparkFileFormatInternalRowReaderContext}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.util
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+class FileGroupReaderBasedParquetFileFormat(tableState: 
Broadcast[HoodieTableState],
+                                            tableSchema: 
Broadcast[HoodieTableSchema],
+                                            tableName: String,
+                                            mergeType: String,
+                                            mandatoryFields: Seq[String],
+                                            isMOR: Boolean,
+                                            isBootstrap: Boolean,
+                                            shouldUseRecordPosition: Boolean
+                                           ) extends ParquetFileFormat with 
SparkAdapterSupport {
+  var isProjected = false
+
+  /**
+   * Support batch needs to remain consistent, even if one side of a bootstrap 
merge can support
+   * while the other side can't
+   */
+  private var supportBatchCalled = false
+  private var supportBatchResult = false
+
+  override def supportBatch(sparkSession: SparkSession, schema: StructType): 
Boolean = {
+    if (!supportBatchCalled) {
+      supportBatchCalled = true
+      supportBatchResult = !isMOR && super.supportBatch(sparkSession, schema)
+    }
+    supportBatchResult
+  }
+
+  override def isSplitable(
+                            sparkSession: SparkSession,
+                            options: Map[String, String],
+                            path: Path): Boolean = {
+    false
+  }
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                              dataSchema: StructType,
+                                              partitionSchema: StructType,
+                                              requiredSchema: StructType,
+                                              filters: Seq[Filter],
+                                              options: Map[String, String],
+                                              hadoopConf: Configuration): 
PartitionedFile => Iterator[InternalRow] = {
+    val requiredSchemaWithMandatory = 
generateRequiredSchemaWithMandatory(requiredSchema, dataSchema)
+    val requiredSchemaSplits = requiredSchemaWithMandatory.fields.partition(f 
=> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name))
+    val requiredMeta = StructType(requiredSchemaSplits._1)
+    val requiredWithoutMeta = StructType(requiredSchemaSplits._2)
+    val (baseFileReader, preMergeBaseFileReader, _, _) = 
buildFileReaders(sparkSession,
+      dataSchema, partitionSchema, requiredSchema, filters, options, 
hadoopConf, requiredSchemaWithMandatory,
+      requiredWithoutMeta, requiredMeta)
+    val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
+
+    (file: PartitionedFile) => {
+      file.partitionValues match {
+        case fileSliceMapping: PartitionFileSliceMapping =>
+          val filePath = 
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)
+          if (FSUtils.isLogFile(filePath)) {
+            // TODO: Use FileGroupReader here.

Review Comment:
   would be nice if you can also add jira IDs - `TODO(HUDI-XXXX): ...`



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/FileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.execution.datasources.parquet
+
+import kotlin.NotImplementedError
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.engine.HoodieReaderContext
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{FileSlice, HoodieBaseFile, HoodieLogFile, 
HoodieRecord}
+import org.apache.hudi.common.table.read.HoodieFileGroupReader
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.{
+  HoodieBaseRelation, HoodieSparkUtils, HoodieTableSchema, HoodieTableState, 
MergeOnReadSnapshotRelation,
+  PartitionFileSliceMapping, SparkAdapterSupport, 
SparkFileFormatInternalRowReaderContext}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.util
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+class FileGroupReaderBasedParquetFileFormat(tableState: 
Broadcast[HoodieTableState],

Review Comment:
   How about `HoodieFileGroupReaderBasedParquetFileFormat`? While we're at it, 
let's rename `NewHoodieParquetFileFormat`. @jonvex @linliu-code maybe you both 
can figure out a better name.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java:
##########
@@ -0,0 +1,260 @@
+/*
+ *
+ *  * 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.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.DeleteRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.log.KeySpec;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.table.log.block.HoodieDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+
+public class HoodieKeyBasedFileGroupRecordBuffer<T> implements 
HoodieFileGroupRecordBuffer<T, String> {
+  private final HoodieReaderContext<T> readerContext;
+  private final Schema readerSchema;
+  private final Schema baseFileSchema;
+  private final Option<String> partitionNameOverrideOpt;
+  private final Option<String[]> partitionPathFieldOpt;
+  private final HoodieRecordMerger recordMerger;
+  private final TypedProperties payloadProps;
+  private final HoodieTableMetaClient hoodieTableMetaClient;
+  private final Map<String, Pair<Option<T>, Map<String, Object>>> records;
+  private ClosableIterator<T> baseFileIterator;
+  private Iterator<Pair<Option<T>, Map<String, Object>>> logRecordIterator;
+  private T nextRecord;
+
+  public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext<T> 
readerContext,
+                                             Schema readerSchema,
+                                             Schema baseFileSchema,
+                                             Option<String> 
partitionNameOverrideOpt,
+                                             Option<String[]> 
partitionPathFieldOpt,
+                                             HoodieRecordMerger recordMerger,
+                                             TypedProperties payloadProps,
+                                             HoodieTableMetaClient 
hoodieTableMetaClient) {
+    this.readerContext = readerContext;
+    this.readerSchema = readerSchema;
+    this.baseFileSchema = baseFileSchema;
+    this.partitionNameOverrideOpt = partitionNameOverrideOpt;
+    this.partitionPathFieldOpt = partitionPathFieldOpt;
+    this.recordMerger = recordMerger;
+    this.payloadProps = payloadProps;
+    this.hoodieTableMetaClient = hoodieTableMetaClient;
+    this.records = new HashMap<>();
+  }
+
+  @Override
+  public BufferType getBufferType() {
+    return BufferType.POSITION_BASED;
+  }
+
+  @Override
+  public void processDataBlock(HoodieDataBlock dataBlock, Option<KeySpec> 
keySpecOpt) throws IOException {
+    checkState(partitionNameOverrideOpt.isPresent() || 
partitionPathFieldOpt.isPresent(),
+        "Either partition-name override or partition-path field had to be 
present");
+
+
+    Pair<ClosableIterator<T>, Schema> recordsIteratorSchemaPair =
+        getRecordsIterator(dataBlock, keySpecOpt);
+
+    try (ClosableIterator<T> recordIterator = 
recordsIteratorSchemaPair.getLeft()) {
+      while (recordIterator.hasNext()) {
+        T nextRecord = recordIterator.next();
+        Map<String, Object> metadata = 
readerContext.generateMetadataForRecord(nextRecord, readerSchema);
+        String recordKey = (String) 
metadata.get(HoodieReaderContext.INTERNAL_META_RECORD_KEY);
+        processNextRecord(nextRecord, metadata, recordKey);
+      }
+    }
+  }
+
+  @Override
+  public void processNextRecord(T record, Map<String, Object> metadata, String 
recordKey) throws IOException {
+    Pair<Option<T>, Map<String, Object>> existingRecordMetadataPair = 
records.get(recordKey);
+
+    if (existingRecordMetadataPair != null) {
+      // Merge and store the combined record
+      // Note that the incoming `record` is from an older commit, so it should 
be put as
+      // the `older` in the merge API
+      HoodieRecord<T> combinedRecord = (HoodieRecord<T>) recordMerger.merge(
+          readerContext.constructHoodieRecord(Option.of(record), metadata, 
readerSchema),
+          readerSchema,
+          
readerContext.constructHoodieRecord(existingRecordMetadataPair.getLeft(), 
existingRecordMetadataPair.getRight(), readerSchema),
+          readerSchema,
+          payloadProps).get().getLeft();
+      // If pre-combine returns existing record, no need to update it
+      if (combinedRecord.getData() != 
existingRecordMetadataPair.getLeft().get()) {
+        records.put(recordKey, 
Pair.of(Option.ofNullable(readerContext.seal(combinedRecord.getData())), 
metadata));
+      }
+    } else {
+      // Put the record as is
+      // NOTE: Record have to be cloned here to make sure if it holds 
low-level engine-specific
+      //       payload pointing into a shared, mutable (underlying) buffer we 
get a clean copy of
+      //       it since these records will be put into records(Map).
+      records.put(recordKey, Pair.of(Option.of(readerContext.seal(record)), 
metadata));
+    }
+  }
+
+  @Override
+  public void processDeleteBlock(HoodieDeleteBlock deleteBlock) throws 
IOException {
+    Iterator<DeleteRecord> it = 
Arrays.stream(deleteBlock.getRecordsToDelete()).iterator();
+    while (it.hasNext()) {
+      DeleteRecord record = it.next();
+      String recordKey = record.getRecordKey();
+      processNextDeletedRecord(record, recordKey);
+    }
+  }
+
+  @Override
+  public void processNextDeletedRecord(DeleteRecord deleteRecord, String 
recordKey) {
+    Pair<Option<T>, Map<String, Object>> existingRecordMetadataPair = 
records.get(recordKey);
+    if (existingRecordMetadataPair != null) {

Review Comment:
   There is still significant duplicate block of code between this and 
`HoodiePositionBasedFileGroupRecordBuffer`. Please see if there are 
opportunities to reuse.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to