You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/26 01:39:16 UTC

[GitHub] [hudi] codope commented on a diff in pull request #5978: [HUDI-4284] Implement bloom lookup tree as red-black tree

codope commented on code in PR #5978:
URL: https://github.com/apache/hudi/pull/5978#discussion_r1005125747


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java:
##########
@@ -165,12 +194,29 @@ private void updateExpectedMatchesToTest(KeyRangeNode toInsert) {
     long startKey = Long.parseLong(toInsert.getMinRecordKey());
     long endKey = Long.parseLong(toInsert.getMaxRecordKey());
     for (long i = startKey; i <= endKey; i++) {
-      String iStr = Long.toString(i);
+      String iStr = alignedNumber(i);
       if (!expectedMatches.containsKey(iStr)) {
         expectedMatches.put(iStr, new HashSet<>());
       }
       expectedMatches.get(iStr).add(toInsert.getFileNameList().get(0));
     }
   }
 
+  /**
+   * Method to align the number of digits. The key's comparison is in lexicographic order, so we need to fill in the numbers.
+   *
+   * @param key the numeric value of the key
+   * @return result of aligned numbers. For example, `1` -> `00001`.
+   */
+  private String alignedNumber(long key) {

Review Comment:
   Can be static?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java:
##########
@@ -51,10 +50,6 @@ class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter {
       allIndexFiles.add(file);
     }));
 
-    // Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
-    // So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be skewed
-    // which could result in N search time instead of NlogN.
-    Collections.shuffle(allIndexFiles);

Review Comment:
   @yabola do you have some micro-benchmark as to how much improvement this change brings?



##########
hudi-common/src/main/java/org/apache/hudi/common/util/rbtree/RedBlackTree.java:
##########
@@ -0,0 +1,419 @@
+/*
+ * 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.util.rbtree;
+
+import java.io.Serializable;
+
+/**
+ * Red–black tree is a kind of self-balancing binary search tree. Each node has its "color" ("red" or "black"),
+ * used to ensure that the tree remains balanced during insertions and deletions. The re-balancing is not perfect,
+ * but guarantees searching in O(log N) time, where n is the number of entries.
+ * In addition to the requirements imposed on a binary search tree, the following must be satisfied by a red–black tree:
+ * 1. Every node has a color either red or black.
+ * 2. The root of the tree is always black.
+ * 3. There are no two adjacent red nodes (A red node cannot have a red parent or red child).
+ * 4. Every path from a node (including root) to any of its descendants NULL nodes has the same number of black nodes.
+ */
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class RedBlackTree<T extends RedBlackTreeNode<K>, K extends Comparable<K>> implements Serializable {

Review Comment:
   Have you considered thread safety of operations on the tree?



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

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