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 2023/01/18 05:36:20 UTC

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

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