alexeykudinkin commented on code in PR #7640:
URL: https://github.com/apache/hudi/pull/7640#discussion_r1072550590


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeylessKeyGenerator.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+/**
+ * This class is used to compute a deterministic key for a record based on the 
contents of the field. Unlike the other KeyGenerators in Hudi, this class does 
not take in any field names as args to
+ * create a "keyless" experience for insert only workloads. The keys are 
guaranteed to be deterministic but not unique, so they can only be used for 
insert workflows with deduplication disabled.
+ * The class attempts to get sufficient uniqueness for keys to prevent 
frequent collisions by choosing the fields it uses in order of decreasing 
likelihood for uniqueness. The ordering is:
+ * <ul>
+ *   <li>timestamp</li>
+ *   <li>numeric values</li>
+ *   <li>string, byte arrays, other types not mentioned</li>
+ *   <li>date, lists, maps, booleans</li>
+ * </ul>
+ * The number of fields is capped to created predictable performance and the 
generator only uses non-null values to help increase uniqueness for sparse 
datasets.
+ */
+public class KeylessKeyGenerator extends CustomAvroKeyGenerator {
+  private static final String HOODIE_PREFIX = "_hoodie";
+  private static final String DOT = ".";
+  private final int maxFieldsToConsider;
+  private final int numFieldsForKey;
+  private final Set<String> partitionFieldNames;
+  private int[][] fieldOrdering;
+
+  public KeylessKeyGenerator(TypedProperties props) {
+    super(props);
+    this.numFieldsForKey = 
props.getInteger(KeyGeneratorOptions.NUM_FIELDS_IN_KEYLESS_GENERATOR.key(), 
KeyGeneratorOptions.NUM_FIELDS_IN_KEYLESS_GENERATOR.defaultValue());
+    // cap the number of fields to order in case of large schemas
+    this.maxFieldsToConsider = numFieldsForKey * 3;
+    this.partitionFieldNames = 
this.getPartitionPathFields().stream().map(field -> 
field.split(SPLIT_REGEX)[0]).collect(Collectors.toSet());
+  }
+
+  @Override
+  public String getRecordKey(GenericRecord record) {
+    return buildKey(getFieldOrdering(record), record);
+  }
+
+  int[][] getFieldOrdering(GenericRecord genericRecord) {
+    if (fieldOrdering == null) {
+      fieldOrdering = 
buildFieldOrdering(genericRecord.getSchema().getFields());
+    }
+    return fieldOrdering;
+  }
+
+  /**
+   * Deterministically builds a key for the input value based on the provided 
fieldOrdering. The first {@link #numFieldsForKey} non-null values will be used 
to generate a string that is passed to
+   * {@link UUID#nameUUIDFromBytes(byte[])}.
+   * @param fieldOrdering an array of integer arrays. The integer arrays 
represent paths to a single field within the input object.
+   * @param input the input object that needs a key
+   * @return a deterministically generated {@link UUID}
+   * @param <T> the input object type
+   */
+  private <T> String buildKey(int[][] fieldOrdering, GenericRecord input) {
+    StringBuilder key = new StringBuilder();
+    int nonNullFields = 0;
+    for (int[] index : fieldOrdering) {
+      Object value = getFieldForRecord(input, index);
+      if (value == null) {
+        continue;
+      }
+      nonNullFields++;
+      key.append(value.hashCode());

Review Comment:
   @the-other-tim-brown @the-other-tim-brown this is incorrect way of hash/key 
generation, we can't distinguish b/w cases of hash_1=12 and hash_1=1, hash_2=2



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

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

Reply via email to