YuweiXiao commented on a change in pull request #4480:
URL: https://github.com/apache/hudi/pull/4480#discussion_r816418906



##########
File path: 
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.keygen.KeyGenUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestBucketIdentifier {
+
+  public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", 
\"name\":\"nested_col\",\"fields\": ["
+      + "{\"name\": \"prop1\",\"type\": \"string\"},{\"name\": \"prop2\", 
\"type\": \"long\"}]}";
+  public static final String EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": 
\"testrec\",\"fields\": [ "
+      + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", 
\"type\": \"string\"},"
+      + "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+      + "{\"name\": \"pii_col\", \"type\": \"string\"},"
+      + "{\"name\": \"nested_col\",\"type\": "
+      + NESTED_COL_SCHEMA + "}"
+      + "]}";
+
+  @Test
+  public void testBucketFileId() {
+    for (int i = 0; i < 1000; i++) {

Review comment:
       I moved it from the original BUCKET_INDEX test. I guess it should be the 
max allowed size of bucket number. So 2^n should be reasonable. maybe 2^16.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.model;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.PropertyAccessor;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * All the metadata that is used for consistent hashing bucket index
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class HoodieConsistentHashingMetadata implements Serializable {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieConsistentHashingMetadata.class);
+
+  public static final int MAX_HASH_VALUE = Integer.MAX_VALUE;
+  public static final String HASHING_METADATA_FILE_SUFFIX = ".hashing_meta";
+
+  protected short version;
+  protected String partitionPath;
+  protected String instant;
+  protected int numBuckets;
+  protected int seqNo;
+  protected List<ConsistentHashingNode> nodes;
+
+  public HoodieConsistentHashingMetadata() {
+  }
+
+  public HoodieConsistentHashingMetadata(String partitionPath, int numBuckets) 
{
+    this((short) 0, partitionPath, HoodieTimeline.INIT_INSTANT_TS, numBuckets);
+  }
+
+  /**
+   * Construct default metadata with all bucket's file group uuid initialized
+   *
+   * @param partitionPath
+   * @param numBuckets
+   */
+  private HoodieConsistentHashingMetadata(short version, String partitionPath, 
String instant, int numBuckets) {
+    this.version = version;
+    this.partitionPath = partitionPath;
+    this.instant = instant;
+    this.numBuckets = numBuckets;
+
+    nodes = new ArrayList<>();
+    long step = ((long) MAX_HASH_VALUE + numBuckets - 1) / numBuckets;
+    for (int i = 1; i <= numBuckets; ++i) {
+      nodes.add(new ConsistentHashingNode((int) Math.min(step * i, 
MAX_HASH_VALUE), FSUtils.createNewFileIdPfx()));
+    }
+  }
+
+  public short getVersion() {
+    return version;
+  }
+
+  public String getPartitionPath() {
+    return partitionPath;
+  }
+
+  public String getInstant() {
+    return instant;
+  }
+
+  public int getNumBuckets() {
+    return numBuckets;
+  }
+
+  public int getSeqNo() {
+    return seqNo;
+  }
+
+  public List<ConsistentHashingNode> getNodes() {
+    return nodes;
+  }
+
+  public void setInstant(String instant) {
+    this.instant = instant;
+  }
+
+  public void setNodes(List<ConsistentHashingNode> nodes) {
+    this.nodes = nodes;
+    this.numBuckets = nodes.size();
+  }
+
+  public void setSeqNo(int seqNo) {
+    this.seqNo = seqNo;
+  }
+
+  public String getFilename() {
+    return instant + HASHING_METADATA_FILE_SUFFIX;
+  }
+
+  public byte[] toBytes() throws IOException {
+    return toJsonString().getBytes(StandardCharsets.UTF_8);
+  }
+
+  public static HoodieConsistentHashingMetadata fromBytes(byte[] bytes) throws 
IOException {
+    try {
+      return fromJsonString(new String(bytes, StandardCharsets.UTF_8), 
HoodieConsistentHashingMetadata.class);
+    } catch (Exception e) {
+      throw new IOException("unable to read hashing metadata", e);
+    }
+  }
+
+  private String toJsonString() throws IOException {
+    return 
getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);
+  }
+
+  protected static <T> T fromJsonString(String jsonStr, Class<T> clazz) throws 
Exception {
+    if (jsonStr == null || jsonStr.isEmpty()) {
+      // For empty commit file (no data or somethings bad happen).
+      return clazz.newInstance();
+    }
+    return getObjectMapper().readValue(jsonStr, clazz);
+  }
+
+  protected static ObjectMapper getObjectMapper() {
+    ObjectMapper mapper = new ObjectMapper();

Review comment:
       Sure




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