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/09/30 07:45:45 UTC

[GitHub] [hudi] wqwl611 opened a new pull request, #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

wqwl611 opened a new pull request, #6636:
URL: https://github.com/apache/hudi/pull/6636

   ### Change Logs
   The rangeBucket is mainly used in the scenario of sync mysql tables to hudi in near real time, which avoids the disadvantage of the fixed number of buckets in simpleBucket.
   
   Usually, in the mysql table, there is an auto-increment id primary key field. In the mysql cdc synchronization scenario, we can use the database name and table name as the partition field of hudi, and id as the primary key field of the hudi table,This can deal with sub-library and sub-table. 
   
   In order to reach better sync performance, we usually use bucket index, but if we use simple bucket index, because the number of buckets is fixed, it is difficult for us to determine a suitable number of buckets, and as the table grows, The previous number of buckets will no longer be appropriate. 
   
   So, I propose rangeBucekt, in the simpleBucket index, the bucket number is (hash % bucketNum), and in rangetBucket, we will use ( id / fixedStep) to determine the bucket number, so that as the id grows,The number of buckets also increases. For example, if step = 10 is set, then, because the id is self-increasing, a bucket will be generated for every 10 pieces of data.
   
   In the actual scenario, I set step=1,000,000, the usual size of each mysql record is similar, then the approximate size of each bucket will be 50M ~ 350M, which avoids the disadvantage of the fixed number of buckets in simpleBucket
   
   
   
   ### Impact
   
   Introduce a new index RANGE_BUCKET, people can ust it like following:
   
         option(HoodieIndexConfig.INDEX_TYPE.key, IndexType.BUCKET.name()).
         option(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key, "RANGE_BUCKET").
         option(HoodieIndexConfig.RANGE_BUCKET_STEP_SIZE.key, 2).
         option(HoodieLayoutConfig.LAYOUT_TYPE.key, "BUCKET").
   
   **Risk level: none | low | medium | high**
   
   low
   
   


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r969195683


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;

Review Comment:
   Thanks, can we add some test cases for this new index then ? Also @alexeykudinkin , do you have interest on this feature ?



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


[GitHub] [hudi] danny0405 commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1241399179

   Nice feature, can we log an JIRA issue and change the commit title to "[HUDI-${JIRA_ID}] ${your title}"


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262069731

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 8530a43c8839ce9ca1c6891585d141eca7641304 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711) 
   * 57c510b4c10006da8a8d901a1f060b12491263d5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
xushiyan commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262152883

   @wqwl611 there are a bunch of flakiness fixes went in master. you might want to rebase


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256939849

   > Hey, thanks for the contribution. It is a great enhancement for bucket index.
   > 
   > On high-level, could we use the current BucketIndex abstraction to unify the implementation of different BucketIndexEngines? Also, the dedicated Partitioner (i.e., SparkRangeBucketIndexPartitioner) may not be necessary, as long as we tag the file id during indexing (checkout consistent hashing which uses default Partitioner).
   
        Right now, rangBucketIndex generate file like "00000009-0_2-12-29_20220924180225595.parquet",
   and it doesn't contain any UUID element,  I think it's ok, am I right?
   
        By this clue, if simpleBucketIndex also act like this, SparkBucketIndexPartitioner may not be necessary eigther?
   and if use default partitioner, it can reduce a lot of empty spark-task。
   @YuweiXiao 
   


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979351457


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   > Be caution the instantTime in record lcoation should never be null.
   
   Becase we know the record location even when record first insert, so here implement like HoodieSparkConsistentBucketIndex set instantTime null and maybe we don't need it.
   @YuweiXiao  please take a look



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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257144913

   > > > Hey, thanks for the contribution. It is a great enhancement for bucket index.
   > > > On high-level, could we use the current BucketIndex abstraction to unify the implementation of different BucketIndexEngines? Also, the dedicated Partitioner (i.e., SparkRangeBucketIndexPartitioner) may not be necessary, as long as we tag the file id during indexing (checkout consistent hashing which uses default Partitioner).
   > > 
   > > 
   > > ```
   > >  Right now, rangBucketIndex generate file like "00000009-0_2-12-29_20220924180225595.parquet", and it doesn't contain any UUID element,  I think it's ok, am I right?
   > >  By this clue, if simpleBucketIndex also act like this, SparkBucketIndexPartitioner may not be necessary eigther? and if use default partitioner, it can reduce a lot of empty spark-task。
   > > ```
   > > 
   > > 
   > >     
   > >       
   > >     
   > > 
   > >       
   > >     
   > > 
   > >     
   > >   
   > > @YuweiXiao
   > 
   > Yeah, I was thinking the same thing, have id as the name rather than concatenating the uuid. But I think the benefit is saving the metadata loading overhead (i.e., listing to get the filename) rather than the one you mentioned. With the default partitioner, it should not be empty partition (`UpsertPartitioner`). Please correct me if I am wrong.
   > 
   > Also, we better to follow the naming convention of the file group, in case of potential compatibility problems.
   
         yes,every empty bucket will access metadata in ‘getBucketInfo’,when partitionNum * bucketNum is very big,it‘s a heary overhead for metadata (and spark driver scheduler don't like it eigther)
   
   More important!I'am afraid that we can't follow ‘uuid naming convention’, because this name is genarated in rdd task one by one record but not a one by one bucket  like simpleBucketIndex rigtht now
   @YuweiXiao 
   


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1242322167

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b837b813fb706508b1fccc0924f839275e9373c3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253) 
   * 2b7e99f0e851f61b92f027928f208180c533c653 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1242327333

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b837b813fb706508b1fccc0924f839275e9373c3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253) 
   * 2b7e99f0e851f61b92f027928f208180c533c653 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245331249

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   * 985e9736cadeeda86a96d5641ec6fd01afdee194 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r969778260


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;

