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/07/07 13:50:32 UTC

[GitHub] [hudi] yuzhaojing opened a new pull request, #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   ## What is the purpose of the pull request
   
   The preparations which changes existing modules for table management service.
   The following commits are in https://github.com/apache/hudi/pull/5926.
   
   ## Brief change log
   
   *(for example:)*
     - Add HoodieTableMangerConfig and HoodieTableManagerClient.
     - Don't execute table services when `hoodie.table.management.service.enable = true`
   
   ## Verify this pull request
   
   This pull request is already covered by existing tests.
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
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] XuQianJin-Stars commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on code in PR #5681:
URL: https://github.com/apache/hudi/pull/5681#discussion_r918477485


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -564,9 +569,13 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
 
       // Do an inline clustering if enabled
       if (config.inlineClusteringEnabled()) {
-        runAnyPendingClustering(table);
         metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true");
-        inlineClustering(extraMetadata);
+        if (delegateToTableManagerService(config, ActionType.replacecommit)) {
+          scheduleClustering(extraMetadata);
+        } else {
+          runAnyPendingClustering(table);
+          inlineClustering(extraMetadata);

Review Comment:
   Ditto



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -865,6 +874,9 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline
         table.getMetaClient().reloadActiveTimeline();
       }
 
+      if (delegateToTableManagerService(config, ActionType.clean)) {
+        return null;

Review Comment:
   Why is `null` returned here?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -547,9 +548,13 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
       }
       // Do an inline compaction if enabled
       if (config.inlineCompactionEnabled()) {
-        runAnyPendingCompactions(table);
         metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "true");
-        inlineCompaction(extraMetadata);
+        if (delegateToTableManagerService(config, ActionType.compaction)) {
+          scheduleCompaction(extraMetadata);
+        } else {
+          runAnyPendingCompactions(table);
+          inlineCompaction(extraMetadata);
+        }

Review Comment:
   L552-L557 Extract into a method?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");

Review Comment:
   submit -> create remove -> delete ?
   https://cloud.google.com/apis/design/standard_methods



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {

Review Comment:
   ditto.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java:
##########
@@ -257,6 +257,7 @@ public static HoodieTableMetaClient initTableIfNotExists(Configuration conf) thr
           .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA))
           .setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
           .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
+          .setDatabaseName(conf.getString(FlinkOptions.DATABASE_NAME))

Review Comment:
   The table service uses db_name.tbl_name to determine the unique hoodie table, but currently Flink does not set it by default db_name.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   * a589f277f3ba2bd94635132fb65f7e12af8f037e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {

Review Comment:
   This is a cleaner implementation, I will modify the implementation. 
   But I don't want to introduce a lock manager in the current version, it is too heavy for the user.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+  }
+
+  private String executeRequest(String requestPath, String instantRange) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+
+    Map<String, String> queryParameters = getParamsWithAdditionalParams(
+        new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, OWNER, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS},
+        new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(),
+            config.getDeployResource(), String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams()});
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void submitCompaction(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_COMPACTION, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule compaction to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClean(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLEAN, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clean to service", e);

Review Comment:
   Sure, I'll reuse this code.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";

Review Comment:
   ok, will update.



-- 
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] xiarixiaoyao commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String CREATE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/create");
+  public static final String DELETE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/delete");
+
+  public static final String CREATE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/create");
+  public static final String DELETE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/delete");
+
+  public static final String CREATE_CLEAN = String.format("%s/%s", BASE_URL, "clean/create");
+  public static final String DELETE_CLEAN = String.format("%s/%s", BASE_URL, "clean/delete");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));

Review Comment:
   maybe it's better to put this exception on line 110



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java:
##########
@@ -37,6 +37,8 @@
 
   // Will be set when auto-commit happens
   private boolean isCommitted;
+  // Will be set when table management service enable
+  private boolean isEmpty;

Review Comment:
   Good suggestion, update later.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fbecda30a47830d20ec52daccd1198fbd43b133 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fbecda30a47830d20ec52daccd1198fbd43b133 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924) 
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f486cddda77a38dc493e042986d30f0f1ae5d687 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java:
##########
@@ -101,9 +105,14 @@ public Option<HoodieCompactionPlan> execute() {
       } catch (IOException ioe) {
         throw new HoodieIOException("Exception scheduling compaction", ioe);
       }
-      return Option.of(plan);
+      option = Option.of(plan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.compaction.name())) {

Review Comment:
   Maybe we could abstract this as an interface in config, e.g., isTableManagerIncludeAction(xxx)



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java:
##########
@@ -236,6 +237,11 @@ private HoodieCleanMetadata runClean(HoodieTable<T, I, K, O> table, HoodieInstan
 
   @Override
   public HoodieCleanMetadata execute() {
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      LOG.warn("Compaction delegate to table management service, do not clean for client!");

Review Comment:
   typo, -> 'clean delegate to'



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java:
##########
@@ -102,6 +107,22 @@ public Option<HoodieClusteringPlan> execute() {
         throw new HoodieIOException("Exception scheduling clustering", ioe);
       }
     }
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.replacecommit.name())) {
+      submitClusteringToService();
+    }
+
     return planOption;
   }
