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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/PartitionBucketIndexCalculator.java:
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StoragePath;
+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.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * A singleton implementation of PartitionBucketIndexCalculator that ensures 
only one instance
+ * exists for each unique hashingInstantToLoad value.
+ */
+public class PartitionBucketIndexCalculator implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = 
LoggerFactory.getLogger(PartitionBucketIndexCalculator.class);
+  // Map to store singleton instances for each instantToLoad + configuration 
hash combination
+  private static final ConcurrentMap<String, PartitionBucketIndexCalculator> 
INSTANCES = new ConcurrentHashMap<>();
+  protected int cacheSize = 100_000;
+  protected PartitionBucketIndexHashingConfig hashingConfig;
+  private int defaultBucketNumber;
+  private String instantToLoad;
+  // Cache for partition to bucket number mapping
+  protected Map<String, Integer> partitionToBucketCache = new 
SerializableLRUMap<>(cacheSize);
+  private RuleEngine ruleEngine;
+
+  /**
+   * Private constructor to prevent direct instantiation
+   *
+   * @param instantToLoad The instant to load
+   * @param hadoopConf The Hadoop configuration
+   */
+  private PartitionBucketIndexCalculator(String instantToLoad, Configuration 
hadoopConf, String basePath) {
+    this.instantToLoad = instantToLoad;
+    StoragePath hashingConfigPath = 
PartitionBucketIndexUtils.getHashingConfigPath(basePath, instantToLoad);
+    try (HoodieHadoopStorage storage = new 
HoodieHadoopStorage(hashingConfigPath, 
HadoopFSUtils.getStorageConf(hadoopConf))) {
+      init(storage, hashingConfigPath);
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to initialize 
PartitionBucketIndexCalculator ", e);
+    }
+  }
+
+  private PartitionBucketIndexCalculator(String instantToLoad, 
HoodieTableMetaClient client) {
+    this.instantToLoad = instantToLoad;
+    String metaPath = client.getHashingMetadataConfigPath();
+    StoragePath hashingConfigPath = new StoragePath(metaPath, instantToLoad + 
PartitionBucketIndexHashingConfig.HASHING_CONFIG_FILE_SUFFIX);
+
+    try (HoodieStorage storage = client.getStorage()) {
+      init(storage, hashingConfigPath);
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to initialize 
PartitionBucketIndexCalculator ", e);
+    }
+  }
+
+  private PartitionBucketIndexCalculator(String instantToLoad, 
PartitionBucketIndexHashingConfig config) {
+    this.hashingConfig = config;
+    this.defaultBucketNumber = config.getDefaultBucketNumber();
+    String expressions = config.getExpressions();
+    String ruleType = config.getRule();
+    this.ruleEngine = createRuleEngine(ruleType, expressions);
+  }
+
+  private void init(HoodieStorage storage, StoragePath hashingConfigPath) {
+    Option<PartitionBucketIndexHashingConfig> config = 
PartitionBucketIndexUtils.loadHashingConfig(storage, hashingConfigPath);
+    ValidationUtils.checkArgument(config.isPresent());

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