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

[GitHub] [hudi] xushiyan opened a new pull request, #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   ### Change Logs
   
   Add abstraction classes for DeltaSyncService and DeltaStreamerMetrics
   
   - HoodieIngestionService
   - HoodieIngestionMetrics
   
   
   ### Impact
   
   NA
   
   ### Risk level
   
   Low
   
   ### Documentation Update
   
   - [ ] document about extendable abstractions
   
   ### 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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582",
       "triggerID" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "343c1b7e79f506e020d255502fa037ac7aa49b13",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15598",
       "triggerID" : "343c1b7e79f506e020d255502fa037ac7aa49b13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6c08fac7778593fff23d9247b999f92ee1cc5ed0 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582) 
   * 343c1b7e79f506e020d255502fa037ac7aa49b13 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15598) 
   
   <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] nsivabalan commented on a diff in pull request #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java:
##########
@@ -883,9 +877,19 @@ public SparkSession getSparkSession() {
     public TypedProperties getProps() {
       return props;
     }
+
+    /**
+     * This API is for testing only.
+     */
+    public DeltaSync getDeltaSync() {
+      return deltaSync;
+    }
   }
 
-  public DeltaSyncService getDeltaSyncService() {
-    return deltaSyncService.get();
+  /**
+   * This API is for testing only.
+   */

Review Comment:
   I am not a fan of exposing internal variables just for testing purpose. lets see  if we can avoid this.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/ingestion/HoodieIngestionService.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.utilities.ingestion;
+
+import org.apache.hudi.async.HoodieAsyncService;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_IS_CONTINUOUS;
+import static org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_MIN_SYNC_INTERNAL_SECONDS;
+
+/**
+ * A generic service to facilitate running data ingestion.
+ */
+public abstract class HoodieIngestionService extends HoodieAsyncService {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieIngestionService.class);
+
+  protected HoodieIngestionConfig ingestionConfig;
+
+  public HoodieIngestionService(HoodieIngestionConfig ingestionConfig) {
+    this.ingestionConfig = ingestionConfig;
+  }
+
+  /**
+   * Entrypoint to start ingestion.

Review Comment:
   lets add some documentation on the flow of calls for syncOnce and continuous mode. esply continuous mode is not very apparent. 
   for eg, startIngestion -> start -> startService { while { ingestOnce() } } 



-- 
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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9f72cf006102bcb34a24935f5da9263d4d98f537 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544) 
   * 6c08fac7778593fff23d9247b999f92ee1cc5ed0 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 a diff in pull request #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java:
##########
@@ -883,9 +877,19 @@ public SparkSession getSparkSession() {
     public TypedProperties getProps() {
       return props;
     }
+
+    /**
+     * This API is for testing only.
+     */
+    public DeltaSync getDeltaSync() {
+      return deltaSync;
+    }
   }
 
-  public DeltaSyncService getDeltaSyncService() {
-    return deltaSyncService.get();
+  /**
+   * This API is for testing only.
+   */

Review Comment:
   this is already there. annotated instead of javadoc



-- 
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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 006ee4a7eee6e217f060b210de92838312384f98 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539) 
   
   <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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582",
       "triggerID" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9f72cf006102bcb34a24935f5da9263d4d98f537 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544) 
   * 6c08fac7778593fff23d9247b999f92ee1cc5ed0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582) 
   
   <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 a diff in pull request #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/ingestion/HoodieIngestionService.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.utilities.ingestion;
+
+import org.apache.hudi.async.HoodieAsyncService;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_IS_CONTINUOUS;
+import static org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_MIN_SYNC_INTERNAL_SECONDS;
+
+/**
+ * A generic service to facilitate running data ingestion.
+ */
+public abstract class HoodieIngestionService extends HoodieAsyncService {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieIngestionService.class);
+
+  protected HoodieIngestionConfig ingestionConfig;
+
+  public HoodieIngestionService(HoodieIngestionConfig ingestionConfig) {
+    this.ingestionConfig = ingestionConfig;
+  }
+
+  /**
+   * Entrypoint to start ingestion.

Review Comment:
   fixed



-- 
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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582",
       "triggerID" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6c08fac7778593fff23d9247b999f92ee1cc5ed0 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582) 
   
   <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 merged pull request #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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


-- 
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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 006ee4a7eee6e217f060b210de92838312384f98 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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 006ee4a7eee6e217f060b210de92838312384f98 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539) 
   
   <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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   ![Screenshot 2023-03-06 at 3 36 18 PM](https://user-images.githubusercontent.com/2701446/223236380-cd9d21f3-1f43-46db-81b6-03fdf133b735.png)
   
   CI was success for the commit. the last commit https://github.com/apache/hudi/pull/8086/commits/343c1b7e79f506e020d255502fa037ac7aa49b13 is just docs/annotation change


-- 
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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 006ee4a7eee6e217f060b210de92838312384f98 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539) 
   * 9f72cf006102bcb34a24935f5da9263d4d98f537 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544) 
   
   <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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 006ee4a7eee6e217f060b210de92838312384f98 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539) 
   * 9f72cf006102bcb34a24935f5da9263d4d98f537 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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9f72cf006102bcb34a24935f5da9263d4d98f537 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544) 
   
   <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 #8086: [HUDI-5872] Abstraction for DeltaSyncService and DeltaStreamerMetrics

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "006ee4a7eee6e217f060b210de92838312384f98",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15539",
       "triggerID" : "006ee4a7eee6e217f060b210de92838312384f98",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15544",
       "triggerID" : "9f72cf006102bcb34a24935f5da9263d4d98f537",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582",
       "triggerID" : "6c08fac7778593fff23d9247b999f92ee1cc5ed0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "343c1b7e79f506e020d255502fa037ac7aa49b13",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "343c1b7e79f506e020d255502fa037ac7aa49b13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6c08fac7778593fff23d9247b999f92ee1cc5ed0 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15582) 
   * 343c1b7e79f506e020d255502fa037ac7aa49b13 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