+
+  private void submitClusteringToService() {
+    HoodieTableMetaClient metaClient = table.getMetaClient();
+    List<String> instantsToSubmit = metaClient.getActiveTimeline()
+        .filterPendingReplaceTimeline()

Review Comment:
   Replace instant may also include `insert overwrite` operation. Will the manager service double-check to ensure it is a clustering instant?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java:
##########
@@ -37,6 +37,8 @@
 
   // Will be set when auto-commit happens
   private boolean isCommitted;
+  // Will be set when table management service enable
+  private boolean isEmpty;

Review Comment:
   `isEmpty` seems a little bit confusing. How about `isSkipped`?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      submitCleanToService();
+    }
+
+    return option;
+  }
+
+  private void submitCleanToService() {

Review Comment:
   These submit function will submit all pending clean/compaction/clustering instant to the service. What happens if we submit the same instant multiple times?



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");

Review Comment:
   will update.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -564,9 +569,13 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
 
       // Do an inline clustering if enabled
       if (config.inlineClusteringEnabled()) {
-        runAnyPendingClustering(table);
         metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true");
-        inlineClustering(extraMetadata);
+        if (delegateToTableManagerService(config, ActionType.replacecommit)) {
+          scheduleClustering(extraMetadata);
+        } else {
+          runAnyPendingClustering(table);
+          inlineClustering(extraMetadata);

Review Comment:
   will update.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -547,9 +548,13 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
       }
       // Do an inline compaction if enabled
       if (config.inlineCompactionEnabled()) {
-        runAnyPendingCompactions(table);
         metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "true");
-        inlineCompaction(extraMetadata);
+        if (delegateToTableManagerService(config, ActionType.compaction)) {
+          scheduleCompaction(extraMetadata);
+        } else {
+          runAnyPendingCompactions(table);
+          inlineCompaction(extraMetadata);
+        }

Review Comment:
   will update.



-- 
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] suryaprasanna commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";

Review Comment:
   Please fix the Typo on word service.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   I noticed that whenever an action is scheduled, instead of sending that particular instant, we are sending all the pending instants for that action as part of the request. This would mean the remote client has to iterate over the instants and update the contents to the mysql/hudi db. 
   
   I am guessing this was done so that in case an action is scheduled but failed to update the table service, then the .requested instant will be left on the timeline.
   
   One way we are trying to workaround this is by sending the request before creating .requested files. 
   
   Timeline of actions will look like this,
   
   1. Generate clustering/compaction plan for an instant `x`
   2. After plan is generated, send request to table service for the instant `x`
   3. Create x.clustering.requested or x.compaction.requested file on the timeline
   
   In case the request is successfully send to table service and the .requested file creation failed then it will be treated as a no-op by table service when it schedules the job. There is a possibility that table service can execute right after point 2, but it can be handled by introducing some delay.
   
   This above approach is useful, if we want to send any spark job specific parameters to the table service for executing that instants, example no. of executors. Since, no.of executors can vary depending on the plan.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 80f748f87ce131827300b54781ae181c62996066 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196",
       "triggerID" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1199016223",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * a9e0129347436e4510e5cb3fd195d6e843701022 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196) 
   
   <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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   > can you please update the description and other elements in the template for the PR
   
   Sure, I will add description for PR later.


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/RunsTableService.java:
##########
@@ -34,4 +35,12 @@ default boolean tableServicesEnabled(HoodieWriteConfig config) {
     }
     return enabled;
   }
+
+  default boolean delegateToTableManagerService(HoodieWriteConfig config, ActionType actionType) {
+    boolean includeAction = config.isTableManagerIncludeAction(actionType);

Review Comment:
   will update 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java:
##########
@@ -50,6 +55,13 @@ public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context,
 
   @Override
   public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.replacecommit.name())) {
+      LOG.warn("Clustering delegate to table management service, do not cluster for client!");

Review Comment:
   Sure, will update 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099) 
   * a9e0129347436e4510e5cb3fd195d6e843701022 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   We can do this in another MR.



-- 
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] prasannarajaperumal commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -1051,13 +1071,29 @@ public void dropIndex(List<MetadataPartitionType> partitionTypes) {
     }
   }
 
