You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "beyond1920 (via GitHub)" <gi...@apache.org> on 2023/06/19 03:02:06 UTC

[GitHub] [hudi] beyond1920 opened a new pull request, #9012: [HUDI-6326] Flink write supports consistent bucket index

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

   ### Change Logs
   
   This pr is the first subtask of [HUDI-4373](https://issues.apache.org/jira/browse/HUDI-4373).
   It focuses on 2 things:
   1. Refactor the code of consistent hashing bucket index, extract common utility to client common module
   2. Flink write progress support consistent hashing bucket index
   It would not cover (would be done in the following subtasks):
   1. generate resize plan
   2. resolve the case which the resize happen during the write process 
   
   ps:  This work is follow up of [pr6737](https://github.com/apache/hudi/pull/6737). Thanks for contribution @YuweiXiao 
   
   ### Impact
   
   NA
   
   ### Risk level (write none, low medium or high below)
   
   NA
   
   ### Documentation Update
   
   All documents update would be introduced in the final subtask of [HUDI-4373](https://issues.apache.org/jira/browse/HUDI-4373).
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597365156

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979",
       "triggerID" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17980",
       "triggerID" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * b208927d059c80455d090eb35083b5743d7b1b0e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979) 
   * 493ab8440ee14b6178f40ba5c0cc0075dc4479f3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17980) 
   
   <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] beyond1920 commented on a diff in pull request #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #9012:
