czy006 commented on code in PR #4107:
URL: https://github.com/apache/amoro/pull/4107#discussion_r2929748523


##########
amoro-ams/src/main/resources/META-INF/services/org.apache.amoro.server.process.executor.ExecuteEngine:
##########


Review Comment:
   org.apache.amoro.process.ExecuteEngine



##########
amoro-common/src/main/java/org/apache/amoro/process/LocalExecutionEngine.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.amoro.process;
+
+import org.apache.amoro.config.ConfigOption;
+import org.apache.amoro.config.ConfigOptions;
+import org.apache.amoro.config.Configurations;
+import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions;
+import 
org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Local execution engine that runs {@link 
org.apache.amoro.process.LocalProcess} instances in AMS
+ * thread pools.
+ *
+ * <p>The engine maintains multiple thread pools keyed by {@link
+ * org.apache.amoro.process.LocalProcess#tag()}.
+ */
+public class LocalExecutionEngine implements ExecuteEngine {
+  private static final Logger LOG = 
LoggerFactory.getLogger(LocalExecutionEngine.class);
+
+  public static final String ENGINE_NAME = "local";
+  public static final String DEFAULT_POOL = "default";
+  public static final String POOL_CONFIG_PREFIX = "pool.";
+  public static final String POOL_SIZE_SUFFIX = ".thread-count";
+  public static final ConfigOption<Integer> DEFAULT_POOL_SIZE =
+      ConfigOptions.key(POOL_CONFIG_PREFIX + DEFAULT_POOL + POOL_SIZE_SUFFIX)
+          .intType()
+          .defaultValue(10);
+
+  private final Map<String, ThreadPoolExecutor> pools = new 
ConcurrentHashMap<>();
+  private final Map<String, Future<?>> activeInstances = new 
ConcurrentHashMap<>();
+  private final Map<String, Future<?>> cancelingInstances = new 
ConcurrentHashMap<>();
+
+  @Override
+  public EngineType engineType() {
+    return EngineType.of(ENGINE_NAME);
+  }
+
+  @Override
+  public ProcessStatus getStatus(String processIdentifier) {
+    if (processIdentifier == null || processIdentifier.isEmpty()) {
+      return ProcessStatus.UNKNOWN;
+    }
+
+    Map<String, Future<?>> instances =
+        cancelingInstances.containsKey(processIdentifier) ? cancelingInstances 
: activeInstances;
+
+    Future<?> future = instances.get(processIdentifier);
+    if (future == null) {
+      return ProcessStatus.KILLED;
+    }
+
+    if (future.isCancelled()) {
+      instances.remove(processIdentifier);
+      return ProcessStatus.CANCELED;
+    }
+
+    if (future.isDone()) {
+      instances.remove(processIdentifier);
+      try {
+        future.get();
+        return ProcessStatus.SUCCESS;
+      } catch (Exception e) {
+        return ProcessStatus.FAILED;
+      }
+    }
+
+    return cancelingInstances.containsKey(processIdentifier)
+        ? ProcessStatus.CANCELING
+        : ProcessStatus.RUNNING;
+  }
+
+  @Override
+  public String submitTableProcess(TableProcess tableProcess) {
+    if (!(tableProcess instanceof LocalProcess)) {
+      throw new IllegalArgumentException(
+          "LocalExecutionEngine only supports LocalProcess, but got: " + 
tableProcess.getClass());
+    }
+
+    LocalProcess localProcess = (LocalProcess) tableProcess;
+    String identifier = UUID.randomUUID().toString();
+
+    ThreadPoolExecutor executor = getPool(localProcess.tag());
+    Future<?> future =
+        executor.submit(
+            () -> {
+              localProcess.run();
+              return null;
+            });
+
+    activeInstances.put(identifier, future);
+    return identifier;
+  }
+
+  @Override
+  public ProcessStatus tryCancelTableProcess(TableProcess tableProcess, String 
processIdentifier) {
+    Future<?> future = activeInstances.get(processIdentifier);
+    if (future == null) {
+      return ProcessStatus.CANCELED;
+    }
+
+    activeInstances.remove(processIdentifier);
+    cancelingInstances.put(processIdentifier, future);
+
+    if (future.isDone()) {
+      try {
+        future.get();
+        return ProcessStatus.SUCCESS;
+      } catch (Exception e) {
+        return ProcessStatus.FAILED;
+      }
+    }
+
+    if (future.isCancelled()) {
+      return ProcessStatus.CANCELED;
+    }
+
+    future.cancel(true);
+    return ProcessStatus.CANCELING;
+  }
+
+  @Override
+  public void open(Map<String, String> properties) {
+    Configurations configs = Configurations.fromMap(properties);
+    int defaultSize = configs.getInteger(DEFAULT_POOL_SIZE);
+    pools.put(DEFAULT_POOL, newFixedPool(DEFAULT_POOL, defaultSize));
+
+    Set<String> customPools =
+        properties.keySet().stream()
+            .filter(key -> key.startsWith(POOL_CONFIG_PREFIX))
+            .map(key -> key.substring(POOL_CONFIG_PREFIX.length()))
+            .map(key -> key.substring(0, key.indexOf(".") + 1))

Review Comment:
   last result is **pool.default..thread-count / 
pool.snapshots-expiring..thread-count**, that's not get the pool



##########
amoro-ams/src/main/java/org/apache/amoro/server/process/iceberg/SnapshotsExpiringProcess.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.amoro.server.process.iceberg;
+
+import org.apache.amoro.Action;
+import org.apache.amoro.AmoroTable;
+import org.apache.amoro.IcebergActions;
+import org.apache.amoro.TableRuntime;
+import org.apache.amoro.maintainer.TableMaintainer;
+import org.apache.amoro.process.ExecuteEngine;
+import org.apache.amoro.process.LocalProcess;
+import org.apache.amoro.process.TableProcess;
+import org.apache.amoro.server.optimizing.maintainer.TableMaintainers;
+import org.apache.amoro.server.table.DefaultTableRuntime;
+import org.apache.amoro.server.table.cleanup.CleanupOperation;
+import org.apache.amoro.shade.guava32.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/** Local table process for expiring Iceberg snapshots. */
+public class SnapshotsExpiringProcess extends TableProcess implements 
LocalProcess {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SnapshotsExpiringProcess.class);
+
+  public SnapshotsExpiringProcess(TableRuntime tableRuntime, ExecuteEngine 
engine) {
+    super(tableRuntime, engine);
+  }
+
+  @Override
+  public String tag() {
+    return getAction().getName().toLowerCase();
+  }
+
+  @Override
+  public void run() {
+    try {
+      AmoroTable<?> amoroTable = tableRuntime.loadTable();

Review Comment:
   The problem is that the new scheduling path no longer preserves the old 
“run, then record cleanup time” behavior for snapshot expiration.
   
   In the old implementation, SnapshotsExpiringExecutor.java executed 
**tableMaintainer.expireSnapshots()** synchronously. Only after that finished 
did PeriodicTableScheduler.java (line 125) update lastCleanTime and schedule 
the next run. So the interval was effectively measured from the end of the 
previous cleanup.
   
   In the new path, **ActionCoordinatorScheduler.java** (line 103) only 
submits/registers a process and returns immediately. After that return, 
PeriodicTableScheduler still updates lastCleanTime right away, even though the 
real cleanup work has not finished yet. The actual cleanup now happens later in 
SnapshotsExpiringProcess.java (line 53).



##########
amoro-ams/src/main/java/org/apache/amoro/server/process/iceberg/IcebergProcessFactory.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.process.iceberg;
+
+import org.apache.amoro.Action;
+import org.apache.amoro.IcebergActions;
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.TableRuntime;
+import org.apache.amoro.config.ConfigOption;
+import org.apache.amoro.config.ConfigOptions;
+import org.apache.amoro.config.Configurations;
+import org.apache.amoro.process.ExecuteEngine;
+import org.apache.amoro.process.LocalExecutionEngine;
+import org.apache.amoro.process.ProcessFactory;
+import org.apache.amoro.process.ProcessTriggerStrategy;
+import org.apache.amoro.process.RecoverProcessFailedException;
+import org.apache.amoro.process.TableProcess;
+import org.apache.amoro.process.TableProcessStore;
+import org.apache.amoro.shade.guava32.com.google.common.collect.Lists;
+import org.apache.amoro.shade.guava32.com.google.common.collect.Maps;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Default process factory for Iceberg-related maintenance actions in AMS. */
+public class IcebergProcessFactory implements ProcessFactory {
+
+  public static final String PLUGIN_NAME = "iceberg";
+  public static final ConfigOption<Boolean> SNAPSHOT_EXPIRE_ENABLED =
+      
ConfigOptions.key("expire-snapshots.enabled").booleanType().defaultValue(true);
+
+  public static final ConfigOption<Duration> SNAPSHOT_EXPIRE_INTERVAL =
+      ConfigOptions.key("expire-snapshot.interval")

Review Comment:
   YAML is **expire-snapshots.interval**



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