+  /**
+   * Performs Clustering for the workload stored in instant-time.
+   *
+   * @param clusteringInstantTime Clustering Instant Time
+   * @return Collection of WriteStatus to inspect errors and counts
+   */
+  public HoodieWriteMetadata<O> cluster(String clusteringInstantTime) {
+    if (delegateToTableManagerService(config, ActionType.replacecommit)) {

Review Comment:
   Refer to my comment above. Would rather have interface having multiple impl than the if conditions here. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   Should we create a clean TableServiceClient abstraction and move all the compaction/clustering/cleaning api into that. We can have multiple engine specific implementation. Thay way we dont combine Write APIs and table services API and avoid multiple if checks here. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {

Review Comment:
   Need to implement the common HoodieTableServiceClient - Referring to my comment above. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {

Review Comment:
   Handcoding a HTTP Rest client seems very painful / error prone. Why dont we standardize and pick a framework to support any rest endpoints. Jersey is good one. Lets you enable REST endpoints with annotations, provides exception recovery hooks, testability etc. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/RunsTableService.java:
##########
@@ -34,4 +35,12 @@ default boolean tableServicesEnabled(HoodieWriteConfig config) {
     }
     return enabled;
   }
+
+  default boolean delegateToTableManagerService(HoodieWriteConfig config, ActionType actionType) {
+    boolean includeAction = config.isTableManagerIncludeAction(actionType);

Review Comment:
   isTableManagerIncludeAction -> isTableManagerSupportAction to make it more readable



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091) 
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 80f748f87ce131827300b54781ae181c62996066 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884) 
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +155,19 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerSupportAction(ActionType.clean)) {
+      createCleanToService();

Review Comment:
   Understand, I will uniformly name the `Service` to `TableManagerService`.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java:
##########
@@ -102,6 +104,16 @@ public Option<HoodieClusteringPlan> execute() {
         throw new HoodieIOException("Exception scheduling clustering", ioe);
       }
     }
+
+    if (config.isTableManagerSupportAction(ActionType.replacecommit)) {
+      createClusteringToService();

Review Comment:
   will update.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      submitCleanToService();
+    }
+
+    return option;
+  }
+
+  private void submitCleanToService() {

Review Comment:
   In the table management service, the received instant will be deduplicated, and the repeatedly accepted instant will not be repeated.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   will update 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      submitCleanToService();
+    }
+
+    return option;
+  }
+
+  private void submitCleanToService() {
+    HoodieTableMetaClient metaClient = table.getMetaClient();
+    List<String> instantsToSubmit = metaClient.getActiveTimeline()

Review Comment:
   We plan to schedule plan and move execution to tms by writer in tms phase1, and support schedule in tms in tms phase2. Because the current concurrency control needs to introduce external dependencies, but we can reserve parameters in the core api. WDYT



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   * a589f277f3ba2bd94635132fb65f7e12af8f037e 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @yuzhaojing let's make sure we link PRs so reviewers know where to follow up
   
   https://github.com/apache/hudi/pull/6732


-- 
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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   > @yuzhaojing let's make sure we link PRs so reviewers know where to follow up
   > 
   > #6732
   
   Sure, my mistake.


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+  }
+
+  private String executeRequest(String requestPath, String instantRange) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+
+    Map<String, String> queryParameters = getParamsWithAdditionalParams(
+        new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, OWNER, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS},
+        new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(),
+            config.getDeployResource(), String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams()});
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void submitCompaction(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_COMPACTION, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule compaction to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClean(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLEAN, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clean to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClustering(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLUSTERING, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clustering to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  private void checkTolerableSchedule(Exception e) {

Review Comment:
   There is already a retry process for request failure, checkTolerableSchedule is just a process to prevent all tasks from failing when table manager service in case have an exception.



-- 
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] vinothchandar commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";

Review Comment:
   /v1/hudi?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1903,6 +1906,10 @@ public HoodieMetadataConfig getMetadataConfig() {
     return metadataConfig;
   }
 
+  public HoodieTableManagerConfig getTableManagerConfig() {

Review Comment:
   Should we have this as a separate module? That is build on top of `hudi-client-common`



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      submitCleanToService();
+    }
+
+    return option;
+  }
+
+  private void submitCleanToService() {
+    HoodieTableMetaClient metaClient = table.getMetaClient();
+    List<String> instantsToSubmit = metaClient.getActiveTimeline()

Review Comment:
   Hmmm. based on this, do you intend to have the scheduling done by the writer still and only move execution to the Table manager service?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java:
##########
@@ -236,6 +237,11 @@ private HoodieCleanMetadata runClean(HoodieTable<T, I, K, O> table, HoodieInstan
 
   @Override
   public HoodieCleanMetadata execute() {
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {
+      LOG.warn("Compaction delegate to table management service, do not clean for client!");

Review Comment:
   this is cleaning? Can we check all log messages like this



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+  }
+
+  private String executeRequest(String requestPath, String instantRange) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+
+    Map<String, String> queryParameters = getParamsWithAdditionalParams(
+        new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, OWNER, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS},
+        new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(),
+            config.getDeployResource(), String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams()});
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void submitCompaction(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_COMPACTION, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule compaction to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClean(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLEAN, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clean to service", e);

Review Comment:
   can we reuse all this code around exception handling? there is 5 lines of code across the submitXXX methods



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java:
##########
@@ -50,6 +55,13 @@ public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context,
 
   @Override
   public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.replacecommit.name())) {
+      LOG.warn("Clustering delegate to table management service, do not cluster for client!");

Review Comment:
   Can we avoid invoking this code path from the client level, rather than returning an empty writeMetadata object



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;

Review Comment:
   did you intend this as static? Seems like all usage is non-static. also its private



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+  }
+
+  private String executeRequest(String requestPath, String instantRange) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+
+    Map<String, String> queryParameters = getParamsWithAdditionalParams(
+        new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, OWNER, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS},
+        new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(),
+            config.getDeployResource(), String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams()});
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void submitCompaction(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_COMPACTION, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule compaction to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClean(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLEAN, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clean to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClustering(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLUSTERING, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clustering to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  private void checkTolerableSchedule(Exception e) {

Review Comment:
   overalll I wonder if there is a simpler way of implementing retryable requests



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");

Review Comment:
   Add a `_ENDPOINT` to the names, so its clear what this is? 
   
   `REGISTER_ENDPOINT` 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {

Review Comment:
   We can then instruct the users to set `hoodie.enable.table.services=false` when running with Table management services. Another point here is that we should check for some lock manager being configured when operating in this mode, so that there is no race between delta commits and compaction scheduling.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +156,26 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.clean.name())) {

Review Comment:
   Instead of leaking the table management service into the action executors., can we layer it on top? Introduce a higher level write config say `hoodie.enable.table.services`, (default: `true`) which when set to `false` will not do any scheduling/execution of any table services. This check can be added at the BaseHoodieWriteClient level when the calls are made for clean, scheduleClean, compact, scheduleCompact etc.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;

Review Comment:
   and this is assuming that there is one request in flight, since all submitXXX methods update this variable. Can we either make this local to submitXXX method or make all those methods synchronized, so that any multi-threaded usage is not suprising



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   probably need to rebase this to log4j2 now



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+  }
+
+  private String executeRequest(String requestPath, String instantRange) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+
+    Map<String, String> queryParameters = getParamsWithAdditionalParams(
+        new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, OWNER, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS},
+        new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(),
+            config.getDeployResource(), String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams()});
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void submitCompaction(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_COMPACTION, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule compaction to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClean(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLEAN, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clean to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  public void submitClustering(List<String> instantsToSubmit) {
+    try {
+      executeRequest(SUBMIT_CLUSTERING, StringUtils.join(instantsToSubmit.toArray(new String[0]), ","));
+    } catch (Exception e) {
+      LOG.error("Failed to schedule clustering to service", e);
+      failTimes++;
+      checkTolerableSchedule(e);
+      return;
+    }
+
+    failTimes = 0;
+  }
+
+  private void checkTolerableSchedule(Exception e) {
+    if (failTimes > config.getConnectionTolerableNum()) {

Review Comment:
   include the url that fails , so we have context? 



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 568181a3e7c1d43a4ee8573d3746600be4f94379 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003) 
   * 75571c4be2855f4c0b6ea5b8a0babaa155604874 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837) 
   * f50bb15f02020f3c345488a8bce6226b07cbe00c 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 568181a3e7c1d43a4ee8573d3746600be4f94379 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   Discussed with @yuzhaojing who has a draft PR abstracting TMS client into a member variable of writeclient. It is not a top-level client as WriteClient but serving the purpose of encapsulating if-checks in one place. I think it is a practical step towards full abstraction of TMS client to the same level as BaseWriteClient, which would involve more many more code changes and not fit into releasing TMS MVP timeline. 


-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196",
       "triggerID" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196",
       "triggerID" : "1199016223",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * a9e0129347436e4510e5cb3fd195d6e843701022 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196) 
   
   <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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @hudi-bot run azure


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -1051,13 +1071,29 @@ public void dropIndex(List<MetadataPartitionType> partitionTypes) {
     }
   }
 
+  /**
+   * Performs Clustering for the workload stored in instant-time.
+   *
+   * @param clusteringInstantTime Clustering Instant Time
+   * @return Collection of WriteStatus to inspect errors and counts
+   */
+  public HoodieWriteMetadata<O> cluster(String clusteringInstantTime) {
+    if (delegateToTableManagerService(config, ActionType.replacecommit)) {

Review Comment:
   Ok, will update.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 568181a3e7c1d43a4ee8573d3746600be4f94379 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003) 
   * 75571c4be2855f4c0b6ea5b8a0babaa155604874 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837) 
   * f50bb15f02020f3c345488a8bce6226b07cbe00c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -865,6 +874,9 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline
         table.getMetaClient().reloadActiveTimeline();
       }
 
+      if (delegateToTableManagerService(config, ActionType.clean)) {
+        return null;

Review Comment:
   Because the clean action has not been carried out at this time. So the cleanMetadata is 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java:
##########
@@ -51,7 +51,7 @@ public HoodieSparkClusteringClient(
   public void cluster(HoodieInstant instant) throws IOException {
     LOG.info("Executing clustering instance " + instant);
     SparkRDDWriteClient<T> writeClient = (SparkRDDWriteClient<T>) clusteringClient;
-    Option<HoodieCommitMetadata> commitMetadata = writeClient.cluster(instant.getTimestamp(), true).getCommitMetadata();
+    Option<HoodieCommitMetadata> commitMetadata = writeClient.cluster(instant.getTimestamp()).getCommitMetadata();

Review Comment:
   This is to prevent the user from directly calling the behavior, if it is considered that no user can use it without modification.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   * a589f277f3ba2bd94635132fb65f7e12af8f037e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090) 
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * a589f277f3ba2bd94635132fb65f7e12af8f037e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090) 
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   Rebase all log4j to log4j2 in the hudi project or only use log4j2 for HoodieTableManagerClient? 



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f486cddda77a38dc493e042986d30f0f1ae5d687 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4d37594e0a819dd66ed405094429693296896596 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975) 
   
   <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] suryaprasanna commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   +1 on this.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   Here is my understanding, we have 3 components, scheduler(Hudi table via write API), storage (state manager or metaserver(can be mysql or hudi table)) and execution(Table services).
   > When the table service is unavailable, the timeline will not be able to generate the requested instant, so it will not be able to operate when the table service is restored, which has a greater impact on the day level write job.
   - If table services is down then there is no one to execute the plan, so no need to create requested file. Initially table generated the plan, similarly it will try to regenerate every time until the table services  accepts the request.
   - There is an advantage to this approach, the latest plan that we generate is also going to include latest changes happened on the table during the period the table service is down.
   
   > When the table service receives the request, it needs to scan the timeline corresponding to the hudi table to determine whether it is invalid, which will make table management service appear a lot of additional operations.
   - I agree, Table service has to be as dumb as possible. Here, it's duty is to read from the storage and execute the spark job. So, basically we are  starting the spark job, and the job will not do anything because it cannot find the .requested file for the instant. So, it will just quit.
   
   Another approach, I can think of is to have async thread running that starts on write path and incrementally post the requests to table service reading through the timeline. We can even update file's extrametadata map with any information if needed.
   
   I am inclined with your approach because it is a clean way to do it, but we need to address following things,
   1. I have seen people configure clustering on non-partitioned tables. If we schedule clustering on them and fail to update the table services then no other clustering job will be able to schedule on them because we don't schedule clustering on pending file groups under clustering. So, the replacecommit.requested can be left alone.
   2. Passing job specific parameters like executor count, either we need to pass this information while calling the table service APIs or the table service API need to read through the plan before scheduling a spark job. Ideally we do not want table services to read through clustering plans.
   3. Also, we can schedule clustering jobs on a small group of filegroups one after the other and these clustering plans can be smaller in size and more in number. If table service is running slow, more and more .requested file will be created and more requests with heavy payload will be send to table service.



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java:
##########
@@ -102,6 +107,22 @@ public Option<HoodieClusteringPlan> execute() {
         throw new HoodieIOException("Exception scheduling clustering", ioe);
       }
     }
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.replacecommit.name())) {
+      submitClusteringToService();
+    }
+
     return planOption;
   }
