You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "codope (via GitHub)" <gi...@apache.org> on 2023/03/24 12:53:42 UTC

[GitHub] [hudi] codope commented on a diff in pull request #8272: use path similar to base file when config is true

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java:
##########
@@ -332,8 +333,13 @@ private HoodieData<HoodieRecord<T>> readRecordsForGroupBaseFiles(JavaSparkContex
           List<Iterator<HoodieRecord<T>>> iteratorsForPartition = new ArrayList<>();
           clusteringOpsPartition.forEachRemaining(clusteringOp -> {
             try {
+              boolean isBootstrapSkeleton = !clusteringOp.getBootstrapFilePath().isEmpty();
               Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema()));
               HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath()));
+              if (isBootstrapSkeleton) {

Review Comment:
   So, for full bootstrap mode, it still goes through the usual base file reader correct?



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java:
##########
@@ -0,0 +1,88 @@
+package org.apache.hudi.io.storage;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+import java.util.Set;
+
+
+public class HoodieBootstrapFileReader<T> implements HoodieFileReader<T> {
+
+  private HoodieFileReader<T> skeletonFileReader;
+  private HoodieFileReader<T> dataFileReader;
+  private Boolean isConsistentLogicalTimestampEnabled;
+
+  public HoodieBootstrapFileReader(HoodieFileReader<T> skeletonFileReader, HoodieFileReader<T> dataFileReader, Boolean isConsistentLogicalTimestampEnabled) {
+    this.skeletonFileReader = skeletonFileReader;
+    this.dataFileReader = dataFileReader;
+    this.isConsistentLogicalTimestampEnabled = isConsistentLogicalTimestampEnabled;
+  }
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    return skeletonFileReader.readMinMaxRecordKeys();
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    return skeletonFileReader.readBloomFilter();
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
+    return skeletonFileReader.filterRowKeys(candidateRowKeys);
+  }
+
+  @Override
+  public ClosableIterator<HoodieRecord<T>> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    ClosableIterator<HoodieRecord<T>> skeletonIterator = skeletonFileReader.getRecordIterator(readerSchema, requestedSchema);
+    ClosableIterator<HoodieRecord<T>> dataFileIterator = dataFileReader.getRecordIterator(HoodieAvroUtils.removeMetadataFields(readerSchema), requestedSchema);
+
+    return new ClosableIterator<HoodieRecord<T>>() {
+      @Override
+      public void close() {
+          skeletonIterator.close();
+          dataFileIterator.close();
+      }
+
+      @Override
+      public boolean hasNext() {
+        return skeletonIterator.hasNext() && dataFileIterator.hasNext();
+      }
+
+      @Override
+      public HoodieRecord<T> next() {
+        HoodieRecord<T> dataRecord = dataFileIterator.next();
+        HoodieRecord<T> skeletonRecord = skeletonIterator.next();
+        HoodieRecord<T> ret = dataRecord.prependMetaFields(readerSchema, readerSchema, new MetadataValues().
+            setCommitTime(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.COMMIT_TIME_METADATA_FIELD ))
+            .setCommitSeqno(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))
+            .setRecordKey(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.RECORD_KEY_METADATA_FIELD))
+            .setPartitionPath(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.PARTITION_PATH_METADATA_FIELD))

Review Comment:
   Is the skeleton record giving the partition path?



-- 
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-unsubscribe@hudi.apache.org

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