diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java index 283d211dd8..997f7b78cf 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java @@ -43,14 +43,14 @@ import org.apache.amoro.server.persistence.SqlSessionFactoryProvider; import org.apache.amoro.server.resource.ContainerMetadata; import org.apache.amoro.server.resource.DefaultOptimizerManager; +import org.apache.amoro.server.resource.InternalContainers; import org.apache.amoro.server.resource.OptimizerManager; -import org.apache.amoro.server.resource.ResourceContainers; +import org.apache.amoro.server.scheduler.inline.InlineTableExecutors; import org.apache.amoro.server.table.DefaultTableManager; import org.apache.amoro.server.table.DefaultTableService; import org.apache.amoro.server.table.RuntimeHandlerChain; import org.apache.amoro.server.table.TableManager; import org.apache.amoro.server.table.TableService; -import org.apache.amoro.server.table.executor.AsyncTableExecutors; import org.apache.amoro.server.terminal.TerminalManager; import org.apache.amoro.server.utils.ThriftServiceProxy; import org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting; @@ -165,18 +165,18 @@ public void startService() throws Exception { new DefaultOptimizingService(serviceConfig, catalogManager, optimizerManager, tableService); LOG.info("Setting up AMS table executors..."); - AsyncTableExecutors.getInstance().setup(tableService, serviceConfig); + InlineTableExecutors.getInstance().setup(tableService, serviceConfig); addHandlerChain(optimizingService.getTableRuntimeHandler()); - addHandlerChain(AsyncTableExecutors.getInstance().getDataExpiringExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getSnapshotsExpiringExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getOrphanFilesCleaningExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getDanglingDeleteFilesCleaningExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getOptimizingCommitExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getOptimizingExpiringExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getBlockerExpiringExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getHiveCommitSyncExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getTableRefreshingExecutor()); - addHandlerChain(AsyncTableExecutors.getInstance().getTagsAutoCreatingExecutor()); + 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"); tableManager.setTableService(tableService); @@ -528,7 +528,7 @@ private void initContainerConfig() { containerList.add(container); } } - ResourceContainers.init(containerList); + InternalContainers.init(containerList); } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java b/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java index 19d0470f72..a6bf70a6f9 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/DefaultOptimizingService.java @@ -47,8 +47,8 @@ import org.apache.amoro.server.resource.OptimizerManager; import org.apache.amoro.server.resource.OptimizerThread; import org.apache.amoro.server.resource.QuotaProvider; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.RuntimeHandlerChain; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import org.apache.amoro.shade.guava32.com.google.common.collect.Sets; @@ -136,17 +136,19 @@ public RuntimeHandlerChain getTableRuntimeHandler() { return tableHandlerChain; } - private void loadOptimizingQueues(List tableRuntimeMetaList) { + private void loadOptimizingQueues(List tableRuntimeList) { List optimizerGroups = getAs(ResourceMapper.class, ResourceMapper::selectResourceGroups); List optimizers = getAs(OptimizerMapper.class, OptimizerMapper::selectAll); - Map> groupToTableRuntimes = - tableRuntimeMetaList.stream() - .collect(Collectors.groupingBy(TableRuntime::getOptimizerGroup)); + Map> groupToTableRuntimes = + tableRuntimeList.stream() + .collect( + Collectors.groupingBy( + tableRuntime -> tableRuntime.getOptimizingState().getOptimizerGroup())); optimizerGroups.forEach( group -> { String groupName = group.getName(); - List tableRuntimes = groupToTableRuntimes.remove(groupName); + List tableRuntimes = groupToTableRuntimes.remove(groupName); OptimizingQueue optimizingQueue = new OptimizingQueue( catalogManager, @@ -283,11 +285,11 @@ public boolean cancelProcess(long processId) throws TException { return false; } long tableId = processMeta.getTableId(); - TableRuntime tableRuntime = tableService.getRuntime(tableId); + DefaultTableRuntime tableRuntime = tableService.getRuntime(tableId); if (tableRuntime == null) { return false; } - OptimizingProcess process = tableRuntime.getOptimizingProcess(); + OptimizingProcess process = tableRuntime.getOptimizingState().getOptimizingProcess(); if (process == null || process.getProcessId() != processId) { return false; } @@ -372,37 +374,39 @@ public int getTotalQuota(String resourceGroup) { private class TableRuntimeHandlerImpl extends RuntimeHandlerChain { @Override - public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { - if (!tableRuntime.getOptimizingStatus().isProcessing()) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + public void handleStatusChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) { + if (!tableRuntime.getOptimizingState().getOptimizingStatus().isProcessing()) { + getOptionalQueueByGroup(tableRuntime.getOptimizingState().getOptimizerGroup()) .ifPresent(q -> q.refreshTable(tableRuntime)); } } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { String originalGroup = originalConfig.getOptimizingConfig().getOptimizerGroup(); - if (!tableRuntime.getOptimizerGroup().equals(originalGroup)) { + if (!tableRuntime.getOptimizingState().getOptimizerGroup().equals(originalGroup)) { getOptionalQueueByGroup(originalGroup).ifPresent(q -> q.releaseTable(tableRuntime)); } - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + getOptionalQueueByGroup(tableRuntime.getOptimizingState().getOptimizerGroup()) .ifPresent(q -> q.refreshTable(tableRuntime)); } @Override - public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + public void handleTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime) { + getOptionalQueueByGroup(tableRuntime.getOptimizingState().getOptimizerGroup()) .ifPresent(q -> q.refreshTable(tableRuntime)); } @Override - public void handleTableRemoved(TableRuntime tableRuntime) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + public void handleTableRemoved(DefaultTableRuntime tableRuntime) { + getOptionalQueueByGroup(tableRuntime.getOptimizingState().getOptimizerGroup()) .ifPresent(queue -> queue.releaseTable(tableRuntime)); } @Override - protected void initHandler(List tableRuntimeList) { + protected void initHandler(List tableRuntimeList) { LOG.info("OptimizerManagementService begin initializing"); loadOptimizingQueues(tableRuntimeList); optimizerKeeper.start(); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerController.java b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerController.java index d5cdc832f1..8075128ade 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerController.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerController.java @@ -25,9 +25,9 @@ import org.apache.amoro.server.DefaultOptimizingService; import org.apache.amoro.server.dashboard.response.OkResponse; import org.apache.amoro.server.resource.ContainerMetadata; +import org.apache.amoro.server.resource.InternalContainers; import org.apache.amoro.server.resource.OptimizerInstance; import org.apache.amoro.server.resource.OptimizerManager; -import org.apache.amoro.server.resource.ResourceContainers; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import java.util.List; @@ -67,7 +67,7 @@ public void releaseOptimizer(Context ctx) { "The resource ID %s has not been indexed" + " to any optimizer.", resourceId)); Resource resource = optimizerManager.getResource(resourceId); resource.getProperties().putAll(optimizerInstances.get(0).getProperties()); - ResourceContainers.get(resource.getContainerName()).releaseOptimizer(resource); + InternalContainers.get(resource.getContainerName()).releaseResource(resource); optimizerManager.deleteResource(resourceId); optimizerManager.deleteOptimizer(resource.getGroupName(), resourceId); ctx.json(OkResponse.of("Success to release optimizer")); @@ -85,7 +85,7 @@ public void createOptimizer(Context ctx) { .setProperties(resourceGroup.getProperties()) .setThreadCount(parallelism) .build(); - ResourceContainers.get(resource.getContainerName()).requestResource(resource); + InternalContainers.get(resource.getContainerName()).requestResource(resource); optimizerManager.createResource(resource); ctx.json(OkResponse.of("success to create optimizer")); } @@ -94,7 +94,7 @@ public void createOptimizer(Context ctx) { public void getContainers(Context ctx) { ctx.json( OkResponse.of( - ResourceContainers.getMetadataList().stream() + InternalContainers.getMetadataList().stream() .map(ContainerMetadata::getName) .collect(Collectors.toList()))); } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java index c579387c97..6bc389613c 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/OptimizerGroupController.java @@ -30,9 +30,9 @@ import org.apache.amoro.server.dashboard.response.PageResult; import org.apache.amoro.server.optimizing.OptimizingStatus; import org.apache.amoro.server.resource.ContainerMetadata; +import org.apache.amoro.server.resource.InternalContainers; import org.apache.amoro.server.resource.OptimizerInstance; import org.apache.amoro.server.resource.OptimizerManager; -import org.apache.amoro.server.resource.ResourceContainers; import org.apache.amoro.server.table.TableManager; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -159,7 +159,7 @@ public void getOptimizerGroups(Context ctx) { optimizerManager.listResourceGroups().stream() .filter( resourceGroup -> - !ResourceContainers.EXTERNAL_CONTAINER_NAME.equals( + !InternalContainers.UNMANAGED_CONTAINER_NAME.equals( resourceGroup.getContainer())) .map( e -> { @@ -209,7 +209,7 @@ public void releaseOptimizer(Context ctx) { "The resource ID %s has not been indexed" + " to any optimizer.", resourceId)); Resource resource = optimizerManager.getResource(resourceId); resource.getProperties().putAll(optimizerInstances.get(0).getProperties()); - ResourceContainers.get(resource.getContainerName()).releaseOptimizer(resource); + InternalContainers.get(resource.getContainerName()).releaseResource(resource); optimizerManager.deleteResource(resourceId); optimizerManager.deleteOptimizer(resource.getGroupName(), resourceId); ctx.json(OkResponse.of("Success to release optimizer")); @@ -228,7 +228,7 @@ public void scaleOutOptimizer(Context ctx) { .setProperties(resourceGroup.getProperties()) .setThreadCount(parallelism) .build(); - ResourceContainers.get(resource.getContainerName()).requestResource(resource); + InternalContainers.get(resource.getContainerName()).requestResource(resource); optimizerManager.createResource(resource); ctx.json(OkResponse.of("success to scaleOut optimizer")); } @@ -303,7 +303,7 @@ public void deleteCheckResourceGroup(Context ctx) { public void getContainers(Context ctx) { ctx.json( OkResponse.of( - ResourceContainers.getMetadataList().stream() + InternalContainers.getMetadataList().stream() .map(ContainerMetadata::getName) .collect(Collectors.toList()))); } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/SettingController.java b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/SettingController.java index 594dde70b9..a0ce1bcbf6 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/SettingController.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/dashboard/controller/SettingController.java @@ -24,8 +24,8 @@ import org.apache.amoro.server.AmoroManagementConf; import org.apache.amoro.server.dashboard.response.OkResponse; import org.apache.amoro.server.resource.ContainerMetadata; +import org.apache.amoro.server.resource.InternalContainers; import org.apache.amoro.server.resource.OptimizerManager; -import org.apache.amoro.server.resource.ResourceContainers; import org.glassfish.jersey.internal.guava.Sets; import java.util.ArrayList; @@ -72,7 +72,7 @@ private void putSetting(Map settingMap, String key, Object value /** Get container settings. */ public void getContainerSetting(Context ctx) { - List containerMetas = ResourceContainers.getMetadataList(); + List containerMetas = InternalContainers.getMetadataList(); List> result = new ArrayList<>(); Objects.requireNonNull(containerMetas) .forEach( diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractResourceContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractOptimizerContainer.java similarity index 97% rename from amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractResourceContainer.java rename to amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractOptimizerContainer.java index 35e69a967c..9e8017429d 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractResourceContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/manager/AbstractOptimizerContainer.java @@ -19,8 +19,8 @@ package org.apache.amoro.server.manager; import org.apache.amoro.OptimizerProperties; +import org.apache.amoro.resource.InternalResourceContainer; import org.apache.amoro.resource.Resource; -import org.apache.amoro.resource.ResourceContainer; import org.apache.amoro.resource.ResourceStatus; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -public abstract class AbstractResourceContainer implements ResourceContainer { +public abstract class AbstractOptimizerContainer implements InternalResourceContainer { private String containerName; private Map containerProperties; diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/manager/FlinkOptimizerContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/manager/FlinkOptimizerContainer.java index c89222efc8..622fe1026e 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/manager/FlinkOptimizerContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/manager/FlinkOptimizerContainer.java @@ -79,7 +79,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -public class FlinkOptimizerContainer extends AbstractResourceContainer { +public class FlinkOptimizerContainer extends AbstractOptimizerContainer { private static final Logger LOG = LoggerFactory.getLogger(FlinkOptimizerContainer.class); public static final String FLINK_HOME_PROPERTY = "flink-home"; @@ -385,7 +385,7 @@ private T fetchCommandOutput(Process exec, Function commandReader } @Override - public void releaseOptimizer(Resource resource) { + public void releaseResource(Resource resource) { if (target.runByFlinkRestClient()) { Preconditions.checkArgument( resource.getProperties().containsKey(SESSION_CLUSTER_JOB_ID), diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/manager/KubernetesOptimizerContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/manager/KubernetesOptimizerContainer.java index 5834e431f7..850fba4eb3 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/manager/KubernetesOptimizerContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/manager/KubernetesOptimizerContainer.java @@ -56,7 +56,7 @@ import java.util.stream.Collectors; /** Kubernetes Optimizer Container with Standalone Optimizer */ -public class KubernetesOptimizerContainer extends AbstractResourceContainer { +public class KubernetesOptimizerContainer extends AbstractOptimizerContainer { private static final Logger LOG = LoggerFactory.getLogger(KubernetesOptimizerContainer.class); @@ -348,7 +348,7 @@ public Deployment initPodTemplateFromFrontEnd( } @Override - public void releaseOptimizer(Resource resource) { + public void releaseResource(Resource resource) { String resourceId = resource.getResourceId(); LOG.info("release Kubernetes Optimizer Container {}", resourceId); String namespace = resource.getProperties().get(NAMESPACE); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/manager/LocalOptimizerContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/manager/LocalOptimizerContainer.java index 3109d58df9..28d9747db9 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/manager/LocalOptimizerContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/manager/LocalOptimizerContainer.java @@ -27,7 +27,7 @@ import java.util.Collections; import java.util.Map; -public class LocalOptimizerContainer extends AbstractResourceContainer { +public class LocalOptimizerContainer extends AbstractOptimizerContainer { private static final Logger LOG = LoggerFactory.getLogger(LocalOptimizerContainer.class); @@ -65,7 +65,7 @@ protected String buildOptimizerStartupArgsString(Resource resource) { } @Override - public void releaseOptimizer(Resource resource) { + public void releaseResource(Resource resource) { long jobId = Long.parseLong(resource.getRequiredProperty(Resource.PROPERTY_JOB_ID)); String os = System.getProperty("os.name").toLowerCase(); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/manager/SparkOptimizerContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/manager/SparkOptimizerContainer.java index 64aee1c6c7..4e82a6e8e6 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/manager/SparkOptimizerContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/manager/SparkOptimizerContainer.java @@ -40,7 +40,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -public class SparkOptimizerContainer extends AbstractResourceContainer { +public class SparkOptimizerContainer extends AbstractOptimizerContainer { private static final Logger LOG = LoggerFactory.getLogger(SparkOptimizerContainer.class); public static final String SPARK_HOME_PROPERTY = "spark-home"; @@ -247,7 +247,7 @@ private T fetchCommandOutput(Process exec, Function commandReader } @Override - public void releaseOptimizer(Resource resource) { + public void releaseResource(Resource resource) { String releaseCommand; if (deployedOnKubernetes()) { releaseCommand = buildReleaseKubernetesCommand(resource); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizerGroupMetrics.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizerGroupMetrics.java index dfa30a5b15..0b0c4cfda5 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizerGroupMetrics.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizerGroupMetrics.java @@ -145,7 +145,7 @@ public void register() { (Gauge) () -> optimizingQueue.getSchedulingPolicy().getTableRuntimeMap().values().stream() - .filter(t -> t.getOptimizingStatus().equals(PLANNING)) + .filter(t -> t.getOptimizingState().getOptimizingStatus().equals(PLANNING)) .count()); registerMetric( registry, @@ -153,7 +153,7 @@ public void register() { (Gauge) () -> optimizingQueue.getSchedulingPolicy().getTableRuntimeMap().values().stream() - .filter(t -> t.getOptimizingStatus().equals(PENDING)) + .filter(t -> t.getOptimizingState().getOptimizingStatus().equals(PENDING)) .count()); registerMetric( registry, @@ -161,7 +161,7 @@ public void register() { (Gauge) () -> optimizingQueue.getSchedulingPolicy().getTableRuntimeMap().values().stream() - .filter(t -> t.getOptimizingStatus().isProcessing()) + .filter(t -> t.getOptimizingState().getOptimizingStatus().isProcessing()) .count()); registerMetric( registry, @@ -169,7 +169,7 @@ public void register() { (Gauge) () -> optimizingQueue.getSchedulingPolicy().getTableRuntimeMap().values().stream() - .filter(t -> t.getOptimizingStatus().equals(IDLE)) + .filter(t -> t.getOptimizingState().getOptimizingStatus().equals(IDLE)) .count()); registerMetric( registry, @@ -177,7 +177,7 @@ public void register() { (Gauge) () -> optimizingQueue.getSchedulingPolicy().getTableRuntimeMap().values().stream() - .filter(t -> t.getOptimizingStatus().equals(COMMITTING)) + .filter(t -> t.getOptimizingState().getOptimizingStatus().equals(COMMITTING)) .count()); registerMetric( diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizingQueue.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizingQueue.java index 3a840570f6..06161ea876 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizingQueue.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/OptimizingQueue.java @@ -39,7 +39,8 @@ import org.apache.amoro.server.persistence.mapper.OptimizingMapper; import org.apache.amoro.server.resource.OptimizerInstance; import org.apache.amoro.server.resource.QuotaProvider; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; @@ -97,7 +98,7 @@ public OptimizingQueue( ResourceGroup optimizerGroup, QuotaProvider quotaProvider, Executor planExecutor, - List tableRuntimeList, + List tableRuntimeList, int maxPlanningParallelism) { Preconditions.checkNotNull(optimizerGroup, "Optimizer group can not be null"); this.planExecutor = planExecutor; @@ -113,17 +114,19 @@ public OptimizingQueue( tableRuntimeList.forEach(this::initTableRuntime); } - private void initTableRuntime(TableRuntime tableRuntime) { - if (tableRuntime.getOptimizingStatus().isProcessing() && tableRuntime.getProcessId() != 0) { + private void initTableRuntime(DefaultTableRuntime tableRuntime) { + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + if (optimizingState.getOptimizingStatus().isProcessing() + && optimizingState.getProcessId() != 0) { tableRuntime.recover(new TableOptimizingProcess(tableRuntime)); } - if (tableRuntime.isOptimizingEnabled()) { - tableRuntime.resetTaskQuotas( + if (optimizingState.isOptimizingEnabled()) { + optimizingState.resetTaskQuotas( System.currentTimeMillis() - AmoroServiceConstants.QUOTA_LOOK_BACK_TIME); // Close the committing process to avoid duplicate commit on the table. - if (tableRuntime.getOptimizingStatus() == OptimizingStatus.COMMITTING) { - OptimizingProcess process = tableRuntime.getOptimizingProcess(); + if (optimizingState.getOptimizingStatus() == OptimizingStatus.COMMITTING) { + OptimizingProcess process = optimizingState.getOptimizingProcess(); if (process != null) { LOG.warn( "Close the committing process {} on table {}", @@ -132,13 +135,13 @@ private void initTableRuntime(TableRuntime tableRuntime) { process.close(); } } - if (!tableRuntime.getOptimizingStatus().isProcessing()) { + if (!optimizingState.getOptimizingStatus().isProcessing()) { scheduler.addTable(tableRuntime); } else { tableQueue.offer(new TableOptimizingProcess(tableRuntime)); } } else { - OptimizingProcess process = tableRuntime.getOptimizingProcess(); + OptimizingProcess process = optimizingState.getOptimizingProcess(); if (process != null) { process.close(); } @@ -149,23 +152,25 @@ public String getContainerName() { return optimizerGroup.getContainer(); } - public void refreshTable(TableRuntime tableRuntime) { - if (tableRuntime.isOptimizingEnabled() && !tableRuntime.getOptimizingStatus().isProcessing()) { + public void refreshTable(DefaultTableRuntime tableRuntime) { + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + if (optimizingState.isOptimizingEnabled() + && !optimizingState.getOptimizingStatus().isProcessing()) { LOG.info( "Bind queue {} success with table {}", optimizerGroup.getName(), tableRuntime.getTableIdentifier()); - tableRuntime.resetTaskQuotas( + optimizingState.resetTaskQuotas( System.currentTimeMillis() - AmoroServiceConstants.QUOTA_LOOK_BACK_TIME); scheduler.addTable(tableRuntime); } } - public void releaseTable(TableRuntime tableRuntime) { + public void releaseTable(DefaultTableRuntime tableRuntime) { scheduler.removeTable(tableRuntime); List processList = tableQueue.stream() - .filter(process -> process.tableRuntime == tableRuntime) + .filter(process -> process.optimizingState == tableRuntime.getOptimizingState()) .collect(Collectors.toList()); for (OptimizingProcess process : processList) { process.close(); @@ -238,7 +243,7 @@ private void scheduleTableIfNecessary(long startTime) { } private void triggerAsyncPlanning( - TableRuntime tableRuntime, Set skipTables, long startTime) { + DefaultTableRuntime tableRuntime, Set skipTables, long startTime) { LOG.info( "Trigger planning table {} by policy {}", tableRuntime.getTableIdentifier(), @@ -253,7 +258,7 @@ private void triggerAsyncPlanning( long currentTime = System.currentTimeMillis(); scheduleLock.lock(); try { - tableRuntime.setLastPlanTime(currentTime); + tableRuntime.getOptimizingState().setLastPlanTime(currentTime); planningTables.remove(tableRuntime.getTableIdentifier()); if (process != null) { tableQueue.offer(process); @@ -284,25 +289,25 @@ private void triggerAsyncPlanning( }); } - private TableOptimizingProcess planInternal(TableRuntime tableRuntime) { - tableRuntime.beginPlanning(); + private TableOptimizingProcess planInternal(DefaultTableRuntime tableRuntime) { + tableRuntime.getOptimizingState().beginPlanning(); try { ServerTableIdentifier identifier = tableRuntime.getTableIdentifier(); AmoroTable table = catalogManager.loadTable(identifier.getIdentifier()); AbstractOptimizingPlanner planner = IcebergTableUtil.createOptimizingPlanner( - tableRuntime.refresh(table), + tableRuntime.getOptimizingState().refresh(table), (MixedTable) table.originalTable(), getAvailableCore(), maxInputSizePerThread()); if (planner.isNecessary()) { return new TableOptimizingProcess(planner, tableRuntime); } else { - tableRuntime.completeEmptyProcess(); + tableRuntime.getOptimizingState().completeEmptyProcess(); return null; } } catch (Throwable throwable) { - tableRuntime.planFailed(); + tableRuntime.getOptimizingState().planFailed(); LOG.error("Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); throw throwable; } @@ -380,7 +385,7 @@ private class TableOptimizingProcess implements OptimizingProcess { private final Lock lock = new ReentrantLock(); private final long processId; private final OptimizingType optimizingType; - private final TableRuntime tableRuntime; + private final DefaultOptimizingState optimizingState; private final long planTime; private final long targetSnapshotId; private final long targetChangeSnapshotId; @@ -396,7 +401,7 @@ private class TableOptimizingProcess implements OptimizingProcess { public TaskRuntime poll() { lock.lock(); try { - return status != ProcessStatus.CLOSED && status != ProcessStatus.FAILED + return status != ProcessStatus.KILLED && status != ProcessStatus.FAILED ? taskQueue.poll() : null; } finally { @@ -404,9 +409,10 @@ public TaskRuntime poll() { } } - public TableOptimizingProcess(AbstractOptimizingPlanner planner, TableRuntime tableRuntime) { + public TableOptimizingProcess( + AbstractOptimizingPlanner planner, DefaultTableRuntime tableRuntime) { processId = planner.getProcessId(); - this.tableRuntime = tableRuntime; + optimizingState = tableRuntime.getOptimizingState(); optimizingType = planner.getOptimizingType(); planTime = planner.getPlanTime(); targetSnapshotId = planner.getTargetSnapshotId(); @@ -417,20 +423,20 @@ public TableOptimizingProcess(AbstractOptimizingPlanner planner, TableRuntime ta beginAndPersistProcess(); } - public TableOptimizingProcess(TableRuntime tableRuntime) { - processId = tableRuntime.getProcessId(); - this.tableRuntime = tableRuntime; - optimizingType = tableRuntime.getOptimizingType(); - targetSnapshotId = tableRuntime.getTargetSnapshotId(); - targetChangeSnapshotId = tableRuntime.getTargetChangeSnapshotId(); - planTime = tableRuntime.getLastPlanTime(); - if (tableRuntime.getFromSequence() != null) { - fromSequence = tableRuntime.getFromSequence(); + public TableOptimizingProcess(DefaultTableRuntime tableRuntime) { + optimizingState = tableRuntime.getOptimizingState(); + processId = optimizingState.getProcessId(); + optimizingType = optimizingState.getOptimizingType(); + targetSnapshotId = optimizingState.getTargetSnapshotId(); + targetChangeSnapshotId = optimizingState.getTargetChangeSnapshotId(); + planTime = optimizingState.getLastPlanTime(); + if (optimizingState.getFromSequence() != null) { + fromSequence = optimizingState.getFromSequence(); } - if (tableRuntime.getToSequence() != null) { - toSequence = tableRuntime.getToSequence(); + if (optimizingState.getToSequence() != null) { + toSequence = optimizingState.getToSequence(); } - if (this.status != ProcessStatus.CLOSED) { + if (this.status != ProcessStatus.KILLED) { tableRuntime.recover(this); } loadTaskRuntimes(this); @@ -470,11 +476,11 @@ private void acceptResult(TaskRuntime taskRuntime) { lock.lock(); try { try { - tableRuntime.addTaskQuota(taskRuntime.getCurrentQuota()); + optimizingState.addTaskQuota(taskRuntime.getCurrentQuota()); } catch (Throwable throwable) { LOG.warn( "{} failed to add task quota {}, ignore it", - tableRuntime.getTableIdentifier(), + optimizingState.getTableIdentifier(), taskRuntime.getTaskId(), throwable); } @@ -488,12 +494,12 @@ private void acceptResult(TaskRuntime taskRuntime) { if (taskRuntime.getStatus() == TaskRuntime.Status.SUCCESS) { // the lock of TableOptimizingProcess makes it thread-safe if (allTasksPrepared() - && tableRuntime.getOptimizingStatus().isProcessing() - && tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { - tableRuntime.beginCommitting(); + && optimizingState.getOptimizingStatus().isProcessing() + && optimizingState.getOptimizingStatus() != OptimizingStatus.COMMITTING) { + optimizingState.beginCommitting(); } } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { - if (taskRuntime.getRetry() < tableRuntime.getMaxExecuteRetryCount()) { + if (taskRuntime.getRetry() < optimizingState.getMaxExecuteRetryCount()) { LOG.info( "Put task {} to retry queue, because {}", taskRuntime.getTaskId(), @@ -513,7 +519,7 @@ private void acceptResult(TaskRuntime taskRuntime) { @Override public boolean isClosed() { - return status == ProcessStatus.CLOSED; + return status == ProcessStatus.KILLED; } @Override @@ -577,14 +583,14 @@ public long getRunningQuotaTime(long calculatingStartTime, long calculatingEndTi public void commit() { LOG.debug( "{} get {} tasks of {} partitions to commit", - tableRuntime.getTableIdentifier(), + optimizingState.getTableIdentifier(), taskMap.size(), taskMap.values()); lock.lock(); try { if (hasCommitted) { - LOG.warn("{} has already committed, give up", tableRuntime.getTableIdentifier()); + LOG.warn("{} has already committed, give up", optimizingState.getTableIdentifier()); try { persistAndSetCompleted(status == ProcessStatus.SUCCESS); } catch (Exception ignored) { @@ -600,10 +606,10 @@ public void commit() { } catch (PersistenceException e) { LOG.warn( "{} failed to persist process completed, will retry next commit", - tableRuntime.getTableIdentifier(), + optimizingState.getTableIdentifier(), e); } catch (Throwable t) { - LOG.error("{} Commit optimizing failed ", tableRuntime.getTableIdentifier(), t); + LOG.error("{} Commit optimizing failed ", optimizingState.getTableIdentifier(), t); status = ProcessStatus.FAILED; failedReason = ExceptionUtil.getErrorMessage(t, 4000); endTime = System.currentTimeMillis(); @@ -629,7 +635,7 @@ private UnKeyedTableCommit buildCommit() { MixedTable table = (MixedTable) catalogManager - .loadTable(tableRuntime.getTableIdentifier().getIdentifier()) + .loadTable(optimizingState.getTableIdentifier().getIdentifier()) .originalTable(); if (table.isUnkeyedTable()) { return new UnKeyedTableCommit(targetSnapshotId, table, taskMap.values()); @@ -666,7 +672,7 @@ private void beginAndPersistProcess() { OptimizingMapper.class, mapper -> mapper.insertOptimizingProcess( - tableRuntime.getTableIdentifier(), + optimizingState.getTableIdentifier(), processId, targetSnapshotId, targetChangeSnapshotId, @@ -681,7 +687,7 @@ private void beginAndPersistProcess() { OptimizingMapper.class, mapper -> mapper.insertTaskRuntimes(Lists.newArrayList(taskMap.values()))), () -> TaskFilesPersistence.persistTaskInputs(processId, taskMap.values()), - () -> tableRuntime.beginProcess(this)); + () -> optimizingState.beginProcess(this)); } private void persistAndSetCompleted(boolean success) { @@ -696,13 +702,13 @@ private void persistAndSetCompleted(boolean success) { OptimizingMapper.class, mapper -> mapper.updateOptimizingProcess( - tableRuntime.getTableIdentifier().getId(), + optimizingState.getTableIdentifier().getId(), processId, status, endTime, getSummary(), getFailedReason())), - () -> tableRuntime.completeProcess(success), + () -> optimizingState.completeProcess(success), () -> clearProcess(this)); } @@ -717,7 +723,7 @@ private void loadTaskRuntimes(OptimizingProcess optimizingProcess) { OptimizingMapper.class, mapper -> mapper.selectTaskRuntimes( - tableRuntime.getTableIdentifier().getId(), processId)); + optimizingState.getTableIdentifier().getId(), processId)); Map inputs = TaskFilesPersistence.loadTaskInputs(processId); taskRuntimes.forEach( taskRuntime -> { @@ -748,7 +754,7 @@ private void loadTaskRuntimes(List taskDescriptors) { new TaskRuntime<>(new OptimizingTaskId(processId, taskId++), taskDescriptor); LOG.info( "{} plan new task {}, summary {}", - tableRuntime.getTableIdentifier(), + optimizingState.getTableIdentifier(), taskRuntime.getTaskId(), taskRuntime.getSummary()); taskRuntime.getCompletedFuture().whenCompleted(() -> acceptResult(taskRuntime)); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/SchedulingPolicy.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/SchedulingPolicy.java index 31209062d3..65ee979a1c 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/SchedulingPolicy.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/SchedulingPolicy.java @@ -23,7 +23,8 @@ import org.apache.amoro.resource.ResourceGroup; import org.apache.amoro.server.optimizing.sorter.QuotaOccupySorter; import org.apache.amoro.server.optimizing.sorter.SorterFactory; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; import org.slf4j.Logger; @@ -46,7 +47,7 @@ public class SchedulingPolicy { private static final String SCHEDULING_POLICY_PROPERTY_NAME = "scheduling-policy"; - private final Map tableRuntimeMap = new HashMap<>(); + private final Map tableRuntimeMap = new HashMap<>(); private volatile String policyName; private final Lock tableLock = new ReentrantLock(); private static final Map sorterFactoryCache = new ConcurrentHashMap<>(); @@ -85,7 +86,7 @@ public String name() { return policyName; } - public TableRuntime scheduleTable(Set skipSet) { + public DefaultTableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { fillSkipSet(skipSet); @@ -98,7 +99,7 @@ public TableRuntime scheduleTable(Set skipSet) { } } - private Comparator createSorterByPolicy() { + private Comparator createSorterByPolicy() { if (sorterFactoryCache.get(policyName) != null) { SorterFactory sorterFactory = sorterFactoryCache.get(policyName); LOG.info( @@ -111,7 +112,7 @@ private Comparator createSorterByPolicy() { } } - public TableRuntime getTableRuntime(ServerTableIdentifier tableIdentifier) { + public DefaultTableRuntime getTableRuntime(ServerTableIdentifier tableIdentifier) { tableLock.lock(); try { return tableRuntimeMap.get(tableIdentifier); @@ -123,23 +124,24 @@ public TableRuntime getTableRuntime(ServerTableIdentifier tableIdentifier) { private void fillSkipSet(Set originalSet) { long currentTime = System.currentTimeMillis(); tableRuntimeMap.values().stream() + .map(DefaultTableRuntime::getOptimizingState) .filter( - tableRuntime -> - !isTablePending(tableRuntime) - || tableRuntime.isBlocked(BlockableOperation.OPTIMIZE) - || currentTime - tableRuntime.getLastPlanTime() - < tableRuntime.getOptimizingConfig().getMinPlanInterval()) + optimizingState -> + !isTablePending(optimizingState) + || optimizingState.isBlocked(BlockableOperation.OPTIMIZE) + || currentTime - optimizingState.getLastPlanTime() + < optimizingState.getOptimizingConfig().getMinPlanInterval()) .forEach(tableRuntime -> originalSet.add(tableRuntime.getTableIdentifier())); } - private boolean isTablePending(TableRuntime tableRuntime) { - return tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING - && (tableRuntime.getLastOptimizedSnapshotId() != tableRuntime.getCurrentSnapshotId() - || tableRuntime.getLastOptimizedChangeSnapshotId() - != tableRuntime.getCurrentChangeSnapshotId()); + private boolean isTablePending(DefaultOptimizingState optimizingState) { + return optimizingState.getOptimizingStatus() == OptimizingStatus.PENDING + && (optimizingState.getLastOptimizedSnapshotId() != optimizingState.getCurrentSnapshotId() + || optimizingState.getLastOptimizedChangeSnapshotId() + != optimizingState.getCurrentChangeSnapshotId()); } - public void addTable(TableRuntime tableRuntime) { + public void addTable(DefaultTableRuntime tableRuntime) { tableLock.lock(); try { tableRuntimeMap.put(tableRuntime.getTableIdentifier(), tableRuntime); @@ -148,7 +150,7 @@ public void addTable(TableRuntime tableRuntime) { } } - public void removeTable(TableRuntime tableRuntime) { + public void removeTable(DefaultTableRuntime tableRuntime) { tableLock.lock(); try { tableRuntimeMap.remove(tableRuntime.getTableIdentifier()); @@ -158,7 +160,7 @@ public void removeTable(TableRuntime tableRuntime) { } @VisibleForTesting - Map getTableRuntimeMap() { + Map getTableRuntimeMap() { return tableRuntimeMap; } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/IcebergTableMaintainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/IcebergTableMaintainer.java index 03e7fa326c..3f45e40b13 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/IcebergTableMaintainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/IcebergTableMaintainer.java @@ -27,9 +27,10 @@ import org.apache.amoro.io.AuthenticatedFileIO; import org.apache.amoro.io.PathInfo; import org.apache.amoro.io.SupportsFileSystemOperations; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting; import org.apache.amoro.shade.guava32.com.google.common.base.Strings; @@ -123,10 +124,10 @@ public IcebergTableMaintainer(Table table) { } @Override - public void cleanOrphanFiles(TableRuntime tableRuntime) { + public void cleanOrphanFiles(DefaultTableRuntime tableRuntime) { TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics = - tableRuntime.getOrphanFilesCleaningMetrics(); + tableRuntime.getOptimizingState().getOrphanFilesCleaningMetrics(); if (!tableConfiguration.isCleanOrphanEnabled()) { return; @@ -144,7 +145,7 @@ public void cleanOrphanFiles(TableRuntime tableRuntime) { } @Override - public void cleanDanglingDeleteFiles(TableRuntime tableRuntime) { + public void cleanDanglingDeleteFiles(DefaultTableRuntime tableRuntime) { TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); if (!tableConfiguration.isDeleteDanglingDeleteFilesEnabled()) { @@ -168,7 +169,7 @@ public void cleanDanglingDeleteFiles(TableRuntime tableRuntime) { } @Override - public void expireSnapshots(TableRuntime tableRuntime) { + public void expireSnapshots(DefaultTableRuntime tableRuntime) { if (!expireSnapshotEnabled(tableRuntime)) { return; } @@ -177,7 +178,7 @@ public void expireSnapshots(TableRuntime tableRuntime) { tableRuntime.getTableConfiguration().getSnapshotMinCount()); } - protected boolean expireSnapshotEnabled(TableRuntime tableRuntime) { + protected boolean expireSnapshotEnabled(DefaultTableRuntime tableRuntime) { TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); return tableConfiguration.isExpireSnapshotEnabled(); } @@ -245,7 +246,7 @@ private void expireSnapshots(long olderThan, int minCount, Set exclude) } @Override - public void expireData(TableRuntime tableRuntime) { + public void expireData(DefaultTableRuntime tableRuntime) { try { DataExpirationConfig expirationConfig = tableRuntime.getTableConfiguration().getExpiringDataConfig(); @@ -309,7 +310,7 @@ public void expireDataFrom(DataExpirationConfig expirationConfig, Instant instan } @Override - public void autoCreateTags(TableRuntime tableRuntime) { + public void autoCreateTags(DefaultTableRuntime tableRuntime) { new AutoCreateIcebergTagAction( table, tableRuntime.getTableConfiguration().getTagConfiguration(), LocalDateTime.now()) .execute(); @@ -349,7 +350,7 @@ protected void cleanDanglingDeleteFiles() { table.name())); } - protected long mustOlderThan(TableRuntime tableRuntime, long now) { + protected long mustOlderThan(DefaultTableRuntime tableRuntime, long now) { return min( // The snapshots keep time now - snapshotsKeepTime(tableRuntime), @@ -361,7 +362,7 @@ protected long mustOlderThan(TableRuntime tableRuntime, long now) { fetchLatestFlinkCommittedSnapshotTime(table)); } - protected long snapshotsKeepTime(TableRuntime tableRuntime) { + protected long snapshotsKeepTime(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().getSnapshotTTLMinutes() * 60 * 1000; } @@ -511,9 +512,11 @@ public static long fetchLatestFlinkCommittedSnapshotTime(Table table) { * @return time of snapshot for optimizing process planned based, return Long.MAX_VALUE if no * optimizing process exists */ - public static long fetchOptimizingPlanSnapshotTime(Table table, TableRuntime tableRuntime) { - if (tableRuntime.getOptimizingStatus().isProcessing()) { - long fromSnapshotId = tableRuntime.getOptimizingProcess().getTargetSnapshotId(); + public static long fetchOptimizingPlanSnapshotTime( + Table table, DefaultTableRuntime tableRuntime) { + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + if (optimizingState.getOptimizingStatus().isProcessing()) { + long fromSnapshotId = optimizingState.getOptimizingProcess().getTargetSnapshotId(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.snapshotId() == fromSnapshotId) { diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/MixedTableMaintainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/MixedTableMaintainer.java index e95a9507e3..a6e998c8a5 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/MixedTableMaintainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/MixedTableMaintainer.java @@ -26,9 +26,9 @@ import org.apache.amoro.config.DataExpirationConfig; import org.apache.amoro.data.FileNameRules; import org.apache.amoro.scan.TableEntriesScan; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.utils.HiveLocationUtil; import org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting; import org.apache.amoro.shade.guava32.com.google.common.collect.Iterables; @@ -93,7 +93,7 @@ public MixedTableMaintainer(MixedTable mixedTable) { } @Override - public void cleanOrphanFiles(TableRuntime tableRuntime) { + public void cleanOrphanFiles(DefaultTableRuntime tableRuntime) { if (changeMaintainer != null) { changeMaintainer.cleanOrphanFiles(tableRuntime); } @@ -101,7 +101,7 @@ public void cleanOrphanFiles(TableRuntime tableRuntime) { } @Override - public void expireSnapshots(TableRuntime tableRuntime) { + public void expireSnapshots(DefaultTableRuntime tableRuntime) { if (changeMaintainer != null) { changeMaintainer.expireSnapshots(tableRuntime); } @@ -117,7 +117,7 @@ protected void expireSnapshots(long mustOlderThan, int minCount) { } @Override - public void expireData(TableRuntime tableRuntime) { + public void expireData(DefaultTableRuntime tableRuntime) { try { DataExpirationConfig expirationConfig = tableRuntime.getTableConfiguration().getExpiringDataConfig(); @@ -243,7 +243,7 @@ private void expireMixedFiles( } @Override - public void autoCreateTags(TableRuntime tableRuntime) { + public void autoCreateTags(DefaultTableRuntime tableRuntime) { throw new UnsupportedOperationException("Mixed table doesn't support auto create tags"); } @@ -297,7 +297,7 @@ void expireSnapshots(long mustOlderThan, int minCount) { } @Override - public void expireSnapshots(TableRuntime tableRuntime) { + public void expireSnapshots(DefaultTableRuntime tableRuntime) { if (!expireSnapshotEnabled(tableRuntime)) { return; } @@ -309,7 +309,7 @@ public void expireSnapshots(TableRuntime tableRuntime) { } @Override - protected long mustOlderThan(TableRuntime tableRuntime, long now) { + protected long mustOlderThan(DefaultTableRuntime tableRuntime, long now) { return min( // The change data ttl time now - snapshotsKeepTime(tableRuntime), @@ -318,7 +318,7 @@ protected long mustOlderThan(TableRuntime tableRuntime, long now) { } @Override - protected long snapshotsKeepTime(TableRuntime tableRuntime) { + protected long snapshotsKeepTime(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().getChangeDataTTLMinutes() * 60 * 1000; } @@ -459,7 +459,7 @@ protected Set expireSnapshotNeedToExcludeFiles() { } @Override - protected long mustOlderThan(TableRuntime tableRuntime, long now) { + protected long mustOlderThan(DefaultTableRuntime tableRuntime, long now) { DataExpirationConfig expiringDataConfig = tableRuntime.getTableConfiguration().getExpiringDataConfig(); long dataExpiringSnapshotTime = diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/TableMaintainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/TableMaintainer.java index a6ce202de5..4db6917a89 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/TableMaintainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/TableMaintainer.java @@ -20,7 +20,7 @@ import org.apache.amoro.AmoroTable; import org.apache.amoro.TableFormat; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.table.MixedTable; import org.apache.iceberg.Table; @@ -34,10 +34,10 @@ public interface TableMaintainer { /** Clean table orphan files. Includes: data files, metadata files. */ - void cleanOrphanFiles(TableRuntime tableRuntime); + void cleanOrphanFiles(DefaultTableRuntime tableRuntime); /** Clean table dangling delete files. */ - default void cleanDanglingDeleteFiles(TableRuntime tableRuntime) { + default void cleanDanglingDeleteFiles(DefaultTableRuntime tableRuntime) { // DO nothing by default } @@ -45,7 +45,7 @@ default void cleanDanglingDeleteFiles(TableRuntime tableRuntime) { * Expire snapshots. The optimizing based on the snapshot that the current table relies on will * not expire according to TableRuntime. */ - void expireSnapshots(TableRuntime tableRuntime); + void expireSnapshots(DefaultTableRuntime tableRuntime); /** * Expire historical data based on the expiration field, and data that exceeds the retention @@ -53,10 +53,10 @@ default void cleanDanglingDeleteFiles(TableRuntime tableRuntime) { * * @param tableRuntime TableRuntime */ - void expireData(TableRuntime tableRuntime); + void expireData(DefaultTableRuntime tableRuntime); /** Auto create tags for table. */ - void autoCreateTags(TableRuntime tableRuntime); + void autoCreateTags(DefaultTableRuntime tableRuntime); static TableMaintainer ofTable(AmoroTable amoroTable) { TableFormat format = amoroTable.format(); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/BalancedSorter.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/BalancedSorter.java index 17963fee0e..6c3ff3c10f 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/BalancedSorter.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/BalancedSorter.java @@ -18,7 +18,7 @@ package org.apache.amoro.server.optimizing.sorter; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultTableRuntime; import java.util.Comparator; @@ -32,18 +32,21 @@ public String getIdentifier() { } @Override - public Comparator createComparator() { - return new Comparator() { + public Comparator createComparator() { + return new Comparator() { @Override - public int compare(TableRuntime one, TableRuntime another) { + public int compare(DefaultTableRuntime one, DefaultTableRuntime another) { return Long.compare( Math.max( - one.getLastFullOptimizingTime(), - Math.max(one.getLastMinorOptimizingTime(), one.getLastMajorOptimizingTime())), + one.getOptimizingState().getLastFullOptimizingTime(), + Math.max( + one.getOptimizingState().getLastMinorOptimizingTime(), + one.getOptimizingState().getLastMajorOptimizingTime())), Math.max( - another.getLastFullOptimizingTime(), + another.getOptimizingState().getLastFullOptimizingTime(), Math.max( - another.getLastMinorOptimizingTime(), another.getLastMajorOptimizingTime()))); + another.getOptimizingState().getLastMinorOptimizingTime(), + another.getOptimizingState().getLastMajorOptimizingTime()))); } }; } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/QuotaOccupySorter.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/QuotaOccupySorter.java index 27af5104de..fd3342a0c6 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/QuotaOccupySorter.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/sorter/QuotaOccupySorter.java @@ -18,7 +18,7 @@ package org.apache.amoro.server.optimizing.sorter; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; import java.util.Comparator; @@ -34,15 +34,17 @@ public String getIdentifier() { } @Override - public Comparator createComparator() { - return new Comparator() { - final Map tableWeightMap = Maps.newHashMap(); + public Comparator createComparator() { + return new Comparator() { + final Map tableWeightMap = Maps.newHashMap(); @Override - public int compare(TableRuntime one, TableRuntime another) { + public int compare(DefaultTableRuntime one, DefaultTableRuntime another) { return Double.compare( - tableWeightMap.computeIfAbsent(one, TableRuntime::calculateQuotaOccupy), - tableWeightMap.computeIfAbsent(another, TableRuntime::calculateQuotaOccupy)); + tableWeightMap.computeIfAbsent( + one, tableRuntime -> tableRuntime.getOptimizingState().calculateQuotaOccupy()), + tableWeightMap.computeIfAbsent( + another, tableRuntime -> tableRuntime.getOptimizingState().calculateQuotaOccupy())); } }; } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/PersistentBase.java b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/PersistentBase.java index e3f822c385..92bc815a91 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/PersistentBase.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/PersistentBase.java @@ -91,6 +91,7 @@ protected final void doAsTransaction(Runnable... operations) { Arrays.stream(operations).forEach(Runnable::run); session.commit(); } catch (Throwable t) { + LOG.error("failed to commit transaction", t); session.rollback(); throw AmoroRuntimeException.wrap(t, PersistenceException::new); } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/ProcessStateMapper.java b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/ProcessStateMapper.java new file mode 100644 index 0000000000..f7b7557819 --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/ProcessStateMapper.java @@ -0,0 +1,87 @@ +/* + * 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.persistence.mapper; + +import org.apache.amoro.process.TableProcessState; +import org.apache.ibatis.annotations.Insert; +import org.apache.ibatis.annotations.Options; +import org.apache.ibatis.annotations.Param; +import org.apache.ibatis.annotations.Result; +import org.apache.ibatis.annotations.ResultMap; +import org.apache.ibatis.annotations.Results; +import org.apache.ibatis.annotations.Select; +import org.apache.ibatis.annotations.Update; + +import java.util.Map; + +public interface ProcessStateMapper { + + @Insert( + "INSERT INTO table_process_state " + + "(process_id, action, table_id, retry_num, status, start_time, end_time, fail_reason, summary) " + + "VALUES " + + "(#{id}, #{action}, #{tableId}, #{retryNumber}, #{status}, #{startTime}, #{endTime}, #{failedReason}, #{summary})") + @Options(useGeneratedKeys = true, keyProperty = "id") + void createProcessState(TableProcessState state); + + @Update( + "UPDATE table_process_state " + + "SET status = #{status}, start_time = #{startTime} " + + "WHERE process_id = #{id} and retry_num = #{retryNumber}") + void updateProcessRunning(TableProcessState state); + + @Update( + "UPDATE table_process_state " + + "SET status = #{status}, end_time = #{endTime} " + + "WHERE process_id = #{id} and retry_num = #{retryNumber}") + void updateProcessCompleted(TableProcessState state); + + @Update( + "UPDATE table_process_state " + + "SET status = #{status}, end_time = #{endTime}, fail_reason = #{failedReason} " + + "WHERE process_id = #{id} and retry_num = #{retryNumber}") + void updateProcessFailed(TableProcessState state); + + @Select( + "SELECT process_id, action, table_id, retry_num, status, start_time, end_time, fail_reason, summary " + + "FROM table_process_state " + + "WHERE process_id = #{processId}") + @Results( + id = "TableProcessStateResultMap", + value = { + @Result(property = "id", column = "process_id"), + @Result(property = "action", column = "action"), + @Result(property = "tableId", column = "table_id"), + @Result(property = "retryNumber", column = "retry_num"), + @Result(property = "status", column = "status"), + @Result(property = "startTime", column = "start_time"), + @Result(property = "endTime", column = "end_time"), + @Result(property = "failedReason", column = "fail_reason"), + @Result(property = "summary", column = "summary", javaType = Map.class) + }) + TableProcessState getProcessStateById(@Param("processId") long processId); + + /** Query TableProcessState by table_id */ + @Select( + "SELECT process_id, action, table_id, retry_num, status, start_time, end_time, fail_reason, summary " + + "FROM table_process_state " + + "WHERE table_id = #{tableId}") + @ResultMap("TableProcessStateResultMap") + TableProcessState getProcessStateByTableId(@Param("tableId") long tableId); +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java index fd6d0ff7ff..1ae224672c 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableMetaMapper.java @@ -26,8 +26,8 @@ import org.apache.amoro.server.persistence.converter.MapLong2StringConverter; import org.apache.amoro.server.persistence.converter.OptimizingStatusConverter; import org.apache.amoro.server.persistence.converter.TableFormatConverter; +import org.apache.amoro.server.table.DefaultOptimizingState; import org.apache.amoro.server.table.TableMetadata; -import org.apache.amoro.server.table.TableRuntime; import org.apache.ibatis.annotations.Delete; import org.apache.ibatis.annotations.Insert; import org.apache.ibatis.annotations.Options; @@ -349,7 +349,7 @@ List selectTableIdentifiersByCatalog( + " table_summary = #{runtime.tableSummary, jdbcType=VARCHAR," + " typeHandler=org.apache.amoro.server.persistence.converter.JsonObjectConverter}" + " WHERE table_id = #{runtime.tableIdentifier.id}") - void updateTableRuntime(@Param("runtime") TableRuntime runtime); + void updateTableRuntime(@Param("runtime") DefaultOptimizingState optimizingState); @Delete("DELETE FROM table_runtime WHERE table_id = #{tableId}") void deleteOptimizingRuntime(@Param("tableId") long tableId); @@ -381,7 +381,7 @@ List selectTableIdentifiersByCatalog( + " typeHandler=org.apache.amoro.server.persistence.converter.JsonObjectConverter}," + " #{runtime.tableSummary, jdbcType=VARCHAR," + " typeHandler=org.apache.amoro.server.persistence.converter.JsonObjectConverter})") - void insertTableRuntime(@Param("runtime") TableRuntime runtime); + void insertTableRuntime(@Param("runtime") DefaultOptimizingState optimizingState); @Select( "SELECT a.table_id, a.catalog_name, a.db_name, a.table_name, i.format, a.current_snapshot_id," diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/resource/ResourceContainers.java b/amoro-ams/src/main/java/org/apache/amoro/server/resource/InternalContainers.java similarity index 81% rename from amoro-ams/src/main/java/org/apache/amoro/server/resource/ResourceContainers.java rename to amoro-ams/src/main/java/org/apache/amoro/server/resource/InternalContainers.java index 84831f89de..c5eecb33d1 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/resource/ResourceContainers.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/resource/InternalContainers.java @@ -18,7 +18,7 @@ package org.apache.amoro.server.resource; -import org.apache.amoro.resource.ResourceContainer; +import org.apache.amoro.resource.InternalResourceContainer; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; @@ -27,15 +27,15 @@ import java.util.Optional; import java.util.stream.Collectors; -public class ResourceContainers { - public static final String EXTERNAL_CONTAINER_NAME = "external"; +public class InternalContainers { + public static final String UNMANAGED_CONTAINER_NAME = "unmanaged"; private static final Map globalContainers = Maps.newHashMap(); private static volatile boolean isInitialized = false; static { - ContainerMetadata metadata = new ContainerMetadata(EXTERNAL_CONTAINER_NAME, ""); + ContainerMetadata metadata = new ContainerMetadata(UNMANAGED_CONTAINER_NAME, ""); ContainerWrapper externalContainer = new ContainerWrapper(metadata, null); - globalContainers.put(EXTERNAL_CONTAINER_NAME, externalContainer); + globalContainers.put(UNMANAGED_CONTAINER_NAME, externalContainer); } public static void init(List containerList) { @@ -46,7 +46,7 @@ public static void init(List containerList) { isInitialized = true; } - public static ResourceContainer get(String name) { + public static InternalResourceContainer get(String name) { checkInitialized(); return Optional.ofNullable(globalContainers.get(name)) .map(ContainerWrapper::getContainer) @@ -70,7 +70,7 @@ public static boolean contains(String name) { } private static class ContainerWrapper { - private final ResourceContainer container; + private final InternalResourceContainer container; private final ContainerMetadata metadata; public ContainerWrapper(ContainerMetadata metadata) { @@ -78,12 +78,12 @@ public ContainerWrapper(ContainerMetadata metadata) { this.container = loadResourceContainer(metadata.getImplClass()); } - ContainerWrapper(ContainerMetadata metadata, ResourceContainer container) { + ContainerWrapper(ContainerMetadata metadata, InternalResourceContainer container) { this.metadata = metadata; this.container = container; } - public ResourceContainer getContainer() { + public InternalResourceContainer getContainer() { return container; } @@ -91,10 +91,11 @@ public ContainerMetadata getMetadata() { return metadata; } - private ResourceContainer loadResourceContainer(String implClass) { + private InternalResourceContainer loadResourceContainer(String implClass) { try { Class clazz = Class.forName(implClass); - ResourceContainer resourceContainer = (ResourceContainer) clazz.newInstance(); + InternalResourceContainer resourceContainer = + (InternalResourceContainer) clazz.newInstance(); resourceContainer.init(metadata.getName(), metadata.getProperties()); return resourceContainer; } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicExternalScheduler.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicExternalScheduler.java new file mode 100644 index 0000000000..b9ad510d38 --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicExternalScheduler.java @@ -0,0 +1,226 @@ +/* + * 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.scheduler; + +import org.apache.amoro.Action; +import org.apache.amoro.TableRuntime; +import org.apache.amoro.process.AmoroProcess; +import org.apache.amoro.process.ManagedProcess; +import org.apache.amoro.process.ProcessFactory; +import org.apache.amoro.process.ProcessStatus; +import org.apache.amoro.process.SimpleFuture; +import org.apache.amoro.process.TableProcess; +import org.apache.amoro.process.TableProcessState; +import org.apache.amoro.resource.ExternalResourceContainer; +import org.apache.amoro.resource.Resource; +import org.apache.amoro.resource.ResourceManager; +import org.apache.amoro.server.persistence.PersistentBase; +import org.apache.amoro.server.persistence.mapper.ProcessStateMapper; +import org.apache.amoro.server.table.DefaultTableRuntime; +import org.apache.amoro.server.table.TableService; + +import java.util.List; + +public abstract class PeriodicExternalScheduler extends PeriodicTableScheduler { + + private final ExternalResourceContainer resourceContainer; + private final ResourceManager resourceManager; + private final ProcessFactory processFactory; + + public PeriodicExternalScheduler( + ResourceManager resourceManager, + ExternalResourceContainer resourceContainer, + Action action, + TableService tableService, + int poolSize) { + super(action, tableService, poolSize); + this.resourceContainer = resourceContainer; + this.resourceManager = resourceManager; + this.processFactory = generateProcessFactory(); + } + + @Override + protected void initHandler(List tableRuntimeList) { + tableRuntimeList.forEach(tableRuntime -> tableRuntime.install(getAction(), processFactory)); + super.initHandler(tableRuntimeList); + } + + @Override + protected boolean enabled(DefaultTableRuntime tableRuntime) { + return tableRuntime.enabled(getAction()); + } + + @Override + protected void execute(DefaultTableRuntime tableRuntime) { + // Trigger a table process and check conflicts by table runtime + // Update process state after process completed, the callback must be register first + AmoroProcess process = tableRuntime.trigger(getAction()); + process.getCompleteFuture().whenCompleted(() -> persistTableProcess(process)); + ManagedProcess managedProcess = new ManagedTableProcess<>(process); + + // Submit the table process to resource manager, this is a sync operation + // update process completed and delete related resources + managedProcess.submit(); + + // Trace the table process by async framework so that process can be called back when completed + trace(process); + } + + protected int getMaxRetryNumber() { + return 1; + } + + protected abstract void trace(AmoroProcess process); + + protected ProcessFactory generateProcessFactory() { + return new ExternalProcessFactory(); + } + + protected void persistTableProcess(AmoroProcess process) { + TableProcessState state = process.getState(); + if (state.getStatus() == ProcessStatus.SUBMITTED) { + new PersistencyHelper().createProcessState(state); + } else if (state.getStatus() == ProcessStatus.RUNNING) { + new PersistencyHelper().updateProcessRunning(state); + } else if (state.getStatus() == ProcessStatus.SUCCESS) { + new PersistencyHelper().updateProcessCompleted(state); + } else if (state.getStatus() == ProcessStatus.FAILED) { + new PersistencyHelper().updateProcessFailed(state); + } + } + + private static class PersistencyHelper extends PersistentBase { + + void createProcessState(TableProcessState state) { + doAs(ProcessStateMapper.class, mapper -> mapper.createProcessState(state)); + } + + void updateProcessRunning(TableProcessState state) { + doAs(ProcessStateMapper.class, mapper -> mapper.updateProcessRunning(state)); + } + + void updateProcessCompleted(TableProcessState state) { + doAs(ProcessStateMapper.class, mapper -> mapper.updateProcessCompleted(state)); + } + + void updateProcessFailed(TableProcessState state) { + doAs(ProcessStateMapper.class, mapper -> mapper.updateProcessFailed(state)); + } + } + + private class ExternalTableProcess extends TableProcess { + + ExternalTableProcess(TableRuntime tableRuntime) { + super( + new TableProcessState( + PeriodicExternalScheduler.this.getAction(), tableRuntime.getTableIdentifier()), + tableRuntime); + } + + ExternalTableProcess(TableRuntime tableRuntime, TableProcessState state) { + super(state, tableRuntime); + } + + @Override + protected void closeInternal() {} + } + + private class ExternalProcessFactory implements ProcessFactory { + + @Override + public AmoroProcess create(TableRuntime tableRuntime, Action action) { + return new ExternalTableProcess(tableRuntime); + } + + @Override + public AmoroProcess recover( + TableRuntime tableRuntime, TableProcessState state) { + return new ExternalTableProcess(tableRuntime, state); + } + } + + protected class ManagedTableProcess implements ManagedProcess { + + private final AmoroProcess process; + + ManagedTableProcess(AmoroProcess process) { + this.process = process; + } + + @Override + public void submit() { + Resource resource = resourceContainer.submit(this); + if (resource == null) { + throw new IllegalStateException("Submit table process can not return null resource"); + } + persistTableProcess(this); + resourceManager.createResource(resource); + getCompleteFuture() + .whenCompleted( + () -> { + resourceManager.deleteResource(resource.getResourceId()); + if (getState().getStatus() == ProcessStatus.FAILED + && getState().getRetryNumber() < getMaxRetryNumber()) { + retry(); + } + }); + getState().setSubmitted(); + getSubmitFuture().complete(); + } + + @Override + public void complete() { + process.getState().setCompleted(); + process.getCompleteFuture().complete(); + } + + @Override + public void complete(String failedReason) { + process.getState().setCompleted(failedReason); + process.getCompleteFuture().complete(); + } + + @Override + public void retry() { + process.getState().addRetryNumber(); + submit(); + } + + @Override + public void kill() { + process.getState().setKilled(); + process.getCompleteFuture().complete(); + } + + @Override + public SimpleFuture getSubmitFuture() { + return process.getSubmitFuture(); + } + + @Override + public SimpleFuture getCompleteFuture() { + return process.getCompleteFuture(); + } + + @Override + public T getState() { + return process.getState(); + } + } +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BaseTableExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicTableScheduler.java similarity index 66% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BaseTableExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicTableScheduler.java index 39f54a1778..ee9a03d3a1 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BaseTableExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/PeriodicTableScheduler.java @@ -16,14 +16,16 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler; +import org.apache.amoro.Action; import org.apache.amoro.AmoroTable; +import org.apache.amoro.IcebergActions; import org.apache.amoro.ServerTableIdentifier; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.OptimizingStatus; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.RuntimeHandlerChain; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang3.StringUtils; @@ -39,18 +41,32 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -public abstract class BaseTableExecutor extends RuntimeHandlerChain { +public abstract class PeriodicTableScheduler extends RuntimeHandlerChain { protected final Logger logger = LoggerFactory.getLogger(getClass()); private static final long START_DELAY = 10 * 1000L; + protected final Set scheduledTables = + Collections.synchronizedSet(new HashSet<>()); + private final Action action; private final ScheduledExecutorService executor; private final TableService tableService; - private final Set scheduledTables = - Collections.synchronizedSet(new HashSet<>()); - protected BaseTableExecutor(TableService tableService, int poolSize) { + protected PeriodicTableScheduler(Action action, TableService tableService, int poolSize) { + this.action = action; + this.tableService = tableService; + this.executor = + Executors.newScheduledThreadPool( + poolSize, + new ThreadFactoryBuilder() + .setDaemon(false) + .setNameFormat("async-" + getThreadName() + "-%d") + .build()); + } + + protected PeriodicTableScheduler(TableService tableService, int poolSize) { + this.action = IcebergActions.SYSTEM; this.tableService = tableService; this.executor = Executors.newScheduledThreadPool( @@ -62,7 +78,7 @@ protected BaseTableExecutor(TableService tableService, int poolSize) { } @Override - protected void initHandler(List tableRuntimeList) { + protected void initHandler(List tableRuntimeList) { tableRuntimeList.stream() .filter(this::enabled) .forEach( @@ -76,7 +92,7 @@ protected void initHandler(List tableRuntimeList) { logger.info("Table executor {} initialized", getClass().getSimpleName()); } - private void executeTask(TableRuntime tableRuntime) { + private void executeTask(DefaultTableRuntime tableRuntime) { try { if (isExecutable(tableRuntime)) { execute(tableRuntime); @@ -87,7 +103,8 @@ private void executeTask(TableRuntime tableRuntime) { } } - protected final void scheduleIfNecessary(TableRuntime tableRuntime, long millisecondsTime) { + protected final void scheduleIfNecessary( + DefaultTableRuntime tableRuntime, long millisecondsTime) { if (isExecutable(tableRuntime)) { if (scheduledTables.add(tableRuntime.getTableIdentifier())) { executor.schedule(() -> executeTask(tableRuntime), millisecondsTime, TimeUnit.MILLISECONDS); @@ -95,37 +112,39 @@ protected final void scheduleIfNecessary(TableRuntime tableRuntime, long millise } } - protected abstract long getNextExecutingTime(TableRuntime tableRuntime); + protected abstract long getNextExecutingTime(DefaultTableRuntime tableRuntime); - protected abstract boolean enabled(TableRuntime tableRuntime); + protected abstract boolean enabled(DefaultTableRuntime tableRuntime); - protected abstract void execute(TableRuntime tableRuntime); + protected abstract void execute(DefaultTableRuntime tableRuntime); protected String getThreadName() { return String.join("-", StringUtils.splitByCharacterTypeCamelCase(getClass().getSimpleName())) .toLowerCase(Locale.ROOT); } - private boolean isExecutable(TableRuntime tableRuntime) { + private boolean isExecutable(DefaultTableRuntime tableRuntime) { return tableService.contains(tableRuntime.getTableIdentifier().getId()) && enabled(tableRuntime); } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { // DO nothing by default } @Override - public void handleTableRemoved(TableRuntime tableRuntime) { + public void handleTableRemoved(DefaultTableRuntime tableRuntime) { // DO nothing, handling would be canceled when calling executeTable } @Override - public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) {} + public void handleStatusChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) {} @Override - public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { + public void handleTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @@ -139,7 +158,11 @@ protected long getStartDelay() { return START_DELAY; } - protected AmoroTable loadTable(TableRuntime tableRuntime) { + protected AmoroTable loadTable(DefaultTableRuntime tableRuntime) { return tableService.loadTable(tableRuntime.getTableIdentifier()); } + + public Action getAction() { + return action; + } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BlockerExpiringExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/BlockerExpiringExecutor.java similarity index 79% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BlockerExpiringExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/BlockerExpiringExecutor.java index b7c0a9259b..93811676b4 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/BlockerExpiringExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/BlockerExpiringExecutor.java @@ -16,14 +16,15 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.server.persistence.PersistentBase; import org.apache.amoro.server.persistence.mapper.TableBlockerMapper; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; -public class BlockerExpiringExecutor extends BaseTableExecutor { +public class BlockerExpiringExecutor extends PeriodicTableScheduler { private final Persistency persistency = new Persistency(); @@ -34,17 +35,17 @@ public BlockerExpiringExecutor(TableService tableService) { } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return INTERVAL; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return true; } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { try { persistency.doExpiring(tableRuntime); } catch (Throwable t) { @@ -54,7 +55,7 @@ protected void execute(TableRuntime tableRuntime) { private static class Persistency extends PersistentBase { - public void doExpiring(TableRuntime tableRuntime) { + public void doExpiring(DefaultTableRuntime tableRuntime) { String catalog = tableRuntime.getTableIdentifier().getCatalog(); String database = tableRuntime.getTableIdentifier().getDatabase(); String table = tableRuntime.getTableIdentifier().getTableName(); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DanglingDeleteFilesCleaningExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DanglingDeleteFilesCleaningExecutor.java similarity index 78% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DanglingDeleteFilesCleaningExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DanglingDeleteFilesCleaningExecutor.java index d2156d6abe..4b53919ca7 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DanglingDeleteFilesCleaningExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DanglingDeleteFilesCleaningExecutor.java @@ -16,18 +16,19 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.maintainer.TableMaintainer; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Clean table dangling delete files */ -public class DanglingDeleteFilesCleaningExecutor extends BaseTableExecutor { +public class DanglingDeleteFilesCleaningExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(DanglingDeleteFilesCleaningExecutor.class); @@ -39,22 +40,23 @@ protected DanglingDeleteFilesCleaningExecutor(TableService tableService, int poo } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return INTERVAL; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().isDeleteDanglingDeleteFilesEnabled(); } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { try { LOG.info("{} start cleaning dangling delete files", tableRuntime.getTableIdentifier()); AmoroTable amoroTable = loadTable(tableRuntime); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DataExpiringExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DataExpiringExecutor.java similarity index 78% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DataExpiringExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DataExpiringExecutor.java index e5b9b593dd..59050694d7 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/DataExpiringExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/DataExpiringExecutor.java @@ -16,19 +16,20 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.maintainer.TableMaintainer; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; -public class DataExpiringExecutor extends BaseTableExecutor { +public class DataExpiringExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(DataExpiringExecutor.class); @@ -40,22 +41,23 @@ protected DataExpiringExecutor(TableService tableService, int poolSize, Duration } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return interval.toMillis(); } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().getExpiringDataConfig().isEnabled(); } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { try { AmoroTable amoroTable = loadTable(tableRuntime); TableMaintainer tableMaintainer = TableMaintainer.ofTable(amoroTable); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/HiveCommitSyncExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/HiveCommitSyncExecutor.java similarity index 84% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/HiveCommitSyncExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/HiveCommitSyncExecutor.java index 14c109d1d6..1ef52c8b8d 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/HiveCommitSyncExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/HiveCommitSyncExecutor.java @@ -16,19 +16,20 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.ServerTableIdentifier; import org.apache.amoro.hive.table.SupportHive; import org.apache.amoro.hive.utils.HiveMetaSynchronizer; import org.apache.amoro.hive.utils.TableTypeUtil; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.table.MixedTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class HiveCommitSyncExecutor extends BaseTableExecutor { +public class HiveCommitSyncExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(HiveCommitSyncExecutor.class); // 10 minutes @@ -39,17 +40,17 @@ public HiveCommitSyncExecutor(TableService tableService, int poolSize) { } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return INTERVAL; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return true; } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { long startTime = System.currentTimeMillis(); ServerTableIdentifier tableIdentifier = tableRuntime.getTableIdentifier(); try { diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/AsyncTableExecutors.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/InlineTableExecutors.java similarity index 96% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/AsyncTableExecutors.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/InlineTableExecutors.java index 560f2bb4b1..d51b02713a 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/AsyncTableExecutors.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/InlineTableExecutors.java @@ -16,15 +16,15 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.config.Configurations; import org.apache.amoro.server.AmoroManagementConf; import org.apache.amoro.server.table.TableService; -public class AsyncTableExecutors { +public class InlineTableExecutors { - private static final AsyncTableExecutors instance = new AsyncTableExecutors(); + private static final InlineTableExecutors instance = new InlineTableExecutors(); private SnapshotsExpiringExecutor snapshotsExpiringExecutor; private TableRuntimeRefreshExecutor tableRefreshingExecutor; private OrphanFilesCleaningExecutor orphanFilesCleaningExecutor; @@ -36,7 +36,7 @@ public class AsyncTableExecutors { private TagsAutoCreatingExecutor tagsAutoCreatingExecutor; private DataExpiringExecutor dataExpiringExecutor; - public static AsyncTableExecutors getInstance() { + public static InlineTableExecutors getInstance() { return instance; } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingCommitExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingCommitExecutor.java similarity index 65% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingCommitExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingCommitExecutor.java index b1d07594e5..f687da85c4 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingCommitExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingCommitExecutor.java @@ -16,16 +16,17 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.server.optimizing.OptimizingStatus; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import java.util.Optional; -public class OptimizingCommitExecutor extends BaseTableExecutor { +public class OptimizingCommitExecutor extends PeriodicTableScheduler { private static final long INTERVAL = 60 * 1000L; // 1min @@ -34,18 +35,18 @@ public OptimizingCommitExecutor(TableService tableService, int poolSize) { } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return INTERVAL; } @Override - protected boolean enabled(TableRuntime tableRuntime) { - return tableRuntime.getOptimizingStatus() == OptimizingStatus.COMMITTING; + protected boolean enabled(DefaultTableRuntime tableRuntime) { + return tableRuntime.getOptimizingState().getOptimizingStatus() == OptimizingStatus.COMMITTING; } @Override - protected void execute(TableRuntime tableRuntime) { - Optional.ofNullable(tableRuntime.getOptimizingProcess()) + protected void execute(DefaultTableRuntime tableRuntime) { + Optional.ofNullable(tableRuntime.getOptimizingState().getOptimizingProcess()) .orElseThrow( () -> new IllegalStateException( @@ -54,12 +55,13 @@ protected void execute(TableRuntime tableRuntime) { } @Override - public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { + public void handleStatusChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @Override - public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) {} + public void handleTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime) {} protected long getStartDelay() { return 0; diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingExpiringExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingExpiringExecutor.java similarity index 84% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingExpiringExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingExpiringExecutor.java index c7230a8316..e0f5c824e7 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OptimizingExpiringExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OptimizingExpiringExecutor.java @@ -16,16 +16,17 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.server.persistence.PersistentBase; import org.apache.amoro.server.persistence.mapper.OptimizingMapper; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class OptimizingExpiringExecutor extends BaseTableExecutor { +public class OptimizingExpiringExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(OptimizingExpiringExecutor.class); private final Persistency persistency = new Persistency(); @@ -39,17 +40,17 @@ public OptimizingExpiringExecutor(TableService tableService, int keepDays, int i } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return interval; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return true; } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { try { persistency.doExpiring(tableRuntime); } catch (Throwable throwable) { @@ -59,7 +60,7 @@ protected void execute(TableRuntime tableRuntime) { } private class Persistency extends PersistentBase { - public void doExpiring(TableRuntime tableRuntime) { + public void doExpiring(DefaultTableRuntime tableRuntime) { long expireTime = System.currentTimeMillis() - keepTime; doAsTransaction( () -> diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OrphanFilesCleaningExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OrphanFilesCleaningExecutor.java similarity index 78% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OrphanFilesCleaningExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OrphanFilesCleaningExecutor.java index 62d7abc624..7d694b2297 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/OrphanFilesCleaningExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/OrphanFilesCleaningExecutor.java @@ -16,19 +16,20 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.maintainer.TableMaintainer; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; -public class OrphanFilesCleaningExecutor extends BaseTableExecutor { +public class OrphanFilesCleaningExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(OrphanFilesCleaningExecutor.class); private final Duration interval; @@ -38,22 +39,23 @@ public OrphanFilesCleaningExecutor(TableService tableService, int poolSize, Dura } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return interval.toMillis(); } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().isCleanOrphanEnabled(); } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @Override - public void execute(TableRuntime tableRuntime) { + public void execute(DefaultTableRuntime tableRuntime) { try { LOG.info("{} start cleaning orphan files", tableRuntime.getTableIdentifier()); AmoroTable amoroTable = loadTable(tableRuntime); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/SnapshotsExpiringExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/SnapshotsExpiringExecutor.java similarity index 77% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/SnapshotsExpiringExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/SnapshotsExpiringExecutor.java index ea39274d04..b4d9da1e55 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/SnapshotsExpiringExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/SnapshotsExpiringExecutor.java @@ -16,18 +16,19 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.maintainer.TableMaintainer; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Service for expiring tables periodically. */ -public class SnapshotsExpiringExecutor extends BaseTableExecutor { +public class SnapshotsExpiringExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(SnapshotsExpiringExecutor.class); private static final long INTERVAL = 60 * 60 * 1000L; // 1 hour @@ -37,22 +38,23 @@ public SnapshotsExpiringExecutor(TableService tableService, int poolSize) { } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return INTERVAL; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().isExpireSnapshotEnabled(); } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @Override - public void execute(TableRuntime tableRuntime) { + public void execute(DefaultTableRuntime tableRuntime) { try { AmoroTable amoroTable = loadTable(tableRuntime); TableMaintainer tableMaintainer = TableMaintainer.ofTable(amoroTable); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TableRuntimeRefreshExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TableRuntimeRefreshExecutor.java similarity index 61% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TableRuntimeRefreshExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TableRuntimeRefreshExecutor.java index 85cd5b5f90..374e52b544 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TableRuntimeRefreshExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TableRuntimeRefreshExecutor.java @@ -16,20 +16,22 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.optimizing.plan.AbstractOptimizingEvaluator; import org.apache.amoro.process.ProcessStatus; import org.apache.amoro.server.optimizing.OptimizingProcess; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.table.MixedTable; /** Executor that refreshes table runtimes and evaluates optimizing status periodically. */ -public class TableRuntimeRefreshExecutor extends BaseTableExecutor { +public class TableRuntimeRefreshExecutor extends PeriodicTableScheduler { // 1 minutes private final long interval; @@ -43,16 +45,20 @@ public TableRuntimeRefreshExecutor( } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return true; } - protected long getNextExecutingTime(TableRuntime tableRuntime) { - return Math.min(tableRuntime.getOptimizingConfig().getMinorLeastInterval() * 4L / 5, interval); + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { + return Math.min( + tableRuntime.getOptimizingState().getOptimizingConfig().getMinorLeastInterval() * 4L / 5, + interval); } - private void tryEvaluatingPendingInput(TableRuntime tableRuntime, MixedTable table) { - if (tableRuntime.isOptimizingEnabled() && !tableRuntime.getOptimizingStatus().isProcessing()) { + private void tryEvaluatingPendingInput(DefaultTableRuntime tableRuntime, MixedTable table) { + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + if (optimizingState.isOptimizingEnabled() + && !optimizingState.getOptimizingStatus().isProcessing()) { AbstractOptimizingEvaluator evaluator = IcebergTableUtil.createOptimizingEvaluator(tableRuntime, table, maxPendingPartitions); if (evaluator.isNecessary()) { @@ -62,20 +68,22 @@ private void tryEvaluatingPendingInput(TableRuntime tableRuntime, MixedTable tab "{} optimizing is necessary and get pending input {}", tableRuntime.getTableIdentifier(), pendingInput); - tableRuntime.setPendingInput(pendingInput); + optimizingState.setPendingInput(pendingInput); } else { - tableRuntime.optimizingNotNecessary(); + optimizingState.optimizingNotNecessary(); } - tableRuntime.setTableSummary(evaluator.getPendingInput()); + optimizingState.setTableSummary(evaluator.getPendingInput()); } } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { // After disabling self-optimizing, close the currently running optimizing process. if (originalConfig.getOptimizingConfig().isEnabled() && !tableRuntime.getTableConfiguration().getOptimizingConfig().isEnabled()) { - OptimizingProcess optimizingProcess = tableRuntime.getOptimizingProcess(); + OptimizingProcess optimizingProcess = + tableRuntime.getOptimizingState().getOptimizingProcess(); if (optimizingProcess != null && optimizingProcess.getStatus() == ProcessStatus.RUNNING) { optimizingProcess.close(); } @@ -83,18 +91,19 @@ public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration or } @Override - public void execute(TableRuntime tableRuntime) { + public void execute(DefaultTableRuntime tableRuntime) { try { - long lastOptimizedSnapshotId = tableRuntime.getLastOptimizedSnapshotId(); - long lastOptimizedChangeSnapshotId = tableRuntime.getLastOptimizedChangeSnapshotId(); + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + long lastOptimizedSnapshotId = optimizingState.getLastOptimizedSnapshotId(); + long lastOptimizedChangeSnapshotId = optimizingState.getLastOptimizedChangeSnapshotId(); AmoroTable table = loadTable(tableRuntime); - tableRuntime.refresh(table); + optimizingState.refresh(table); MixedTable mixedTable = (MixedTable) table.originalTable(); if ((mixedTable.isKeyedTable() - && (lastOptimizedSnapshotId != tableRuntime.getCurrentSnapshotId() - || lastOptimizedChangeSnapshotId != tableRuntime.getCurrentChangeSnapshotId())) + && (lastOptimizedSnapshotId != optimizingState.getCurrentSnapshotId() + || lastOptimizedChangeSnapshotId != optimizingState.getCurrentChangeSnapshotId())) || (mixedTable.isUnkeyedTable() - && lastOptimizedSnapshotId != tableRuntime.getCurrentSnapshotId())) { + && lastOptimizedSnapshotId != optimizingState.getCurrentSnapshotId())) { tryEvaluatingPendingInput(tableRuntime, mixedTable); } } catch (Throwable throwable) { diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TagsAutoCreatingExecutor.java b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TagsAutoCreatingExecutor.java similarity index 78% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TagsAutoCreatingExecutor.java rename to amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TagsAutoCreatingExecutor.java index c549b93619..144aa567b0 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/executor/TagsAutoCreatingExecutor.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TagsAutoCreatingExecutor.java @@ -16,19 +16,20 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.AmoroTable; import org.apache.amoro.TableFormat; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.server.optimizing.maintainer.TableMaintainer; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.scheduler.PeriodicTableScheduler; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Service for automatically creating tags for table periodically. */ -public class TagsAutoCreatingExecutor extends BaseTableExecutor { +public class TagsAutoCreatingExecutor extends PeriodicTableScheduler { private static final Logger LOG = LoggerFactory.getLogger(TagsAutoCreatingExecutor.class); private final long interval; @@ -39,18 +40,18 @@ protected TagsAutoCreatingExecutor(TableService tableService, int poolSize, long } @Override - protected long getNextExecutingTime(TableRuntime tableRuntime) { + protected long getNextExecutingTime(DefaultTableRuntime tableRuntime) { return interval; } @Override - protected boolean enabled(TableRuntime tableRuntime) { + protected boolean enabled(DefaultTableRuntime tableRuntime) { return tableRuntime.getTableConfiguration().getTagConfiguration().isAutoCreateTag() && tableRuntime.getFormat() == TableFormat.ICEBERG; } @Override - protected void execute(TableRuntime tableRuntime) { + protected void execute(DefaultTableRuntime tableRuntime) { try { AmoroTable amoroTable = loadTable(tableRuntime); TableMaintainer tableMaintainer = TableMaintainer.ofTable(amoroTable); @@ -61,7 +62,8 @@ protected void execute(TableRuntime tableRuntime) { } @Override - public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleConfigChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { scheduleIfNecessary(tableRuntime, getStartDelay()); } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntime.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultOptimizingState.java similarity index 91% rename from amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntime.java rename to amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultOptimizingState.java index 2bd5ae30cc..12c3106ae3 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntime.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultOptimizingState.java @@ -18,16 +18,19 @@ package org.apache.amoro.server.table; +import org.apache.amoro.Action; import org.apache.amoro.AmoroTable; +import org.apache.amoro.IcebergActions; import org.apache.amoro.ServerTableIdentifier; import org.apache.amoro.StateField; -import org.apache.amoro.TableFormat; import org.apache.amoro.api.BlockableOperation; import org.apache.amoro.config.OptimizingConfig; import org.apache.amoro.config.TableConfiguration; import org.apache.amoro.iceberg.Constants; import org.apache.amoro.optimizing.OptimizingType; import org.apache.amoro.optimizing.plan.AbstractOptimizingEvaluator; +import org.apache.amoro.process.ProcessState; +import org.apache.amoro.process.ProcessStatus; import org.apache.amoro.server.AmoroServiceConstants; import org.apache.amoro.server.metrics.MetricRegistry; import org.apache.amoro.server.optimizing.OptimizingProcess; @@ -60,23 +63,21 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -public class TableRuntime extends StatedPersistentBase { +public class DefaultOptimizingState extends StatedPersistentBase implements ProcessState { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultOptimizingState.class); - private static final Logger LOG = LoggerFactory.getLogger(TableRuntime.class); private final Lock tableLock = new ReentrantLock(); private final TableRuntimeHandler tableHandler; private final ServerTableIdentifier tableIdentifier; - private final List taskQuotas = new CopyOnWriteArrayList<>(); - // for unKeyedTable or base table @StateField private volatile long currentSnapshotId = Constants.INVALID_SNAPSHOT_ID; + private final List taskQuotas = new CopyOnWriteArrayList<>(); @StateField private volatile long lastOptimizedSnapshotId = Constants.INVALID_SNAPSHOT_ID; - @StateField private volatile long lastOptimizedChangeSnapshotId = Constants.INVALID_SNAPSHOT_ID; // for change table @StateField private volatile long currentChangeSnapshotId = Constants.INVALID_SNAPSHOT_ID; - @StateField private volatile OptimizingStatus optimizingStatus = OptimizingStatus.IDLE; @StateField private volatile long currentStatusStartTime = System.currentTimeMillis(); @StateField private volatile long lastMajorOptimizingTime; @@ -97,15 +98,17 @@ public class TableRuntime extends StatedPersistentBase { private Map fromSequence; private Map toSequence; private OptimizingType optimizingType; + private DefaultTableRuntime tableRuntime; - public TableRuntime( - ServerTableIdentifier tableIdentifier, + public DefaultOptimizingState( + DefaultTableRuntime tableRuntime, TableRuntimeHandler tableHandler, Map properties) { - Preconditions.checkNotNull(tableIdentifier, "ServerTableIdentifier must not be null."); + Preconditions.checkNotNull(tableRuntime, "ServerTableIdentifier must not be null."); Preconditions.checkNotNull(tableHandler, "TableRuntimeHandler must not be null."); + this.tableRuntime = tableRuntime; this.tableHandler = tableHandler; - this.tableIdentifier = tableIdentifier; + this.tableIdentifier = tableRuntime.getTableIdentifier(); this.tableConfiguration = TableConfigurations.parseTableConfig(properties); this.optimizerGroup = tableConfiguration.getOptimizingConfig().getOptimizerGroup(); persistTableRuntime(); @@ -114,10 +117,14 @@ public TableRuntime( tableSummaryMetrics = new TableSummaryMetrics(tableIdentifier); } - public TableRuntime(TableRuntimeMeta tableRuntimeMeta, TableRuntimeHandler tableHandler) { + public DefaultOptimizingState( + DefaultTableRuntime tableRuntime, + TableRuntimeMeta tableRuntimeMeta, + TableRuntimeHandler tableHandler) { Preconditions.checkNotNull(tableRuntimeMeta, "TableRuntimeMeta must not be null."); Preconditions.checkNotNull(tableHandler, "TableRuntimeHandler must not be null."); + this.tableRuntime = tableRuntime; this.tableHandler = tableHandler; this.tableIdentifier = ServerTableIdentifier.of( @@ -196,7 +203,7 @@ public void beginPlanning() { OptimizingStatus originalStatus = optimizingStatus; updateOptimizingStatus(OptimizingStatus.PLANNING); persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); }); } @@ -207,14 +214,14 @@ public void planFailed() { OptimizingStatus originalStatus = optimizingStatus; updateOptimizingStatus(OptimizingStatus.PENDING); persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); }); } catch (Exception e) { OptimizingStatus originalStatus = optimizingStatus; updateOptimizingStatus(OptimizingStatus.PENDING); LOG.warn( "Persistent database failed, only the optimizing state in the memory was changed.", e); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); } } @@ -228,7 +235,7 @@ public void beginProcess(OptimizingProcess optimizingProcess) { OptimizingStatus.ofOptimizingType(optimizingProcess.getOptimizingType())); this.pendingInput = null; persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); }); } @@ -238,7 +245,7 @@ public void beginCommitting() { OptimizingStatus originalStatus = optimizingStatus; updateOptimizingStatus(OptimizingStatus.COMMITTING); persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); }); } @@ -253,12 +260,12 @@ public void setPendingInput(AbstractOptimizingEvaluator.PendingInput pendingInpu "{} status changed from idle to pending with pendingInput {}", tableIdentifier, pendingInput); - tableHandler.handleTableChanged(this, OptimizingStatus.IDLE); + tableHandler.handleTableChanged(tableRuntime, OptimizingStatus.IDLE); } }); } - public TableRuntime refresh(AmoroTable table) { + public DefaultTableRuntime refresh(AmoroTable table) { return invokeConsistency( () -> { TableConfiguration configuration = tableConfiguration; @@ -267,9 +274,9 @@ public TableRuntime refresh(AmoroTable table) { persistUpdatingRuntime(); } if (configChanged) { - tableHandler.handleTableChanged(this, configuration); + tableHandler.handleTableChanged(tableRuntime, configuration); } - return this; + return tableRuntime; }); } @@ -296,7 +303,7 @@ public void completeEmptyProcess() { lastOptimizedSnapshotId = currentSnapshotId; lastOptimizedChangeSnapshotId = currentChangeSnapshotId; persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, optimizingStatus); + tableHandler.handleTableChanged(tableRuntime, optimizingStatus); } }); } @@ -350,7 +357,7 @@ public void completeProcess(boolean success) { updateOptimizingStatus(OptimizingStatus.IDLE); optimizingProcess = null; persistUpdatingRuntime(); - tableHandler.handleTableChanged(this, originalStatus); + tableHandler.handleTableChanged(tableRuntime, originalStatus); }); } @@ -466,10 +473,6 @@ public ServerTableIdentifier getTableIdentifier() { return tableIdentifier; } - public TableFormat getFormat() { - return tableIdentifier.getFormat(); - } - public OptimizingStatus getOptimizingStatus() { return optimizingStatus; } @@ -657,4 +660,44 @@ protected void invokeConsistency(Runnable runnable) { tableLock.unlock(); } } + + @Override + public long getId() { + return optimizingProcess.getProcessId(); + } + + @Override + public String getName() { + return optimizingType.name(); + } + + @Override + public long getStartTime() { + return optimizingProcess.getPlanTime(); + } + + @Override + public Action getAction() { + return IcebergActions.REWRITE; + } + + @Override + public ProcessStatus getStatus() { + return optimizingProcess.getStatus(); + } + + @Override + public Map getSummary() { + return null; + } + + @Override + public String getFailedReason() { + return null; + } + + @Override + public long getQuotaRuntime() { + return getQuotaTime(); + } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java new file mode 100644 index 0000000000..5ab6b02073 --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java @@ -0,0 +1,164 @@ +/* + * 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.Action; +import org.apache.amoro.ServerTableIdentifier; +import org.apache.amoro.SupportsProcessPlugins; +import org.apache.amoro.TableRuntime; +import org.apache.amoro.config.TableConfiguration; +import org.apache.amoro.process.AmoroProcess; +import org.apache.amoro.process.ProcessFactory; +import org.apache.amoro.process.TableProcessState; +import org.apache.amoro.server.metrics.MetricRegistry; +import org.apache.amoro.server.optimizing.OptimizingProcess; +import org.apache.amoro.server.persistence.StatedPersistentBase; +import org.apache.amoro.server.persistence.TableRuntimeMeta; +import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.shaded.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +public class DefaultTableRuntime extends StatedPersistentBase + implements TableRuntime, SupportsProcessPlugins { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultTableRuntime.class); + + private final ServerTableIdentifier tableIdentifier; + private final DefaultOptimizingState optimizingState; + private final Map processContainerMap = Maps.newConcurrentMap(); + + public DefaultTableRuntime( + ServerTableIdentifier tableIdentifier, + TableRuntimeHandler tableHandler, + Map properties) { + Preconditions.checkNotNull(tableIdentifier, "ServerTableIdentifier must not be null."); + Preconditions.checkNotNull(tableHandler, "TableRuntimeHandler must not be null."); + this.tableIdentifier = tableIdentifier; + this.optimizingState = new DefaultOptimizingState(this, tableHandler, properties); + } + + public DefaultTableRuntime(TableRuntimeMeta tableRuntimeMeta, TableRuntimeHandler tableHandler) { + Preconditions.checkNotNull(tableRuntimeMeta, "TableRuntimeMeta must not be null."); + Preconditions.checkNotNull(tableHandler, "TableRuntimeHandler must not be null."); + + this.tableIdentifier = + ServerTableIdentifier.of( + tableRuntimeMeta.getTableId(), + tableRuntimeMeta.getCatalogName(), + tableRuntimeMeta.getDbName(), + tableRuntimeMeta.getTableName(), + tableRuntimeMeta.getFormat()); + this.optimizingState = new DefaultOptimizingState(this, tableRuntimeMeta, tableHandler); + } + + public DefaultOptimizingState getOptimizingState() { + return optimizingState; + } + + public void recover(OptimizingProcess optimizingProcess) { + optimizingState.recover(optimizingProcess); + } + + public void registerMetric(MetricRegistry metricRegistry) { + optimizingState.registerMetric(metricRegistry); + } + + public void dispose() { + optimizingState.dispose(); + } + + @Override + public AmoroProcess trigger(Action action) { + return Optional.ofNullable(processContainerMap.get(action)) + .map(container -> container.trigger(action)) + // Define a related exception + .orElseThrow(() -> new IllegalArgumentException("No ProcessFactory for action " + action)); + } + + @Override + public void install(Action action, ProcessFactory processFactory) { + if (processContainerMap.putIfAbsent(action, new TableProcessContainer(processFactory)) + != null) { + throw new IllegalStateException("ProcessFactory for action " + action + " already exists"); + } + } + + @Override + public boolean enabled(Action action) { + return processContainerMap.get(action) != null; + } + + @Override + public List getProcessStates() { + return processContainerMap.values().stream() + .flatMap(container -> container.getProcessStates().stream()) + .collect(Collectors.toList()); + } + + @Override + public List getProcessStates(Action action) { + return processContainerMap.get(action).getProcessStates(); + } + + @Override + public ServerTableIdentifier getTableIdentifier() { + return tableIdentifier; + } + + @Override + public TableConfiguration getTableConfiguration() { + return optimizingState.getTableConfiguration(); + } + + private class TableProcessContainer { + private final Lock processLock = new ReentrantLock(); + private final ProcessFactory processFactory; + private final Map> processMap = + Maps.newConcurrentMap(); + + TableProcessContainer(ProcessFactory processFactory) { + this.processFactory = processFactory; + } + + public AmoroProcess trigger(Action action) { + processLock.lock(); + try { + AmoroProcess process = + processFactory.create(DefaultTableRuntime.this, action); + process.getCompleteFuture().whenCompleted(() -> processMap.remove(process.getId())); + processMap.put(process.getId(), process); + return process; + } finally { + processLock.unlock(); + } + } + + public List getProcessStates() { + return processMap.values().stream().map(AmoroProcess::getState).collect(Collectors.toList()); + } + } +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableService.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableService.java index 7dbe0dd596..519570c98c 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableService.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableService.java @@ -70,7 +70,7 @@ public class DefaultTableService extends PersistentBase implements TableService public static final Logger LOG = LoggerFactory.getLogger(DefaultTableService.class); private final long externalCatalogRefreshingInterval; - private final Map tableRuntimeMap = new ConcurrentHashMap<>(); + private final Map tableRuntimeMap = new ConcurrentHashMap<>(); private final ScheduledExecutorService tableExplorerScheduler = Executors.newSingleThreadScheduledExecutor( @@ -128,14 +128,16 @@ public void addHandlerChain(RuntimeHandlerChain handler) { } @Override - public void handleTableChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { + public void handleTableChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) { if (headHandler != null) { headHandler.fireStatusChanged(tableRuntime, originalStatus); } } @Override - public void handleTableChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + public void handleTableChanged( + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { if (headHandler != null) { headHandler.fireConfigChanged(tableRuntime, originalConfig); } @@ -147,10 +149,10 @@ public void initialize() { List tableRuntimeMetaList = getAs(TableMetaMapper.class, TableMetaMapper::selectTableRuntimeMetas); - List tableRuntimes = new ArrayList<>(tableRuntimeMetaList.size()); + List tableRuntimes = new ArrayList<>(tableRuntimeMetaList.size()); tableRuntimeMetaList.forEach( tableRuntimeMeta -> { - TableRuntime tableRuntime = new TableRuntime(tableRuntimeMeta, this); + DefaultTableRuntime tableRuntime = new DefaultTableRuntime(tableRuntimeMeta, this); tableRuntimeMap.put(tableRuntimeMeta.getTableId(), tableRuntime); tableRuntime.registerMetric(MetricManager.getInstance().getGlobalRegistry()); tableRuntimes.add(tableRuntime); @@ -182,9 +184,9 @@ public void initialize() { initialized.complete(true); } - private TableRuntime getAndCheckExist(ServerTableIdentifier tableIdentifier) { + private DefaultTableRuntime getAndCheckExist(ServerTableIdentifier tableIdentifier) { Preconditions.checkArgument(tableIdentifier != null, "tableIdentifier cannot be null"); - TableRuntime tableRuntime = getRuntime(tableIdentifier.getId()); + DefaultTableRuntime tableRuntime = getRuntime(tableIdentifier.getId()); if (tableRuntime == null) { throw new ObjectNotExistsException(tableIdentifier); } @@ -192,7 +194,7 @@ private TableRuntime getAndCheckExist(ServerTableIdentifier tableIdentifier) { } @Override - public TableRuntime getRuntime(Long tableId) { + public DefaultTableRuntime getRuntime(Long tableId) { checkStarted(); return tableRuntimeMap.get(tableId); } @@ -253,7 +255,7 @@ void exploreTableRuntimes() { catalogManager.listCatalogMetas().stream() .map(CatalogMeta::getCatalogName) .collect(Collectors.toSet()); - for (TableRuntime tableRuntime : tableRuntimeMap.values()) { + for (DefaultTableRuntime tableRuntime : tableRuntimeMap.values()) { if (!catalogNames.contains(tableRuntime.getTableIdentifier().getCatalog())) { disposeTable(tableRuntime.getTableIdentifier()); } @@ -446,7 +448,8 @@ private boolean triggerTableAdded( return false; } } - TableRuntime tableRuntime = new TableRuntime(serverTableIdentifier, this, table.properties()); + DefaultTableRuntime tableRuntime = + new DefaultTableRuntime(serverTableIdentifier, this, table.properties()); tableRuntimeMap.put(serverTableIdentifier.getId(), tableRuntime); tableRuntime.registerMetric(MetricManager.getInstance().getGlobalRegistry()); if (headHandler != null) { @@ -466,7 +469,7 @@ private void revertTableRuntimeAdded( } private void disposeTable(ServerTableIdentifier tableIdentifier) { - TableRuntime existedTableRuntime = tableRuntimeMap.get(tableIdentifier.getId()); + DefaultTableRuntime existedTableRuntime = tableRuntimeMap.get(tableIdentifier.getId()); try { doAsTransaction( () -> diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/RuntimeHandlerChain.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/RuntimeHandlerChain.java index 7554eca0ae..0ab56c4a2b 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/RuntimeHandlerChain.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/RuntimeHandlerChain.java @@ -51,8 +51,8 @@ protected void appendNext(RuntimeHandlerChain handler) { } } - public final void initialize(List tableRuntimes) { - List supportedtableRuntimeList = + public final void initialize(List tableRuntimes) { + List supportedtableRuntimeList = tableRuntimes.stream() .filter(runtime -> formatSupported(runtime.getFormat())) .collect(Collectors.toList()); @@ -63,7 +63,8 @@ public final void initialize(List tableRuntimes) { } } - public final void fireStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { + public final void fireStatusChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) { if (!initialized) { return; } @@ -76,7 +77,7 @@ public final void fireStatusChanged(TableRuntime tableRuntime, OptimizingStatus } public final void fireConfigChanged( - TableRuntime tableRuntime, TableConfiguration originalConfig) { + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { if (!initialized) { return; } @@ -89,7 +90,7 @@ public final void fireConfigChanged( } } - public final void fireTableAdded(AmoroTable table, TableRuntime tableRuntime) { + public final void fireTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime) { if (!initialized) { return; } @@ -102,7 +103,7 @@ public final void fireTableAdded(AmoroTable table, TableRuntime tableRuntime) } } - public final void fireTableRemoved(TableRuntime tableRuntime) { + public final void fireTableRemoved(DefaultTableRuntime tableRuntime) { if (!initialized) { return; } @@ -137,16 +138,16 @@ protected boolean formatSupported(TableFormat format) { } protected abstract void handleStatusChanged( - TableRuntime tableRuntime, OptimizingStatus originalStatus); + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus); protected abstract void handleConfigChanged( - TableRuntime tableRuntime, TableConfiguration originalConfig); + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig); - protected abstract void handleTableAdded(AmoroTable table, TableRuntime tableRuntime); + protected abstract void handleTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime); - protected abstract void handleTableRemoved(TableRuntime tableRuntime); + protected abstract void handleTableRemoved(DefaultTableRuntime tableRuntime); - protected abstract void initHandler(List tableRuntimeList); + protected abstract void initHandler(List tableRuntimeList); protected abstract void doDispose(); } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntimeHandler.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntimeHandler.java index 32c58923fc..45a032a911 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntimeHandler.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableRuntimeHandler.java @@ -25,7 +25,7 @@ public interface TableRuntimeHandler { void addHandlerChain(RuntimeHandlerChain handler); - void handleTableChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus); + void handleTableChanged(DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus); - void handleTableChanged(TableRuntime tableRuntime, TableConfiguration originalConfig); + void handleTableChanged(DefaultTableRuntime tableRuntime, TableConfiguration originalConfig); } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableService.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableService.java index c1a265800d..68c1ff8f7f 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableService.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableService.java @@ -32,7 +32,7 @@ public interface TableService extends TableRuntimeHandler { void onTableDropped(InternalCatalog catalog, ServerTableIdentifier identifier); - TableRuntime getRuntime(Long tableId); + DefaultTableRuntime getRuntime(Long tableId); default boolean contains(Long tableId) { return getRuntime(tableId) != null; diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/utils/IcebergTableUtil.java b/amoro-ams/src/main/java/org/apache/amoro/server/utils/IcebergTableUtil.java index 0df0637254..2393b29496 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/utils/IcebergTableUtil.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/utils/IcebergTableUtil.java @@ -33,7 +33,8 @@ import org.apache.amoro.optimizing.plan.MixedIcebergOptimizingEvaluator; import org.apache.amoro.optimizing.plan.MixedIcebergOptimizingPlanner; import org.apache.amoro.scan.TableEntriesScan; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import org.apache.amoro.shade.guava32.com.google.common.base.Predicate; import org.apache.amoro.shade.guava32.com.google.common.collect.Iterables; @@ -88,12 +89,13 @@ public static long getSnapshotId(Table table, boolean refresh) { } } - public static TableSnapshot getSnapshot(MixedTable mixedTable, TableRuntime tableRuntime) { + public static TableSnapshot getSnapshot(MixedTable mixedTable, DefaultTableRuntime tableRuntime) { if (mixedTable.isUnkeyedTable()) { - return new BasicTableSnapshot(tableRuntime.getCurrentSnapshotId()); + return new BasicTableSnapshot(tableRuntime.getOptimizingState().getCurrentSnapshotId()); } else { return new KeyedTableSnapshot( - tableRuntime.getCurrentSnapshotId(), tableRuntime.getCurrentChangeSnapshotId()); + tableRuntime.getOptimizingState().getCurrentSnapshotId(), + tableRuntime.getOptimizingState().getCurrentChangeSnapshotId()); } } @@ -219,14 +221,15 @@ public static Set getAllManifestFiles(Table table) { } public static AbstractOptimizingEvaluator createOptimizingEvaluator( - TableRuntime tableRuntime, + DefaultTableRuntime tableRuntime, MixedTable table, TableSnapshot snapshot, int maxPendingPartitions) { ServerTableIdentifier identifier = tableRuntime.getTableIdentifier(); - OptimizingConfig config = tableRuntime.getOptimizingConfig(); - long lastMinor = tableRuntime.getLastMinorOptimizingTime(); - long lastFull = tableRuntime.getLastFullOptimizingTime(); + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); + OptimizingConfig config = optimizingState.getOptimizingConfig(); + long lastMinor = optimizingState.getLastMinorOptimizingTime(); + long lastFull = optimizingState.getLastFullOptimizingTime(); if (TableFormat.ICEBERG.equals(table.format())) { return new IcebergOptimizerEvaluator( identifier, config, table, snapshot, maxPendingPartitions, lastMinor, lastFull); @@ -241,20 +244,22 @@ public static AbstractOptimizingEvaluator createOptimizingEvaluator( } public static AbstractOptimizingEvaluator createOptimizingEvaluator( - TableRuntime tableRuntime, MixedTable table, int maxPendingPartitions) { + DefaultTableRuntime tableRuntime, MixedTable table, int maxPendingPartitions) { TableSnapshot snapshot = IcebergTableUtil.getSnapshot(table, tableRuntime); return createOptimizingEvaluator(tableRuntime, table, snapshot, maxPendingPartitions); } public static AbstractOptimizingPlanner createOptimizingPlanner( - TableRuntime tableRuntime, + DefaultTableRuntime tableRuntime, MixedTable table, double availableCore, long maxInputSizePerThread) { + DefaultOptimizingState optimizingState = tableRuntime.getOptimizingState(); Expression partitionFilter = - tableRuntime.getPendingInput() == null + optimizingState.getPendingInput() == null ? Expressions.alwaysTrue() - : tableRuntime.getPendingInput().getPartitions().entrySet().stream() + : tableRuntime.getOptimizingState().getPendingInput().getPartitions().entrySet() + .stream() .map( entry -> ExpressionUtil.convertPartitionDataToDataFilter( @@ -262,11 +267,11 @@ public static AbstractOptimizingPlanner createOptimizingPlanner( .reduce(Expressions::or) .orElse(Expressions.alwaysTrue()); long planTime = System.currentTimeMillis(); - long processId = Math.max(tableRuntime.getNewestProcessId() + 1, planTime); + long processId = Math.max(optimizingState.getNewestProcessId() + 1, planTime); ServerTableIdentifier identifier = tableRuntime.getTableIdentifier(); - OptimizingConfig config = tableRuntime.getOptimizingConfig(); - long lastMinor = tableRuntime.getLastMinorOptimizingTime(); - long lastFull = tableRuntime.getLastFullOptimizingTime(); + OptimizingConfig config = optimizingState.getOptimizingConfig(); + long lastMinor = optimizingState.getLastMinorOptimizingTime(); + long lastFull = optimizingState.getLastFullOptimizingTime(); TableSnapshot snapshot = IcebergTableUtil.getSnapshot(table, tableRuntime); if (TableFormat.ICEBERG.equals(table.format())) { return new IcebergOptimizingPlanner( diff --git a/amoro-ams/src/main/resources/mysql/ams-mysql-init.sql b/amoro-ams/src/main/resources/mysql/ams-mysql-init.sql index b72fa61013..b3322d2e2b 100644 --- a/amoro-ams/src/main/resources/mysql/ams-mysql-init.sql +++ b/amoro-ams/src/main/resources/mysql/ams-mysql-init.sql @@ -179,6 +179,21 @@ CREATE TABLE `task_runtime` KEY `table_index` (`table_id`, `process_id`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COMMENT 'Optimize task basic information'; +CREATE TABLE `table_process_state` +( + `process_id` bigint(20) NOT NULL COMMENT 'optimizing_procedure UUID', + `action` varchar(16) NOT NULL COMMENT 'process action', + `table_id` bigint(20) NOT NULL, + `retry_num` int(11) DEFAULT NULL COMMENT 'Retry times', + `status` varchar(10) NOT NULL COMMENT 'Direct to TableOptimizingStatus', + `start_time` timestamp DEFAULT CURRENT_TIMESTAMP COMMENT 'First plan time', + `end_time` timestamp NULL DEFAULT NULL COMMENT 'finish time or failed time', + `fail_reason` varchar(4096) DEFAULT NULL COMMENT 'Error message after task failed', + `summary` mediumtext COMMENT 'state summary, usually a map', + PRIMARY KEY (`process_id`), + KEY `table_index` (`table_id`, `plan_time`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COMMENT 'History of optimizing after each commit'; + CREATE TABLE `optimizing_task_quota` ( `process_id` bigint(20) NOT NULL COMMENT 'Optimize type: Major, Minor, FullMajor', diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/AmsEnvironment.java b/amoro-ams/src/test/java/org/apache/amoro/server/AmsEnvironment.java index 1e37a071c9..f20e6d1d55 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/AmsEnvironment.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/AmsEnvironment.java @@ -31,8 +31,8 @@ import org.apache.amoro.resource.ResourceGroup; import org.apache.amoro.server.catalog.DefaultCatalogManager; import org.apache.amoro.server.catalog.ServerCatalog; +import org.apache.amoro.server.resource.InternalContainers; import org.apache.amoro.server.resource.OptimizerManager; -import org.apache.amoro.server.resource.ResourceContainers; import org.apache.amoro.server.table.DefaultTableService; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; import org.apache.amoro.shade.guava32.com.google.common.io.MoreFiles; @@ -310,7 +310,7 @@ public void stopOptimizer() { .listOptimizers() .forEach( resource -> { - ResourceContainers.get(resource.getContainerName()).releaseOptimizer(resource); + InternalContainers.get(resource.getContainerName()).releaseResource(resource); }); } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/RestCatalogServiceTestBase.java b/amoro-ams/src/test/java/org/apache/amoro/server/RestCatalogServiceTestBase.java index fe798dc787..7d9691367b 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/RestCatalogServiceTestBase.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/RestCatalogServiceTestBase.java @@ -25,8 +25,8 @@ import org.apache.amoro.properties.CatalogMetaProperties; import org.apache.amoro.server.catalog.CatalogManager; import org.apache.amoro.server.catalog.InternalCatalog; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableMetadata; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; import org.apache.amoro.table.PrimaryKeySpec; @@ -121,13 +121,13 @@ protected TableMetadata getTableMetadata(TableIdentifier identifier) { return internalCatalog.loadTableMetadata(identifier.getDatabase(), identifier.getTableName()); } - protected TableRuntime getTableRuntime(TableIdentifier identifier) { + protected DefaultTableRuntime getTableRuntime(TableIdentifier identifier) { ServerTableIdentifier serverTableIdentifier = getServerTableIdentifier(identifier); return tableService.getRuntime(serverTableIdentifier.getId()); } protected void assertTableRuntime(TableIdentifier identifier, TableFormat format) { - TableRuntime runtime = getTableRuntime(identifier); + DefaultTableRuntime runtime = getTableRuntime(identifier); Assertions.assertNotNull(runtime, "table runtime is not exists after created"); Assertions.assertEquals(format, runtime.getFormat()); } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java index 55fac3a497..cd69f30025 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestDefaultOptimizingService.java @@ -45,10 +45,10 @@ import org.apache.amoro.server.optimizing.OptimizingStatus; import org.apache.amoro.server.optimizing.TaskRuntime; import org.apache.amoro.server.resource.OptimizerInstance; +import org.apache.amoro.server.scheduler.inline.TableRuntimeRefreshExecutor; import org.apache.amoro.server.table.AMSTableTestBase; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableManager; -import org.apache.amoro.server.table.TableRuntime; -import org.apache.amoro.server.table.executor.TableRuntimeRefreshExecutor; 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.amoro.table.MixedTable; @@ -128,9 +128,9 @@ private void initTableWithFiles() { (MixedTable) tableService().loadTable(serverTableIdentifier()).originalTable(); appendData(mixedTable.asUnkeyedTable(), 1); appendData(mixedTable.asUnkeyedTable(), 2); - TableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); + DefaultTableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); - runtime.refresh(tableService().loadTable(serverTableIdentifier())); + runtime.getOptimizingState().refresh(tableService().loadTable(serverTableIdentifier())); } private void appendData(UnkeyedTable table, int id) { @@ -328,10 +328,16 @@ public void testReloadCompletedTask() { reload(); // Committing process will be closed when reloading Assertions.assertNull( - tableService().getRuntime(serverTableIdentifier().getId()).getOptimizingProcess()); + tableService() + .getRuntime(serverTableIdentifier().getId()) + .getOptimizingState() + .getOptimizingProcess()); Assertions.assertEquals( OptimizingStatus.IDLE, - tableService().getRuntime(serverTableIdentifier().getId()).getOptimizingStatus()); + tableService() + .getRuntime(serverTableIdentifier().getId()) + .getOptimizingState() + .getOptimizingStatus()); } @Test @@ -383,151 +389,151 @@ public void testGetRuntimes() { // 1.1 add tables with IDLE status // the status will be OptimizingStatus.IDLE default String idle1InGroup1 = "idle1InGroup1"; - TableRuntime idle1 = - new TableRuntime( + DefaultTableRuntime idle1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10001L, catalog, db1, idle1InGroup1, TableFormat.ICEBERG), tableService(), properties); // the status will be OptimizingStatus.IDLE default String idle2InGroup1 = "idle2InGroup1"; - TableRuntime idle2 = - new TableRuntime( + DefaultTableRuntime idle2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10002L, catalog, db1, idle2InGroup1, TableFormat.ICEBERG), tableService(), properties); // 1.2 add tables with PENDING status String pending1InGroup1 = "pending1InGroup1"; - TableRuntime pending1 = - new TableRuntime( + DefaultTableRuntime pending1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10003L, catalog, db1, pending1InGroup1, TableFormat.ICEBERG), tableService(), properties); // update status - pending1.setPendingInput(new AbstractOptimizingEvaluator.PendingInput()); + pending1.getOptimizingState().setPendingInput(new AbstractOptimizingEvaluator.PendingInput()); String pending2InGroup1 = "pending2InGroup1"; - TableRuntime pending2 = - new TableRuntime( + DefaultTableRuntime pending2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10004L, catalog, db1, pending2InGroup1, TableFormat.ICEBERG), tableService(), properties); // update status - pending2.setPendingInput(new AbstractOptimizingEvaluator.PendingInput()); + pending2.getOptimizingState().setPendingInput(new AbstractOptimizingEvaluator.PendingInput()); // 1.3 add tables with PLANNING status String db2 = "db2"; String plan1InGroup1 = "plan1InGroup1"; - TableRuntime plan1 = - new TableRuntime( + DefaultTableRuntime plan1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10005L, catalog, db2, plan1InGroup1, TableFormat.ICEBERG), tableService(), properties); - plan1.beginPlanning(); + plan1.getOptimizingState().beginPlanning(); String plan2InGroup1 = "plan2InGroup1"; - TableRuntime plan2 = - new TableRuntime( + DefaultTableRuntime plan2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10006L, catalog, db2, plan2InGroup1, TableFormat.ICEBERG), tableService(), properties); - plan2.beginPlanning(); + plan2.getOptimizingState().beginPlanning(); // 1.4 add tables with COMMITTING status String committing1InGroup1 = "committing1InGroup1"; - TableRuntime committing1 = - new TableRuntime( + DefaultTableRuntime committing1 = + new DefaultTableRuntime( ServerTableIdentifier.of( 10007L, catalog, db2, committing1InGroup1, TableFormat.ICEBERG), tableService(), properties); - committing1.beginCommitting(); + committing1.getOptimizingState().beginCommitting(); String commiting2InGroup1 = "committing2InGroup1"; - TableRuntime committing2 = - new TableRuntime( + DefaultTableRuntime committing2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10008L, catalog, db2, commiting2InGroup1, TableFormat.ICEBERG), tableService(), properties); - committing2.beginCommitting(); + committing2.getOptimizingState().beginCommitting(); // 1.5 add tables with MINOR_OPTIMIZING status String minor1InGroup1 = "minor1InGroup1"; - TableRuntime minor1 = - new TableRuntime( + DefaultTableRuntime minor1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10009L, catalog, db2, minor1InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process = mock(OptimizingProcess.class); doReturn(1L).when(process).getProcessId(); doReturn(OptimizingType.MINOR).when(process).getOptimizingType(); - minor1.beginProcess(process); + minor1.getOptimizingState().beginProcess(process); String minor2InGroup1 = "minor2InGroup1"; - TableRuntime minor2 = - new TableRuntime( + DefaultTableRuntime minor2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10010L, catalog, db2, minor2InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process2 = mock(OptimizingProcess.class); doReturn(2L).when(process2).getProcessId(); doReturn(OptimizingType.MINOR).when(process2).getOptimizingType(); - minor2.beginProcess(process2); + minor2.getOptimizingState().beginProcess(process2); // 1.6 add tables with MAJOR_OPTIMIZING status String major1InGroup1 = "major1InGroup1"; - TableRuntime major1 = - new TableRuntime( + DefaultTableRuntime major1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10011L, catalog, db1, major1InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process3 = mock(OptimizingProcess.class); doReturn(3L).when(process3).getProcessId(); doReturn(OptimizingType.MAJOR).when(process3).getOptimizingType(); - major1.beginProcess(process3); + major1.getOptimizingState().beginProcess(process3); String major2InGroup1 = "major2InGroup1"; - TableRuntime major2 = - new TableRuntime( + DefaultTableRuntime major2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10012L, catalog, db1, major2InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process4 = mock(OptimizingProcess.class); doReturn(4L).when(process4).getProcessId(); doReturn(OptimizingType.MAJOR).when(process4).getOptimizingType(); - major2.beginProcess(process4); + major2.getOptimizingState().beginProcess(process4); // 1.7 add tables with FULL_OPTIMIZING status String full1InGroup1 = "full1InGroup1"; - TableRuntime full1 = - new TableRuntime( + DefaultTableRuntime full1 = + new DefaultTableRuntime( ServerTableIdentifier.of(10013L, catalog, db1, full1InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process5 = mock(OptimizingProcess.class); doReturn(5L).when(process5).getProcessId(); doReturn(OptimizingType.FULL).when(process5).getOptimizingType(); - full1.beginProcess(process5); + full1.getOptimizingState().beginProcess(process5); String full2InGroup1 = "full2InGroup1"; - TableRuntime full2 = - new TableRuntime( + DefaultTableRuntime full2 = + new DefaultTableRuntime( ServerTableIdentifier.of(10014L, catalog, db1, full2InGroup1, TableFormat.ICEBERG), tableService(), properties); OptimizingProcess process6 = mock(OptimizingProcess.class); doReturn(6L).when(process6).getProcessId(); doReturn(OptimizingType.FULL).when(process6).getOptimizingType(); - full2.beginProcess(process6); + full2.getOptimizingState().beginProcess(process6); // 1.8 add tables in other group with MINOR_OPTIMIZING status // table in other group. String opGroup2 = "opGroup2-other"; properties.put(TableProperties.SELF_OPTIMIZING_GROUP, opGroup2); String minor1InOtherGroup1 = "minor1-InOtherGroup"; - TableRuntime minor1Other = - new TableRuntime( + DefaultTableRuntime minor1Other = + new DefaultTableRuntime( ServerTableIdentifier.of( 10015L, catalog, db1, minor1InOtherGroup1, TableFormat.ICEBERG), tableService(), @@ -535,11 +541,11 @@ public void testGetRuntimes() { OptimizingProcess process7 = mock(OptimizingProcess.class); doReturn(7L).when(process7).getProcessId(); doReturn(OptimizingType.MINOR).when(process7).getOptimizingType(); - minor1Other.beginProcess(process7); + minor1Other.getOptimizingState().beginProcess(process7); String minor2InOtherGroup1 = "minor2-InOtherGroup"; - TableRuntime minor2Other = - new TableRuntime( + DefaultTableRuntime minor2Other = + new DefaultTableRuntime( ServerTableIdentifier.of( 10016L, catalog, db1, minor2InOtherGroup1, TableFormat.ICEBERG), tableService(), @@ -547,11 +553,11 @@ public void testGetRuntimes() { OptimizingProcess process8 = mock(OptimizingProcess.class); doReturn(8L).when(process8).getProcessId(); doReturn(OptimizingType.MINOR).when(process8).getOptimizingType(); - minor2Other.beginProcess(process8); + minor2Other.getOptimizingState().beginProcess(process8); String minor3InOtherGroup1 = "minor3-InOtherGroup"; - TableRuntime minor3Other = - new TableRuntime( + DefaultTableRuntime minor3Other = + new DefaultTableRuntime( ServerTableIdentifier.of( 10017L, catalog, db1, minor3InOtherGroup1, TableFormat.ICEBERG), tableService(), @@ -559,7 +565,7 @@ public void testGetRuntimes() { OptimizingProcess process9 = mock(OptimizingProcess.class); doReturn(9L).when(process9).getProcessId(); doReturn(OptimizingType.MINOR).when(process9).getOptimizingType(); - minor3Other.beginProcess(process9); + minor3Other.getOptimizingState().beginProcess(process9); // 2 test and assert the result // 2.1 only optimize group filter set @@ -680,11 +686,15 @@ private void assertTaskCompleted(TaskRuntime taskRuntime) { ProcessStatus.RUNNING, tableService() .getRuntime(serverTableIdentifier().getId()) + .getOptimizingState() .getOptimizingProcess() .getStatus()); Assertions.assertEquals( OptimizingStatus.COMMITTING, - tableService().getRuntime(serverTableIdentifier().getId()).getOptimizingStatus()); + tableService() + .getRuntime(serverTableIdentifier().getId()) + .getOptimizingState() + .getOptimizingStatus()); } protected void reload() { diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/dashboard/TestOverviewManager.java b/amoro-ams/src/test/java/org/apache/amoro/server/dashboard/TestOverviewManager.java index acffaa942b..1cf06a5d69 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/dashboard/TestOverviewManager.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/dashboard/TestOverviewManager.java @@ -31,9 +31,9 @@ import org.apache.amoro.catalog.CatalogTestHelper; import org.apache.amoro.io.MixedDataTestHelpers; import org.apache.amoro.server.dashboard.model.OverviewTopTableItem; +import org.apache.amoro.server.scheduler.inline.TableRuntimeRefreshExecutor; import org.apache.amoro.server.table.AMSTableTestBase; -import org.apache.amoro.server.table.TableRuntime; -import org.apache.amoro.server.table.executor.TableRuntimeRefreshExecutor; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; import org.apache.amoro.table.MixedTable; import org.apache.amoro.table.UnkeyedTable; @@ -91,8 +91,8 @@ private void initTableWithFiles() { .asUnkeyedTable(); appendData(table, 1); appendData(table, 2); - TableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); - runtime.refresh(tableService().loadTable(serverTableIdentifier())); + DefaultTableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); + runtime.getOptimizingState().refresh(tableService().loadTable(serverTableIdentifier())); } private void appendData(UnkeyedTable table, int id) { diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/TestOptimizingQueue.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/TestOptimizingQueue.java index 84ebcd3860..506624e199 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/TestOptimizingQueue.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/TestOptimizingQueue.java @@ -52,8 +52,8 @@ import org.apache.amoro.server.resource.OptimizerThread; import org.apache.amoro.server.resource.QuotaProvider; import org.apache.amoro.server.table.AMSTableTestBase; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; import org.apache.amoro.table.MixedTable; @@ -106,7 +106,7 @@ protected static ResourceGroup testResourceGroup() { return new ResourceGroup.Builder("test", "local").build(); } - protected OptimizingQueue buildOptimizingGroupService(TableRuntime tableRuntime) { + protected OptimizingQueue buildOptimizingGroupService(DefaultTableRuntime tableRuntime) { return new OptimizingQueue( CATALOG_MANAGER, testResourceGroup(), @@ -128,7 +128,7 @@ private OptimizingQueue buildOptimizingGroupService() { @Test public void testPollNoTask() { - TableRuntime tableRuntimeMeta = + DefaultTableRuntime tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); Assert.assertNull(queue.pollTask(0)); @@ -139,7 +139,7 @@ public void testPollNoTask() { public void testRefreshAndReleaseTable() { OptimizingQueue queue = buildOptimizingGroupService(); Assert.assertEquals(0, queue.getSchedulingPolicy().getTableRuntimeMap().size()); - TableRuntime tableRuntime = + DefaultTableRuntime tableRuntime = buildTableRuntimeMeta(OptimizingStatus.IDLE, defaultResourceGroup()); queue.refreshTable(tableRuntime); Assert.assertEquals(1, queue.getSchedulingPolicy().getTableRuntimeMap().size()); @@ -156,7 +156,7 @@ public void testRefreshAndReleaseTable() { @Test public void testPollTask() { - TableRuntime tableRuntime = initTableWithFiles(); + DefaultTableRuntime tableRuntime = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntime); // 1.poll task @@ -170,7 +170,7 @@ public void testPollTask() { @Test public void testRetryTask() { - TableRuntime tableRuntimeMeta = initTableWithFiles(); + DefaultTableRuntime tableRuntimeMeta = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); // 1.poll task @@ -203,7 +203,7 @@ public void testRetryTask() { @Test public void testCommitTask() { - TableRuntime tableRuntime = initTableWithFiles(); + DefaultTableRuntime tableRuntime = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntime); Assert.assertEquals(0, queue.collectTasks().size()); @@ -219,11 +219,11 @@ public void testCommitTask() { Assert.assertEquals(TaskRuntime.Status.SUCCESS, task.getStatus()); // 7.commit - OptimizingProcess optimizingProcess = tableRuntime.getOptimizingProcess(); + OptimizingProcess optimizingProcess = tableRuntime.getOptimizingState().getOptimizingProcess(); Assert.assertEquals(ProcessStatus.RUNNING, optimizingProcess.getStatus()); optimizingProcess.commit(); Assert.assertEquals(ProcessStatus.SUCCESS, optimizingProcess.getStatus()); - Assert.assertNull(tableRuntime.getOptimizingProcess()); + Assert.assertNull(tableRuntime.getOptimizingState().getOptimizingProcess()); // 8.commit again, throw exceptions, and status not changed. Assert.assertThrows(IllegalStateException.class, optimizingProcess::commit); @@ -235,7 +235,7 @@ public void testCommitTask() { @Test public void testCollectingTasks() { - TableRuntime tableRuntime = initTableWithFiles(); + DefaultTableRuntime tableRuntime = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntime); Assert.assertEquals(0, queue.collectTasks().size()); @@ -250,7 +250,7 @@ public void testCollectingTasks() { @Test public void testTaskAndTableMetrics() { - TableRuntime tableRuntime = initTableWithFiles(); + DefaultTableRuntime tableRuntime = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntime); MetricRegistry registry = MetricManager.getInstance().getGlobalRegistry(); Map tagValues = ImmutableMap.of(GROUP_TAG, testResourceGroup().getName()); @@ -316,7 +316,7 @@ public void testTaskAndTableMetrics() { Assert.assertEquals(0, idleTablesGauge.getValue().longValue()); Assert.assertEquals(1, committingTablesGauge.getValue().longValue()); - OptimizingProcess optimizingProcess = tableRuntime.getOptimizingProcess(); + OptimizingProcess optimizingProcess = tableRuntime.getOptimizingState().getOptimizingProcess(); optimizingProcess.commit(); Assert.assertEquals(0, queueTasksGauge.getValue().longValue()); Assert.assertEquals(0, executingTasksGauge.getValue().longValue()); @@ -364,19 +364,20 @@ public void testAddAndRemoveOptimizers() { queue.dispose(); } - protected TableRuntime initTableWithFiles() { + protected DefaultTableRuntime initTableWithFiles() { MixedTable mixedTable = (MixedTable) tableService().loadTable(serverTableIdentifier()).originalTable(); appendData(mixedTable.asUnkeyedTable(), 1); appendData(mixedTable.asUnkeyedTable(), 2); - TableRuntime tableRuntime = + DefaultTableRuntime tableRuntime = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); - tableRuntime.refresh(tableService().loadTable(serverTableIdentifier())); + tableRuntime.getOptimizingState().refresh(tableService().loadTable(serverTableIdentifier())); return tableRuntime; } - private TableRuntime buildTableRuntimeMeta(OptimizingStatus status, ResourceGroup resourceGroup) { + private DefaultTableRuntime buildTableRuntimeMeta( + OptimizingStatus status, ResourceGroup resourceGroup) { MixedTable mixedTable = (MixedTable) tableService().loadTable(serverTableIdentifier()).originalTable(); TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); @@ -388,7 +389,7 @@ private TableRuntime buildTableRuntimeMeta(OptimizingStatus status, ResourceGrou tableRuntimeMeta.setTableStatus(status); tableRuntimeMeta.setTableConfig(TableConfigurations.parseTableConfig(mixedTable.properties())); tableRuntimeMeta.setOptimizerGroup(resourceGroup.getName()); - return new TableRuntime(tableRuntimeMeta, tableService()); + return new DefaultTableRuntime(tableRuntimeMeta, tableService()); } private void appendData(UnkeyedTable table, int id) { diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/flow/CompleteOptimizingFlow.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/flow/CompleteOptimizingFlow.java index e4c7dd78e7..4662ee3f84 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/flow/CompleteOptimizingFlow.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/flow/CompleteOptimizingFlow.java @@ -39,8 +39,9 @@ import org.apache.amoro.server.optimizing.KeyedTableCommit; import org.apache.amoro.server.optimizing.TaskRuntime; import org.apache.amoro.server.optimizing.UnKeyedTableCommit; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.table.MixedTable; import org.apache.amoro.utils.MixedDataFiles; @@ -182,18 +183,22 @@ private List> mockTaskRuntime( private AbstractOptimizingPlanner planner() { table.refresh(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); - Mockito.when(tableRuntime.getCurrentSnapshotId()).thenAnswer(f -> getCurrentSnapshotId()); - Mockito.when(tableRuntime.getCurrentChangeSnapshotId()) + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); + Mockito.when(optimizingState.getCurrentSnapshotId()).thenAnswer(f -> getCurrentSnapshotId()); + Mockito.when(optimizingState.getCurrentChangeSnapshotId()) .thenAnswer(f -> getCurrentChangeSnapshotId()); - Mockito.when(tableRuntime.getNewestProcessId()).thenReturn(1L); - Mockito.when(tableRuntime.getPendingInput()).thenReturn(null); - Mockito.doCallRealMethod().when(tableRuntime).getLastMinorOptimizingTime(); - Mockito.doCallRealMethod().when(tableRuntime).getLastMajorOptimizingTime(); - Mockito.doCallRealMethod().when(tableRuntime).getLastFullOptimizingTime(); - Mockito.when(tableRuntime.getOptimizingConfig()).thenAnswer(f -> optimizingConfig()); + Mockito.when(optimizingState.getNewestProcessId()).thenReturn(1L); + Mockito.when(optimizingState.getPendingInput()).thenReturn(null); + Mockito.doCallRealMethod().when(optimizingState).getLastMinorOptimizingTime(); + Mockito.doCallRealMethod().when(optimizingState).getLastMajorOptimizingTime(); + Mockito.doCallRealMethod().when(optimizingState).getLastFullOptimizingTime(); + Mockito.when(optimizingState.getOptimizingConfig()).thenAnswer(f -> optimizingConfig()); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(1L, "a", "b", "c", table.format())); + Mockito.when(optimizingState.getTableIdentifier()) + .thenReturn(ServerTableIdentifier.of(1L, "a", "b", "c", table.format())); return IcebergTableUtil.createOptimizingPlanner( tableRuntime, table, diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestDataExpire.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestDataExpire.java index f54c81decc..ebec3c8f95 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestDataExpire.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestDataExpire.java @@ -36,8 +36,8 @@ import org.apache.amoro.optimizing.scan.TableFileScanHelper; import org.apache.amoro.optimizing.scan.UnkeyedTableFileScanHelper; import org.apache.amoro.server.optimizing.OptimizingTestHelpers; +import org.apache.amoro.server.scheduler.inline.ExecutorTestBase; import org.apache.amoro.server.table.TableConfigurations; -import org.apache.amoro.server.table.executor.ExecutorTestBase; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; import org.apache.amoro.table.KeyedTable; diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java index 0de861e2a9..5df5b47b13 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java @@ -27,10 +27,11 @@ import org.apache.amoro.TableTestHelper; import org.apache.amoro.catalog.BasicCatalogTestHelper; import org.apache.amoro.catalog.CatalogTestHelper; +import org.apache.amoro.server.scheduler.inline.ExecutorTestBase; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics; -import org.apache.amoro.server.table.TableRuntime; -import org.apache.amoro.server.table.executor.ExecutorTestBase; import org.apache.amoro.table.TableIdentifier; import org.apache.amoro.table.TableProperties; import org.apache.amoro.table.UnkeyedTable; @@ -395,13 +396,17 @@ public void testGcDisabled() throws IOException { Assert.assertTrue(getMixedTable().io().exists(changeOrphanFilePath)); } - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(baseTable.id(), getTestFormat())); + Mockito.when(optimizingState.getTableConfiguration()) + .thenReturn(TableConfigurations.parseTableConfig(baseTable.properties())); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(baseTable.properties())); - Mockito.when(tableRuntime.getOrphanFilesCleaningMetrics()) + Mockito.when(tableRuntime.getOptimizingState().getOrphanFilesCleaningMetrics()) .thenReturn( new TableOrphanFilesCleaningMetrics( ServerTableIdentifier.of(baseTable.id(), getTestFormat()))); diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestSnapshotExpire.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestSnapshotExpire.java index e068e59c71..d5326dfab6 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestSnapshotExpire.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestSnapshotExpire.java @@ -30,9 +30,10 @@ import org.apache.amoro.data.ChangeAction; import org.apache.amoro.server.optimizing.OptimizingProcess; import org.apache.amoro.server.optimizing.OptimizingStatus; +import org.apache.amoro.server.scheduler.inline.ExecutorTestBase; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; -import org.apache.amoro.server.table.TableRuntime; -import org.apache.amoro.server.table.executor.ExecutorTestBase; import org.apache.amoro.shade.guava32.com.google.common.collect.Iterables; import org.apache.amoro.shade.guava32.com.google.common.collect.Iterators; import org.apache.amoro.table.BaseTable; @@ -166,10 +167,14 @@ public void testNotExpireFlinkLatestCommit4ChangeTable() { Snapshot lastSnapshot = testKeyedTable.changeTable().currentSnapshot(); testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(testKeyedTable.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); + Mockito.when(optimizingState.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(optimizingState.getTableConfiguration()) + .thenReturn(TableConfigurations.parseTableConfig((testKeyedTable.properties()))); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig((testKeyedTable.properties()))); @@ -208,10 +213,13 @@ public void testNotExpireFlinkLatestCommit4All() { Snapshot lastSnapshot = table.currentSnapshot(); table.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(table.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingStatus()) + .thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(table.properties())); @@ -247,12 +255,16 @@ public void testNotExpireOptimizedSequenceCommit4All() { Snapshot lastSnapshot = table.currentSnapshot(); table.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(table.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(optimizingState.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(table.properties())); + Mockito.when(optimizingState.getTableConfiguration()) + .thenReturn(TableConfigurations.parseTableConfig(table.properties())); Assert.assertEquals(4, Iterables.size(table.snapshots())); @@ -277,10 +289,13 @@ public void testNotExpireOptimizeCommit4All() { table.newAppend().commit(); table.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(table.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingStatus()) + .thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(table.properties())); @@ -293,8 +308,10 @@ public void testNotExpireOptimizeCommit4All() { long optimizeSnapshotId = table.currentSnapshot().snapshotId(); OptimizingProcess optimizingProcess = Mockito.mock(OptimizingProcess.class); Mockito.when(optimizingProcess.getTargetSnapshotId()).thenReturn(optimizeSnapshotId); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.COMMITTING); - Mockito.when(tableRuntime.getOptimizingProcess()).thenReturn(optimizingProcess); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingStatus()) + .thenReturn(OptimizingStatus.COMMITTING); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingProcess()) + .thenReturn(optimizingProcess); List expectedSnapshots = new ArrayList<>(); expectedSnapshots.add(table.currentSnapshot()); @@ -463,12 +480,16 @@ public void testChangeTableGcDisabled() { Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(testKeyedTable.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(optimizingState.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(testKeyedTable.properties())); + Mockito.when(optimizingState.getTableConfiguration()) + .thenReturn(TableConfigurations.parseTableConfig(testKeyedTable.properties())); MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); @@ -493,10 +514,13 @@ public void testBaseTableGcDisabled() { Assert.assertEquals(2, Iterables.size(testUnkeyedTable.snapshots())); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(testUnkeyedTable.id(), getTestFormat())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingStatus()) + .thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(testUnkeyedTable.properties())); @@ -529,12 +553,15 @@ public void testRetainMinSnapshot() { table.updateProperties().set(TableProperties.SNAPSHOT_KEEP_DURATION, "0s").commit(); table.updateProperties().set(TableProperties.SNAPSHOT_MIN_COUNT, "3").commit(); - TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); + DefaultTableRuntime tableRuntime = Mockito.mock(DefaultTableRuntime.class); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); Mockito.when(tableRuntime.getTableIdentifier()) .thenReturn(ServerTableIdentifier.of(table.id(), getTestFormat())); Mockito.when(tableRuntime.getTableConfiguration()) .thenReturn(TableConfigurations.parseTableConfig(table.properties())); - Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); + Mockito.when(tableRuntime.getOptimizingState().getOptimizingStatus()) + .thenReturn(OptimizingStatus.IDLE); new MixedTableMaintainer(table).expireSnapshots(tableRuntime); Assert.assertEquals(2, Iterables.size(table.snapshots())); diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/MixedTablePlanTestBase.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/MixedTablePlanTestBase.java index 6b6386a163..f54d19f241 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/MixedTablePlanTestBase.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/MixedTablePlanTestBase.java @@ -31,8 +31,9 @@ import org.apache.amoro.optimizing.plan.AbstractPartitionPlan; import org.apache.amoro.optimizing.scan.TableFileScanHelper; import org.apache.amoro.server.optimizing.OptimizingTestHelpers; +import org.apache.amoro.server.table.DefaultOptimizingState; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableConfigurations; -import org.apache.amoro.server.table.TableRuntime; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; @@ -66,7 +67,7 @@ public abstract class MixedTablePlanTestBase extends TableTestBase { - protected TableRuntime tableRuntime; + protected DefaultTableRuntime tableRuntime; public MixedTablePlanTestBase( CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { @@ -75,13 +76,15 @@ public MixedTablePlanTestBase( @Before public void mock() { - tableRuntime = Mockito.mock(TableRuntime.class); + tableRuntime = Mockito.mock(DefaultTableRuntime.class); ServerTableIdentifier id = ServerTableIdentifier.of(getMixedTable().id(), getTestFormat()); id.setId(0L); + DefaultOptimizingState optimizingState = Mockito.mock(DefaultOptimizingState.class); Mockito.when(tableRuntime.getTableIdentifier()).thenReturn(id); - Mockito.when(tableRuntime.getOptimizingConfig()).thenAnswer(f -> getConfig()); - Mockito.when(tableRuntime.getCurrentSnapshotId()).thenAnswer(f -> getCurrentSnapshotId()); - Mockito.when(tableRuntime.getCurrentChangeSnapshotId()) + Mockito.when(tableRuntime.getOptimizingState()).thenReturn(optimizingState); + Mockito.when(optimizingState.getOptimizingConfig()).thenAnswer(f -> getConfig()); + Mockito.when(optimizingState.getCurrentSnapshotId()).thenAnswer(f -> getCurrentSnapshotId()); + Mockito.when(optimizingState.getCurrentChangeSnapshotId()) .thenAnswer(f -> getCurrentChangeSnapshotId()); } @@ -555,7 +558,7 @@ protected long beginTransaction() { } } - protected TableRuntime getTableRuntime() { + protected DefaultTableRuntime getTableRuntime() { return tableRuntime; } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveKeyedPartitionPlan.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveKeyedPartitionPlan.java index 412defbd9d..44b256ccf6 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveKeyedPartitionPlan.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveKeyedPartitionPlan.java @@ -86,12 +86,12 @@ protected AbstractPartitionPlan getPartitionPlan() { return new MixedHivePartitionPlan( getTableRuntime().getTableIdentifier(), getMixedTable(), - getTableRuntime().getOptimizingConfig(), + getTableRuntime().getOptimizingState().getOptimizingConfig(), getPartition(), hiveLocation, System.currentTimeMillis(), - getTableRuntime().getLastMinorOptimizingTime(), - getTableRuntime().getLastFullOptimizingTime()); + getTableRuntime().getOptimizingState().getLastMinorOptimizingTime(), + getTableRuntime().getOptimizingState().getLastFullOptimizingTime()); } @Test diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveUnkeyedPartitionPlan.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveUnkeyedPartitionPlan.java index 82097d92c1..1ea7ef1f28 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveUnkeyedPartitionPlan.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestHiveUnkeyedPartitionPlan.java @@ -85,12 +85,12 @@ protected AbstractPartitionPlan getPartitionPlan() { return new MixedHivePartitionPlan( getTableRuntime().getTableIdentifier(), getMixedTable(), - getTableRuntime().getOptimizingConfig(), + getTableRuntime().getOptimizingState().getOptimizingConfig(), getPartition(), hiveLocation, System.currentTimeMillis(), - getTableRuntime().getLastMinorOptimizingTime(), - getTableRuntime().getLastFullOptimizingTime()); + getTableRuntime().getOptimizingState().getLastMinorOptimizingTime(), + getTableRuntime().getOptimizingState().getLastFullOptimizingTime()); } @Test diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestIcebergPartitionPlan.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestIcebergPartitionPlan.java index 7d2b85caa4..3b2c832005 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestIcebergPartitionPlan.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestIcebergPartitionPlan.java @@ -29,7 +29,7 @@ import org.apache.amoro.optimizing.plan.IcebergPartitionPlan; import org.apache.amoro.optimizing.scan.IcebergTableFileScanHelper; import org.apache.amoro.optimizing.scan.TableFileScanHelper; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.utils.IcebergTableUtil; import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; import org.junit.Test; @@ -75,15 +75,15 @@ public void testOnlyOneFragmentFiles() { @Override protected AbstractPartitionPlan getPartitionPlan() { - TableRuntime tableRuntime = getTableRuntime(); + DefaultTableRuntime tableRuntime = getTableRuntime(); return new IcebergPartitionPlan( tableRuntime.getTableIdentifier(), - tableRuntime.getOptimizingConfig(), + tableRuntime.getOptimizingState().getOptimizingConfig(), getMixedTable(), getPartition(), System.currentTimeMillis(), - tableRuntime.getLastMinorOptimizingTime(), - tableRuntime.getLastFullOptimizingTime()); + tableRuntime.getOptimizingState().getLastMinorOptimizingTime(), + tableRuntime.getOptimizingState().getLastFullOptimizingTime()); } @Override diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestKeyedPartitionPlan.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestKeyedPartitionPlan.java index 4a4b38472a..02a4e6901c 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestKeyedPartitionPlan.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestKeyedPartitionPlan.java @@ -218,11 +218,11 @@ protected AbstractPartitionPlan getPartitionPlan() { return new MixedIcebergPartitionPlan( getTableRuntime().getTableIdentifier(), getMixedTable(), - getTableRuntime().getOptimizingConfig(), + getTableRuntime().getOptimizingState().getOptimizingConfig(), getPartition(), System.currentTimeMillis(), - getTableRuntime().getLastMinorOptimizingTime(), - getTableRuntime().getLastFullOptimizingTime()); + getTableRuntime().getOptimizingState().getLastMinorOptimizingTime(), + getTableRuntime().getOptimizingState().getLastFullOptimizingTime()); } @Override diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestUnkeyedPartitionPlan.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestUnkeyedPartitionPlan.java index 33d7aa33b8..054bae354f 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestUnkeyedPartitionPlan.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/plan/TestUnkeyedPartitionPlan.java @@ -84,11 +84,11 @@ protected AbstractPartitionPlan getPartitionPlan() { return new MixedIcebergPartitionPlan( getTableRuntime().getTableIdentifier(), getMixedTable(), - getTableRuntime().getOptimizingConfig(), + getTableRuntime().getOptimizingState().getOptimizingConfig(), getPartition(), System.currentTimeMillis(), - getTableRuntime().getLastMinorOptimizingTime(), - getTableRuntime().getLastFullOptimizingTime()); + getTableRuntime().getOptimizingState().getLastMinorOptimizingTime(), + getTableRuntime().getOptimizingState().getLastFullOptimizingTime()); } @Override diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/executor/ExecutorTestBase.java b/amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/ExecutorTestBase.java similarity index 98% rename from amoro-ams/src/test/java/org/apache/amoro/server/table/executor/ExecutorTestBase.java rename to amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/ExecutorTestBase.java index 4d5a230877..744f03305e 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/executor/ExecutorTestBase.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/ExecutorTestBase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.TableTestHelper; import org.apache.amoro.catalog.CatalogTestHelper; diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/executor/TestBlockerExpiringExecutor.java b/amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/TestBlockerExpiringExecutor.java similarity index 95% rename from amoro-ams/src/test/java/org/apache/amoro/server/table/executor/TestBlockerExpiringExecutor.java rename to amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/TestBlockerExpiringExecutor.java index afba8fab6b..eb79622b54 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/executor/TestBlockerExpiringExecutor.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/scheduler/inline/TestBlockerExpiringExecutor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.amoro.server.table.executor; +package org.apache.amoro.server.scheduler.inline; import org.apache.amoro.ServerTableIdentifier; import org.apache.amoro.TableFormat; @@ -24,7 +24,7 @@ import org.apache.amoro.server.AMSServiceTestBase; import org.apache.amoro.server.persistence.PersistentBase; import org.apache.amoro.server.persistence.mapper.TableBlockerMapper; -import org.apache.amoro.server.table.TableRuntime; +import org.apache.amoro.server.table.DefaultTableRuntime; import org.apache.amoro.server.table.TableService; import org.apache.amoro.server.table.blocker.TableBlocker; import org.junit.Assert; @@ -41,12 +41,12 @@ public class TestBlockerExpiringExecutor extends AMSServiceTestBase { 0L, "test_catalog", "test_db", "test_table_blocker", TableFormat.MIXED_ICEBERG); private final Persistency persistency = new Persistency(); - private TableRuntime tableRuntime; + private DefaultTableRuntime tableRuntime; private TableService tableService; @Before public void mock() { - tableRuntime = Mockito.mock(TableRuntime.class); + tableRuntime = Mockito.mock(DefaultTableRuntime.class); tableService = Mockito.mock(TableService.class); Mockito.when(tableRuntime.getTableIdentifier()).thenReturn(tableIdentifier); } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/AMSTableTestBase.java b/amoro-ams/src/test/java/org/apache/amoro/server/table/AMSTableTestBase.java index 352f0280e0..67c666bd78 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/AMSTableTestBase.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/table/AMSTableTestBase.java @@ -300,7 +300,7 @@ protected void validateMixedTable(MixedTable mixedTable) { tableTestHelper().primaryKeySpec().primaryKeyExisted(), mixedTable.isKeyedTable()); } - protected void validateTableRuntime(TableRuntime tableRuntime) { + protected void validateTableRuntime(DefaultTableRuntime tableRuntime) { Assert.assertEquals(serverTableIdentifier(), tableRuntime.getTableIdentifier()); } } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeHandler.java b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeHandler.java similarity index 81% rename from amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeHandler.java rename to amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeHandler.java index 230bd59e1b..0a0dd445b8 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeHandler.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeHandler.java @@ -44,7 +44,7 @@ import java.util.List; @RunWith(Parameterized.class) -public class TestTableRuntimeHandler extends AMSTableTestBase { +public class TestDefaultTableRuntimeHandler extends AMSTableTestBase { private DefaultTableService tableService; @@ -60,7 +60,7 @@ public static Object[] parameters() { }; } - public TestTableRuntimeHandler( + public TestDefaultTableRuntimeHandler( CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { super(catalogTestHelper, tableTestHelper, false); } @@ -101,6 +101,7 @@ public void testInitialize() throws Exception { mixedTable.updateProperties().set(TableProperties.ENABLE_ORPHAN_CLEAN, "true").commit(); tableService() .getRuntime(createTableId.getId()) + .getOptimizingState() .refresh(tableService.loadTable(serverTableIdentifier())); Assert.assertEquals(1, handler.getConfigChangedTables().size()); validateTableRuntime(handler.getConfigChangedTables().get(0).first()); @@ -132,38 +133,39 @@ protected DefaultTableService tableService() { static class TestHandler extends RuntimeHandlerChain { - private final List initTables = Lists.newArrayList(); - private final List> statusChangedTables = + private final List initTables = Lists.newArrayList(); + private final List> statusChangedTables = Lists.newArrayList(); - private final List> configChangedTables = + private final List> configChangedTables = Lists.newArrayList(); - private final List> addedTables = Lists.newArrayList(); - private final List removedTables = Lists.newArrayList(); + private final List> addedTables = Lists.newArrayList(); + private final List removedTables = Lists.newArrayList(); private boolean disposed = false; @Override - protected void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { + protected void handleStatusChanged( + DefaultTableRuntime tableRuntime, OptimizingStatus originalStatus) { statusChangedTables.add(Pair.of(tableRuntime, originalStatus)); } @Override protected void handleConfigChanged( - TableRuntime tableRuntime, TableConfiguration originalConfig) { + DefaultTableRuntime tableRuntime, TableConfiguration originalConfig) { configChangedTables.add(Pair.of(tableRuntime, originalConfig)); } @Override - protected void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { + protected void handleTableAdded(AmoroTable table, DefaultTableRuntime tableRuntime) { addedTables.add(Pair.of((MixedTable) table.originalTable(), tableRuntime)); } @Override - protected void handleTableRemoved(TableRuntime tableRuntime) { + protected void handleTableRemoved(DefaultTableRuntime tableRuntime) { removedTables.add(tableRuntime); } @Override - protected void initHandler(List tableRuntimeList) { + protected void initHandler(List tableRuntimeList) { initTables.addAll(tableRuntimeList); } @@ -172,23 +174,23 @@ protected void doDispose() { disposed = true; } - public List getInitTables() { + public List getInitTables() { return initTables; } - public List> getStatusChangedTables() { + public List> getStatusChangedTables() { return statusChangedTables; } - public List> getConfigChangedTables() { + public List> getConfigChangedTables() { return configChangedTables; } - public List> getAddedTables() { + public List> getAddedTables() { return addedTables; } - public List getRemovedTables() { + public List getRemovedTables() { return removedTables; } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeManager.java b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeManager.java similarity index 92% rename from amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeManager.java rename to amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeManager.java index f77996d363..51e8829c7a 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableRuntimeManager.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestDefaultTableRuntimeManager.java @@ -34,7 +34,7 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) -public class TestTableRuntimeManager extends AMSTableTestBase { +public class TestDefaultTableRuntimeManager extends AMSTableTestBase { @Parameterized.Parameters(name = "{0}, {1}") public static Object[] parameters() { @@ -48,7 +48,7 @@ public static Object[] parameters() { }; } - public TestTableRuntimeManager( + public TestDefaultTableRuntimeManager( CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { super(catalogTestHelper, tableTestHelper, true); } @@ -71,7 +71,7 @@ public void testLoadTable() { @Test public void testTableRuntime() { - TableRuntime tableRuntime = tableService().getRuntime(serverTableIdentifier().getId()); + DefaultTableRuntime tableRuntime = tableService().getRuntime(serverTableIdentifier().getId()); validateTableRuntime(tableRuntime); } } diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableManager.java b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableManager.java index 17c5b20fbd..7332abd39a 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableManager.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableManager.java @@ -346,7 +346,10 @@ private boolean isBlocked(BlockableOperation operation) { } private boolean isTableRuntimeBlocked(BlockableOperation operation) { - return tableService().getRuntime(serverTableIdentifier().getId()).isBlocked(operation); + return tableService() + .getRuntime(serverTableIdentifier().getId()) + .getOptimizingState() + .isBlocked(operation); } private void assertBlockerCnt(int i) { diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableSummaryMetrics.java b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableSummaryMetrics.java index bd2436ccb4..61454fff32 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableSummaryMetrics.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/table/TestTableSummaryMetrics.java @@ -47,7 +47,7 @@ import org.apache.amoro.metrics.MetricKey; import org.apache.amoro.server.manager.MetricManager; import org.apache.amoro.server.optimizing.OptimizingTestHelpers; -import org.apache.amoro.server.table.executor.TableRuntimeRefreshExecutor; +import org.apache.amoro.server.scheduler.inline.TableRuntimeRefreshExecutor; import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap; import org.apache.amoro.shade.guava32.com.google.common.collect.Lists; import org.apache.amoro.table.MixedTable; @@ -105,8 +105,8 @@ private void initTableWithFiles() { .asUnkeyedTable(); appendData(table); appendPosDelete(table); - TableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); - runtime.refresh(tableService().loadTable(serverTableIdentifier())); + DefaultTableRuntime runtime = tableService().getRuntime(serverTableIdentifier().getId()); + runtime.getOptimizingState().refresh(tableService().loadTable(serverTableIdentifier())); } private void appendData(UnkeyedTable table) { diff --git a/amoro-common/src/main/java/org/apache/amoro/Action.java b/amoro-common/src/main/java/org/apache/amoro/Action.java index 9f07473504..42671a6e1d 100644 --- a/amoro-common/src/main/java/org/apache/amoro/Action.java +++ b/amoro-common/src/main/java/org/apache/amoro/Action.java @@ -18,91 +18,34 @@ package org.apache.amoro; -import org.apache.amoro.shade.guava32.com.google.common.collect.Maps; -import org.apache.amoro.shade.thrift.org.apache.commons.lang3.tuple.Pair; +import org.apache.amoro.shade.guava32.com.google.common.base.Preconditions; import java.util.Arrays; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; +import java.util.Objects; public final class Action { - private static final Map ACTIONS = Maps.newConcurrentMap(); - - private static final TableFormat[] DEFAULT_FORMATS = - new TableFormat[] {TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE}; - - static { - // default optimizing action - register(1, 10, "rewrite"); - // expire all metadata and data files necessarily. - register(4, 1, "expire-data"); - // delete orphan files - register(5, 2, "delete-orphans"); - // sync optimizing commit to hive - register(6, 3, "sync-hive"); - } + private static final int MAX_NAME_LENGTH = 16; /** supported table formats of this action */ private final TableFormat[] formats; - /** - * storage code of this action, normally this code should be identical within supported formats - */ - private final int code; + + private final String name; /** * the weight number of this action, the bigger the weight number, the higher positions of * schedulers or front pages */ private final int weight; - /** description of this action, will be shown in front pages */ - private final String desc; - private Action(TableFormat[] formats, int code, int weight, String desc) { + public Action(TableFormat[] formats, int weight, String name) { + Preconditions.checkArgument( + name.length() <= MAX_NAME_LENGTH, + "Action name length should be less than " + MAX_NAME_LENGTH); this.formats = formats; - this.code = code; - this.desc = desc; + this.name = name; this.weight = weight; } - public static Action valueOf(int code) { - return ACTIONS.get(code); - } - - public static synchronized void register(int code, int weight, String desc) { - register(DEFAULT_FORMATS, code, weight, desc); - } - - public static synchronized void register( - TableFormat[] formats, int code, int weight, String desc) { - Map> format2Actions = buildMapFromActions(); - for (TableFormat format : formats) { - if (format2Actions.get(format).contains(desc)) { - throw new IllegalArgumentException("Duplicated action: " + desc + " in format: " + format); - } - } - if (ACTIONS.put(code, new Action(formats, code, weight, desc)) != null) { - throw new IllegalArgumentException("Duplicated action code: " + code); - } - } - - private static Map> buildMapFromActions() { - return ACTIONS.values().stream() - .flatMap( - action -> Arrays.stream(action.formats).map(format -> Pair.of(format, action.desc))) - .collect( - Collectors.groupingBy( - Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toSet()))); - } - - public int getCode() { - return code; - } - - public String getDesc() { - return desc; - } - public int getWeight() { return weight; } @@ -111,6 +54,10 @@ public TableFormat[] supportedFormats() { return formats; } + public String getName() { + return name; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -120,11 +67,13 @@ public boolean equals(Object o) { return false; } Action action = (Action) o; - return code == action.code; + return Objects.equals(name, action.name) && Arrays.equals(formats, action.formats); } @Override public int hashCode() { - return code; + int result = Objects.hash(name); + result = 31 * result + Arrays.hashCode(formats); + return result; } } diff --git a/amoro-common/src/main/java/org/apache/amoro/IcebergActions.java b/amoro-common/src/main/java/org/apache/amoro/IcebergActions.java new file mode 100644 index 0000000000..76f470d98c --- /dev/null +++ b/amoro-common/src/main/java/org/apache/amoro/IcebergActions.java @@ -0,0 +1,31 @@ +/* + * 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; + +public class IcebergActions { + + private static final TableFormat[] DEFAULT_FORMATS = + new TableFormat[] {TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE}; + + public static final Action SYSTEM = new Action(DEFAULT_FORMATS, 0, "system"); + public static final Action REWRITE = new Action(DEFAULT_FORMATS, 10, "rewrite"); + public static final Action DELETE_ORPHANS = new Action(DEFAULT_FORMATS, 2, "delete-orphans"); + public static final Action SYNC_HIVE = new Action(DEFAULT_FORMATS, 3, "sync-hive"); + public static final Action EXPIRE_DATA = new Action(DEFAULT_FORMATS, 1, "expire-data"); +} diff --git a/amoro-common/src/main/java/org/apache/amoro/SupportsProcessPlugins.java b/amoro-common/src/main/java/org/apache/amoro/SupportsProcessPlugins.java new file mode 100644 index 0000000000..f15b6f4d81 --- /dev/null +++ b/amoro-common/src/main/java/org/apache/amoro/SupportsProcessPlugins.java @@ -0,0 +1,50 @@ +/* + * 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; + +import org.apache.amoro.process.AmoroProcess; +import org.apache.amoro.process.ProcessFactory; +import org.apache.amoro.process.TableProcessState; + +public interface SupportsProcessPlugins { + + /** + * Trigger a table process on an action. + * + * @param action + * @return created table process + */ + AmoroProcess trigger(Action action); + + /** + * Install a process factory for an action. + * + * @param action action type + * @param processFactory process factory + */ + void install(Action action, ProcessFactory processFactory); + + /** + * Check if an action is enabled. + * + * @param action + * @return true if the action is enabled + */ + boolean enabled(Action action); +} diff --git a/amoro-common/src/main/java/org/apache/amoro/TableRuntime.java b/amoro-common/src/main/java/org/apache/amoro/TableRuntime.java index b0a3636696..6509843efc 100644 --- a/amoro-common/src/main/java/org/apache/amoro/TableRuntime.java +++ b/amoro-common/src/main/java/org/apache/amoro/TableRuntime.java @@ -19,7 +19,6 @@ package org.apache.amoro; import org.apache.amoro.config.TableConfiguration; -import org.apache.amoro.process.OptimizingState; import org.apache.amoro.process.ProcessFactory; import org.apache.amoro.process.TableProcessState; @@ -33,22 +32,20 @@ public interface TableRuntime { /** - * Get the list of optimizing process states. Normally, the list contains one default optimizing - * state at least. There could be more than one states if multiple optimizing processes are - * running. + * Get the list of process states. which belong to all running table processes. There could be + * more than one external process states depending on scheduler implementation. * - * @return the list of optimizing process states + * @return the list of arbitrary process states */ - List getOptimizingStates(); + List getProcessStates(); /** - * Get the list of arbitrary process states. One arbitrary state belongs to one arbitrary process - * related to one {@link Action#ARBITRARY_ACTIONS}. There could be more than one arbitrary states - * depending on scheduler implementation. + * Get the list of process states. which belong to all running table processes. There could be + * more than one external process states depending on scheduler implementation. * * @return the list of arbitrary process states */ - List getArbitraryStates(); + List getProcessStates(Action action); /** * Get the table identifier containing server side id and table format. @@ -63,4 +60,9 @@ public interface TableRuntime { * @return the table configuration */ TableConfiguration getTableConfiguration(); + + /** Get table format */ + default TableFormat getFormat() { + return getTableIdentifier().getFormat(); + } } diff --git a/amoro-common/src/main/java/org/apache/amoro/hive/AuthenticatedHiveClientPool.java b/amoro-common/src/main/java/org/apache/amoro/hive/AuthenticatedHiveClientPool.java index 6c5c0107b8..bdfbb00a4e 100644 --- a/amoro-common/src/main/java/org/apache/amoro/hive/AuthenticatedHiveClientPool.java +++ b/amoro-common/src/main/java/org/apache/amoro/hive/AuthenticatedHiveClientPool.java @@ -65,6 +65,7 @@ public static HMSClient createHiveMetaStoreClient(HiveConf hiveConf) { } catch (MetaException e) { throw new RuntimeException("Failed to connect to Hive Metastore", e); } catch (Throwable t) { + LOG.error("Failed to call createHiveMetaStoreClient", t); if (t.getMessage().contains("Another instance of Derby may have already booted")) { throw new RuntimeException( "Failed to start an embedded metastore because embedded " diff --git a/amoro-common/src/main/java/org/apache/amoro/process/AmoroProcess.java b/amoro-common/src/main/java/org/apache/amoro/process/AmoroProcess.java index 46e3048760..3bdf65dfd2 100644 --- a/amoro-common/src/main/java/org/apache/amoro/process/AmoroProcess.java +++ b/amoro-common/src/main/java/org/apache/amoro/process/AmoroProcess.java @@ -32,13 +32,6 @@ */ public interface AmoroProcess { - /** - * Submit the process to Amoro. The process will be handled by Amoro after submitted. If the - * process is already submitted, this method will do nothing. For external optimizing, the process - * will be submitted to external resources like Yarn. - */ - void submit(); - /** * return submit future of the process. This method always returns the same future object even if * submit() has not been called @@ -55,12 +48,6 @@ public interface AmoroProcess { */ SimpleFuture getCompleteFuture(); - /** - * Cancel and close this process, related resources will be released. This method will block until - * getStatus() return CLOSED, but related resource could be released later. - */ - void close(); - /** * Get {@link ProcessState} of the process * @@ -87,15 +74,6 @@ default ProcessStatus getStatus() { return getState().getStatus(); } - /** - * Check if the process is closed - * - * @return true if the process is closed, false otherwise - */ - default boolean isClosed() { - return getStatus() == ProcessStatus.CLOSED; - } - /** * Get the id of the process * diff --git a/amoro-common/src/main/java/org/apache/amoro/process/ManagedProcess.java b/amoro-common/src/main/java/org/apache/amoro/process/ManagedProcess.java new file mode 100644 index 0000000000..6a9fde9f62 --- /dev/null +++ b/amoro-common/src/main/java/org/apache/amoro/process/ManagedProcess.java @@ -0,0 +1,42 @@ +/* + * 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; + +/** + * ManagedProcess is the interface for AMS modules to manage process. + * + * @param + */ +public interface ManagedProcess extends AmoroProcess { + + /** Submit this process to ResourceContainer. */ + void submit(); + + /** Mark this process as completed, trigger callbacks */ + void complete(); + + /** Mark this process as failed, trigger callbacks */ + void complete(String failedReason); + + /** Reset process status and re-submit it */ + void retry(); + + /** Kill this process */ + void kill(); +} diff --git a/amoro-common/src/main/java/org/apache/amoro/process/ProcessStatus.java b/amoro-common/src/main/java/org/apache/amoro/process/ProcessStatus.java index 43f9471b5e..8d41edeb18 100644 --- a/amoro-common/src/main/java/org/apache/amoro/process/ProcessStatus.java +++ b/amoro-common/src/main/java/org/apache/amoro/process/ProcessStatus.java @@ -26,6 +26,7 @@ public enum ProcessStatus { RUNNING, SUCCESS, CLOSED, + KILLED, FAILED; public ProcessStage toStage() { diff --git a/amoro-common/src/main/java/org/apache/amoro/process/TableProcess.java b/amoro-common/src/main/java/org/apache/amoro/process/TableProcess.java index f9c32e3317..3882564bc8 100644 --- a/amoro-common/src/main/java/org/apache/amoro/process/TableProcess.java +++ b/amoro-common/src/main/java/org/apache/amoro/process/TableProcess.java @@ -31,35 +31,10 @@ public abstract class TableProcess implements Amoro protected final TableRuntime tableRuntime; private final SimpleFuture submitFuture = new SimpleFuture(); private final SimpleFuture completeFuture = new SimpleFuture(); - private volatile ProcessStatus status = ProcessStatus.RUNNING; - private volatile String failedReason; protected TableProcess(T state, TableRuntime tableRuntime) { this.state = state; this.tableRuntime = tableRuntime; - this.completeFuture.whenCompleted( - () -> { - if (status == ProcessStatus.FAILED) { - state.setFailedReason(failedReason); - } else { - state.setStatus(status); - } - }); - } - - protected void completeSubmitting() { - submitFuture.complete(); - } - - protected void complete() { - status = ProcessStatus.SUCCESS; - completeFuture.complete(); - } - - protected void complete(String errorMessage) { - status = ProcessStatus.FAILED; - failedReason = errorMessage; - completeFuture.complete(); } public TableRuntime getTableRuntime() { @@ -71,16 +46,9 @@ public T getState() { return state; } - @Override - public void close() { - closeInternal(); - status = ProcessStatus.CLOSED; - completeFuture.complete(); - } - @Override public ProcessStatus getStatus() { - return status; + return state.getStatus(); } protected abstract void closeInternal(); diff --git a/amoro-common/src/main/java/org/apache/amoro/process/TableProcessState.java b/amoro-common/src/main/java/org/apache/amoro/process/TableProcessState.java index 335ab82b5f..9c97c9d25e 100644 --- a/amoro-common/src/main/java/org/apache/amoro/process/TableProcessState.java +++ b/amoro-common/src/main/java/org/apache/amoro/process/TableProcessState.java @@ -30,6 +30,7 @@ public class TableProcessState implements ProcessState { @StateField private volatile long id; private final Action action; private final ServerTableIdentifier tableIdentifier; + @StateField private int retryNumber; @StateField private long startTime; @StateField private long endTime = -1L; @StateField private ProcessStatus status = ProcessStatus.SUBMITTED; @@ -53,7 +54,7 @@ public long getId() { } public String getName() { - return action.getDesc(); + return action.getName(); } public Action getAction() { @@ -106,7 +107,7 @@ protected void setStartTime(long startTime) { protected void setStatus(ProcessStatus status) { if (status == ProcessStatus.SUCCESS || status == ProcessStatus.FAILED - || status == ProcessStatus.CLOSED) { + || status == ProcessStatus.KILLED) { endTime = System.currentTimeMillis(); } else if (this.status != ProcessStatus.SUBMITTED && status == ProcessStatus.SUBMITTED) { endTime = -1L; @@ -120,12 +121,6 @@ public String getFailedReason() { return failedReason; } - protected void setFailedReason(String failedReason) { - this.status = ProcessStatus.FAILED; - this.failedReason = failedReason; - this.endTime = System.currentTimeMillis(); - } - public ProcessStage getStage() { return status.toStage(); } @@ -133,4 +128,35 @@ public ProcessStage getStage() { protected void setId(long processId) { this.id = processId; } + + public void setSubmitted() { + this.status = ProcessStatus.SUBMITTED; + this.startTime = System.currentTimeMillis(); + } + + public void addRetryNumber() { + this.retryNumber += 1; + this.status = ProcessStatus.PENDING; + this.failedReason = null; + } + + public void setCompleted() { + this.status = ProcessStatus.SUCCESS; + this.endTime = System.currentTimeMillis(); + } + + public void setKilled() { + this.status = ProcessStatus.KILLED; + this.endTime = System.currentTimeMillis(); + } + + public void setCompleted(String failedReason) { + this.status = ProcessStatus.FAILED; + this.failedReason = failedReason; + this.endTime = System.currentTimeMillis(); + } + + public int getRetryNumber() { + return retryNumber; + } } diff --git a/amoro-common/src/main/java/org/apache/amoro/resource/ExternalResourceContainer.java b/amoro-common/src/main/java/org/apache/amoro/resource/ExternalResourceContainer.java new file mode 100644 index 0000000000..da57eacfb2 --- /dev/null +++ b/amoro-common/src/main/java/org/apache/amoro/resource/ExternalResourceContainer.java @@ -0,0 +1,45 @@ +/* + * 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.resource; + +import org.apache.amoro.process.AmoroProcess; +import org.apache.amoro.process.TableProcessState; + +/** + * ExternalResourceContainer is the key interface for the AMS framework to interact with the + * external resource. Typically, it is used to submit and release resource on YARN/K8S. + */ +public interface ExternalResourceContainer extends ResourceContainer { + + /** + * Submit a process and get the related resource, when the process is completed, the resource will + * be automatically released. + * + * @param process + * @return the resource + */ + Resource submit(AmoroProcess process); + + /** + * Release a resource by resource id manually, usually means a killing operation + * + * @param resourceId + */ + void release(String resourceId); +} diff --git a/amoro-common/src/main/java/org/apache/amoro/resource/InternalResourceContainer.java b/amoro-common/src/main/java/org/apache/amoro/resource/InternalResourceContainer.java new file mode 100644 index 0000000000..1cc5a8b478 --- /dev/null +++ b/amoro-common/src/main/java/org/apache/amoro/resource/InternalResourceContainer.java @@ -0,0 +1,43 @@ +/* + * 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.resource; + +/** + * The interface of internal resource container For {@link InternalResourceContainer}, resources are + * managed by AMS, and resources are decoupled from processes, which means one process could run any + * number of resources and a resource could run any number of processes. For {@link + * ExternalResourceContainer} resources are managed outside(by some JobManager or application + * master), resources and processes are one-to-one mapping. + */ +public interface InternalResourceContainer extends ResourceContainer { + + /** + * Start a new optimizer. + * + * @param resource resource information to start the optimizer + */ + void requestResource(Resource resource); + + /** + * Release a optimizer + * + * @param resource resource information to release the optimizer + */ + void releaseResource(Resource resource); +} diff --git a/amoro-common/src/main/java/org/apache/amoro/resource/ResourceContainer.java b/amoro-common/src/main/java/org/apache/amoro/resource/ResourceContainer.java index c2a7dac2bb..b0324d53a6 100644 --- a/amoro-common/src/main/java/org/apache/amoro/resource/ResourceContainer.java +++ b/amoro-common/src/main/java/org/apache/amoro/resource/ResourceContainer.java @@ -32,18 +32,10 @@ public interface ResourceContainer { void init(String name, Map containerProperties); /** - * Start a new optimizer. + * Get resource status * - * @param resource resource information to start the optimizer + * @param resourceId + * @return resource status */ - void requestResource(Resource resource); - - /** - * Release a optimizer - * - * @param resource resource information to release the optimizer - */ - void releaseOptimizer(Resource resource); - ResourceStatus getStatus(String resourceId); }