+
+  private void submitClusteringToService() {
+    HoodieTableMetaClient metaClient = table.getMetaClient();
+    List<String> instantsToSubmit = metaClient.getActiveTimeline()
+        .filterPendingReplaceTimeline()

Review Comment:
   Yes, we will ensure clustering instant in replaceMetadata operationType. In table manager service we can only deserialize replaceMetadata once.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java:
##########
@@ -101,9 +105,14 @@ public Option<HoodieCompactionPlan> execute() {
       } catch (IOException ioe) {
         throw new HoodieIOException("Exception scheduling compaction", ioe);
       }
-      return Option.of(plan);
+      option = Option.of(plan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerEnabled() && config.getTableManagerConfig().getTableManagerActions().contains(ActionType.compaction.name())) {

Review Comment:
   Good suggestion, update later.



-- 
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 pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   can you please update the description and other elements in the template for the 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;

Review Comment:
   ok, will update 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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @suryaprasanna Could you 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] hudi-bot commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4d37594e0a819dd66ed405094429693296896596 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975) 
   * dee99d00d3076291b077637a26bd48c3031376d2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4d37594e0a819dd66ed405094429693296896596 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975) 
   * dee99d00d3076291b077637a26bd48c3031376d2 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;

Review Comment:
   Table Manager Service use the javalin, the implemention of HoodieTableManagerClient align with RemoteHoodieTableFileSystemView.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196",
       "triggerID" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * a9e0129347436e4510e5cb3fd195d6e843701022 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 568181a3e7c1d43a4ee8573d3746600be4f94379 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003) 
   * 75571c4be2855f4c0b6ea5b8a0babaa155604874 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f486cddda77a38dc493e042986d30f0f1ae5d687 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928) 
   * 4d37594e0a819dd66ed405094429693296896596 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String CREATE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/create");
