YuweiXiao commented on code in PR #4480:
URL: https://github.com/apache/hudi/pull/4480#discussion_r866555531


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.index.bucket;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.apache.hudi.common.model.ConsistentHashingNode;
+import org.apache.hudi.common.model.HoodieConsistentHashingMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Predicate;
+
+/**
+ * Consistent hashing bucket index implementation, with auto-adjust bucket 
number.
+ * NOTE: bucket resizing is triggered by clustering.
+ */
+public class HoodieSparkConsistentBucketIndex extends HoodieBucketIndex {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieSparkConsistentBucketIndex.class);
+
+  private Map<String, ConsistentBucketIdentifier> partitionToIdentifier;
+
+  public HoodieSparkConsistentBucketIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> 
writeStatuses, HoodieEngineContext context, HoodieTable hoodieTable) throws 
HoodieIndexException {
+    return writeStatuses;
+  }
+
+  /**
+   * Do nothing.
+   * A failed write may create a hashing metadata for a partition. In this 
case, we still do nothing when rolling back
+   * the failed write. Because the hashing metadata created by a writer must 
have 00000000000000 timestamp and can be viewed
+   * as the initialization of a partition rather than as a part of the failed 
write.
+   *
+   * @param instantTime
+   * @return
+   */
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    return true;
+  }
+
+  /**
+   * Initialize bucket metadata for each partition
+   *
+   * @param table
+   * @param partitions partitions that need to be initialized
+   */
+  @Override
+  protected void initialize(HoodieTable table, List<String> partitions) {
+    partitionToIdentifier = new HashMap(partitions.size() + partitions.size() 
/ 3);
+
+    // TODO maybe parallel
+    partitions.stream().forEach(p -> {
+      HoodieConsistentHashingMetadata metadata = loadOrCreateMetadata(table, 
p);
+      ConsistentBucketIdentifier identifier = new 
ConsistentBucketIdentifier(metadata);
+      partitionToIdentifier.put(p, identifier);
+    });
+  }
+
+  /**
+   * Get bucket location for given key and partition
+   *
+   * @param key
+   * @param partitionPath
+   * @return
+   */
+  @Override
+  protected HoodieRecordLocation getBucket(HoodieKey key, String 
partitionPath) {
+    ConsistentHashingNode node = 
partitionToIdentifier.get(partitionPath).getBucket(key, indexKeyFields);
+    if (node.getFileIdPfx() != null && !node.getFileIdPfx().isEmpty()) {
+      /**
+       * Dynamic Bucket Index doesn't need the instant time of the latest file 
group.
+       * We add suffix 0 here to the file uuid, following the naming 
convention, i.e., fileId = [uuid]_[numWrites]
+       */
+      return new HoodieRecordLocation(null, 
FSUtils.createNewFileId(node.getFileIdPfx(), 0));
+    }
+
+    LOG.error("Consistent hashing node has no file group, partition: " + 
partitionPath + ", meta: "
+        + partitionToIdentifier.get(partitionPath).getMetadata().getFilename() 
+ ", record_key: " + key.toString());
+    throw new HoodieIndexException("Failed to getBucket as hashing node has no 
file group");
+  }
+
+  /**
+   * Load hashing metadata of the given partition, if it is not existed, 
create a new one (also persist it into storage)
+   *
+   * @param table     hoodie table
+   * @param partition table partition
+   * @return Consistent hashing metadata
+   */
+  public HoodieConsistentHashingMetadata loadOrCreateMetadata(HoodieTable 
table, String partition) {
+    HoodieConsistentHashingMetadata metadata = loadMetadata(table, partition);
+    if (metadata != null) {
+      return metadata;
+    }
+
+    // There is no metadata, so try to create a new one and save it.
+    metadata = new HoodieConsistentHashingMetadata(partition, numBuckets);
+    if (saveMetadata(table, metadata, false)) {
+      return metadata;
+    }
+
+    // The creation failed, so try load metadata again. Concurrent creation of 
metadata should have succeeded.
+    // Note: the consistent problem of cloud storage is handled internal in 
the HoodieWrapperFileSystem, i.e., ConsistentGuard
+    metadata = loadMetadata(table, partition);
+    ValidationUtils.checkState(metadata != null, "Failed to load or create 
metadata, partition: " + partition);
+    return metadata;
+  }
+
+  /**
+   * Load hashing metadata of the given partition, if it is not existed, 
return null
+   *
+   * @param table     hoodie table
+   * @param partition table partition
+   * @return Consistent hashing metadata or null if it does not exist
+   */
+  public static HoodieConsistentHashingMetadata loadMetadata(HoodieTable 
table, String partition) {
+    Path metadataPath = 
FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), 
partition);
+
+    try {
+      if (!table.getMetaClient().getFs().exists(metadataPath)) {
+        return null;
+      }
+      FileStatus[] metaFiles = 
table.getMetaClient().getFs().listStatus(metadataPath);
+      final HoodieTimeline completedCommits = 
table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants();
+      Predicate<FileStatus> metaFilePredicate = fileStatus -> {
+        String filename = fileStatus.getPath().getName();
+        if 
(!filename.contains(HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX))
 {
+          return false;
+        }
+        String timestamp = 
HoodieConsistentHashingMetadata.getTimestampFromFile(filename);
+        return completedCommits.containsInstant(timestamp) || 
timestamp.equals(HoodieTimeline.INIT_INSTANT_TS);
+      };
+
+      // Get a valid hashing metadata with the largest (latest) timestamp
+      FileStatus metaFile = Arrays.stream(metaFiles).filter(metaFilePredicate)
+          .max(Comparator.comparing(a -> a.getPath().getName())).orElse(null);
+
+      if (metaFile == null) {
+        return null;
+      }
+
+      byte[] content = 
FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath()));
+      return HoodieConsistentHashingMetadata.fromBytes(content);
+    } catch (IOException e) {
+      LOG.warn("Error when loading hashing metadata, partition: " + partition, 
e);
+      throw new HoodieIndexException("Error while loading hashing metadata", 
e);
+    }
+  }
+
+  /**
+   * Save metadata into storage
+   *
+   * @param table
+   * @param metadata
+   * @param overwrite
+   * @return
+   */
+  private static boolean saveMetadata(HoodieTable table, 
HoodieConsistentHashingMetadata metadata, boolean overwrite) {
+    FSDataOutputStream fsOut = null;
+    HoodieWrapperFileSystem fs = table.getMetaClient().getFs();
+    Path dir = 
FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), 
metadata.getPartitionPath());
+    Path fullPath = new Path(dir, metadata.getFilename());
+    try {
+      byte[] bytes = metadata.toBytes();
+      fsOut = fs.create(fullPath, overwrite);
+      fsOut.write(bytes);
+      fsOut.close();
+      return true;
+    } catch (IOException e) {
+      LOG.warn("Failed to update bucket metadata: " + metadata, e);
+    } finally {
+      try {
+        if (fsOut != null) {
+          fsOut.close();

Review Comment:
   Thanks! Fixed.



-- 
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]

Reply via email to