Review Comment:
   @danny0405 ok, a base test was added.



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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r978390561


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##########
@@ -64,6 +69,29 @@ public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatu
     return writeStatuses;
   }
 
+  protected Map<Integer, HoodieRecordLocation> loadPartitionBucketIdFileIdMapping(

Review Comment:
   @YuweiXiao 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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256940118

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 2f75d8a39026d6d2039754eccb90b7b3ed0dcfee Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608) 
   * 0dd96e8c8209bf3a6f3faced96c867d888974260 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342023


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+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.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    initSparkContexts();
+    initPath();
+    initFileSystem();
+    // We have some records to be tagged (two different partitions)
+    initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+    cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+    boolean makeConfigSuccess = false;
+    try {
+      makeConfig("_row_key,time");
+      makeConfigSuccess = true;
+    } catch (HoodieIndexException e) {
+      LOG.error(e.getMessage());
+      assertFalse(makeConfigSuccess);
+      makeConfig("_row_key");
+      makeConfigSuccess = true;
+    }
+    assertTrue(makeConfigSuccess);
+  }
+
+  @Test
+  public void testTagLocation() throws Exception {
+    long rowKey1 = 1;
+    long rowKey2 = 9;
+    long rowKey3 = 20;
+    String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+    String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+    String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+    String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
+    RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+    HoodieRecord record1 = new HoodieAvroRecord(
+        new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
+    RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+    HoodieRecord record2 = new HoodieAvroRecord(
+        new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
+    RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+    HoodieRecord record3 = new HoodieAvroRecord(
+        new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
+    RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
+    HoodieRecord record4 = new HoodieAvroRecord(
+        new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
+    JavaRDD<HoodieRecord<HoodieAvroRecord>> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
+    HoodieWriteConfig config = makeConfig("_row_key");
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
+    int bucketRangeStepSize = config.getBucketRangeStepSize();
+    HoodieRangeBucketIndex bucketIndex = new HoodieRangeBucketIndex(config);
+    HoodieData<HoodieRecord<HoodieAvroRecord>> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
+    // even first insert, we should know the location
+    assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> !r.isCurrentLocationKnown()));
+
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA);
+    testTable.addCommit("001").withInserts("2016/01/31", getRecordFileId(record1), record1);
+    testTable.addCommit("002").withInserts("2016/01/31", getRecordFileId(record2), record2);
+    testTable.addCommit("003").withInserts("2016/01/31", getRecordFileId(record3), record3);
+    taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context,
+        HoodieSparkTable.create(config, context, metaClient));
+    boolean present = taggedRecordRDD.collectAsList().stream().filter(r -> r.isCurrentLocationKnown())
+        .filter(r -> BucketIdentifier.bucketIdFromFileId(r.getCurrentLocation().getFileId()) != getRecordBucketId(r)).findAny().isPresent();
+    assertFalse(present);
+    boolean condition = taggedRecordRDD.collectAsList().stream().filter(r -> r.getPartitionPath().equals("2015/01/31") && r.isCurrentLocationKnown()).count() == 1L;
+    assertTrue(condition);

Review Comment:
   assert the location cnt directly instead of the booleans.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341988


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+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.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    initSparkContexts();
+    initPath();
+    initFileSystem();
+    // We have some records to be tagged (two different partitions)
+    initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+    cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+    boolean makeConfigSuccess = false;
+    try {
+      makeConfig("_row_key,time");
+      makeConfigSuccess = true;
+    } catch (HoodieIndexException e) {
+      LOG.error(e.getMessage());
+      assertFalse(makeConfigSuccess);
+      makeConfig("_row_key");
+      makeConfigSuccess = true;
+    }
+    assertTrue(makeConfigSuccess);
+  }
+
+  @Test
+  public void testTagLocation() throws Exception {
+    long rowKey1 = 1;
+    long rowKey2 = 9;
+    long rowKey3 = 20;
+    String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+    String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+    String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+    String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
+    RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+    HoodieRecord record1 = new HoodieAvroRecord(
+        new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
+    RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+    HoodieRecord record2 = new HoodieAvroRecord(
+        new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
+    RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+    HoodieRecord record3 = new HoodieAvroRecord(
+        new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
+    RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
+    HoodieRecord record4 = new HoodieAvroRecord(
+        new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
+    JavaRDD<HoodieRecord<HoodieAvroRecord>> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
+    HoodieWriteConfig config = makeConfig("_row_key");
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
+    int bucketRangeStepSize = config.getBucketRangeStepSize();
+    HoodieRangeBucketIndex bucketIndex = new HoodieRangeBucketIndex(config);
+    HoodieData<HoodieRecord<HoodieAvroRecord>> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
+    // even first insert, we should know the location
+    assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> !r.isCurrentLocationKnown()));
+
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA);
+    testTable.addCommit("001").withInserts("2016/01/31", getRecordFileId(record1), record1);
+    testTable.addCommit("002").withInserts("2016/01/31", getRecordFileId(record2), record2);
+    testTable.addCommit("003").withInserts("2016/01/31", getRecordFileId(record3), record3);
+    taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context,
+        HoodieSparkTable.create(config, context, metaClient));
+    boolean present = taggedRecordRDD.collectAsList().stream().filter(r -> r.isCurrentLocationKnown())
+        .filter(r -> BucketIdentifier.bucketIdFromFileId(r.getCurrentLocation().getFileId()) != getRecordBucketId(r)).findAny().isPresent();

Review Comment:
   `present` -> `hasWrongBucketIds`



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257144417

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1f939eeefe4435a9cb009c0648bcaffcaa990a1a Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707) 
   * 8530a43c8839ce9ca1c6891585d141eca7641304 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1263436625

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11924",
       "triggerID" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11924) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979354166


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   > I guess the log files can also be indexed here ?
   
   may be indexed log files is not so useful in bucket scene. and in BaseSparkDeltaCommitActionExecutor.handleInsert, 
   if set this true, the first insert will just write a log file but not baseFile, it's not a good chioce.
   @YuweiXiao please help check this.



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256968507

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 0dd96e8c8209bf3a6f3faced96c867d888974260 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255452368

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1ef2c6f59d7ce1616b4cf36494373007e2ad9450 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1258907260

   
   @alexeykudinkin I update this PR Change Logs, please check it


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1240711307

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b837b813fb706508b1fccc0924f839275e9373c3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245511345

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   * 985e9736cadeeda86a96d5641ec6fd01afdee194 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326) 
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
xushiyan commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1296886534

   The work continues in https://github.com/apache/hudi/pull/6858


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r978271511


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##########
@@ -64,6 +69,29 @@ public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatu
     return writeStatuses;
   }
 