+  public static final String DELETE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/delete");
+
+  public static final String CREATE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/create");
+  public static final String DELETE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/delete");
+
+  public static final String CREATE_CLEAN = String.format("%s/%s", BASE_URL, "clean/create");
+  public static final String DELETE_CLEAN = String.format("%s/%s", BASE_URL, "clean/delete");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));

Review Comment:
   sure, will 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] suryaprasanna commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   Here is my understanding, we have 3 components, scheduler(Hudi table via write API), storage (state manager or metaserver(can be mysql or hudi table)) and execution(Table services).
   > When the table service is unavailable, the timeline will not be able to generate the requested instant, so it will not be able to operate when the table service is restored, which has a greater impact on the day level write job.
   - If table services is down then there is no one to execute the plan, so no need to create requested file. Initially table generated the plan, similarly it will try to regenerate every time until the table services  accepts the request.
   - There is an advantage to this approach, the latest plan that we generate is also going to include latest changes happened on the table during the period the table service is down.
   
   > When the table service receives the request, it needs to scan the timeline corresponding to the hudi table to determine whether it is invalid, which will make table management service appear a lot of additional operations.
   - I agree, Table service has to be as dumb as possible. Here, it's duty is to read from the storage and execute the spark job. So, basically we are  starting the spark job, and the job will not do anything because it cannot find the .requested file for the instant. So, it will just quit.
   
   Another approach, I can think of is to have async thread running that starts on write path and incrementally post the requests to table service reading through the timeline. We can even update file's extrametadata map with any information if needed.
   
   I am inclined with your approach because it is a clean way to do it, but we need to address following things,
   1. I have seen people configure clustering on non-partitioned tables. If we schedule clustering on them and fail to update the table services then no other clustering job will be able to schedule on them because we don't schedule clustering on pending file groups under clustering. So, the replacecommit.requested can be left alone.
   2. Passing job specific parameters like executor count, either we need to pass this information while calling the table service APIs or the table service API need to read through the plan before scheduling a spark job. Ideally we do not want table services to read through clustering plans.
   3. Also, we can schedule clustering jobs on a small group of filegroups one after the other and these clustering plans can be smaller in size and more in number. If table service is running slow, more and more .requested file will be created and more requests will be send to table service.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 75571c4be2855f4c0b6ea5b8a0babaa155604874 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837) 
   * f50bb15f02020f3c345488a8bce6226b07cbe00c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/service";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String CREATE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/create");
+  public static final String DELETE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/delete");
+
+  public static final String CREATE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/create");
+  public static final String DELETE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/delete");
+
+  public static final String CREATE_CLEAN = String.format("%s/%s", BASE_URL, "clean/create");
+  public static final String DELETE_CLEAN = String.format("%s/%s", BASE_URL, "clean/delete");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void createCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(CREATE_COMPACTION, getDefaultParams(instantRange));

