zhangyue19921010 commented on code in PR #13017:
URL: https://github.com/apache/hudi/pull/13017#discussion_r2011355721


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/PartitionBucketIndexUtils.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.common.model.PartitionBucketIndexHashingConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.HoodieInstantWriter;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StorageConfiguration;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.StoragePathInfo;
+import org.apache.hudi.storage.hadoop.HoodieHadoopStorage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class PartitionBucketIndexUtils {
+  public static final String INITIAL_HASHING_CONFIG_INSTANT = 
HoodieTimeline.INIT_INSTANT_TS;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PartitionBucketIndexUtils.class);
+
+  public static boolean isPartitionSimpleBucketIndex(Configuration conf, 
String basePath) {
+    return isPartitionSimpleBucketIndex(HadoopFSUtils.getStorageConf(conf), 
basePath);
+  }
+
+  public static boolean isPartitionSimpleBucketIndex(StorageConfiguration 
conf, String basePath) {
+    StoragePath storagePath = getHashingConfigStorageFolder(basePath);
+    try (HoodieHadoopStorage storage = new HoodieHadoopStorage(storagePath, 
conf)) {
+      return storage.exists(storagePath);
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to list 
PARTITION_BUCKET_INDEX_HASHING_FOLDER folder ", e);
+    }
+  }
+
+  public static StoragePath getHashingConfigStorageFolder(String basePath) {
+    StoragePath metaPath = new StoragePath(basePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+    return new StoragePath(metaPath, 
HoodieTableMetaClient.BUCKET_INDEX_METAFOLDER_CONFIG_FOLDER);
+  }
+
+  public static StoragePath getHashingConfigPath(String basePath, String 
instantToLoad) {
+    StoragePath hashingBase = getHashingConfigStorageFolder(basePath);
+    return new StoragePath(hashingBase, instantToLoad + 
PartitionBucketIndexHashingConfig.HASHING_CONFIG_FILE_SUFFIX);
+  }
+
+  public static boolean initHashingConfig(HoodieTableMetaClient metaClient,
+                                          String expressions,
+                                          String rule,
+                                          int defaultBucketNumber,
+                                          String instant) {
+    if (StringUtils.isNullOrEmpty(expressions)) {
+      return false;
+    }
+    String hashingInstant = StringUtils.isNullOrEmpty(instant) ? 
INITIAL_HASHING_CONFIG_INSTANT : instant;
+    PartitionBucketIndexHashingConfig hashingConfig =
+        new PartitionBucketIndexHashingConfig(expressions, 
defaultBucketNumber, rule, PartitionBucketIndexHashingConfig.CURRENT_VERSION, 
hashingInstant);
+    return saveHashingConfig(hashingConfig, metaClient);
+  }
+
+  public static boolean saveHashingConfig(PartitionBucketIndexHashingConfig 
hashingConfig, HoodieTableMetaClient metaClient) {
+    StoragePath hashingConfigPath = new 
StoragePath(metaClient.getHashingMetadataConfigPath(), 
hashingConfig.getFilename());
+    HoodieStorage storage = metaClient.getStorage();
+    try {
+      Option<byte []> content = 
Option.of(hashingConfig.toJsonString().getBytes(StandardCharsets.UTF_8));
+      storage.createImmutableFileInPath(hashingConfigPath, 
content.map(HoodieInstantWriter::convertByteArrayToWriter));
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Failed to initHashingConfig ", ioe);
+    }
+    return true;
+  }
+
+  public static Option<PartitionBucketIndexHashingConfig> 
loadHashingConfig(HoodieStorage storage, StoragePathInfo hashingConfig) {
+    return loadHashingConfig(storage, hashingConfig.getPath());
+  }
+
+  public static Option<PartitionBucketIndexHashingConfig> 
loadHashingConfig(HoodieStorage storage, StoragePath hashingConfig) {
+    if (hashingConfig == null) {
+      return Option.empty();
+    }
+    try (InputStream is = storage.open(hashingConfig)) {
+      byte[] content = FileIOUtils.readAsByteArray(is);
+      return Option.of(PartitionBucketIndexHashingConfig.fromBytes(content));
+    } catch (IOException e) {
+      LOG.error("Error when loading hashing config, for path: " + 
hashingConfig.getName(), e);
+      throw new HoodieIOException("Error while loading hashing config", e);
+    }
+  }
+
+  public static String getHashingConfigInstantToLoad(HoodieTableMetaClient 
metaClient) {

Review Comment:
   done.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BucketIndexBulkInsertPartitionerWithRows.java:
##########
@@ -31,15 +35,29 @@ public class BucketIndexBulkInsertPartitionerWithRows 
implements BulkInsertParti
 
   private final String indexKeyFields;
   private final int bucketNum;
+  private boolean isPartitionBucketIndexEnable = false;
+  private PartitionBucketIndexCalculator calc;
 
-  public BucketIndexBulkInsertPartitionerWithRows(String indexKeyFields, int 
bucketNum) {
+  public BucketIndexBulkInsertPartitionerWithRows(String indexKeyFields, int 
bucketNum, HoodieTable table) {
     this.indexKeyFields = indexKeyFields;
     this.bucketNum = bucketNum;
+    String hashingInstantToLoad = 
table.getConfig().getHashingConfigInstantToLoad();

Review Comment:
   done.



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