xxubai commented on code in PR #4073:
URL: https://github.com/apache/amoro/pull/4073#discussion_r2786706009


##########
amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java:
##########
@@ -239,33 +240,33 @@ public void startOptimizingService() throws Exception {
         new DefaultOptimizingService(serviceConfig, catalogManager, 
optimizerManager, tableService);
 
     processService = new ProcessService(serviceConfig, tableService);
-
-    LOG.info("Setting up AMS table executors...");
-    InlineTableExecutors.getInstance().setup(tableService, serviceConfig);
-    addHandlerChain(optimizingService.getTableRuntimeHandler());
-    addHandlerChain(processService.getTableHandlerChain());
-    
addHandlerChain(InlineTableExecutors.getInstance().getDataExpiringExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getSnapshotsExpiringExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getOrphanFilesCleaningExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getDanglingDeleteFilesCleaningExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getOptimizingCommitExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getOptimizingExpiringExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getBlockerExpiringExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getHiveCommitSyncExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getTableRefreshingExecutor());
-    
addHandlerChain(InlineTableExecutors.getInstance().getTagsAutoCreatingExecutor());
-    tableService.initialize();
-    LOG.info("AMS table service have been initialized");
+    tableService.initialize(initTablePlugins());
     tableManager.setTableService(tableService);
+    LOG.info("AMS table service have been initialized");
 
     initThriftService();
     startThriftService();
   }
 
-  private void addHandlerChain(RuntimeHandlerChain chain) {
-    if (chain != null) {
-      tableService.addHandlerChain(chain);
-    }
+  private List<TableRuntimePlugin> initTablePlugins() {
+    LOG.info("Setting up AMS table executors...");
+    InlineTableExecutors.getInstance().setup(tableService, serviceConfig);
+    IcebergTablePlugin icebergTablePlugin =

Review Comment:
   How to load different table runtime plugins?



##########
amoro-ams/src/main/java/org/apache/amoro/server/table/IcebergTablePlugin.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.amoro.server.table;
+
+import org.apache.amoro.AmoroTable;
+import org.apache.amoro.ServerTableIdentifier;
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.TableRuntime;
+import org.apache.amoro.config.TableConfiguration;
+import org.apache.amoro.server.optimizing.OptimizingStatus;
+import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
+import org.apache.amoro.table.TableRuntimeFactory;
+import org.apache.amoro.table.TableRuntimeStore;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class IcebergTablePlugin implements TableRuntimePlugin, 
TableRuntimeHandler {
+
+  private final RuntimeHandlerChain headHandler;
+
+  private IcebergTablePlugin(RuntimeHandlerChain headHandler) {
+    Preconditions.checkNotNull(headHandler);
+    this.headHandler = headHandler;
+  }
+
+  @Override
+  public boolean accept(ServerTableIdentifier tableIdentifier) {
+    return tableIdentifier.getFormat() == TableFormat.ICEBERG

Review Comment:
   IcebergTablePlugin will accept other formats? It should not be named as 
`IcebergTablePlugin`



##########
amoro-ams/src/main/java/org/apache/amoro/server/table/IcebergTablePlugin.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.amoro.server.table;
+
+import org.apache.amoro.AmoroTable;
+import org.apache.amoro.ServerTableIdentifier;
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.TableRuntime;
+import org.apache.amoro.config.TableConfiguration;
+import org.apache.amoro.server.optimizing.OptimizingStatus;
+import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
+import org.apache.amoro.table.TableRuntimeFactory;
+import org.apache.amoro.table.TableRuntimeStore;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class IcebergTablePlugin implements TableRuntimePlugin, 
TableRuntimeHandler {
+
+  private final RuntimeHandlerChain headHandler;
+
+  private IcebergTablePlugin(RuntimeHandlerChain headHandler) {
+    Preconditions.checkNotNull(headHandler);
+    this.headHandler = headHandler;
+  }
+
+  @Override
+  public boolean accept(ServerTableIdentifier tableIdentifier) {
+    return tableIdentifier.getFormat() == TableFormat.ICEBERG
+        || tableIdentifier.getFormat() == TableFormat.MIXED_HIVE
+        || tableIdentifier.getFormat() == TableFormat.MIXED_ICEBERG;
+  }
+
+  @Override
+  public TableRuntime createTableRuntime(
+      TableRuntimeFactory.Creator creator, TableRuntimeStore store) {
+    if (!(store instanceof DefaultTableRuntimeStore)) {
+      throw new IllegalStateException("Only support DefaultTableRuntimeStore");
+    }
+    DefaultTableRuntimeStore icebergRuntimeStore = (DefaultTableRuntimeStore) 
store;
+    icebergRuntimeStore.setRuntimeHandler(this);
+    return creator.create(store);
+  }
+
+  @Override
+  public void initialize(List<TableRuntime> tableRuntimes) {
+    if (headHandler != null) {

Review Comment:
   Already checked null in construct function



##########
amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableService.java:
##########
@@ -182,23 +136,46 @@ public void initialize() {
         continue;
       }
       List<TableRuntimeState> states = 
statesMap.get(tableRuntimeMeta.getTableId());
-      Optional<TableRuntime> tableRuntime =
-          createTableRuntime(identifier, tableRuntimeMeta, states);
-      if (!tableRuntime.isPresent()) {
-        LOG.warn("No available table runtime factory found for table {}", 
identifier);
-        continue;
-      }
-      tableRuntime.ifPresent(
-          t -> {
-            t.registerMetric(MetricManager.getInstance().getGlobalRegistry());
-            tableRuntimeMap.put(t.getTableIdentifier().getId(), t);
-            tableRuntimes.add(t);
-          });
+      createTableRuntime(identifier, tableRuntimeMeta, states)
+          .ifPresentOrElse(
+              tableRuntime ->
+                  
tableRuntimeMap.put(tableRuntime.getTableIdentifier().getId(), tableRuntime),
+              () -> LOG.warn("No available table runtime factory found for 
table {}", identifier));
     }
+  }
 
-    if (headHandler != null) {
-      headHandler.initialize(tableRuntimes);
-    }
+  private void initTableRuntimePlugins() {
+    List<TableRuntime> tableRuntimes = new 
ArrayList<>(tableRuntimeMap.values());
+    tableRuntimePlugins.forEach(plugin -> plugin.initialize(tableRuntimes));

Review Comment:
   Plugin is invoked in `initTableRuntimes` when create `tableRuntime`. Should 
we check the method workflow?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to