Review Comment:
   Ok, will update them.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091) 
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099) 
   
   <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] vinothchandar commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   We just needto co ordinate with RDD APl users like uber, Since it's a breaking change. We can file a code cleanup follow-up JIRA, Orthogonal to this 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   Rebase all log4j to log4j2 in the hudi project or only use log4j2 for HoodieTableManagerClient? 



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * dee99d00d3076291b077637a26bd48c3031376d2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f486cddda77a38dc493e042986d30f0f1ae5d687 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928) 
   * 4d37594e0a819dd66ed405094429693296896596 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fbecda30a47830d20ec52daccd1198fbd43b133 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   It's a good suggestion, Can we refactor this later?



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 80f748f87ce131827300b54781ae181c62996066 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884) 
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 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] prasannarajaperumal commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {

Review Comment:
   javalin seems okay to me. We should use that 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] hudi-bot commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f50bb15f02020f3c345488a8bce6226b07cbe00c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842) 
   
   <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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   > +1 on keeping the table manager/service decouple as discussed.
   > 
   > 1. Wont' 1 be handled automatically because we resend all the pending instants again to the table manager.
   > 2. I would imagine 2 to be a config specified at the table manager service level and not passed through from the writer/
   > 3. there is no backpressure mechanism like what you are describing today. I think your point exists in async clustering/compaction today, regardless of table service or not?
   
   +1 from me. 



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   * a589f277f3ba2bd94635132fb65f7e12af8f037e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090) 
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/RunsTableService.java:
##########
@@ -34,4 +35,12 @@ default boolean tableServicesEnabled(HoodieWriteConfig config) {
     }
     return enabled;
   }
+
+  default boolean delegateToTableManagerService(HoodieWriteConfig config, ActionType actionType) {
+    boolean supportAction = config.isTableManagerSupportAction(actionType);

Review Comment:
   Sure, will move it to TableManagementConfig.



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * a589f277f3ba2bd94635132fb65f7e12af8f037e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090) 
   * dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091) 
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");

Review Comment:
   ok, will update 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * dee99d00d3076291b077637a26bd48c3031376d2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750) 
   * 568181a3e7c1d43a4ee8573d3746600be4f94379 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] vinothchandar commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @yuzhaojing Can you please fill out the pull request template so we have more context heading into the review?
   


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1903,6 +1906,10 @@ public HoodieMetadataConfig getMetadataConfig() {
     return metadataConfig;
   }
 
+  public HoodieTableManagerConfig getTableManagerConfig() {

Review Comment:
   I think TableManagerConfig is a public config, clients can get it through HoodieWriteConfig, so I think the config in HoodieWriteConfig is better.



-- 
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] yuzhaojing closed pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

Posted by GitBox <gi...@apache.org>.
yuzhaojing closed pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service
URL: https://github.com/apache/hudi/pull/5681


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   This will cause two problems: 
   1. When the table service is unavailable, the timeline will not be able to generate the requested instant, so it will not be able to operate when the table service is restored, which has a greater impact on the day level write job.
   2. When the table service receives the request, it needs to scan the timeline corresponding to the hudi table to determine whether it is invalid, which will make table management service appear a lot of additional operations. 
   
   Considering these two points, I think sending all the pending instant is a better way. WDYT



-- 
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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @vinothchandar Thanks for review, I'll fill in the template later and consider an updated scenario based on the comments.


-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 7ea9c28a75e97a6543fa8c131fe1035034fe66c4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f50bb15f02020f3c345488a8bce6226b07cbe00c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842) 
   * 80f748f87ce131827300b54781ae181c62996066 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f50bb15f02020f3c345488a8bce6226b07cbe00c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842) 
   * 80f748f87ce131827300b54781ae181c62996066 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884) 
   
   <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] yuzhaojing commented on pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   @vinothchandar Modified some code, Major changes: 
   1. Extract validation logic from action executor to BaseHoodieWriteClient.
   2. Removed failure tolerance logic.


-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableManagerConfig.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.common.config;
+
+import javax.annotation.concurrent.Immutable;
+
+import java.util.Properties;
+
+/**
+ * Configurations used by the HUDI Table Management Service.
+ */
+@Immutable
+@ConfigClassProperty(name = "Table Management Service Configs",
+    groupName = ConfigGroups.Names.WRITE_CLIENT,
+    description = "Configurations used by the Hudi Table Management Service.")
+public class HoodieTableManagerConfig extends HoodieConfig {
+
+  public static final String TABLE_MANAGEMENT_SERVICE_PREFIX = "hoodie.table.management.service";

Review Comment:
   Get 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] vinothchandar commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/RunsTableService.java:
##########
@@ -34,4 +35,12 @@ default boolean tableServicesEnabled(HoodieWriteConfig config) {
     }
     return enabled;
   }
+
+  default boolean delegateToTableManagerService(HoodieWriteConfig config, ActionType actionType) {
+    boolean supportAction = config.isTableManagerSupportAction(actionType);

Review Comment:
   rename: supports Action



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java:
##########
@@ -102,6 +104,16 @@ public Option<HoodieClusteringPlan> execute() {
         throw new HoodieIOException("Exception scheduling clustering", ioe);
       }
     }