+  protected Map<Integer, HoodieRecordLocation> loadPartitionBucketIdFileIdMapping(

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

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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256246931

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * cdbc804bd8ecd88141ec461fd16716c66397c8db Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341202


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   I guess the log files can also be indexed here ?



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979369308


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   Just checked you new commits. Even you set true here, the first write should produce base file, right? As you guard the the empty file group writing.



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1242448980

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b7e99f0e851f61b92f027928f208180c533c653 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1241430662

   > Nice feature, can we log an JIRA issue and change the commit title to "[HUDI-${JIRA_ID}] ${your title}"
   
   yes


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1263286348

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 84076b27bdf11455c54a0d878089724e65e9daf9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904) 
   * 5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r978391152


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##########
@@ -64,6 +69,29 @@ public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatu
     return writeStatuses;
   }
 
+  protected Map<Integer, HoodieRecordLocation> loadPartitionBucketIdFileIdMapping(

Review Comment:
   @YuweiXiao please help review it. Thanks.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979338893


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java:
##########
@@ -674,6 +685,12 @@ public EngineType getEngineType() {
 
     private void validateBucketIndexConfig() {
       if (hoodieIndexConfig.getString(INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.toString())) {
+        if (HoodieIndex.BucketIndexEngineType.RANGE_BUCKET.toString().equalsIgnoreCase(hoodieIndexConfig.getString(BUCKET_INDEX_ENGINE_TYPE))) {
+          if (hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME).contains(",")) {
+            throw new HoodieIndexException("Range Bucket index only support single-record-key. Best auto-increment key.");
+          }
+          return;

Review Comment:
   `single-record-key` -> `single record key field`



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979367701


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   The null value will be overwritten in the write handle. 
   
   ps. As we will write log files most of the time, the instant will always be overwrite by the one of base files.



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


[GitHub] [hudi] YuweiXiao commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257139465

   > > Hey, thanks for the contribution. It is a great enhancement for bucket index.
   > > On high-level, could we use the current BucketIndex abstraction to unify the implementation of different BucketIndexEngines? Also, the dedicated Partitioner (i.e., SparkRangeBucketIndexPartitioner) may not be necessary, as long as we tag the file id during indexing (checkout consistent hashing which uses default Partitioner).
   > 
   > ```
   >  Right now, rangBucketIndex generate file like "00000009-0_2-12-29_20220924180225595.parquet", and it doesn't contain any UUID element,  I think it's ok, am I right?
   >  By this clue, if simpleBucketIndex also act like this, SparkBucketIndexPartitioner may not be necessary eigther? and if use default partitioner, it can reduce a lot of empty spark-task。
   > ```
   > 
   > @YuweiXiao
   
   Yeah, I was thinking the same thing, have id as the name rather than concatenating the uuid. But I think the benefit is saving the metadata loading overhead (i.e., listing to get the filename) rather than the one you mentioned. With the default partitioner, it should not be empty partition (`UpsertPartitioner`). Please correct me if I am wrong.
   
   Also, we better to follow the naming convention of the file group, in case of potential compatibility problems. 


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r983340547


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   I use a fixed uuid(00000000-0000-0000-0000-000000000000) to spell fileId, as shown below, do you think this is ok? please review my new commit。
   @YuweiXiao 
   
   
   <img width="979" alt="image" src="https://user-images.githubusercontent.com/67826098/193003627-e716d5cd-faa7-40dd-8961-8a77a7b20f12.png">
   



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


[GitHub] [hudi] xushiyan commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
xushiyan commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1277614369

   @wqwl611 thanks for the work but looks like this PR is identical to #6858 . should we continue using this one given we have reviews here already and close #6858 ?


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1241210655

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b837b813fb706508b1fccc0924f839275e9373c3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262153425

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 57c510b4c10006da8a8d901a1f060b12491263d5 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262777534

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 84076b27bdf11455c54a0d878089724e65e9daf9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1242259656

   https://issues.apache.org/jira/browse/HUDI-4824
   @danny0405 


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r973901133


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkRangeBucketIndexPartitioner.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.table.action.commit;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.index.bucket.HoodieRangeBucketIndex;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.WorkloadProfile;
+import org.apache.hudi.table.WorkloadStat;
+import scala.Tuple2;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+/**
+ * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
+ */
+public class SparkRangeBucketIndexPartitioner<T extends HoodieRecordPayload<T>> extends
+    SparkHoodiePartitioner<T> {
+
+  private final int bucketRangeStepSize;
+
+  private final Map<Pair<String, Integer>, Integer> partitionBucketNumIndexMap;
+
+  private final Map<Integer, Pair<String, Integer>> indexPartitionBucketNumMap;
+
+  /**
+   * Partition path and file groups in it pair. Decide the file group an incoming update should go to.
+   */
+  private Map<String, Set<String>> updatePartitionPathFileIds;
+
+  public SparkRangeBucketIndexPartitioner(WorkloadProfile profile,
+                                          HoodieEngineContext context,
+                                          HoodieTable table,
+                                          HoodieWriteConfig config) {
+    super(profile, table);
+    if (!(table.getIndex() instanceof HoodieRangeBucketIndex)) {
+      throw new HoodieException(
+        " Bucket index partitioner should only be used by BucketIndex other than "
+          + table.getIndex().getClass().getSimpleName());
+    }
+    this.bucketRangeStepSize = config.getBucketRangeStepSize();
+    this.indexPartitionBucketNumMap = ((HoodieRangeBucketIndex) table.getIndex()).getIndexPartitionBucketNumMap();
+    this.partitionBucketNumIndexMap = ((HoodieRangeBucketIndex) table.getIndex()).getPartitionBucketNumIndexMap();
+
+    assignUpdates(profile);
+  }
+
+  private void assignUpdates(WorkloadProfile profile) {
+    updatePartitionPathFileIds = new HashMap<>();
+    // each update location gets a partition
+    Set<Entry<String, WorkloadStat>> partitionStatEntries = profile.getInputPartitionPathStatMap()
+        .entrySet();
+    for (Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
+      if (!updatePartitionPathFileIds.containsKey(partitionStat.getKey())) {
+        updatePartitionPathFileIds.put(partitionStat.getKey(), new HashSet<>());
+      }
+      for (Entry<String, Pair<String, Long>> updateLocEntry :
+          partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
+        updatePartitionPathFileIds.get(partitionStat.getKey()).add(updateLocEntry.getKey());
+      }
+    }
+  }
+
+  @Override
+  public BucketInfo getBucketInfo(int partitionNum) {
+    Pair<String, Integer> pmod = indexPartitionBucketNumMap.get(partitionNum);
+    String partitionPath = pmod.getLeft();
+    String bucketId = BucketIdentifier.bucketIdStr(pmod.getRight());
+    Option<String> fileIdOption = Option.fromJavaOptional(updatePartitionPathFileIds
+        .getOrDefault(partitionPath, Collections.emptySet()).stream()
+        .filter(e -> e.startsWith(bucketId))
+        .findFirst());
+    if (fileIdOption.isPresent()) {
+      return new BucketInfo(BucketType.UPDATE, fileIdOption.get(), partitionPath);
+    } else {
+      return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath);
+    }
+  }
+
+  @Override
+  public int numPartitions() {
+    return indexPartitionBucketNumMap.size();
+  }
+
+  @Override
+  public int getPartition(Object key) {
+    Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
+    String partitionPath = keyLocation._1.getPartitionPath();
+
+    String recordKey = keyLocation._1.getRecordKey();
+    if (recordKey.contains(":")) {
+      recordKey = recordKey.substring(recordKey.indexOf(":") + 1);
+    }
+    long l = Long.parseLong(recordKey) / bucketRangeStepSize;

Review Comment:
   Maybe extract the bucket number calculate logic to BucketIdentifier.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieIndex<Object, Object> {

Review Comment:
   Could we extend `HoodieBucketIndex` to use the `LocationMapper` abstraction, in order to reuse code and unify the implementation?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieIndex<Object, Object> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieRangeBucketIndex.class);
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatuses,
+                                                HoodieEngineContext context,
+                                                HoodieTable hoodieTable)
+      throws HoodieIndexException {
+    return writeStatuses;
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> tagLocation(
+      HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
+      HoodieTable hoodieTable)
+      throws HoodieIndexException {
+    List<Pair<String, Integer>> partitionPathAndBucketNumPair = records.map((SerializableFunction<HoodieRecord<R>, Pair<String, Integer>>) v1 -> {
+      String partitionPath = v1.getPartitionPath();
+      String recordKey = v1.getKey().getRecordKey();

Review Comment:
   Seems we cannot handle multi-record-key case? like `record_key=a:1,b:2` and range partition on column a. Could we add that as well? or add validation for single-record-key only at the very beginning.



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256354109

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 2f75d8a39026d6d2039754eccb90b7b3ed0dcfee Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979338948


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1648,6 +1648,10 @@ public int getBucketIndexNumBuckets() {
     return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS);
   }
 
+  public int getBucketRangeStepSize() {
+    return getIntOrDefault(HoodieIndexConfig.RANGE_BUCKET_STEP_SIZE);
+  }

Review Comment:
   `getBucketRangeStepSize` -> `getRangeBucketStepSize`



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257142796

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 0dd96e8c8209bf3a6f3faced96c867d888974260 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692) 
   * 1f939eeefe4435a9cb009c0648bcaffcaa990a1a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979340081


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java:
##########
@@ -42,6 +50,20 @@ public static int getBucketId(HoodieKey hoodieKey, String indexKeyFields, int nu
     return (getHashKeys(hoodieKey, indexKeyFields).hashCode() & Integer.MAX_VALUE) % numBuckets;
   }
 
+  public static int getRangeBucketNum(String recordKey, int bucketRangeStepSize) {
+    int index = recordKey.indexOf(":");
+    if (index >= 0) {
+      recordKey = recordKey.substring(index + 1);
+    }
+    try {
+      int bucketNum = (int) (Long.parseLong(recordKey) / bucketRangeStepSize);
+      return bucketNum;
+    } catch (Exception e) {
+      LOG.error("RANGE_BUCKET index need a primary key like Interger.", e);
+      throw e;

Review Comment:
   `need a primary key like Interger` -> `needs integer type primary key`



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


[GitHub] [hudi] wqwl611 commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1241432142

   > Nice feature, can we log an JIRA issue and change the commit title to "[HUDI-${JIRA_ID}] ${your title}"
   
   @danny0405 yes,Thanks。


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1244906545

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b7e99f0e851f61b92f027928f208180c533c653 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273) 
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1240718297

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b837b813fb706508b1fccc0924f839275e9373c3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1244998559

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1263291647

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11924",
       "triggerID" : "5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 84076b27bdf11455c54a0d878089724e65e9daf9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904) 
   * 5bcf55eb76b9e7abf7a18ffcedfe638f4d11c643 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11924) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1263252224

   > @wqwl611 there are a bunch of flakiness fixes went in master. you might want to rebase
   
   @xushiyan I miss rebase my repo,and I delete it and refork it,apply all the changes,how can I use the old pr。


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341382


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   Be caution the instantTime in record lcoation should never be null.



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


[GitHub] [hudi] alexeykudinkin commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1258544325

   @wqwl611 would really love to understand your use-case a little better and how `RANGE_BUCKET` can address it better than existing implementations (for ex Bucket Index `SIMPLE` engine-type). Can you please elaborate on this in the PR description?


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255309399

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1ef2c6f59d7ce1616b4cf36494373007e2ad9450 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979351673


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   > what is the `partitionPathFileIDList` used for ? To support changing the step size (or buckets) ?
   
   It's a map, which  record the existed bucket location in each partition, it's not  To support changing the step size (or buckets) , it's for tag a record location.
   



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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1279798025

   > @wqwl611 thanks for the work but looks like this PR is identical to #6858 . should we continue using this one given we have reviews here already and close #6858 ?
   
   I deleted my fork by mistake before, and now I have made changes in the new branch (HUDI-4824) in my new fork, so I  created a new pr(6858), and I don't confirm the  current pr if is available?
   @xushiyan 


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245604298

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 985e9736cadeeda86a96d5641ec6fd01afdee194 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326) 
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245337477

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   * 985e9736cadeeda86a96d5641ec6fd01afdee194 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341444


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   what is the `partitionPathFileIDList` used for ? To support changing the step size (or buckets) ?



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255811097

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1ef2c6f59d7ce1616b4cf36494373007e2ad9450 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585) 
   * cdbc804bd8ecd88141ec461fd16716c66397c8db UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979351673


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   > what is the `partitionPathFileIDList` used for ? To support changing the step size (or buckets) ?
   
   It's a map, which  holds the existed ( bucketId -> HoodieLocation) in each partition, it's not  To support changing the step size (or buckets) , it's for tag a record location.
   



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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979371451


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   @YuweiXiao when init a big hudi table,produce log-file-only is not good chioce。
   now I found the solution, please help check my new commits,  find is it proper?becase if not ‘super.handleInsert(fileId, recordItr);’, the sub handleInsert will write a log file when ‘table.getIndex().canIndexLogFiles()’ is true。
   



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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r977731323


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieIndex<Object, Object> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieRangeBucketIndex.class);
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatuses,
+                                                HoodieEngineContext context,
+                                                HoodieTable hoodieTable)
+      throws HoodieIndexException {
+    return writeStatuses;
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> tagLocation(
+      HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
+      HoodieTable hoodieTable)
+      throws HoodieIndexException {
+    List<Pair<String, Integer>> partitionPathAndBucketNumPair = records.map((SerializableFunction<HoodieRecord<R>, Pair<String, Integer>>) v1 -> {
+      String partitionPath = v1.getPartitionPath();
+      String recordKey = v1.getKey().getRecordKey();

Review Comment:
   @YuweiXiao I optimization my code folloy your guide, please help my review it.
   And I think single-record-key is enough,and I add validation.



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256941093

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 2f75d8a39026d6d2039754eccb90b7b3ed0dcfee Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608) 
   * 0dd96e8c8209bf3a6f3faced96c867d888974260 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979368160


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }

Review Comment:
   @wqwl611 You are right, the first written will be log file only file group. I guess the overhead is ok as you need a compaction sooner or later when you have follow-up updates. 
   
   By the way, if you set it to false, you will also produce log-file-only file group if you do not have your own partitioner.



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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r977731639


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkRangeBucketIndexPartitioner.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.table.action.commit;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.index.bucket.HoodieRangeBucketIndex;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.WorkloadProfile;
+import org.apache.hudi.table.WorkloadStat;
+import scala.Tuple2;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+/**
+ * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
+ */
+public class SparkRangeBucketIndexPartitioner<T extends HoodieRecordPayload<T>> extends
+    SparkHoodiePartitioner<T> {
+
+  private final int bucketRangeStepSize;
+
+  private final Map<Pair<String, Integer>, Integer> partitionBucketNumIndexMap;
+
+  private final Map<Integer, Pair<String, Integer>> indexPartitionBucketNumMap;
+
+  /**
+   * Partition path and file groups in it pair. Decide the file group an incoming update should go to.
+   */
+  private Map<String, Set<String>> updatePartitionPathFileIds;
+
+  public SparkRangeBucketIndexPartitioner(WorkloadProfile profile,
+                                          HoodieEngineContext context,
+                                          HoodieTable table,
+                                          HoodieWriteConfig config) {
+    super(profile, table);
+    if (!(table.getIndex() instanceof HoodieRangeBucketIndex)) {
+      throw new HoodieException(
+        " Bucket index partitioner should only be used by BucketIndex other than "
+          + table.getIndex().getClass().getSimpleName());
+    }
+    this.bucketRangeStepSize = config.getBucketRangeStepSize();
+    this.indexPartitionBucketNumMap = ((HoodieRangeBucketIndex) table.getIndex()).getIndexPartitionBucketNumMap();
+    this.partitionBucketNumIndexMap = ((HoodieRangeBucketIndex) table.getIndex()).getPartitionBucketNumIndexMap();
+
+    assignUpdates(profile);
+  }
+
+  private void assignUpdates(WorkloadProfile profile) {
+    updatePartitionPathFileIds = new HashMap<>();
+    // each update location gets a partition
+    Set<Entry<String, WorkloadStat>> partitionStatEntries = profile.getInputPartitionPathStatMap()
+        .entrySet();
+    for (Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
+      if (!updatePartitionPathFileIds.containsKey(partitionStat.getKey())) {
+        updatePartitionPathFileIds.put(partitionStat.getKey(), new HashSet<>());
+      }
+      for (Entry<String, Pair<String, Long>> updateLocEntry :
+          partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
+        updatePartitionPathFileIds.get(partitionStat.getKey()).add(updateLocEntry.getKey());
+      }
+    }
+  }
+
+  @Override
+  public BucketInfo getBucketInfo(int partitionNum) {
+    Pair<String, Integer> pmod = indexPartitionBucketNumMap.get(partitionNum);
+    String partitionPath = pmod.getLeft();
+    String bucketId = BucketIdentifier.bucketIdStr(pmod.getRight());
+    Option<String> fileIdOption = Option.fromJavaOptional(updatePartitionPathFileIds
+        .getOrDefault(partitionPath, Collections.emptySet()).stream()
+        .filter(e -> e.startsWith(bucketId))
+        .findFirst());
+    if (fileIdOption.isPresent()) {
+      return new BucketInfo(BucketType.UPDATE, fileIdOption.get(), partitionPath);
+    } else {
+      return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath);
+    }
+  }
+
+  @Override
+  public int numPartitions() {
+    return indexPartitionBucketNumMap.size();
+  }
+
+  @Override
+  public int getPartition(Object key) {
+    Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
+    String partitionPath = keyLocation._1.getPartitionPath();
+
+    String recordKey = keyLocation._1.getRecordKey();
+    if (recordKey.contains(":")) {
+      recordKey = recordKey.substring(recordKey.indexOf(":") + 1);
+    }
+    long l = Long.parseLong(recordKey) / bucketRangeStepSize;

Review Comment:
   done



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieIndex<Object, Object> {

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

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342473


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }

Review Comment:
   Log some info about the range bucket just like the base class.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342415


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##########
@@ -45,12 +48,14 @@ public abstract class HoodieBucketIndex extends HoodieIndex<Object, Object> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class);
 
-  protected final int numBuckets;
-  protected final List<String> indexKeyFields;
+  protected int numBuckets;
+  protected List<String> indexKeyFields;
 
   public HoodieBucketIndex(HoodieWriteConfig config) {
     super(config);
-
+    if (config.getBucketIndexEngineType() == BucketIndexEngineType.RANGE_BUCKET) {
+      return;
+    }

Review Comment:
   We should not write sub-class specific logic in base classes. So let's not return early here.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r978203697


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##########
@@ -64,6 +69,29 @@ public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatu
     return writeStatuses;
   }
 
+  protected Map<Integer, HoodieRecordLocation> loadPartitionBucketIdFileIdMapping(

Review Comment:
   Maybe add some comments, to clarify it is used by SimpleBucket & RangeBucket.



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


[GitHub] [hudi] xushiyan closed pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
xushiyan closed pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table
URL: https://github.com/apache/hudi/pull/6636


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262523328

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 57c510b4c10006da8a8d901a1f060b12491263d5 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893) 
   * 84076b27bdf11455c54a0d878089724e65e9daf9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11904) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979351673


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+    super(config);
+    bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) {
+    return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements BucketIndexLocationMapper {
+
+    /**
+     * Mapping from partitionPath -> bucketId -> fileInfo
+     */
+    private final Map<String, Map<Integer, HoodieRecordLocation>> partitionPathFileIDList;
+
+    public RangeBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
+      partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+    }
+
+    @Override
+    public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key) {
+      int bucketId = BucketIdentifier.getRangeBucketId(key, bucketRangeStepSize);
+      Map<Integer, HoodieRecordLocation> bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath());
+      if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+        return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+      } else {
+        return Option.ofNullable(new HoodieRecordLocation(null, BucketIdentifier.bucketIdStr(bucketId)));
+      }

Review Comment:
   > what is the `partitionPathFileIDList` used for ? To support changing the step size (or buckets) ?
   It's a map, which  record the existed bucket location in each partition, it's not  To support changing the step size (or buckets) , it's for tag a record location.
   



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257188145

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 8530a43c8839ce9ca1c6891585d141eca7641304 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 closed pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 closed pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table
URL: https://github.com/apache/hudi/pull/6636


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245695553

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] wqwl611 commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
wqwl611 commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1265711269

   I accidentally deleted my previous fork, so I opened a new PR, and modified the code according to the previous review requirements, please help me review it,thanks.
   @danny0405 @YuweiXiao @xushiyan @alexeykudinkin 
   
   https://github.com/apache/hudi/pull/6858


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262451400

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "84076b27bdf11455c54a0d878089724e65e9daf9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 57c510b4c10006da8a8d901a1f060b12491263d5 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893) 
   * 84076b27bdf11455c54a0d878089724e65e9daf9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1262074982

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893",
       "triggerID" : "57c510b4c10006da8a8d901a1f060b12491263d5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 8530a43c8839ce9ca1c6891585d141eca7641304 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711) 
   * 57c510b4c10006da8a8d901a1f060b12491263d5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11893) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1245519400

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   * 985e9736cadeeda86a96d5641ec6fd01afdee194 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326) 
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1244941654

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b7e99f0e851f61b92f027928f208180c533c653 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273) 
   * b999bb8532df8c5cdf236b3fc891eaf9b90a8487 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1256348260

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * cdbc804bd8ecd88141ec461fd16716c66397c8db Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612) 
   * 2f75d8a39026d6d2039754eccb90b7b3ed0dcfee UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341624


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+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.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    initSparkContexts();
+    initPath();
+    initFileSystem();
+    // We have some records to be tagged (two different partitions)
+    initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+    cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+    boolean makeConfigSuccess = false;
+    try {
+      makeConfig("_row_key,time");
+      makeConfigSuccess = true;
+    } catch (HoodieIndexException e) {
+      LOG.error(e.getMessage());
+      assertFalse(makeConfigSuccess);
+      makeConfig("_row_key");

Review Comment:
   Use assertThrows instead



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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255116409

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257153664

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1f939eeefe4435a9cb009c0648bcaffcaa990a1a Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707) 
   * 8530a43c8839ce9ca1c6891585d141eca7641304 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11711) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255303799

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * adfccbc445f31f6e109dfced1fb47beccc6c7dae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329) 
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1ef2c6f59d7ce1616b4cf36494373007e2ad9450 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1255844460

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 1ef2c6f59d7ce1616b4cf36494373007e2ad9450 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585) 
   * cdbc804bd8ecd88141ec461fd16716c66397c8db Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1257143605

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11253",
       "triggerID" : "b837b813fb706508b1fccc0924f839275e9373c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11273",
       "triggerID" : "2b7e99f0e851f61b92f027928f208180c533c653",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11317",
       "triggerID" : "b999bb8532df8c5cdf236b3fc891eaf9b90a8487",
       "triggerType" : "PUSH"
     }, {
       "hash" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11326",
       "triggerID" : "985e9736cadeeda86a96d5641ec6fd01afdee194",
       "triggerType" : "PUSH"
     }, {
       "hash" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11329",
       "triggerID" : "adfccbc445f31f6e109dfced1fb47beccc6c7dae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "47864db3234b5dd7115006db07019481c91d65f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47864db3234b5dd7115006db07019481c91d65f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11585",
       "triggerID" : "1ef2c6f59d7ce1616b4cf36494373007e2ad9450",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11612",
       "triggerID" : "cdbc804bd8ecd88141ec461fd16716c66397c8db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11608",
       "triggerID" : "2f75d8a39026d6d2039754eccb90b7b3ed0dcfee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692",
       "triggerID" : "0dd96e8c8209bf3a6f3faced96c867d888974260",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707",
       "triggerID" : "1f939eeefe4435a9cb009c0648bcaffcaa990a1a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8530a43c8839ce9ca1c6891585d141eca7641304",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47864db3234b5dd7115006db07019481c91d65f8 UNKNOWN
   * 0dd96e8c8209bf3a6f3faced96c867d888974260 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11692) 
   * 1f939eeefe4435a9cb009c0648bcaffcaa990a1a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11707) 
   * 8530a43c8839ce9ca1c6891585d141eca7641304 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

Posted by GitBox <gi...@apache.org>.
xushiyan commented on PR #6636:
URL: https://github.com/apache/hudi/pull/6636#issuecomment-1296881222

   > > @wqwl611 thanks for the work but looks like this PR is identical to #6858 . should we continue using this one given we have reviews here already and close #6858 ?
   > 
   > I deleted my fork by mistake before, and now I have made changes in the new branch ([HUDI-4824](https://issues.apache.org/jira/browse/HUDI-4824)) in my new fork, so I created a new pr(6858), and I don't confirm the current pr if is available? @xushiyan
   
   @wqwl611 sorry i don't quite get what you meant. I'll close https://github.com/apache/hudi/pull/6858 to keep the discussion consolidated here.


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