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/06/21 08:53:30 UTC

[GitHub] [hudi] yuzhaojing opened a new pull request, #5926: [HUDI-3475] Initialize hudi table management module

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

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## 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] Zouxxyy commented on a diff in pull request #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/resources/mybatis/Instance.xml:
##########
@@ -0,0 +1,178 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+
+<mapper namespace="Instance">
+
+  <resultMap type="org.apache.hudi.table.service.manager.entity.Instance" id="InstanceMapping">
+    <result column="id" property="id" javaType="java.lang.Long"/>
+    <result column="db_name" property="dbName"/>
+    <result column="table_name" property="tableName"/>
+    <result column="base_path" property="basePath"/>
+    <result column="execution_engine" property="executionEngine"/>
+    <result column="user_name" property="userName"/>
+    <result column="queue" property="queue"/>
+    <result column="resource" property="resource"/>
+    <result column="parallelism" property="parallelism"/>
+    <result column="instant" property="instant"/>
+    <result column="action" property="action" javaType="java.lang.Integer"/>
+    <result column="status" property="status" javaType="java.lang.Integer"/>
+    <result column="run_times" property="runTimes" javaType="java.lang.Integer"/>
+    <result column="application_id" property="applicationId"/>
+    <result column="schedule_time" property="scheduleTime" javaType="java.util.Date"/>
+    <result column="create_time" property="createTime" javaType="java.util.Date"/>
+    <result column="update_time" property="updateTime" javaType="java.util.Date"/>
+  </resultMap>
+
+  <update id="createInstance">

Review Comment:
   It might be misunderstood with `saveInstance` below, maybe we can call it `createInstanceTable`?



##########
hudi-platform-service/hudi-table-service-manager/src/main/resources/mybatis/Instance.xml:
##########
@@ -0,0 +1,178 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+
+<mapper namespace="Instance">
+
+  <resultMap type="org.apache.hudi.table.service.manager.entity.Instance" id="InstanceMapping">
+    <result column="id" property="id" javaType="java.lang.Long"/>
+    <result column="db_name" property="dbName"/>
+    <result column="table_name" property="tableName"/>