+
+    if (config.isTableManagerSupportAction(ActionType.replacecommit)) {
+      createClusteringToService();

Review Comment:
   rename all such createXXX to delegateXXX to clearly capture what's going on. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;

Review Comment:
   Can we just align this with how the timeline server is built i.e via javalin instead of the raw HTTP client



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java:
##########
@@ -101,9 +104,14 @@ public Option<HoodieCompactionPlan> execute() {
       } catch (IOException ioe) {
         throw new HoodieIOException("Exception scheduling compaction", ioe);
       }
-      return Option.of(plan);
+      option = Option.of(plan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerSupportAction(ActionType.compaction)) {

Review Comment:
   @prasannarajaperumal can you think any way where can avoid this if switch here. the `ActionExecutor` hierarchy is pretty deep. So wonder if there is a better way to do without writing a bunch of scaffolding/small subclasses and such/



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java:
##########
@@ -152,9 +155,19 @@ protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
         LOG.error("Got exception when saving cleaner requested file", e);
         throw new HoodieIOException(e.getMessage(), e);
       }
-      return Option.of(cleanerPlan);
+      option = Option.of(cleanerPlan);
     }
-    return Option.empty();
+
+    if (config.isTableManagerSupportAction(ActionType.clean)) {
+      createCleanToService();

Review Comment:
   rename: delegateCleanExecutionToTableManager() ? In general, lets strive for consistent naming of similar concepts across the PR. e.g pick of of the following - `TableManagerService`, `TableManager`, `Service` (probably not ideal since we dont know what "service" is) 



##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableManagerConfig.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.common.config;
+
+import javax.annotation.concurrent.Immutable;
+
+import java.util.Properties;
+
+/**
+ * Configurations used by the HUDI Table Management Service.
+ */
+@Immutable
+@ConfigClassProperty(name = "Table Management Service Configs",
+    groupName = ConfigGroups.Names.WRITE_CLIENT,
+    description = "Configurations used by the Hudi Table Management Service.")
+public class HoodieTableManagerConfig extends HoodieConfig {
+
+  public static final String TABLE_MANAGEMENT_SERVICE_PREFIX = "hoodie.table.management.service";

Review Comment:
   same feedback. Consistent naming - table.manager.service vs table.management.service. IMHO we can just keep it short as `table.manager` (its kind of conceivable manager is a service)



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/RunsTableService.java:
##########
@@ -34,4 +35,12 @@ default boolean tableServicesEnabled(HoodieWriteConfig config) {
     }
     return enabled;
   }
+
+  default boolean delegateToTableManagerService(HoodieWriteConfig config, ActionType actionType) {
+    boolean supportAction = config.isTableManagerSupportAction(actionType);

Review Comment:
   rename: is Table Manager Supported Action?
   
   Also can this check move out of WriteConfig? We want to keep methods there pretty light - Small validations on data type conversions



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java:
##########
@@ -51,7 +51,7 @@ public HoodieSparkClusteringClient(
   public void cluster(HoodieInstant instant) throws IOException {
     LOG.info("Executing clustering instance " + instant);
     SparkRDDWriteClient<T> writeClient = (SparkRDDWriteClient<T>) clusteringClient;
-    Option<HoodieCommitMetadata> commitMetadata = writeClient.cluster(instant.getTimestamp(), true).getCommitMetadata();
+    Option<HoodieCommitMetadata> commitMetadata = writeClient.cluster(instant.getTimestamp()).getCommitMetadata();

Review Comment:
   who calls this today? Are they are side effects from not forcing the completion?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java:
##########
@@ -257,6 +257,7 @@ public static HoodieTableMetaClient initTableIfNotExists(Configuration conf) thr
           .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA))
           .setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
           .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
+          .setDatabaseName(conf.getString(FlinkOptions.DATABASE_NAME))

Review Comment:
   side effects?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/service";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String CREATE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/create");
+  public static final String DELETE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/delete");
+
+  public static final String CREATE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/create");
+  public static final String DELETE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/delete");
+
+  public static final String CREATE_CLEAN = String.format("%s/%s", BASE_URL, "clean/create");
+  public static final String DELETE_CLEAN = String.format("%s/%s", BASE_URL, "clean/delete");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void createCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(CREATE_COMPACTION, getDefaultParams(instantRange));

Review Comment:
   the naming of these endpoints can be more intuitive IMO. 
   Instead of CREATE_XXX , can we call them EXECUTE_XXX where xxx is the action?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String OWNER = "owner";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static int failTimes;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);