URL: https://github.com/apache/hudi/pull/9012#discussion_r1233630877


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignerCoordinator.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.sink.bucket;
+
+import org.apache.hudi.adapter.OperatorCoordinatorAdapter;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.index.bucket.ConsistentBucketIndexUtils;
+import org.apache.hudi.sink.event.CreatePartitionMetadataEvent;
+import org.apache.hudi.sink.utils.NonThrownExecutor;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.util.FlinkWriteClients;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.hudi.util.StreamerUtil.initTableIfNotExists;
+
+/**
+ * {@link OperatorCoordinator} for {@link ConsistentBucketAssignFunction}.
+ *

Review Comment:
   I'm not sure `FileSystemBasedLockProvider ` is suitable for other object store storage?
   Here is another option, which is to create a fixed file id for each bucket in new partition. It could also solve the inconsistent hashing metadata if multiple subtask create it for the same partition.



-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597174253

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 9d791e58566786d23128fca462149c4c674424ce Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973) 
   * 0ddc29e42c96c2327712ea12fa78ae8a62b1b794 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597157859

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971) 
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 9d791e58566786d23128fca462149c4c674424ce Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973) 
   * 0ddc29e42c96c2327712ea12fa78ae8a62b1b794 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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596715245

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959) 
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596431576

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d 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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #9012:
URL: https://github.com/apache/hudi/pull/9012#discussion_r1233550345


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignerCoordinator.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.sink.bucket;
+
+import org.apache.hudi.adapter.OperatorCoordinatorAdapter;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.index.bucket.ConsistentBucketIndexUtils;
+import org.apache.hudi.sink.event.CreatePartitionMetadataEvent;
+import org.apache.hudi.sink.utils.NonThrownExecutor;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.util.FlinkWriteClients;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.hudi.util.StreamerUtil.initTableIfNotExists;
+
+/**
+ * {@link OperatorCoordinator} for {@link ConsistentBucketAssignFunction}.
+ *

Review Comment:
   Maybe a simple `FileSystemBasedLockProvider` for metadata creation coordination is enough ~



-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597255764

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 9d791e58566786d23128fca462149c4c674424ce Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973) 
   * 0ddc29e42c96c2327712ea12fa78ae8a62b1b794 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978) 
   * b208927d059c80455d090eb35083b5743d7b1b0e 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 merged pull request #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #9012:
URL: https://github.com/apache/hudi/pull/9012


-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596699394

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959) 
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #9012:
URL: https://github.com/apache/hudi/pull/9012#discussion_r1233802138


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignerFactory.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.sink.bucket;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.ProcessOperator;
+import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
+
+public class ConsistentBucketAssignerFactory
+    extends SimpleUdfStreamOperatorFactory<HoodieRecord>
+    implements OneInputStreamOperatorFactory<HoodieRecord, HoodieRecord> {

Review Comment:
   Do we still need this factory?



-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596809531

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959) 
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971) 
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a 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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #9012:
URL: https://github.com/apache/hudi/pull/9012#discussion_r1233808423


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java:
##########
@@ -327,14 +337,51 @@ public static DataStream<HoodieRecord> rowDataToHoodieRecord(Configuration conf,
    */
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, DataStream<HoodieRecord> dataStream) {
     if (OptionsResolver.isBucketIndexType(conf)) {
-      WriteOperatorFactory<HoodieRecord> operatorFactory = BucketStreamWriteOperator.getFactory(conf);
-      int bucketNum = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
-      String indexKeyFields = OptionsResolver.getIndexKeyField(conf);
-      BucketIndexPartitioner<HoodieKey> partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields);
-      return dataStream.partitionCustom(partitioner, HoodieRecord::getKey)
-          .transform(opName("bucket_write", conf), TypeInformation.of(Object.class), operatorFactory)
-          .uid(opUID("bucket_write", conf))
-          .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
+      HoodieIndex.BucketIndexEngineType bucketIndexEngineType = OptionsResolver.getBucketEngineType(conf);
+      switch (bucketIndexEngineType) {
+        case SIMPLE:
+          int bucketNum = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+          String indexKeyFields = OptionsResolver.getIndexKeyField(conf);
+          BucketIndexPartitioner<HoodieKey> partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields);
+          return dataStream.partitionCustom(partitioner, HoodieRecord::getKey)
+              .transform(
+                  opName("bucket_write", conf),
+                  TypeInformation.of(Object.class),
+                  BucketStreamWriteOperator.getFactory(conf))
+              .uid(opUID("bucket_write", conf))
+              .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
+        case CONSISTENT_HASHING:
+          String writeOperation = conf.get(FlinkOptions.OPERATION);
+          switch (WriteOperationType.fromValue(writeOperation)) {
+            case INSERT:
+            case UPSERT:

Review Comment:
   We can use `OptionsResolver.isInsertOverwrite`



-- 
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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596453895

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596823708

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971) 
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 9d791e58566786d23128fca462149c4c674424ce 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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1596887581

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7953e9b91b00d0ed06fd2115b1756c96fc222461 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971) 
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 9d791e58566786d23128fca462149c4c674424ce Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597553443

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979",
       "triggerID" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17980",
       "triggerID" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 493ab8440ee14b6178f40ba5c0cc0075dc4479f3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17980) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597273321

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979",
       "triggerID" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 0ddc29e42c96c2327712ea12fa78ae8a62b1b794 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978) 
   * b208927d059c80455d090eb35083b5743d7b1b0e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979) 
   
   <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 #9012: [HUDI-6326] Flink write supports consistent bucket index

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #9012:
URL: https://github.com/apache/hudi/pull/9012#issuecomment-1597350996

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17959",
       "triggerID" : "f7ea0e2080d7453bb2ea7b52eda8dd3c479e0d5d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17971",
       "triggerID" : "7953e9b91b00d0ed06fd2115b1756c96fc222461",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9d791e58566786d23128fca462149c4c674424ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17973",
       "triggerID" : "9d791e58566786d23128fca462149c4c674424ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978",
       "triggerID" : "0ddc29e42c96c2327712ea12fa78ae8a62b1b794",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979",
       "triggerID" : "b208927d059c80455d090eb35083b5743d7b1b0e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "493ab8440ee14b6178f40ba5c0cc0075dc4479f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d128349f4d36cc47fc5ddedbffbe4c9eb1600c3a UNKNOWN
   * 0ddc29e42c96c2327712ea12fa78ae8a62b1b794 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17978) 
   * b208927d059c80455d090eb35083b5743d7b1b0e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=17979) 
   * 493ab8440ee14b6178f40ba5c0cc0075dc4479f3 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