Review Comment:
   Is the tbl table canceled? I noticed that tbl_id is used here in rfc



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/common/CommandConfig.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.common;
+
+import com.beust.jcommander.Parameter;
+
+public class CommandConfig {
+  @Parameter(names = {"--server-port", "-p"}, description = "Server Port")
+  public Integer serverPort = 9092;
+
+  @Parameter(names = {"-schedule-interval-ms"}, description = "Schedule Interval Ms")
+  public Long scheduleIntervalMs = 30000L;
+
+  @Parameter(names = {"-schedule-core-executor-size"}, description = "Schedule Core Execute Size")
+  public Integer scheduleCoreExecuteSize = 300;
+
+  @Parameter(names = {"-schedule-max-executor-size"}, description = "Schedule Max Execute Size")
+  public Integer scheduleMaxExecuteSize = 1000;
+
+  @Parameter(names = {"-metadata-store-class"}, description = "Metadata Store Class")
+  public String metadataStoreClass = "org.apache.hudi.table.service.manager.store.impl.RelationDBBasedStore";
+
+  @Parameter(names = {"-instance-cache-enable"}, description = "Instance Cache Enable")
+  public boolean instanceCacheEnable = true;
+
+  @Parameter(names = {"-instance-max-retry-num"}, description = "Instance Max Retry Num")
+  public Integer instanceMaxRetryNum = 3;
+
+  @Parameter(names = {"-instance-submit-timeout-sec"}, description = "Instance Submit Timeout Sec")
+  public Integer instanceSubmitTimeoutSec = 600;
+
+  @Parameter(names = {"-spark-submit-jar-path"}, description = "Spark Submit Jar Path")

Review Comment:
   Does `spark-submit-jar` refer to hudi-cli?  Its parameters seem to be 
   ```java
   sparkLauncher.addAppArgs("COMPACT_RUN", master, sparkMemory, instance.getBasePath(), 
       instance.getTableName(), instance.getInstant(), parallelism, "", maxRetryNum, "")
   ```
   



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/handlers/CompactionHandler.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.handlers;
+
+import org.apache.hudi.table.service.manager.common.ServiceContext;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.store.MetadataStore;
+
+import org.jetbrains.annotations.NotNull;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * REST Handler servicing compaction requests.
+ */
+public class CompactionHandler {
+  private static Logger LOG = LogManager.getLogger(CompactionHandler.class);
+  protected boolean cacheEnable;
+
+  public CompactionHandler(boolean cacheEnable) {
+    this.cacheEnable = cacheEnable;
+  }
+
+  public void scheduleCompaction(MetadataStore metadataStore,
+                                 Instance instance) {
+    String recordKey = instance.getRecordKey();
+    LOG.info("Start register compaction instance: " + recordKey);
+    if ((cacheEnable && ServiceContext.containsPendingInstant(recordKey))
+        || metadataStore.getInstance(instance) != null) {
+      LOG.warn("Instance has existed, instance: " + instance);
+    } else {
+      metadataStore.saveInstance(instance);
+    }
+    if (cacheEnable) {
+      ServiceContext.refreshPendingInstant(recordKey);
+    }
+  }
+
+  public void removeCompaction(@NotNull MetadataStore metadataStore,
+                               Instance instance) {
+    LOG.info("Start remove compaction instance: " + instance.getIdentifier());
+    // 1. check instance exist
+    Instance result = metadataStore.getInstance(instance);
+    if (result == null) {
+      throw new RuntimeException("Instance not exist: " + instance);
+    }
+    // 2. update status
+    metadataStore.updateStatus(instance);

Review Comment:
   It feels like no changes have been made here



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/entity/AssistQueryEntity.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.entity;
+
+import org.apache.hudi.table.service.manager.common.ServiceConfig;
+import org.apache.hudi.table.service.manager.util.DateTimeUtils;
+
+import lombok.Getter;
+
+import java.util.Date;
+
+@Getter
+public class AssistQueryEntity {
+
+  private int maxRetry;
+
+  private Date queryStartTime = DateTimeUtils.addDay(-3);

Review Comment:
   What does this -3 mean ~



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/executor/submitter/SparkEngine.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.executor.submitter;
+
+import org.apache.hudi.cli.commands.SparkMain;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.common.ServiceConfig;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+import org.apache.hudi.table.service.manager.store.impl.InstanceService;
+
+import org.apache.spark.launcher.SparkAppHandle;
+import org.apache.spark.launcher.SparkLauncher;
+import org.apache.spark.util.Utils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.File;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.spark.launcher.SparkAppHandle.State.FINISHED;
+import static org.apache.spark.launcher.SparkAppHandle.State.SUBMITTED;
+
+public class SparkEngine extends ExecutionEngine {
+
+  private static final Logger LOG = LogManager.getLogger(SparkEngine.class);
+
+  public SparkEngine(InstanceService instanceDao, HoodieTableServiceManagerConfig config) {
+    super(instanceDao, config);
+  }
+
+  @Override
+  public Map<String, String> getJobParams(Instance instance) {
+    Map<String, String> sparkParams = new HashMap<>();
+    String parallelism = StringUtils.isNullOrEmpty(instance.getParallelism())
+        ? String.valueOf(config.getSparkMaxExecutors())
+        : instance.getParallelism();
+
+    sparkParams.put("spark.dynamicAllocation.maxExecutors", parallelism);
+    sparkParams.put("spark.dynamicAllocation.minExecutors", String.valueOf(config.getSparkMinExecutors()));
+    sparkParams.put("spark.speculation", "false");
+    String driverResource;
+    String executorResource;
+    String resource = instance.getResource().trim();
+
+    if (StringUtils.isNullOrEmpty(resource)) {
+      driverResource = config.getSparkDriverMemory();
+      executorResource = config.getSparkExecutorMemory();
+    } else {
+      String[] resourceArray = resource.split(":");
+      if (resourceArray.length == 1) {
+        driverResource = resourceArray[0];
+        executorResource = resourceArray[0];
+      } else if (resourceArray.length == 2) {
+        driverResource = resourceArray[0];
+        executorResource = resourceArray[1];
+      } else {
+        throw new RuntimeException(
+            "Invalid conf: " + instance.getIdentifier() + ", resource: " + resource);
+      }
+    }
+
+    sparkParams.put("spark.executor.cores", String.valueOf(config.getSparkExecutorCores()));
+    sparkParams.put("spark.executor.memory", executorResource);
+    sparkParams.put("spark.driver.memory", driverResource);
+    sparkParams.put("spark.executor.memoryOverhead", config.getSparkExecutorMemoryOverhead());
+
+    return sparkParams;
+  }
+
+  @Override
+  public void launchJob(String jobName, Instance instance) throws HoodieTableServiceManagerException {
+    String sparkPropertiesPath =
+        Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
+    SparkLauncher sparkLauncher;
+    try {
+      sparkLauncher = initLauncher(sparkPropertiesPath, instance);
+    } catch (URISyntaxException e) {
+      LOG.error("Failed to init spark launcher");
+      throw new HoodieTableServiceManagerException("Failed to init spark launcher", e);
+    }
+
+    try {
+      final boolean[] isFinished = new boolean[1];
+      SparkAppHandle sparkAppHandle = sparkLauncher.startApplication(new SparkAppHandle.Listener() {
+        @Override
+        public void stateChanged(SparkAppHandle handle) {
+          LOG.info("****************************");
+          LOG.info("State Changed [state={}]", handle.getState());
+          LOG.info("AppId={}", handle.getAppId());
+
+          if (handle.getState() == SUBMITTED) {
+            LOG.info("Submit job in application id: " + handle.getAppId());
+            instance.setApplicationId(handle.getAppId());
+            instanceDao.updateExecutionInfo(instance);
+          } else if (isCompleted(handle.getState())) {
+            isFinished[0] = true;
+            LOG.info("Completed job in state: " + handle.getState());
+            if (handle.getState() == FINISHED) {
+              instance.setStatus(InstanceStatus.COMPLETED.getStatus());
+            } else {
+              instance.setStatus(InstanceStatus.FAILED.getStatus());
+            }
+            instanceDao.updateStatus(instance);
+          }
+        }
+
+        @Override
+        public void infoChanged(SparkAppHandle handle) {
+          // no OP
+        }
+      });
+
+      while (!isFinished[0]) {
+        TimeUnit.SECONDS.sleep(5);
+        LOG.info("Waiting for job {} finished.", jobName);
+      }
+
+      LOG.info("Stop job when job is finished.");
+      sparkAppHandle.kill();
+    } catch (Throwable e) {
+      LOG.error("Failed to launcher spark process");
+      throw new HoodieTableServiceManagerException("Failed to init spark launcher", e);
+    }
+  }
+
+  private boolean isCompleted(SparkAppHandle.State state) {
+    switch (state) {
+      case FINISHED:
+      case FAILED:
+      case KILLED:
+      case LOST:
+        return true;
+    }
+    return false;
+  }
+
+  private SparkLauncher initLauncher(String propertiesFile, Instance instance) throws URISyntaxException {
+    String currentJar = StringUtils.isNullOrEmpty(config.getSparkSubmitJarPath())
+        ? config.getSparkSubmitJarPath()
+        : SparkEngine.class.getProtectionDomain().getCodeSource().getLocation().getFile();
+    System.out.println("currentJar = " + currentJar);
+    Map<String, String> env = System.getenv();
+    String master = config.getSparkMaster();
+
+    SparkLauncher sparkLauncher =
+        new SparkLauncher(env)
+            .setDeployMode("cluster")
+            .setMaster(master)
+            .setAppResource(currentJar)
+            .setMainClass(SparkMain.class.getName());
+
+    if (!StringUtils.isNullOrEmpty(propertiesFile)) {
+      sparkLauncher.setPropertiesFile(propertiesFile);
+    }
+
+    File libDirectory = new File(new File(currentJar).getParent(), "lib");
+    // This lib directory may be not required, such as providing libraries through a bundle jar
+    if (libDirectory.exists()) {
+      Arrays.stream(Objects.requireNonNull(libDirectory.list())).forEach(library -> {
+        if (!library.startsWith("hadoop-hdfs")) {
+          sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
+        }
+      });
+    }
+
+    Map<String, String> jobParams = getJobParams(instance);
+
+    for (Map.Entry<String, String> entry : jobParams.entrySet()) {
+      sparkLauncher.setConf(entry.getKey(), entry.getValue());
+    }
+
+    sparkLauncher.addSparkArg("--queue", instance.getQueue());
+    String sparkMemory = jobParams.get("spark.executor.memory");
+    String parallelism = String.valueOf(config.getSparkParallelism());
+    String maxRetryNum = String.valueOf(config.getInstanceMaxRetryNum());
+
+    //    sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), master, sparkMemory, client.getBasePath(),
+    //        client.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath,
+    //        retry, propsFilePath);
+    sparkLauncher.addAppArgs("COMPACT_RUN", master, sparkMemory, instance.getBasePath(),

Review Comment:
   `COMPACT_RUN` might be set as a parameter



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/executor/submitter/SparkEngine.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.executor.submitter;
+
+import org.apache.hudi.cli.commands.SparkMain;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.common.ServiceConfig;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+import org.apache.hudi.table.service.manager.store.impl.InstanceService;
+
+import org.apache.spark.launcher.SparkAppHandle;
+import org.apache.spark.launcher.SparkLauncher;
+import org.apache.spark.util.Utils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.File;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.spark.launcher.SparkAppHandle.State.FINISHED;
+import static org.apache.spark.launcher.SparkAppHandle.State.SUBMITTED;
+
+public class SparkEngine extends ExecutionEngine {
+
+  private static final Logger LOG = LogManager.getLogger(SparkEngine.class);
+
+  public SparkEngine(InstanceService instanceDao, HoodieTableServiceManagerConfig config) {
+    super(instanceDao, config);
+  }
+
+  @Override
+  public Map<String, String> getJobParams(Instance instance) {
+    Map<String, String> sparkParams = new HashMap<>();
+    String parallelism = StringUtils.isNullOrEmpty(instance.getParallelism())
+        ? String.valueOf(config.getSparkMaxExecutors())
+        : instance.getParallelism();
+
+    sparkParams.put("spark.dynamicAllocation.maxExecutors", parallelism);
+    sparkParams.put("spark.dynamicAllocation.minExecutors", String.valueOf(config.getSparkMinExecutors()));
+    sparkParams.put("spark.speculation", "false");
+    String driverResource;
+    String executorResource;
+    String resource = instance.getResource().trim();
+
+    if (StringUtils.isNullOrEmpty(resource)) {
+      driverResource = config.getSparkDriverMemory();
+      executorResource = config.getSparkExecutorMemory();
+    } else {
+      String[] resourceArray = resource.split(":");
+      if (resourceArray.length == 1) {
+        driverResource = resourceArray[0];
+        executorResource = resourceArray[0];
+      } else if (resourceArray.length == 2) {
+        driverResource = resourceArray[0];
+        executorResource = resourceArray[1];
+      } else {
+        throw new RuntimeException(
+            "Invalid conf: " + instance.getIdentifier() + ", resource: " + resource);
+      }
+    }
+
+    sparkParams.put("spark.executor.cores", String.valueOf(config.getSparkExecutorCores()));
+    sparkParams.put("spark.executor.memory", executorResource);
+    sparkParams.put("spark.driver.memory", driverResource);
+    sparkParams.put("spark.executor.memoryOverhead", config.getSparkExecutorMemoryOverhead());
+
+    return sparkParams;
+  }
+
+  @Override
+  public void launchJob(String jobName, Instance instance) throws HoodieTableServiceManagerException {
+    String sparkPropertiesPath =
+        Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
+    SparkLauncher sparkLauncher;
+    try {
+      sparkLauncher = initLauncher(sparkPropertiesPath, instance);
+    } catch (URISyntaxException e) {
+      LOG.error("Failed to init spark launcher");
+      throw new HoodieTableServiceManagerException("Failed to init spark launcher", e);
+    }
+
+    try {
+      final boolean[] isFinished = new boolean[1];
+      SparkAppHandle sparkAppHandle = sparkLauncher.startApplication(new SparkAppHandle.Listener() {
+        @Override
+        public void stateChanged(SparkAppHandle handle) {
+          LOG.info("****************************");
+          LOG.info("State Changed [state={}]", handle.getState());
+          LOG.info("AppId={}", handle.getAppId());
+
+          if (handle.getState() == SUBMITTED) {
+            LOG.info("Submit job in application id: " + handle.getAppId());
+            instance.setApplicationId(handle.getAppId());
+            instanceDao.updateExecutionInfo(instance);
+          } else if (isCompleted(handle.getState())) {
+            isFinished[0] = true;
+            LOG.info("Completed job in state: " + handle.getState());
+            if (handle.getState() == FINISHED) {
+              instance.setStatus(InstanceStatus.COMPLETED.getStatus());
+            } else {
+              instance.setStatus(InstanceStatus.FAILED.getStatus());
+            }
+            instanceDao.updateStatus(instance);
+          }
+        }
+
+        @Override
+        public void infoChanged(SparkAppHandle handle) {
+          // no OP
+        }
+      });
+
+      while (!isFinished[0]) {
+        TimeUnit.SECONDS.sleep(5);
+        LOG.info("Waiting for job {} finished.", jobName);
+      }
+
+      LOG.info("Stop job when job is finished.");
+      sparkAppHandle.kill();
+    } catch (Throwable e) {
+      LOG.error("Failed to launcher spark process");
+      throw new HoodieTableServiceManagerException("Failed to init spark launcher", e);
+    }
+  }
+
+  private boolean isCompleted(SparkAppHandle.State state) {
+    switch (state) {
+      case FINISHED:
+      case FAILED:
+      case KILLED:
+      case LOST:
+        return true;
+    }
+    return false;
+  }
+
+  private SparkLauncher initLauncher(String propertiesFile, Instance instance) throws URISyntaxException {
+    String currentJar = StringUtils.isNullOrEmpty(config.getSparkSubmitJarPath())
+        ? config.getSparkSubmitJarPath()
+        : SparkEngine.class.getProtectionDomain().getCodeSource().getLocation().getFile();
+    System.out.println("currentJar = " + currentJar);
+    Map<String, String> env = System.getenv();
+    String master = config.getSparkMaster();
+
+    SparkLauncher sparkLauncher =
+        new SparkLauncher(env)
+            .setDeployMode("cluster")
+            .setMaster(master)
+            .setAppResource(currentJar)
+            .setMainClass(SparkMain.class.getName());
+
+    if (!StringUtils.isNullOrEmpty(propertiesFile)) {
+      sparkLauncher.setPropertiesFile(propertiesFile);
+    }
+
+    File libDirectory = new File(new File(currentJar).getParent(), "lib");
+    // This lib directory may be not required, such as providing libraries through a bundle jar
+    if (libDirectory.exists()) {
+      Arrays.stream(Objects.requireNonNull(libDirectory.list())).forEach(library -> {
+        if (!library.startsWith("hadoop-hdfs")) {
+          sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
+        }
+      });
+    }
+
+    Map<String, String> jobParams = getJobParams(instance);
+
+    for (Map.Entry<String, String> entry : jobParams.entrySet()) {
+      sparkLauncher.setConf(entry.getKey(), entry.getValue());
+    }
+
+    sparkLauncher.addSparkArg("--queue", instance.getQueue());

Review Comment:
   Maybe we can use `spark.yarn.queue` and put it in the `jobParams` above?



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4fbf616dfd3913da00e9644163ce434a585bf6e2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4dc0fc0118ee468737f7bcfc4587341e047a19a3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/jdbc/SqlSessionFactoryUtil.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.jdbc;
+
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+
+import org.apache.ibatis.io.Resources;
+import org.apache.ibatis.session.SqlSession;
+import org.apache.ibatis.session.SqlSessionFactory;
+import org.apache.ibatis.session.SqlSessionFactoryBuilder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.PreparedStatement;
+import java.util.stream.Collectors;
+
+public class SqlSessionFactoryUtil {

Review Comment:
   In the follow-up PR, hudi-platform-common will be extracted for unification, let us follow up.



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/HoodieTableServiceManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.table.service.manager.common.CommandConfig;
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.service.BaseService;
+import org.apache.hudi.table.service.manager.service.CleanService;
+import org.apache.hudi.table.service.manager.service.ExecutorService;
+import org.apache.hudi.table.service.manager.service.MonitorService;
+import org.apache.hudi.table.service.manager.service.RestoreService;
+import org.apache.hudi.table.service.manager.service.RetryService;
+import org.apache.hudi.table.service.manager.service.ScheduleService;
+import org.apache.hudi.table.service.manager.store.MetadataStore;
+
+import com.beust.jcommander.JCommander;
+import io.javalin.Javalin;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Main class of hoodie table service manager.
+ *
+ * @Experimental
+ * @since 0.13.0
+ */
+public class HoodieTableServiceManager {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableServiceManager.class);
+
+  private final int serverPort;
+  private final Configuration conf;
+  private transient Javalin app = null;
+  private List<BaseService> services;
+  private final MetadataStore metadataStore;
+  private final HoodieTableServiceManagerConfig tableServiceManagerConfig;
+
+  public HoodieTableServiceManager(CommandConfig config) {
+    this.conf = FSUtils.prepareHadoopConf(new Configuration());
+    this.tableServiceManagerConfig = CommandConfig.toTableServiceManagerConfig(config);
+    this.serverPort = config.serverPort;
+    this.metadataStore = initMetadataStore();
+  }
+
+  public void startService() {
+    app = Javalin.create();
+    RequestHandler requestHandler = new RequestHandler(app, conf, metadataStore);
+    app.get("/", ctx -> ctx.result("Hello World"));
+    requestHandler.register();
+    app.start(serverPort);
+    registerService();
+    initAndStartRegisterService();
+  }
+
+  private MetadataStore initMetadataStore() {
+    String metadataStoreClass = tableServiceManagerConfig.getMetadataStoreClass();
+    MetadataStore metadataStore = (MetadataStore) ReflectionUtils.loadClass(metadataStoreClass,
+        new Class<?>[] {HoodieTableServiceManagerConfig.class}, tableServiceManagerConfig);
+    metadataStore.init();
+    LOG.info("Finish init metastore : " + metadataStoreClass);
+    return metadataStore;
+  }
+
+  private void registerService() {
+    services = new ArrayList<>();
+    ExecutorService executorService = new ExecutorService(metadataStore);
+    services.add(executorService);
+    services.add(new ScheduleService(executorService, metadataStore));
+    services.add(new RetryService(metadataStore));
+    services.add(new RestoreService(metadataStore));
+    services.add(new MonitorService());
+    services.add(new CleanService());
+  }
+
+  private void initAndStartRegisterService() {
+    for (BaseService service : services) {
+      service.init();
+      service.startService();
+    }
+  }
+
+  private void stopRegisterService() {
+    for (BaseService service : services) {
+      service.stop();
+    }
+  }
+
+  public void run() throws IOException {
+    startService();
+    Runtime.getRuntime()
+        .addShutdownHook(
+            new Thread(
+                () -> {
+                  // Use stderr here since the logger may have been reset by its JVM shutdown hook.
+                  System.err.println(
+                      "*** shutting down table service manager since JVM is shutting down");
+                  try {
+                    HoodieTableServiceManager.this.stop();
+                  } catch (InterruptedException e) {
+                    e.printStackTrace(System.err);
+                  }
+                  System.err.println("*** Table table service manager shut down");
+                }));
+  }
+
+  /**
+   * Stop serving requests and shutdown resources.
+   */
+  public void stop() throws InterruptedException {
+    if (app != null) {
+      LOG.info("Stop table service manager...");
+      this.app.stop();
+      this.app = null;
+    }
+    stopRegisterService();
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.out.println("SPARK_HOME = " + System.getenv("SPARK_HOME"));

Review Comment:
   Sure.



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d737c7d73ef7f907d04d77fa6eb87deeae521ed Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124) 
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd 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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/RequestHandler.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager;
+
+import org.apache.hudi.client.HoodieTableServiceManagerClient;
+import org.apache.hudi.table.service.manager.entity.Action;
+import org.apache.hudi.table.service.manager.entity.Engine;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+import org.apache.hudi.table.service.manager.handlers.ActionHandler;
+import org.apache.hudi.table.service.manager.store.MetadataStore;
+import org.apache.hudi.table.service.manager.util.InstanceUtil;
+
+import io.javalin.Context;
+import io.javalin.Handler;
+import io.javalin.Javalin;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.Locale;
+
+/**
+ * Main REST Handler class that handles and delegates calls to timeline relevant handlers.
+ */
+public class RequestHandler {
+
+  private static final Logger LOG = LogManager.getLogger(RequestHandler.class);
+
+  private final Javalin app;
+  private final ActionHandler actionHandler;
+
+  public RequestHandler(Javalin app,
+                        Configuration conf,
+                        MetadataStore metadataStore) {
+    this.app = app;
+    this.actionHandler = new ActionHandler(conf, metadataStore);
+  }
+
+  public void register() {
+    registerCompactionAPI();
+    registerClusteringAPI();
+    registerCleanAPI();
+  }
+
+  /**
+   * Register Compaction API calls.
+   */
+  private void registerCompactionAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_COMPACTION, new ViewHandler(ctx -> {
+      for (String instant : ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow().split(",")) {
+        Instance instance = Instance.builder()
+            .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+            .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+            .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+            .action(Action.COMPACTION.getValue())
+            .instant(instant)
+            .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+            .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+            .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+            .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+            .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+            .status(InstanceStatus.SCHEDULED.getStatus())
+            .build();
+        InstanceUtil.checkArgument(instance);
+        actionHandler.scheduleCompaction(instance);
+      }
+    }));
+  }
+
+  /**
+   * Register Clustering API calls.
+   */
+  private void registerClusteringAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_CLUSTERING, new ViewHandler(ctx -> {
+      Instance instance = Instance.builder()
+          .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+          .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+          .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+          .action(Action.CLUSTERING.getValue())
+          .instant(ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow())
+          .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+          .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+          .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+          .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+          .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+          .status(InstanceStatus.SCHEDULED.getStatus())
+          .build();
+      InstanceUtil.checkArgument(instance);
+      actionHandler.scheduleClustering(instance);
+    }));
+  }
+
+  /**
+   * Register Clean API calls.
+   */
+  private void registerCleanAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_CLEAN, new ViewHandler(ctx -> {
+      Instance instance = Instance.builder()
+          .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+          .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+          .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+          .action(Action.CLEAN.getValue())
+          .instant(ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow())
+          .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+          .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+          .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+          .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+          .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+          .status(InstanceStatus.SCHEDULED.getStatus())
+          .build();
+      InstanceUtil.checkArgument(instance);
+      actionHandler.scheduleClustering(instance);
+    }));
+  }
+
+  /**
+   * Used for logging and performing refresh check.
+   */
+  private class ViewHandler implements Handler {
+
+    private final Handler handler;
+
+    ViewHandler(Handler handler) {
+      this.handler = handler;
+    }
+
+    @Override
+    public void handle(@NotNull Context context) throws Exception {
+      boolean success = true;
+      long beginTs = System.currentTimeMillis();

Review Comment:
   Ok.



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/jdbc/JdbcMapper.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.jdbc;
+
+import java.util.List;
+import org.apache.ibatis.session.RowBounds;
+import org.apache.ibatis.session.SqlSession;
+
+public class JdbcMapper {

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] hudi-bot commented on pull request #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d737c7d73ef7f907d04d77fa6eb87deeae521ed Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094) 
   * 4fbf616dfd3913da00e9644163ce434a585bf6e2 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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497) 
   * 1d737c7d73ef7f907d04d77fa6eb87deeae521ed 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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124) 
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * a94346128d6b22fec262f74d7c2c9d7d342a0a3c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14378",
       "triggerID" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * a94346128d6b22fec262f74d7c2c9d7d342a0a3c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364) 
   * 091943461a6aa7e7dab9364813fb867f6a8771f6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14378) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497) 
   * 1d737c7d73ef7f907d04d77fa6eb87deeae521ed Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   @yuzhaojing - I have given @prasannarajaperumal full context on this. and he will take over this. and I ll check-in/chime in as well


-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/store/impl/RelationDBBasedStore.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.store.impl;
+
+import org.apache.hudi.table.management.common.ServiceContext;
+import org.apache.hudi.table.management.entity.AssistQueryEntity;
+import org.apache.hudi.table.management.entity.Instance;
+import org.apache.hudi.table.management.store.MetadataStore;
+
+import java.util.List;
+
+public class RelationDBBasedStore implements MetadataStore {

Review Comment:
   Ideally we want this to be built over the HudiTimeline persistence of RFC-36 (Metastore server). These 2 RFC's have persistence layer shared. I am okay to track this as an action item and proceed here.



##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/entity/InstanceStatus.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.entity;
+
+public enum InstanceStatus {

Review Comment:
   May be we can split State (SCHEDULED, RUNNING, COMPLETED) and Status (SUCESS, FAIL). Completed state can either be a success or fail. 



##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/entity/Instance.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+
+import java.util.Date;
+
+@Builder
+@Getter
+@Setter
+@ToString
+@NoArgsConstructor
+@AllArgsConstructor
+public class Instance {

Review Comment:
   Better name for this would TableManagementAction? Also rename Action as ActionType



##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/executor/submitter/SparkEngine.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.executor.submitter;
+
+import org.apache.hudi.cli.commands.SparkMain;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.table.management.common.ServiceConfig;
+import org.apache.hudi.table.management.entity.Instance;
+import org.apache.hudi.table.management.entity.InstanceStatus;
+import org.apache.hudi.table.management.exception.HoodieTableManagementException;
+import org.apache.hudi.table.management.store.impl.InstanceService;
+
+import org.apache.spark.launcher.SparkAppHandle;
+import org.apache.spark.launcher.SparkLauncher;
+import org.apache.spark.util.Utils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.File;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.spark.launcher.SparkAppHandle.State.FINISHED;
+import static org.apache.spark.launcher.SparkAppHandle.State.SUBMITTED;
+
+public class SparkEngine extends ExecutionEngine {
+
+  private static final Logger LOG = LogManager.getLogger(SparkEngine.class);
+
+  public SparkEngine(InstanceService instanceDao) {
+    super(instanceDao);
+  }
+
+  @Override
+  public Map<String, String> getJobParams(Instance instance) {
+    Map<String, String> sparkParams = new HashMap<>();

Review Comment:
   We could possibly register a execution engine and config with a seperate API endpoint and use that engineConfig as an input to run a table service. ServiceConfig is not generalized to any engine today - would rather get this as a input through the rest API. 
   



##########
hudi-table-management-service/pom.xml:
##########
@@ -0,0 +1,397 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hudi</artifactId>
+        <groupId>org.apache.hudi</groupId>
+        <version>0.13.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>hudi-table-management-service</artifactId>
+
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+        <mybatis.version>3.4.6</mybatis.version>
+    </properties>
+
+    <dependencies>
+        <!-- Hoodie -->
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-cli</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-client-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-utilities_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <!-- Spark -->
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <scope>compile</scope>
+        </dependency>
+
+        <!-- Fasterxml -->
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+
+        <!-- Httpcomponents -->
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>fluent-hc</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>io.javalin</groupId>
+            <artifactId>javalin</artifactId>
+            <version>2.8.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>com.beust</groupId>
+            <artifactId>jcommander</artifactId>
+        </dependency>
+
+        <!-- Hadoop -->
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.mortbay.jetty</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>tools</artifactId>
+                    <groupId>com.sun</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>tools</artifactId>
+                    <groupId>com.sun</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-auth</artifactId>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-common</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>log4j</artifactId>
+                    <groupId>log4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>jersey-core</artifactId>
+                    <groupId>com.sun.jersey</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jersey-server</artifactId>
+                    <groupId>com.sun.jersey</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jersey-client</artifactId>
+                    <groupId>com.sun.jersey</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jersey-json</artifactId>
+                    <groupId>com.sun.jersey</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jersey-guice</artifactId>
+                    <groupId>com.sun.jersey.contribs</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-java-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.mybatis</groupId>
+            <artifactId>mybatis</artifactId>
+            <version>${mybatis.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>1.18.24</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>${avro.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.25</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>1.7.25</version>
+        </dependency>
+
+        <dependency>

Review Comment:
   Can you check on the license of this third-party? Seems like apache 2.0 (https://github.com/brettwooldridge/HikariCP/blob/dev/LICENSE) but want to make sure. 



##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/entity/Instance.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+
+import java.util.Date;
+
+@Builder
+@Getter
+@Setter
+@ToString
+@NoArgsConstructor
+@AllArgsConstructor
+public class Instance {

Review Comment:
   JavaDoc and comments on the fields used here will be helpful. 



##########
hudi-table-management-service/src/main/java/org/apache/hudi/table/management/executor/BaseActionExecutor.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.management.executor;
+
+import org.apache.hudi.table.management.common.ServiceConfig;
+import org.apache.hudi.table.management.common.ServiceContext;
+import org.apache.hudi.table.management.entity.Instance;
+import org.apache.hudi.table.management.executor.submitter.ExecutionEngine;
+import org.apache.hudi.table.management.executor.submitter.SparkEngine;
+import org.apache.hudi.table.management.store.impl.InstanceService;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public abstract class BaseActionExecutor implements Runnable {
+
+  private static final Logger LOG = LogManager.getLogger(BaseActionExecutor.class);
+
+  protected InstanceService instanceDao;
+  protected Instance instance;
+  protected int maxFailTolerance;
+  protected ExecutionEngine engine;
+
+  public BaseActionExecutor(Instance instance) {
+    this.instance = instance;
+    this.instanceDao = ServiceContext.getInstanceDao();
+    this.maxFailTolerance = ServiceConfig.getInstance()
+        .getInt(ServiceConfig.ServiceConfVars.MaxFailTolerance);
+    switch (instance.getExecutionEngine()) {
+      case SPARK:
+        engine = new SparkEngine(instanceDao);
+        break;
+      case FLINK:
+      default:
+        throw new IllegalStateException("Unexpected value: " + instance.getExecutionEngine());

Review Comment:
   Engine {0} not supported



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/entity/AssistQueryEntity.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.entity;
+
+import org.apache.hudi.table.service.manager.common.ServiceConfig;
+import org.apache.hudi.table.service.manager.util.DateTimeUtils;
+
+import lombok.Getter;
+
+import java.util.Date;

Review Comment:
   Ok.



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4dc0fc0118ee468737f7bcfc4587341e047a19a3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430) 
   * 6a6b31a23cdee09dcf5f285cbae08df5a5720427 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14378",
       "triggerID" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * 091943461a6aa7e7dab9364813fb867f6a8771f6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14378) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] xushiyan commented on a diff in pull request #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/util/DateTimeUtils.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.util;
+
+import java.util.Calendar;
+import java.util.Date;
+
+public class DateTimeUtils {

Review Comment:
   can we consolidate this with `org.apache.hudi.common.util.DateTimeUtils` ? and pls add some UTs 



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/RequestHandler.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager;
+
+import org.apache.hudi.client.HoodieTableServiceManagerClient;
+import org.apache.hudi.table.service.manager.entity.Action;
+import org.apache.hudi.table.service.manager.entity.Engine;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+import org.apache.hudi.table.service.manager.handlers.ActionHandler;
+import org.apache.hudi.table.service.manager.store.MetadataStore;
+import org.apache.hudi.table.service.manager.util.InstanceUtil;
+
+import io.javalin.Context;
+import io.javalin.Handler;
+import io.javalin.Javalin;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.Locale;
+
+/**
+ * Main REST Handler class that handles and delegates calls to timeline relevant handlers.
+ */
+public class RequestHandler {
+
+  private static final Logger LOG = LogManager.getLogger(RequestHandler.class);
+
+  private final Javalin app;
+  private final ActionHandler actionHandler;
+
+  public RequestHandler(Javalin app,
+                        Configuration conf,
+                        MetadataStore metadataStore) {
+    this.app = app;
+    this.actionHandler = new ActionHandler(conf, metadataStore);
+  }
+
+  public void register() {
+    registerCompactionAPI();
+    registerClusteringAPI();
+    registerCleanAPI();
+  }
+
+  /**
+   * Register Compaction API calls.
+   */
+  private void registerCompactionAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_COMPACTION, new ViewHandler(ctx -> {
+      for (String instant : ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow().split(",")) {
+        Instance instance = Instance.builder()
+            .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+            .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+            .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+            .action(Action.COMPACTION.getValue())
+            .instant(instant)
+            .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+            .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+            .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+            .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+            .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+            .status(InstanceStatus.SCHEDULED.getStatus())
+            .build();
+        InstanceUtil.checkArgument(instance);
+        actionHandler.scheduleCompaction(instance);
+      }
+    }));
+  }
+
+  /**
+   * Register Clustering API calls.
+   */
+  private void registerClusteringAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_CLUSTERING, new ViewHandler(ctx -> {
+      Instance instance = Instance.builder()
+          .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+          .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+          .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+          .action(Action.CLUSTERING.getValue())
+          .instant(ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow())
+          .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+          .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+          .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+          .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+          .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+          .status(InstanceStatus.SCHEDULED.getStatus())
+          .build();
+      InstanceUtil.checkArgument(instance);
+      actionHandler.scheduleClustering(instance);
+    }));
+  }
+
+  /**
+   * Register Clean API calls.
+   */
+  private void registerCleanAPI() {
+    app.get(HoodieTableServiceManagerClient.EXECUTE_CLEAN, new ViewHandler(ctx -> {
+      Instance instance = Instance.builder()
+          .basePath(ctx.validatedQueryParam(HoodieTableServiceManagerClient.BASEPATH_PARAM).getOrThrow())
+          .dbName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.DATABASE_NAME_PARAM).getOrThrow())
+          .tableName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.TABLE_NAME_PARAM).getOrThrow())
+          .action(Action.CLEAN.getValue())
+          .instant(ctx.validatedQueryParam(HoodieTableServiceManagerClient.INSTANT_PARAM).getOrThrow())
+          .executionEngine(Engine.valueOf(ctx.validatedQueryParam(HoodieTableServiceManagerClient.EXECUTION_ENGINE).getOrThrow().toUpperCase(Locale.ROOT)))
+          .userName(ctx.validatedQueryParam(HoodieTableServiceManagerClient.USERNAME).getOrThrow())
+          .queue(ctx.validatedQueryParam(HoodieTableServiceManagerClient.QUEUE).getOrThrow())
+          .resource(ctx.validatedQueryParam(HoodieTableServiceManagerClient.RESOURCE).getOrThrow())
+          .parallelism(ctx.validatedQueryParam(HoodieTableServiceManagerClient.PARALLELISM).getOrThrow())
+          .status(InstanceStatus.SCHEDULED.getStatus())
+          .build();
+      InstanceUtil.checkArgument(instance);
+      actionHandler.scheduleClustering(instance);
+    }));
+  }
+
+  /**
+   * Used for logging and performing refresh check.
+   */
+  private class ViewHandler implements Handler {
+
+    private final Handler handler;
+
+    ViewHandler(Handler handler) {
+      this.handler = handler;
+    }
+
+    @Override
+    public void handle(@NotNull Context context) throws Exception {
+      boolean success = true;
+      long beginTs = System.currentTimeMillis();

Review Comment:
   use `org.apache.hudi.common.util.HoodieTimer` as a standard for compute code execution time



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/executor/submitter/ExecutionEngine.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.executor.submitter;
+
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+import org.apache.hudi.table.service.manager.store.impl.InstanceService;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Map;
+
+public abstract class ExecutionEngine {

Review Comment:
   can you please review all new classes for TSM models and add javadoc to explain the use case ?



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/jdbc/JdbcMapper.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.jdbc;
+
+import java.util.List;
+import org.apache.ibatis.session.RowBounds;
+import org.apache.ibatis.session.SqlSession;
+
+public class JdbcMapper {

Review Comment:
   we should standardize jdbc interaction, at least for metaserver and TSM. We can have a follow up jira for this - having a module `hudi-platform-service/hudi-platform-common` for common components & classes maybe ?



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/jdbc/SqlSessionFactoryUtil.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.jdbc;
+
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+
+import org.apache.ibatis.io.Resources;
+import org.apache.ibatis.session.SqlSession;
+import org.apache.ibatis.session.SqlSessionFactory;
+import org.apache.ibatis.session.SqlSessionFactoryBuilder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.PreparedStatement;
+import java.util.stream.Collectors;
+
+public class SqlSessionFactoryUtil {

Review Comment:
   can we consolidate this with `org.apache.hudi.metaserver.store.jdbc.SqlSessionFactoryUtils` ? and more UTs?



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/HoodieTableServiceManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.table.service.manager.common.CommandConfig;
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.service.BaseService;
+import org.apache.hudi.table.service.manager.service.CleanService;
+import org.apache.hudi.table.service.manager.service.ExecutorService;
+import org.apache.hudi.table.service.manager.service.MonitorService;
+import org.apache.hudi.table.service.manager.service.RestoreService;
+import org.apache.hudi.table.service.manager.service.RetryService;
+import org.apache.hudi.table.service.manager.service.ScheduleService;
+import org.apache.hudi.table.service.manager.store.MetadataStore;
+
+import com.beust.jcommander.JCommander;
+import io.javalin.Javalin;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Main class of hoodie table service manager.
+ *
+ * @Experimental
+ * @since 0.13.0
+ */
+public class HoodieTableServiceManager {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieTableServiceManager.class);
+
+  private final int serverPort;
+  private final Configuration conf;
+  private transient Javalin app = null;
+  private List<BaseService> services;
+  private final MetadataStore metadataStore;
+  private final HoodieTableServiceManagerConfig tableServiceManagerConfig;
+
+  public HoodieTableServiceManager(CommandConfig config) {
+    this.conf = FSUtils.prepareHadoopConf(new Configuration());
+    this.tableServiceManagerConfig = CommandConfig.toTableServiceManagerConfig(config);
+    this.serverPort = config.serverPort;
+    this.metadataStore = initMetadataStore();
+  }
+
+  public void startService() {
+    app = Javalin.create();
+    RequestHandler requestHandler = new RequestHandler(app, conf, metadataStore);
+    app.get("/", ctx -> ctx.result("Hello World"));
+    requestHandler.register();
+    app.start(serverPort);
+    registerService();
+    initAndStartRegisterService();
+  }
+
+  private MetadataStore initMetadataStore() {
+    String metadataStoreClass = tableServiceManagerConfig.getMetadataStoreClass();
+    MetadataStore metadataStore = (MetadataStore) ReflectionUtils.loadClass(metadataStoreClass,
+        new Class<?>[] {HoodieTableServiceManagerConfig.class}, tableServiceManagerConfig);
+    metadataStore.init();
+    LOG.info("Finish init metastore : " + metadataStoreClass);
+    return metadataStore;
+  }
+
+  private void registerService() {
+    services = new ArrayList<>();
+    ExecutorService executorService = new ExecutorService(metadataStore);
+    services.add(executorService);
+    services.add(new ScheduleService(executorService, metadataStore));
+    services.add(new RetryService(metadataStore));
+    services.add(new RestoreService(metadataStore));
+    services.add(new MonitorService());
+    services.add(new CleanService());
+  }
+
+  private void initAndStartRegisterService() {
+    for (BaseService service : services) {
+      service.init();
+      service.startService();
+    }
+  }
+
+  private void stopRegisterService() {
+    for (BaseService service : services) {
+      service.stop();
+    }
+  }
+
+  public void run() throws IOException {
+    startService();
+    Runtime.getRuntime()
+        .addShutdownHook(
+            new Thread(
+                () -> {
+                  // Use stderr here since the logger may have been reset by its JVM shutdown hook.
+                  System.err.println(
+                      "*** shutting down table service manager since JVM is shutting down");
+                  try {
+                    HoodieTableServiceManager.this.stop();
+                  } catch (InterruptedException e) {
+                    e.printStackTrace(System.err);
+                  }
+                  System.err.println("*** Table table service manager shut down");
+                }));
+  }
+
+  /**
+   * Stop serving requests and shutdown resources.
+   */
+  public void stop() throws InterruptedException {
+    if (app != null) {
+      LOG.info("Stop table service manager...");
+      this.app.stop();
+      this.app = null;
+    }
+    stopRegisterService();
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.out.println("SPARK_HOME = " + System.getenv("SPARK_HOME"));

Review Comment:
   can you review all occurrence of system.out/err.print and replace with logger ? 



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/impl/InstanceService.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.impl;
+
+import org.apache.hudi.table.service.manager.common.ServiceContext;
+import org.apache.hudi.table.service.manager.entity.AssistQueryEntity;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+
+import org.apache.hudi.table.service.manager.store.jdbc.JdbcMapper;
+
+import org.apache.ibatis.session.RowBounds;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class InstanceService {
+
+  private static Logger LOG = LogManager.getLogger(InstanceService.class);
+
+  private JdbcMapper jdbcMapper = ServiceContext.getJdbcMapper();
+
+  private static final String NAMESPACE = "Instance";
+
+  public void createInstance() {
+    try {
+      jdbcMapper.updateObject(statement(NAMESPACE, "createInstance"), null);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void saveInstance(Instance instance) {
+    try {
+      jdbcMapper.saveObject(statement(NAMESPACE, "saveInstance"), instance);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void updateStatus(Instance instance) {
+    try {
+      int ret = jdbcMapper.updateObject(statement(NAMESPACE, getUpdateStatusSqlId(instance)), instance);
+      if (ret != 1) {
+        LOG.error("Fail update status instance: " + instance);
+        throw new RuntimeException("Fail update status instance: " + instance.getIdentifier());
+      }
+      LOG.info("Success update status instance: " + instance.getIdentifier());
+    } catch (Exception e) {
+      LOG.error("Fail update status, instance: " + instance.getIdentifier() + ", errMsg: ", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void updateExecutionInfo(Instance instance) {
+    int retryNum = 0;
+    try {
+      while (retryNum++ < 3) {

Review Comment:
   use retryhelper here?



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/service/BaseService.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.service;
+
+public interface BaseService {
+
+  void init();
+
+  void startService();

Review Comment:
   why not just call it `start()` ? . :)



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/entity/AssistQueryEntity.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.entity;
+
+import org.apache.hudi.table.service.manager.common.ServiceConfig;
+import org.apache.hudi.table.service.manager.util.DateTimeUtils;
+
+import lombok.Getter;
+
+import java.util.Date;

Review Comment:
   pls avoid all java.util.Date, which is not thread safe. let's change all to java.time.* instead. Also for timestamp, can you see if better with all `Long` type? 



##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/common/CommandConfig.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.common;
+
+import com.beust.jcommander.Parameter;
+
+public class CommandConfig {
+  @Parameter(names = {"--server-port", "-p"}, description = "Server Port")
+  public Integer serverPort = 9092;
+
+  @Parameter(names = {"-schedule-interval-ms"}, description = "Schedule Interval Ms")
+  public Long scheduleIntervalMs = 30000L;
+
+  @Parameter(names = {"-schedule-core-executor-size"}, description = "Schedule Core Execute Size")
+  public Integer scheduleCoreExecuteSize = 300;
+
+  @Parameter(names = {"-schedule-max-executor-size"}, description = "Schedule Max Execute Size")
+  public Integer scheduleMaxExecuteSize = 1000;
+
+  @Parameter(names = {"-metadata-store-class"}, description = "Metadata Store Class")
+  public String metadataStoreClass = "org.apache.hudi.table.service.manager.store.impl.RelationDBBasedStore";
+
+  @Parameter(names = {"-instance-cache-enable"}, description = "Instance Cache Enable")
+  public boolean instanceCacheEnable = true;
+
+  @Parameter(names = {"-instance-max-retry-num"}, description = "Instance Max Retry Num")
+  public Integer instanceMaxRetryNum = 3;
+
+  @Parameter(names = {"-instance-submit-timeout-sec"}, description = "Instance Submit Timeout Sec")
+  public Integer instanceSubmitTimeoutSec = 600;
+
+  @Parameter(names = {"-spark-submit-jar-path"}, description = "Spark Submit Jar Path")

Review Comment:
   this is spark-specific? it's under `common/CommandConfig` so it better be engine agnostic. 



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/service/BaseService.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.service;
+
+public interface BaseService {
+
+  void init();
+
+  void startService();

Review Comment:
   Ok.



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4dc0fc0118ee468737f7bcfc4587341e047a19a3 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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d737c7d73ef7f907d04d77fa6eb87deeae521ed Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094) 
   * 4fbf616dfd3913da00e9644163ce434a585bf6e2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a6b31a23cdee09dcf5f285cbae08df5a5720427 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606) 
   * e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6 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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/store/impl/InstanceService.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.store.impl;
+
+import org.apache.hudi.table.service.manager.common.ServiceContext;
+import org.apache.hudi.table.service.manager.entity.AssistQueryEntity;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.entity.InstanceStatus;
+
+import org.apache.hudi.table.service.manager.store.jdbc.JdbcMapper;
+
+import org.apache.ibatis.session.RowBounds;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class InstanceService {
+
+  private static Logger LOG = LogManager.getLogger(InstanceService.class);
+
+  private JdbcMapper jdbcMapper = ServiceContext.getJdbcMapper();
+
+  private static final String NAMESPACE = "Instance";
+
+  public void createInstance() {
+    try {
+      jdbcMapper.updateObject(statement(NAMESPACE, "createInstance"), null);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void saveInstance(Instance instance) {
+    try {
+      jdbcMapper.saveObject(statement(NAMESPACE, "saveInstance"), instance);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void updateStatus(Instance instance) {
+    try {
+      int ret = jdbcMapper.updateObject(statement(NAMESPACE, getUpdateStatusSqlId(instance)), instance);
+      if (ret != 1) {
+        LOG.error("Fail update status instance: " + instance);
+        throw new RuntimeException("Fail update status instance: " + instance.getIdentifier());
+      }
+      LOG.info("Success update status instance: " + instance.getIdentifier());
+    } catch (Exception e) {
+      LOG.error("Fail update status, instance: " + instance.getIdentifier() + ", errMsg: ", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void updateExecutionInfo(Instance instance) {
+    int retryNum = 0;
+    try {
+      while (retryNum++ < 3) {

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] hudi-bot commented on pull request #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4fbf616dfd3913da00e9644163ce434a585bf6e2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098) 
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4fbf616dfd3913da00e9644163ce434a585bf6e2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098) 
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * a94346128d6b22fec262f74d7c2c9d7d342a0a3c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4dc0fc0118ee468737f7bcfc4587341e047a19a3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430) 
   * 6a6b31a23cdee09dcf5f285cbae08df5a5720427 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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a6b31a23cdee09dcf5f285cbae08df5a5720427 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a6b31a23cdee09dcf5f285cbae08df5a5720427 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606) 
   * e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   @prasannarajaperumal Can you have a look for 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] hudi-bot commented on pull request #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9c6308712dc95b2062fd0dfe64163e723aa46561 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124) 
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * a94346128d6b22fec262f74d7c2c9d7d342a0a3c 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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/executor/submitter/ExecutionEngine.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.executor.submitter;
+
+import org.apache.hudi.table.service.manager.common.HoodieTableServiceManagerConfig;
+import org.apache.hudi.table.service.manager.entity.Instance;
+import org.apache.hudi.table.service.manager.exception.HoodieTableServiceManagerException;
+import org.apache.hudi.table.service.manager.store.impl.InstanceService;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Map;
+
+public abstract class ExecutionEngine {

Review Comment:
   Sure, will add doc.



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4dc0fc0118ee468737f7bcfc4587341e047a19a3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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

   > @yuzhaojing - I have given @prasannarajaperumal full context on this. and he will take over this. and I ll check-in/chime in as well as needed
   
   Ok, will communicate with him. 


-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497) 
   
   <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 #5926: [HUDI-3475] Initialize hudi table management module

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


##########
hudi-platform-service/hudi-table-service-manager/src/main/java/org/apache/hudi/table/service/manager/util/DateTimeUtils.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.service.manager.util;
+
+import java.util.Calendar;
+import java.util.Date;
+
+public class DateTimeUtils {

Review Comment:
   Sure.



-- 
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 #5926: [HUDI-3475] Initialize hudi table management module

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9430",
       "triggerID" : "4dc0fc0118ee468737f7bcfc4587341e047a19a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9606",
       "triggerID" : "6a6b31a23cdee09dcf5f285cbae08df5a5720427",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10497",
       "triggerID" : "e94d6fccd8021dc0e7dd0ea780866b3aa9d542b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14094",
       "triggerID" : "1d737c7d73ef7f907d04d77fa6eb87deeae521ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14098",
       "triggerID" : "4fbf616dfd3913da00e9644163ce434a585bf6e2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14124",
       "triggerID" : "9c6308712dc95b2062fd0dfe64163e723aa46561",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364",
       "triggerID" : "a94346128d6b22fec262f74d7c2c9d7d342a0a3c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "091943461a6aa7e7dab9364813fb867f6a8771f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ed783b49dbeec18cca93a9fe43f1c4f8ee9ae6dd UNKNOWN
   * a94346128d6b22fec262f74d7c2c9d7d342a0a3c Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14364) 
   * 091943461a6aa7e7dab9364813fb867f6a8771f6 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