Review Comment:
   lets actually fix this in this PR please, now that log4j2 is landed. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   Reading this again, a separate client impl is probably cleaner even for this PR. These if checks will grow harder to maintain over time. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {
+
+  private static final String BASE_URL = "/v1/hoodie/serivce";
+
+  public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register");
+
+  public static final String SUBMIT_COMPACTION = String.format("%s/%s", BASE_URL, "compact/submit");
+  public static final String REMOVE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/remove");
+
+  public static final String SUBMIT_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/submit");
+  public static final String REMOVE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/remove");
+
+  public static final String SUBMIT_CLEAN = String.format("%s/%s", BASE_URL, "clean/submit");
+  public static final String REMOVE_CLEAN = String.format("%s/%s", BASE_URL, "clean/remove");
+
+  public static final String DATABASE_NAME_PARAM = "db_name";
+  public static final String TABLE_NAME_PARAM = "table_name";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String USERNAME = "username";
+  public static final String CLUSTER = "cluster";
+  public static final String QUEUE = "queue";
+  public static final String RESOURCE = "resource";
+  public static final String PARALLELISM = "parallelism";
+  public static final String EXTRA_PARAMS = "extra_params";
+  public static final String EXECUTION_ENGINE = "execution_engine";
+
+  private final HoodieTableManagerConfig config;
+  private final HoodieTableMetaClient metaClient;
+  private final String host;
+  private final int port;
+  private final String basePath;
+  private final String dbName;
+  private final String tableName;
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class);
+
+  public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) {
+    this.basePath = metaClient.getBasePathV2().toString();
+    this.dbName = metaClient.getTableConfig().getDatabaseName();
+    this.tableName = metaClient.getTableConfig().getTableName();
+    this.host = config.getTableManagerHost();
+    this.port = config.getTableManagerPort();
+    this.config = config;
+    this.metaClient = metaClient;
+  }
+
+  private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException {
+    URIBuilder builder =
+        new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http");
+    queryParameters.forEach(builder::addParameter);
+
+    String url = builder.toString();
+    LOG.info("Sending request to table management service : (" + url + ")");
+    Response response;
+    int timeout = this.config.getConnectionTimeout() * 1000; // msec
+    int requestRetryLimit = config.getConnectionRetryLimit();
+    int retry = 0;
+
+    while (retry < requestRetryLimit) {
+      try {
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        return response.returnContent().asString();
+      } catch (IOException e) {
+        retry++;
+        LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e);
+        if (requestRetryLimit == retry) {
+          throw e;
+        }
+      }
+
+      try {
+        TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay());
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit));
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  public void register() {
+    try {
+      executeRequest(REGISTER_ENDPOINT, getDefaultParams(null));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitCompaction() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .filterPendingCompactionTimeline()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_COMPACTION, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClean() {
+    try {
+      String instantRange = StringUtils.join(metaClient.reloadActiveTimeline()
+          .getCleanerTimeline()
+          .filterInflightsAndRequested()
+          .getInstants()
+          .map(HoodieInstant::getTimestamp)
+          .toArray(String[]::new), ",");
+
+      executeRequest(SUBMIT_CLEAN, getDefaultParams(instantRange));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public void submitClustering() {
+    try {
+      metaClient.reloadActiveTimeline();
+      String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient)

Review Comment:
   +1 on keeping the table manager/service decouple as discussed. 
   1) Wont' 1 be handled automatically because we resend all the pending instants again to the table manager. 
   
   2) I would imagine 2 to be a config specified at the table manager service level and not passed through from the writer/
   
   3) there is no backpressure mechanism like what you are describing today. I think your point exists in async clustering/compaction today, regardless of table service or not?
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/table/manager/HoodieTableManagerClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.client.table.manager;
+
+import org.apache.hudi.common.config.HoodieTableManagerConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieRemoteException;
+
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client which send the table service instants to the table management service.
+ */
+public class HoodieTableManagerClient {

Review Comment:
   We already use javalin and the rest end points generated/wired that way. Can we just reuse that. it will save all the new bundling work needed for a new dependency. @prasannarajaperumal wdyt?



-- 
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] yuzhaojing commented on a diff in pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -581,6 +580,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
     }
   }
 
+  private void inlineCompaction(HoodieTable table, Option<Map<String, String>> extraMetadata) {

Review Comment:
   I have built a Jira to trace, we can do a refactoring of this later, Jira: https://issues.apache.org/jira/browse/HUDI-4407, WDYT?



-- 
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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8928",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d37594e0a819dd66ed405094429693296896596",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8975",
       "triggerID" : "4d37594e0a819dd66ed405094429693296896596",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9750",
       "triggerID" : "dee99d00d3076291b077637a26bd48c3031376d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9003",
       "triggerID" : "568181a3e7c1d43a4ee8573d3746600be4f94379",
       "triggerType" : "PUSH"
     }, {
       "hash" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9837",
       "triggerID" : "75571c4be2855f4c0b6ea5b8a0babaa155604874",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9842",
       "triggerID" : "f50bb15f02020f3c345488a8bce6226b07cbe00c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80f748f87ce131827300b54781ae181c62996066",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9884",
       "triggerID" : "80f748f87ce131827300b54781ae181c62996066",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10007",
       "triggerID" : "7ea9c28a75e97a6543fa8c131fe1035034fe66c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10090",
       "triggerID" : "a589f277f3ba2bd94635132fb65f7e12af8f037e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10091",
       "triggerID" : "dad49f19ecc25ada1b9727ddfae7f6d4c02da9c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099",
       "triggerID" : "4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196",
       "triggerID" : "a9e0129347436e4510e5cb3fd195d6e843701022",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * 4da8bf7a4e18a41cae6b754934bd1ac1cd197cc4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10099) 
   * a9e0129347436e4510e5cb3fd195d6e843701022 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10196) 
   
   <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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f486cddda77a38dc493e042986d30f0f1ae5d687",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fbecda30a47830d20ec52daccd1198fbd43b133 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924) 
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb UNKNOWN
   * f486cddda77a38dc493e042986d30f0f1ae5d687 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 #5681: [HUDI-4148] Preparations and client for hudi table manager service

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924",
       "triggerID" : "1fbecda30a47830d20ec52daccd1198fbd43b133",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6566d731e9d462a78a2a85a37c09d5a24ffc45fb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fbecda30a47830d20ec52daccd1198fbd43b133 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8924) 
   * 6566d731e9d462a78a2a85a37c09d5a24ffc45fb 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] yuzhaojing closed pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service

Posted by GitBox <gi...@apache.org>.
yuzhaojing closed pull request #5681: [HUDI-4148] Preparations and client for hudi table manager service
URL: https://github.com/apache/hudi/pull/5681


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