From a5ed5ad69fc40718bc648b8626624f22b7677b83 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Tue, 31 Oct 2023 19:24:25 +0800 Subject: [PATCH 01/23] [AMORO-1951] Support parallelized planning in one optimizer group --- .../ams/api/resource/ResourceManager.java | 2 + .../arctic/server/ArcticManagementConf.java | 6 + .../server/DefaultOptimizingService.java | 227 +++++++--- .../server/optimizing/OptimizingQueue.java | 420 +++++++----------- .../server/optimizing/SchedulingPolicy.java | 28 +- .../arctic/server/optimizing/TaskRuntime.java | 64 ++- .../persistence/mapper/OptimizingMapper.java | 2 +- .../server/resource/OptimizerInstance.java | 12 +- .../server/resource/OptimizerThread.java | 42 ++ .../arctic/server/resource/QuotaProvider.java | 9 + .../server/table/DefaultTableService.java | 7 +- .../server/TestDefaultOptimizingService.java | 272 ++++++++++++ .../optimizing/TestOptimizingQueue.java | 391 ++++------------ .../server/table/TableServiceTestBase.java | 15 + 14 files changed, 828 insertions(+), 669 deletions(-) create mode 100644 ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java create mode 100644 ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java diff --git a/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java b/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java index 2520b153f4..cd3af0eefb 100644 --- a/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java +++ b/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java @@ -23,4 +23,6 @@ public interface ResourceManager { List listResourcesByGroup(String groupName); Resource getResource(String resourceId); + + void dispose(); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 6ca9f21e04..8f1a560df0 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -218,6 +218,12 @@ public class ArcticManagementConf { .defaultValue(30000L) .withDescription("Timeout duration for task acknowledgment."); + public static final ConfigOption OPTIMIZER_MAX_PLANNING_PARALLELISM = + ConfigOptions.key("optimizer.task-ack-timeout") + .intType() + .defaultValue(1) + .withDescription("Max planning parallelism in one optimizer group."); + /** * config key prefix of terminal */ diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 5725c54fe9..4791c5d5e7 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -29,25 +29,30 @@ import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import com.netease.arctic.ams.api.resource.Resource; import com.netease.arctic.ams.api.resource.ResourceGroup; -import com.netease.arctic.ams.api.resource.ResourceManager; import com.netease.arctic.server.exception.ObjectNotExistsException; import com.netease.arctic.server.exception.PluginRetryAuthException; +import com.netease.arctic.server.exception.TaskNotFoundException; import com.netease.arctic.server.optimizing.OptimizingQueue; import com.netease.arctic.server.optimizing.OptimizingStatus; +import com.netease.arctic.server.optimizing.TaskRuntime; import com.netease.arctic.server.persistence.StatedPersistentBase; import com.netease.arctic.server.persistence.mapper.OptimizerMapper; import com.netease.arctic.server.persistence.mapper.ResourceMapper; import com.netease.arctic.server.resource.OptimizerInstance; import com.netease.arctic.server.resource.OptimizerManager; +import com.netease.arctic.server.resource.OptimizerThread; +import com.netease.arctic.server.resource.QuotaProvider; import com.netease.arctic.server.table.DefaultTableService; import com.netease.arctic.server.table.RuntimeHandlerChain; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.TableRuntimeMeta; +import com.netease.arctic.server.table.TableService; import com.netease.arctic.server.utils.Configurations; import com.netease.arctic.table.TableProperties; -import org.apache.commons.collections.CollectionUtils; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,9 +61,14 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; /** @@ -70,97 +80,152 @@ * The code also includes a TimerTask for detecting and removing expired optimizers and suspending tasks. */ public class DefaultOptimizingService extends StatedPersistentBase implements OptimizingService.Iface, - OptimizerManager, ResourceManager { + OptimizerManager, QuotaProvider { private static final Logger LOG = LoggerFactory.getLogger(DefaultOptimizingService.class); + private static final long POLLING_TIMEOUT_MS = 10000; private final long optimizerTouchTimeout; private final long taskAckTimeout; - @StatedPersistentBase.StateField + private final int maxPlanningParallelism; private final Map optimizingQueueByGroup = new ConcurrentHashMap<>(); private final Map optimizingQueueByToken = new ConcurrentHashMap<>(); - private final DefaultTableService tableManager; + private final Map authOptimizers = new ConcurrentHashMap<>(); + private final TableService tableService; private final RuntimeHandlerChain tableHandlerChain; + private final Executor planExecutor; private Timer optimizerMonitorTimer; public DefaultOptimizingService(Configurations serviceConfig, DefaultTableService tableService) { this.optimizerTouchTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_HB_TIMEOUT); this.taskAckTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_TASK_ACK_TIMEOUT); - this.tableManager = tableService; + this.maxPlanningParallelism = serviceConfig.getInteger(ArcticManagementConf.OPTIMIZER_MAX_PLANNING_PARALLELISM); + this.tableService = tableService; this.tableHandlerChain = new TableRuntimeHandlerImpl(); + this.planExecutor = Executors.newCachedThreadPool(new ThreadFactory() { + private final AtomicInteger threadId = new AtomicInteger(0); + @Override + public Thread newThread(@NotNull Runnable r) { + Thread thread = new Thread(r, "plan-executor-thread-" + threadId.incrementAndGet()); + thread.setDaemon(true); + return thread; + } + }); } public RuntimeHandlerChain getTableRuntimeHandler() { return tableHandlerChain; } - //TODO optimizing code - public void loadOptimizingQueues(List tableRuntimeMetaList) { + private void loadOptimizingQueues(List tableRuntimeMetaList) { List optimizerGroups = getAs(ResourceMapper.class, ResourceMapper::selectResourceGroups); List optimizers = getAs(OptimizerMapper.class, OptimizerMapper::selectAll); - Map> optimizersByGroup = - optimizers.stream().collect(Collectors.groupingBy(OptimizerInstance::getGroupName)); Map> groupToTableRuntimes = tableRuntimeMetaList.stream() .collect(Collectors.groupingBy(TableRuntimeMeta::getOptimizerGroup)); optimizerGroups.forEach(group -> { String groupName = group.getName(); List tableRuntimeMetas = groupToTableRuntimes.remove(groupName); - List optimizersUnderGroup = optimizersByGroup.get(groupName); - OptimizingQueue optimizingQueue = new OptimizingQueue(tableManager, group, + OptimizingQueue optimizingQueue = new OptimizingQueue( + tableService, + group, + this, + planExecutor, Optional.ofNullable(tableRuntimeMetas).orElseGet(ArrayList::new), - Optional.ofNullable(optimizersUnderGroup).orElseGet(ArrayList::new), - optimizerTouchTimeout, taskAckTimeout); + maxPlanningParallelism); optimizingQueueByGroup.put(groupName, optimizingQueue); - if (CollectionUtils.isNotEmpty(optimizersUnderGroup)) { - optimizersUnderGroup.forEach(optimizer -> optimizingQueueByToken.put(optimizer.getToken(), optimizingQueue)); - } }); + optimizers.forEach(this::registerOptimizer); groupToTableRuntimes.keySet().forEach(groupName -> LOG.warn("Unloaded task runtime in group " + groupName)); } + private void registerOptimizer(OptimizerInstance optimizer) { + String token = optimizer.getToken(); + authOptimizers.put(token, optimizer); + optimizingQueueByToken.put(token, optimizingQueueByGroup.get(optimizer.getGroupName())); + } + + private void unregisterOptimizer(String token) { + optimizingQueueByToken.remove(token); + authOptimizers.remove(token); + } + @Override public void ping() { } + public List listTasks(String optimizerGroup) { + return getQueueByGroup(optimizerGroup).collectTasks(); + } + @Override public void touch(String authToken) { - LOG.debug("Optimizer {} touching", authToken); - OptimizingQueue queue = getQueueByToken(authToken); - queue.touch(authToken); + OptimizerInstance optimizer = getAuthenticatedOptimizer(authToken).touch(); + if (LOG.isDebugEnabled()) { + LOG.debug("Optimizer {} touch time: {}", optimizer.getToken(), optimizer.getTouchTime()); + } + doAs(OptimizerMapper.class, mapper -> mapper.updateTouchTime(optimizer.getToken())); + } + + private OptimizerInstance getAuthenticatedOptimizer(String authToken) { + org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument(authToken != null, + "authToken can not be null"); + return Optional.ofNullable(authOptimizers.get(authToken)) + .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); } @Override public OptimizingTask pollTask(String authToken, int threadId) { - LOG.debug("Optimizer {} (threadId {}) try polling task", authToken, threadId); + if (LOG.isDebugEnabled()) { + LOG.debug("Optimizer {} (threadId {}) try polling task", authToken, threadId); + } OptimizingQueue queue = getQueueByToken(authToken); - OptimizingTask task = queue.pollTask(authToken, threadId); - if (task != null) { - LOG.info("Optimizer {} (threadId {}) polled task {}", authToken, threadId, task.getTaskId()); + return Optional.ofNullable(queue.pollTask(POLLING_TIMEOUT_MS)) + .map(task -> extractOptimizingTask(task, + getAuthenticatedOptimizer(authToken).getThread(threadId), queue)) + .orElse(null); + } + + private OptimizingTask extractOptimizingTask(TaskRuntime task, + OptimizerThread optimizerThread, + OptimizingQueue queue) { + try { + task.schedule(optimizerThread); + LOG.info("OptimizerThread {} polled task {}", optimizerThread, task.getTaskId()); + return task.getOptimizingTask(); + } catch (Throwable throwable) { + LOG.error("Schedule task {} failed, put it to retry queue", task.getTaskId(), throwable); + queue.retryTask(task, false); + return null; } - return task; } @Override public void ackTask(String authToken, int threadId, OptimizingTaskId taskId) { LOG.info("Ack task {} by optimizer {} (threadId {})", taskId, authToken, threadId); OptimizingQueue queue = getQueueByToken(authToken); - queue.ackTask(authToken, threadId, taskId); + Optional.ofNullable(queue.getTask(taskId)) + .orElseThrow(() -> new TaskNotFoundException(taskId)) + .ack(getAuthenticatedOptimizer(authToken).getThread(threadId)); } @Override public void completeTask(String authToken, OptimizingTaskResult taskResult) { LOG.info("Optimizer {} complete task {}", authToken, taskResult.getTaskId()); OptimizingQueue queue = getQueueByToken(authToken); - queue.completeTask(authToken, taskResult); + OptimizerThread thread = getAuthenticatedOptimizer(authToken).getThread(taskResult.getThreadId()); + Optional.ofNullable(queue.getTask(taskResult.getTaskId())) + .orElseThrow(() -> new TaskNotFoundException(taskResult.getTaskId())) + .complete(thread, taskResult); } @Override public String authenticate(OptimizerRegisterInfo registerInfo) { LOG.info("Register optimizer {}.", registerInfo); OptimizingQueue queue = getQueueByGroup(registerInfo.getGroupName()); - String token = queue.authenticate(registerInfo); - optimizingQueueByToken.put(token, queue); - return token; + OptimizerInstance optimizer = new OptimizerInstance(registerInfo, queue.getContainerName()); + doAs(OptimizerMapper.class, mapper -> mapper.insertOptimizer(optimizer)); + registerOptimizer(optimizer); + return optimizer.getToken(); } /** @@ -190,54 +255,47 @@ private OptimizingQueue getQueueByToken(String token) { @Override public List listOptimizers() { - return optimizingQueueByGroup.values() - .stream() - .flatMap(queue -> queue.getOptimizers().stream()) - .collect(Collectors.toList()); + return ImmutableList.copyOf(authOptimizers.values()); } @Override public List listOptimizers(String group) { - return getQueueByGroup(group).getOptimizers(); + return authOptimizers.values().stream() + .filter(optimizer -> optimizer.getGroupName().equals(group)) + .collect(Collectors.toList()); } @Override public void deleteOptimizer(String group, String resourceId) { - getQueueByGroup(group).removeOptimizer(resourceId); List deleteOptimizers = getAs(OptimizerMapper.class, mapper -> mapper.selectByResourceId(resourceId)); deleteOptimizers.forEach(optimizer -> { String token = optimizer.getToken(); - optimizingQueueByToken.remove(token); doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token)); + unregisterOptimizer(token); }); } @Override public void createResourceGroup(ResourceGroup resourceGroup) { - invokeConsisitency(() -> - doAsTransaction(() -> { - doAs(ResourceMapper.class, mapper -> mapper.insertResourceGroup(resourceGroup)); - String groupName = resourceGroup.getName(); - OptimizingQueue optimizingQueue = new OptimizingQueue( - tableManager, - resourceGroup, - new ArrayList<>(), - new ArrayList<>(), - optimizerTouchTimeout, - taskAckTimeout); - optimizingQueueByGroup.put(groupName, optimizingQueue); - }) - ); + doAsTransaction(() -> { + doAs(ResourceMapper.class, mapper -> mapper.insertResourceGroup(resourceGroup)); + OptimizingQueue optimizingQueue = new OptimizingQueue( + tableService, + resourceGroup, + this, + planExecutor, + new ArrayList<>(), + maxPlanningParallelism); + optimizingQueueByGroup.put(resourceGroup.getName(), optimizingQueue); + }); } @Override public void deleteResourceGroup(String groupName) { if (canDeleteResourceGroup(groupName)) { - invokeConsisitency(() -> { - optimizingQueueByGroup.remove(groupName); - doAs(ResourceMapper.class, mapper -> mapper.deleteResourceGroup(groupName)); - }); + doAs(ResourceMapper.class, mapper -> mapper.deleteResourceGroup(groupName)); + optimizingQueueByGroup.remove(groupName); } else { throw new RuntimeException(String.format("The resource group %s cannot be deleted because it is currently in " + "use.", groupName)); @@ -289,8 +347,16 @@ public Resource getResource(String resourceId) { return getAs(ResourceMapper.class, mapper -> mapper.selectResource(resourceId)); } + @Override + public void dispose() { + tableHandlerChain.dispose(); + optimizingQueueByGroup.clear(); + optimizingQueueByToken.clear(); + authOptimizers.clear(); + } + public boolean canDeleteResourceGroup(String name) { - for (CatalogMeta catalogMeta : tableManager.listCatalogMetas()) { + for (CatalogMeta catalogMeta : tableService.listCatalogMetas()) { if (catalogMeta.getCatalogProperties() != null && catalogMeta.getCatalogProperties() .getOrDefault( @@ -305,7 +371,7 @@ public boolean canDeleteResourceGroup(String name) { return false; } } - for (ServerTableIdentifier identifier : tableManager.listManagedTables()) { + for (ServerTableIdentifier identifier : tableService.listManagedTables()) { if (optimizingQueueByGroup.containsKey(name) && optimizingQueueByGroup.get(name).containsTable(identifier)) { return false; } @@ -313,6 +379,14 @@ public boolean canDeleteResourceGroup(String name) { return true; } + @Override + public int getTotalQuota(String resourceGroup) { + return authOptimizers.values().stream() + .filter(optimizer -> optimizer.getGroupName().equals(resourceGroup)) + .mapToInt(OptimizerInstance::getThreadCount) + .sum(); + } + private class TableRuntimeHandlerImpl extends RuntimeHandlerChain { @Override @@ -368,16 +442,43 @@ private class SuspendingDetector extends TimerTask { @Override public void run() { try { - optimizingQueueByGroup.values().forEach(optimizingQueue -> { - List expiredOptimizers = optimizingQueue.checkSuspending(); - expiredOptimizers.forEach(optimizerToken -> { - doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(optimizerToken)); - optimizingQueueByToken.remove(optimizerToken); - }); + long currentTime = System.currentTimeMillis(); + Set expiredTokens = authOptimizers.values().stream() + .filter(optimizer -> currentTime - optimizer.getTouchTime() > optimizerTouchTimeout) + .map(OptimizerInstance::getToken) + .collect(Collectors.toSet()); + + expiredTokens.forEach(authOptimizers.keySet()::remove); + if (!expiredTokens.isEmpty()) { + LOG.info("Expired optimizers: {}", expiredTokens); + } + + for (OptimizingQueue queue : optimizingQueueByGroup.values()) { + queue.collectRunningTasks().stream() + .filter(task -> isTaskExpired(task, currentTime, expiredTokens, authOptimizers.keySet())) + .forEach(task -> { + LOG.info("Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", + task.getTaskId(), task.getResourceDesc()); + //optimizing task of suspending optimizer would not be counted for retrying + queue.retryTask(task, false); + }); + } + + expiredTokens.forEach(token -> { + doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token)); + unregisterOptimizer(token); }); } catch (RuntimeException e) { LOG.error("Update optimizer status abnormal failed. try next round", e); } } + + private boolean isTaskExpired(TaskRuntime task, long currentTime, + Set expiredTokens, Set authTokens) { + return task.getStatus() == TaskRuntime.Status.SCHEDULED + && currentTime - task.getStartTime() > taskAckTimeout || + expiredTokens.contains(task.getToken()) || + !authTokens.contains(task.getToken()); + } } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 0dd8ef83ba..61a9cc47e9 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -1,89 +1,85 @@ package com.netease.arctic.server.optimizing; -import com.google.common.base.Objects; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.BlockableOperation; -import com.netease.arctic.ams.api.OptimizerRegisterInfo; -import com.netease.arctic.ams.api.OptimizingService; -import com.netease.arctic.ams.api.OptimizingTask; import com.netease.arctic.ams.api.OptimizingTaskId; -import com.netease.arctic.ams.api.OptimizingTaskResult; -import com.netease.arctic.ams.api.resource.Resource; import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.optimizing.RewriteFilesInput; import com.netease.arctic.server.ArcticServiceConstants; import com.netease.arctic.server.exception.OptimizingClosedException; -import com.netease.arctic.server.exception.PluginRetryAuthException; -import com.netease.arctic.server.exception.TaskNotFoundException; import com.netease.arctic.server.optimizing.plan.OptimizingPlanner; import com.netease.arctic.server.optimizing.plan.TaskDescriptor; import com.netease.arctic.server.persistence.PersistentBase; import com.netease.arctic.server.persistence.TaskFilesPersistence; -import com.netease.arctic.server.persistence.mapper.OptimizerMapper; import com.netease.arctic.server.persistence.mapper.OptimizingMapper; -import com.netease.arctic.server.resource.OptimizerInstance; +import com.netease.arctic.server.resource.QuotaProvider; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.TableRuntimeMeta; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.utils.ArcticDataFiles; import com.netease.arctic.utils.ExceptionUtil; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.LinkedTransferQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class OptimizingQueue extends PersistentBase implements OptimizingService.Iface { +public class OptimizingQueue extends PersistentBase { private static final Logger LOG = LoggerFactory.getLogger(OptimizingQueue.class); - private final long optimizerTouchTimeout; - private final long taskAckTimeout; - private final Lock planLock = new ReentrantLock(); - private ResourceGroup optimizerGroup; - private final Queue taskQueue = new LinkedTransferQueue<>(); - private final Queue retryQueue = new LinkedTransferQueue<>(); - private final SchedulingPolicy schedulingPolicy; - // keeps the SCHEDULED and ACKED tasks - private final Map executingTaskMap = new ConcurrentHashMap<>(); - private final Map authOptimizers = new ConcurrentHashMap<>(); + private static final long TABLE_PLANNING_MIN_INTERVAL = 60 * 1000L; + private final QuotaProvider quotaProvider; + private final Queue tableQueue = new LinkedTransferQueue<>(); + private final Queue retryTaskQueue = new LinkedTransferQueue<>(); + private final SchedulingPolicy schedulingPolicy; private final TableManager tableManager; + private final Executor planExecutor; + private final Map plannedKeepingTables = new HashMap<>(); + private final Set planningTables = new HashSet<>(); + private final Lock scheduleLock = new ReentrantLock(); + private final Condition planningCompleted = scheduleLock.newCondition(); + private final int maxPlanningParallelism; + private ResourceGroup optimizerGroup; public OptimizingQueue( TableManager tableManager, ResourceGroup optimizerGroup, + QuotaProvider quotaProvider, + Executor planExecutor, List tableRuntimeMetaList, - List authOptimizers, - long optimizerTouchTimeout, - long taskAckTimeout) { - Preconditions.checkNotNull(optimizerGroup, "optimizerGroup can not be null"); - this.optimizerTouchTimeout = optimizerTouchTimeout; - this.taskAckTimeout = taskAckTimeout; + int maxPlanningParallelism) { + Preconditions.checkNotNull(optimizerGroup, "Optimizer group can not be null"); + this.planExecutor = planExecutor; this.optimizerGroup = optimizerGroup; + this.quotaProvider = quotaProvider; this.schedulingPolicy = new SchedulingPolicy(optimizerGroup); this.tableManager = tableManager; - this.authOptimizers.putAll(authOptimizers.stream().collect(Collectors.toMap( - OptimizerInstance::getToken, optimizer -> optimizer))); + this.maxPlanningParallelism = maxPlanningParallelism; tableRuntimeMetaList.forEach(this::initTableRuntime); } @@ -101,14 +97,7 @@ private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING) { schedulingPolicy.addTable(tableRuntime); } else if (tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { - TableOptimizingProcess process = new TableOptimizingProcess(tableRuntimeMeta); - process.getTaskMap().entrySet().stream().filter( - entry -> entry.getValue().getStatus() == TaskRuntime.Status.SCHEDULED || - entry.getValue().getStatus() == TaskRuntime.Status.ACKED) - .forEach(entry -> executingTaskMap.put(entry.getKey(), entry.getValue())); - process.getTaskMap().values().stream() - .filter(task -> task.getStatus() == TaskRuntime.Status.PLANNED) - .forEach(taskQueue::offer); + tableQueue.offer(new TableOptimizingProcess(tableRuntimeMeta)); } } else { OptimizingProcess process = tableRuntime.getOptimizingProcess(); @@ -118,6 +107,10 @@ private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { } } + public String getContainerName() { + return optimizerGroup.getContainer(); + } + public void refreshTable(TableRuntime tableRuntime) { if (tableRuntime.isOptimizingEnabled() && !tableRuntime.getOptimizingStatus().isProcessing()) { LOG.info("Bind queue {} success with table {}", optimizerGroup.getName(), tableRuntime.getTableIdentifier()); @@ -135,140 +128,57 @@ public boolean containsTable(ServerTableIdentifier identifier) { return this.schedulingPolicy.containsTable(identifier); } - public List getOptimizers() { - return ImmutableList.copyOf(authOptimizers.values()); - } - - public void removeOptimizer(String resourceId) { - authOptimizers.entrySet().removeIf(op -> op.getValue().getResourceId().equals(resourceId)); - } - - private void clearTasks(TableOptimizingProcess optimizingProcess) { - retryQueue.removeIf(taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); - taskQueue.removeIf(taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); - executingTaskMap.entrySet().removeIf(entry -> entry.getValue().getProcessId() == optimizingProcess.getProcessId()); - } - - @Override - public void ping() { + private void clearProcess(TableOptimizingProcess optimizingProcess) { + tableQueue.removeIf(process -> process.getProcessId() == optimizingProcess.getProcessId()); + retryTaskQueue.removeIf(taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); } - @Override - public void touch(String authToken) { - OptimizerInstance optimizer = getAuthenticatedOptimizer(authToken).touch(); - LOG.debug("Optimizer {} touch time: {}", optimizer.getToken(), optimizer.getTouchTime()); - doAs(OptimizerMapper.class, mapper -> mapper.updateTouchTime(optimizer.getToken())); - } - - private OptimizerInstance getAuthenticatedOptimizer(String authToken) { - Preconditions.checkArgument(authToken != null, "authToken can not be null"); - return Optional.ofNullable(authOptimizers.get(authToken)) - .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); - } - - @Override - public OptimizingTask pollTask(String authToken, int threadId) { - getAuthenticatedOptimizer(authToken); - TaskRuntime task = Optional.ofNullable(retryQueue.poll()) - .orElseGet(this::pollOrPlan); - - if (task != null) { - safelySchedule(task, new OptimizingThread(authToken, threadId)); - executingTaskMap.putIfAbsent(task.getTaskId(), task); + public TaskRuntime pollTask(long maxWaitTime) { + long deadline = calculateDeadline(maxWaitTime); + TaskRuntime task = fetchTask(); + while (task == null && waitTask(deadline)) { + task = fetchTask(); } - return task != null ? task.getOptimizingTask() : null; - } - - private void safelySchedule(TaskRuntime task, OptimizingThread thread) { - try { - task.schedule(thread); - } catch (Throwable throwable) { - LOG.error("Schedule task {} failed, put it to retry queue", task.getTaskId(), throwable); - retryTask(task, false); - throw throwable; - } - } - - private void retryTask(TaskRuntime taskRuntime, boolean incRetryCount) { - taskRuntime.reset(incRetryCount); - retryQueue.offer(taskRuntime); + return task; } - @Override - public void ackTask(String authToken, int threadId, OptimizingTaskId taskId) { - getAuthenticatedOptimizer(authToken); - Optional.ofNullable(executingTaskMap.get(taskId)) - .orElseThrow(() -> new TaskNotFoundException(taskId)) - .ack(new OptimizingThread(authToken, threadId)); + private long calculateDeadline(long maxWaitTime) { + long deadline = System.currentTimeMillis() + maxWaitTime; + return deadline <= 0 ? Long.MAX_VALUE : deadline; } - @Override - public void completeTask(String authToken, OptimizingTaskResult taskResult) { - getAuthenticatedOptimizer(authToken); - OptimizingThread thread = new OptimizingThread(authToken, taskResult.getThreadId()); - TaskRuntime task = executingTaskMap.remove(taskResult.getTaskId()); - try { - Optional.ofNullable(task) - .orElseThrow(() -> new TaskNotFoundException(taskResult.getTaskId())) - .complete(thread, taskResult); - } catch (Throwable t) { - if (task != null) { - executingTaskMap.put(taskResult.getTaskId(), task); - } - throw t; - } + private TaskRuntime fetchTask() { + return Optional.ofNullable(retryTaskQueue.poll()) + .orElse(Optional.ofNullable(tableQueue.peek()) + .map(TableOptimizingProcess::poll) + .orElse(null)); } - @Override - public String authenticate(OptimizerRegisterInfo registerInfo) { - OptimizerInstance optimizer = new OptimizerInstance(registerInfo, optimizerGroup.getContainer()); - LOG.debug("Register optimizer: {}", optimizer); - doAs(OptimizerMapper.class, mapper -> mapper.insertOptimizer(optimizer)); - authOptimizers.put(optimizer.getToken(), optimizer); - return optimizer.getToken(); + public TaskRuntime getTask(OptimizingTaskId taskId) { + return tableQueue.stream() + .filter(p -> p.getProcessId() == taskId.getProcessId()) + .findFirst() + .map(p -> p.getTaskMap().get(taskId)) + .orElse(null); } - public List checkSuspending() { - long currentTime = System.currentTimeMillis(); - List expiredOptimizers = authOptimizers.values().stream() - .filter(optimizer -> currentTime - optimizer.getTouchTime() > optimizerTouchTimeout) - .map(OptimizerInstance::getToken) + public List collectTasks() { + return tableQueue.stream() + .flatMap(p -> p.getTaskMap().values().stream()) .collect(Collectors.toList()); + } - expiredOptimizers.forEach(authOptimizers.keySet()::remove); - if (!expiredOptimizers.isEmpty()) { - LOG.info("Expired optimizers: {}", expiredOptimizers); - } - - List canceledTasks = executingTaskMap.values().stream() - .filter(task -> task.getStatus() == TaskRuntime.Status.CANCELED) + public List collectRunningTasks() { + return tableQueue.stream() + .flatMap(p -> p.getTaskMap().values().stream()) + .filter(task -> task.getStatus() == TaskRuntime.Status.ACKED || + task.getStatus() == TaskRuntime.Status.SCHEDULED) .collect(Collectors.toList()); - canceledTasks.forEach(task -> { - LOG.info("Task {} is canceled, remove it from executing task map", task.getTaskId()); - executingTaskMap.remove(task.getTaskId()); - }); + } - List suspendingTasks = executingTaskMap.values().stream() - .filter(task -> task.getStatus().equals(TaskRuntime.Status.SCHEDULED) || - task.getStatus().equals(TaskRuntime.Status.ACKED)) - .filter(task -> task.isSuspending(currentTime, taskAckTimeout) || - expiredOptimizers.contains(task.getOptimizingThread().getToken()) || - !authOptimizers.containsKey(task.getOptimizingThread().getToken())) - .collect(Collectors.toList()); - suspendingTasks.forEach(task -> { - LOG.info("Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", - task.getTaskId(), task.getOptimizingThread()); - executingTaskMap.remove(task.getTaskId()); - try { - //optimizing task of suspending optimizer would not be counted for retrying - retryTask(task, false); - } catch (Throwable t) { - LOG.error("Retry task {} failed, put it back to executing tasks", task.getTaskId(), t); - // retry next task, not throw exception - executingTaskMap.put(task.getTaskId(), task); - } - }); - return expiredOptimizers; + public void retryTask(TaskRuntime taskRuntime, boolean incRetryCount) { + taskRuntime.reset(incRetryCount); + retryTaskQueue.offer(taskRuntime); } public void updateOptimizerGroup(ResourceGroup optimizerGroup) { @@ -279,67 +189,97 @@ public void updateOptimizerGroup(ResourceGroup optimizerGroup) { schedulingPolicy.setTableSorterIfNeeded(optimizerGroup); } - @VisibleForTesting - Map getExecutingTaskMap() { - return executingTaskMap; - } - - private TaskRuntime pollOrPlan() { - planLock.lock(); + private boolean waitTask(long waitDeadline) { + scheduleLock.lock(); try { - if (taskQueue.isEmpty()) { - planTasks(); - } - return taskQueue.poll(); + long currentTime = System.currentTimeMillis(); + scheduleTableIfNecessary(currentTime); + return waitDeadline > currentTime && + planningCompleted.await(waitDeadline - currentTime, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.error("Schedule table interrupted", e); + return false; } finally { - planLock.unlock(); + scheduleLock.unlock(); } } - private void planTasks() { - long startTime = System.currentTimeMillis(); - List scheduledTables = schedulingPolicy.scheduleTables(); - LOG.debug("Calculating and sorting tables by quota : {}", scheduledTables); - - if (scheduledTables.size() <= 0) { - return; + private void scheduleTableIfNecessary(long startTime) { + if (planningTables.size() < maxPlanningParallelism) { + Set skipTables = new HashSet<>(planningTables); + plannedKeepingTables.entrySet().stream() + .filter(entry -> startTime - entry.getValue() < TABLE_PLANNING_MIN_INTERVAL) + .filter(entry -> isOptimizingBlocked(entry.getKey())) + .map(Map.Entry::getKey) + .forEach(skipTables::add); + Optional.ofNullable(schedulingPolicy.scheduleTable(skipTables)) + .ifPresent(tableRuntime -> triggerAsyncPlanning(tableRuntime, skipTables, startTime)); } - List plannedTables = Lists.newArrayList(); - for (TableRuntime tableRuntime : scheduledTables) { - LOG.debug("Planning table {}", tableRuntime.getTableIdentifier()); + } + + private void triggerAsyncPlanning(TableRuntime tableRuntime, Set skipTables, long startTime) { + LOG.info("Trigger planning table {}", tableRuntime.getTableIdentifier()); + planningTables.add(tableRuntime); + doPlanning(tableRuntime).whenComplete((process, throwable) -> { + long currentTime = System.currentTimeMillis(); + scheduleLock.lock(); try { - AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); - OptimizingPlanner planner = new OptimizingPlanner( - tableRuntime.refresh(table), - (ArcticTable) table.originalTable(), - getAvailableCore()); - if (tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) { - LOG.info("{} optimize is blocked, continue", tableRuntime.getTableIdentifier()); - continue; - } - plannedTables.add(table.id()); - if (planner.isNecessary()) { - TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); - LOG.info("{} after plan get {} tasks", tableRuntime.getTableIdentifier(), - optimizingProcess.getTaskMap().size()); - optimizingProcess.taskMap.values().forEach(taskQueue::offer); - break; + plannedKeepingTables.put(tableRuntime, System.currentTimeMillis()); + planningTables.remove(tableRuntime); + if (process != null) { + tableQueue.offer(process); + LOG.info("{} completes planning {} tasks with a total cost of {} ms, skipping tables, {}", + optimizerGroup.getName(), + process.getTaskMap().size(), + currentTime - startTime, + skipTables); } else { - tableRuntime.cleanPendingInput(); + if (throwable != null) { + LOG.error("Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); + } + LOG.info("{} skip planning table {} with a total cost of {} ms.", + optimizerGroup.getName(), + tableRuntime.getTableIdentifier(), + currentTime - startTime); } - } catch (Throwable e) { - LOG.error(tableRuntime.getTableIdentifier() + " plan failed, continue", e); + planningCompleted.signalAll(); + } finally { + scheduleLock.unlock(); + } + }); + } + + private CompletableFuture doPlanning(TableRuntime tableRuntime) { + CompletableFuture future = new CompletableFuture<>(); + planExecutor.execute(() -> { + AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + OptimizingPlanner planner = new OptimizingPlanner( + tableRuntime.refresh(table), + (ArcticTable) table.originalTable(), + getAvailableCore()); + if (planner.isNecessary()) { + TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); + future.complete(optimizingProcess); + } else { + tableRuntime.cleanPendingInput(); + future.complete(null); } + }); + return future; + } + + private boolean isOptimizingBlocked(TableRuntime tableRuntime) { + if (tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) { + LOG.info("{} optimize is blocked, continue", tableRuntime.getTableIdentifier()); + return false; + } else { + return true; } - long end = System.currentTimeMillis(); - LOG.info("{} completes planning tasks with a total cost of {} ms, which involves {}/{}(planned/pending) tables, {}", - optimizerGroup.getName(), end - startTime, plannedTables.size(), scheduledTables.size(), plannedTables); } private double getAvailableCore() { - int totalCore = authOptimizers.values().stream().mapToInt(Resource::getThreadCount).sum(); // the available core should be at least 1 - return Math.max(totalCore, 1); + return Math.max(quotaProvider.getTotalQuota(optimizerGroup.getName()), 1); } @VisibleForTesting @@ -355,16 +295,24 @@ private class TableOptimizingProcess implements OptimizingProcess, TaskRuntime.T private final long targetSnapshotId; private final long targetChangeSnapshotId; private final Map taskMap = Maps.newHashMap(); + private final Queue taskQueue = new LinkedList<>(); private final Lock lock = new ReentrantLock(); private volatile Status status = OptimizingProcess.Status.RUNNING; private volatile String failedReason; private long endTime = ArcticServiceConstants.INVALID_TIME; - private Map fromSequence = Maps.newHashMap(); private Map toSequence = Maps.newHashMap(); - private boolean hasCommitted = false; + public TaskRuntime poll() { + lock.lock(); + try { + return taskQueue.poll(); + } finally { + lock.unlock(); + } + } + public TableOptimizingProcess(OptimizingPlanner planner) { processId = planner.getProcessId(); tableRuntime = planner.getTableRuntime(); @@ -414,10 +362,10 @@ public Status getStatus() { public void close() { lock.lock(); try { - clearTasks(this); + persistProcessCompleted(false); + clearProcess(this); this.status = OptimizingProcess.Status.CLOSED; this.endTime = System.currentTimeMillis(); - persistProcessCompleted(false); } finally { lock.unlock(); } @@ -446,7 +394,7 @@ public void acceptResult(TaskRuntime taskRuntime) { if (taskRuntime.getRetry() <= tableRuntime.getMaxExecuteRetryCount()) { retryTask(taskRuntime, true); } else { - clearTasks(this); + clearProcess(this); this.failedReason = taskRuntime.getFailReason(); this.status = OptimizingProcess.Status.FAILED; this.endTime = taskRuntime.getEndTime(); @@ -545,6 +493,7 @@ public void commit() { endTime = System.currentTimeMillis(); persistProcessCompleted(false); } finally { + clearProcess(this); lock.unlock(); } } @@ -619,6 +568,7 @@ private void loadTaskRuntimes() { taskRuntime.claimOwnership(this); taskRuntime.setInput(inputs.get(taskRuntime.getTaskId().getTaskId())); taskMap.put(taskRuntime.getTaskId(), taskRuntime); + taskQueue.offer(taskRuntime); }); } @@ -630,54 +580,8 @@ private void loadTaskRuntimes(List taskDescriptors) { LOG.info("{} plan new task {}, summary {}", tableRuntime.getTableIdentifier(), taskRuntime.getTaskId(), taskRuntime.getSummary()); taskMap.put(taskRuntime.getTaskId(), taskRuntime.claimOwnership(this)); + taskQueue.offer(taskRuntime); } } } - - public static class OptimizingThread { - - private String token; - private int threadId; - - public OptimizingThread(String token, int threadId) { - this.token = token; - this.threadId = threadId; - } - - public OptimizingThread() { - } - - public String getToken() { - return token; - } - - public int getThreadId() { - return threadId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OptimizingThread that = (OptimizingThread) o; - return threadId == that.threadId && Objects.equal(token, that.token); - } - - @Override - public int hashCode() { - return Objects.hashCode(token, threadId); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("token", token) - .add("threadId", threadId) - .toString(); - } - } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 74b0b88d00..8801e66569 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -4,6 +4,7 @@ import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableRuntime; +import com.netease.arctic.server.table.TableRuntimeMeta; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import java.util.Comparator; @@ -11,15 +12,16 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; public class SchedulingPolicy { - private static final String SCHEDULING_POLICY_PROPERTY_NAME = "scheduling-policy"; - private static final String QUOTA = "quota"; - private static final String BALANCED = "balanced"; + protected static final String SCHEDULING_POLICY_PROPERTY_NAME = "scheduling-policy"; + protected static final String QUOTA = "quota"; + protected static final String BALANCED = "balanced"; private final Map tableRuntimeMap = new HashMap<>(); private Comparator tableSorter; @@ -46,20 +48,28 @@ public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { } } - public List scheduleTables() { + public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { return tableRuntimeMap.values().stream() - .filter(tableRuntime -> tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING && - (tableRuntime.getLastOptimizedSnapshotId() != tableRuntime.getCurrentSnapshotId() || - tableRuntime.getLastOptimizedChangeSnapshotId() != tableRuntime.getCurrentChangeSnapshotId())) - .sorted(tableSorter) - .collect(Collectors.toList()); + .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) + .min(tableSorter) + .orElse(null); } finally { tableLock.unlock(); } } + private boolean shouldSkip(Set skipSet, TableRuntime tableRuntime) { + return skipSet.contains(tableRuntime) || !isTablePending(tableRuntime); + } + + private boolean isTablePending(TableRuntime tableRuntime) { + return tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING && + (tableRuntime.getLastOptimizedSnapshotId() != tableRuntime.getCurrentSnapshotId() || + tableRuntime.getLastOptimizedChangeSnapshotId() != tableRuntime.getCurrentChangeSnapshotId()); + } + public void addTable(TableRuntime tableRuntime) { tableLock.lock(); try { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java index 5f5e0e13b7..11eba6b54e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java @@ -33,6 +33,7 @@ import com.netease.arctic.server.persistence.StatedPersistentBase; import com.netease.arctic.server.persistence.TaskFilesPersistence; import com.netease.arctic.server.persistence.mapper.OptimizingMapper; +import com.netease.arctic.server.resource.OptimizerThread; import com.netease.arctic.utils.SerializationUtil; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -56,7 +57,9 @@ public class TaskRuntime extends StatedPersistentBase { @StateField private long costTime = 0; @StateField - private OptimizingQueue.OptimizingThread optimizingThread; + private String token; + @StateField + private int threadId = -1; @StateField private String failReason; private TaskOwner owner; @@ -82,7 +85,7 @@ public TaskRuntime( this.properties = properties; } - public void complete(OptimizingQueue.OptimizingThread thread, OptimizingTaskResult result) { + public void complete(OptimizerThread thread, OptimizingTaskResult result) { invokeConsisitency(() -> { validThread(thread); if (result.getErrorMessage() != null) { @@ -91,7 +94,8 @@ public void complete(OptimizingQueue.OptimizingThread thread, OptimizingTaskResu finish(TaskFilesPersistence.loadTaskOutput(result.getTaskOutput())); } owner.acceptResult(this); - optimizingThread = null; + token = null; + threadId = -1; }); } @@ -131,16 +135,17 @@ void reset(boolean incRetryCount) { }); } - void schedule(OptimizingQueue.OptimizingThread thread) { + public void schedule(OptimizerThread thread) { invokeConsisitency(() -> { statusMachine.accept(Status.SCHEDULED); - optimizingThread = thread; + token = thread.getToken(); + threadId = thread.getThreadId(); startTime = System.currentTimeMillis(); persistTaskRuntime(this); }); } - void ack(OptimizingQueue.OptimizingThread thread) { + public void ack(OptimizerThread thread) { invokeConsisitency(() -> { validThread(thread); statusMachine.accept(Status.ACKED); @@ -150,7 +155,7 @@ void ack(OptimizingQueue.OptimizingThread thread) { }); } - void tryCanceling() { + public void tryCanceling() { invokeConsisitency(() -> { if (statusMachine.tryAccepting(Status.CANCELED)) { costTime = System.currentTimeMillis() - startTime; @@ -196,8 +201,16 @@ public long getProcessId() { return taskId.getProcessId(); } - public OptimizingQueue.OptimizingThread getOptimizingThread() { - return optimizingThread; + public String getResourceDesc() { + return token + ":" + threadId; + } + + public String getToken() { + return token; + } + + public int getThreadId() { + return threadId; } public OptimizingTask getOptimizingTask() { @@ -284,15 +297,15 @@ public String toString() { .add("startTime", startTime) .add("endTime", endTime) .add("costTime", costTime) - .add("optimizingThread", optimizingThread) + .add("resourceThread", getResourceDesc()) .add("failReason", failReason) .add("summary", summary) .add("properties", properties) .toString(); } - private void validThread(OptimizingQueue.OptimizingThread thread) { - if (!thread.equals(this.optimizingThread)) { + private void validThread(OptimizerThread thread) { + if (!thread.getToken().equals(getToken()) || thread.getThreadId() != threadId) { throw new DuplicateRuntimeException("Task already acked by optimizer thread + " + thread); } } @@ -308,11 +321,6 @@ public TaskQuota getCurrentQuota() { return new TaskQuota(this); } - public boolean isSuspending(long determineTime, long ackTimeout) { - return status == TaskRuntime.Status.SCHEDULED && - determineTime - startTime > ackTimeout; - } - private static final Map> nextStatusMap = new HashMap<>(); static { @@ -396,7 +404,6 @@ public static class TaskQuota { private long tableId; public TaskQuota() { - } public TaskQuota(TaskRuntime task) { @@ -416,27 +423,6 @@ public long getProcessId() { return processId; } - public int getTaskId() { - return taskId; - } - - - public int getRetryNum() { - return retryNum; - } - - public long getEndTime() { - return endTime; - } - - public String getFailReason() { - return failReason; - } - - public long getTableId() { - return tableId; - } - public long getQuotaTime(long calculatingStartTime) { long lastingTime = endTime - Math.max(startTime, calculatingStartTime); return Math.max(0, lastingTime); diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java index a3b43e229b..fd0d206da7 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java @@ -109,7 +109,7 @@ List selectOptimizingProcesses( "#{taskRuntime.startTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + " #{taskRuntime.endTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}, " + "#{taskRuntime.status}, #{taskRuntime.failReason, jdbcType=VARCHAR}," + - " #{taskRuntime.optimizingThread.token, jdbcType=VARCHAR}, #{taskRuntime.optimizingThread.threadId, " + + " #{taskRuntime.token, jdbcType=VARCHAR}, #{taskRuntime.threadId, " + "jdbcType=INTEGER}, #{taskRuntime.output, jdbcType=BLOB, " + " typeHandler=com.netease.arctic.server.persistence.converter.Object2ByteArrayConvert}," + " #{taskRuntime.summary, typeHandler=com.netease.arctic.server.persistence.converter.JsonObjectConverter}," + diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java index 57dffaebf4..450ebb0f47 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java @@ -20,8 +20,8 @@ import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.resource.Resource; -import com.netease.arctic.server.optimizing.OptimizingQueue; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import java.util.UUID; @@ -30,6 +30,7 @@ public class OptimizerInstance extends Resource { private String token; private long startTime; private long touchTime; + private OptimizerThread[] threads; public OptimizerInstance() { } @@ -39,6 +40,10 @@ public OptimizerInstance(OptimizerRegisterInfo registerInfo, String containerNam this.token = UUID.randomUUID().toString(); this.touchTime = System.currentTimeMillis(); this.startTime = registerInfo.getStartTime(); + this.threads = new OptimizerThread[registerInfo.getThreadCount()]; + for (int index = 0; index < threads.length; index++) { + threads[index] = new OptimizerThread(index, this); + } } public OptimizerInstance touch() { @@ -65,7 +70,8 @@ public long getStartTime() { } - public OptimizingQueue.OptimizingThread getThread(int threadId) { - return new OptimizingQueue.OptimizingThread(token, threadId); + public OptimizerThread getThread(int threadId) { + Preconditions.checkElementIndex(threadId, threads.length); + return threads[threadId]; } } \ No newline at end of file diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java new file mode 100644 index 0000000000..e365241429 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -0,0 +1,42 @@ +package com.netease.arctic.server.resource; + +import java.util.Objects; + +public class OptimizerThread { + private final int threadId; + private final OptimizerInstance optimizer; + + protected OptimizerThread(int threadId, OptimizerInstance optimizer) { + this.threadId = threadId; + this.optimizer = optimizer; + } + + public String getToken() { + return optimizer.getToken(); + } + + public int getThreadId() { + return threadId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + OptimizerThread that = (OptimizerThread) o; + return threadId == that.threadId && Objects.equals(optimizer, that.optimizer); + } + + @Override + public int hashCode() { + return Objects.hash(threadId, optimizer); + } + + @Override + public String toString() { + return "OptimizerThread{" + + "threadId=" + threadId + + ", optimizer=" + optimizer + + '}'; + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java new file mode 100644 index 0000000000..d0da963706 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java @@ -0,0 +1,9 @@ +package com.netease.arctic.server.resource; + +import com.netease.arctic.ams.api.resource.ResourceGroup; +import com.netease.arctic.server.table.TableRuntime; + +public interface QuotaProvider { + + public int getTotalQuota(String resourceGroup); +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java b/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java index c66710e903..70dce065e4 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java @@ -293,6 +293,10 @@ public void handleTableChanged(TableRuntime tableRuntime, TableConfiguration ori } } + protected List loadTableRuntimeMetadata() { + return getAs(TableMetaMapper.class, TableMetaMapper::selectTableRuntimeMetas); + } + @Override public void initialize() { checkNotStarted(); @@ -300,8 +304,7 @@ public void initialize() { getAs(CatalogMetaMapper.class, CatalogMetaMapper::getCatalogs); catalogMetas.forEach(this::initServerCatalog); - List tableRuntimeMetaList = - getAs(TableMetaMapper.class, TableMetaMapper::selectTableRuntimeMetas); + List tableRuntimeMetaList = loadTableRuntimeMetadata(); tableRuntimeMetaList.forEach(tableRuntimeMeta -> { TableRuntime tableRuntime = tableRuntimeMeta.constructTableRuntime(this); tableRuntimeMap.put(tableRuntime.getTableIdentifier(), tableRuntime); diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java new file mode 100644 index 0000000000..dc0690caed --- /dev/null +++ b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java @@ -0,0 +1,272 @@ +package com.netease.arctic.server; + +import com.netease.arctic.TableTestHelper; +import com.netease.arctic.ams.api.OptimizerRegisterInfo; +import com.netease.arctic.ams.api.OptimizingTask; +import com.netease.arctic.ams.api.OptimizingTaskId; +import com.netease.arctic.ams.api.OptimizingTaskResult; +import com.netease.arctic.ams.api.resource.ResourceGroup; +import com.netease.arctic.catalog.CatalogTestHelper; +import com.netease.arctic.io.MixedDataTestHelpers; +import com.netease.arctic.optimizing.RewriteFilesOutput; +import com.netease.arctic.optimizing.TableOptimizing; +import com.netease.arctic.server.exception.IllegalTaskStateException; +import com.netease.arctic.server.exception.PluginRetryAuthException; +import com.netease.arctic.server.optimizing.OptimizingStatus; +import com.netease.arctic.server.optimizing.TaskRuntime; +import com.netease.arctic.server.resource.OptimizerInstance; +import com.netease.arctic.server.table.AMSTableTestBase; +import com.netease.arctic.server.table.TableConfiguration; +import com.netease.arctic.server.table.TableRuntime; +import com.netease.arctic.server.table.TableRuntimeMeta; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.SerializationUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +public class TestDefaultOptimizingService extends AMSTableTestBase { + + private String token; + + public TestDefaultOptimizingService(CatalogTestHelper catalogTestHelper, + TableTestHelper tableTestHelper) { + super(catalogTestHelper, tableTestHelper, false); + } + + @BeforeEach + public void prepare() { + token = optimizingService().authenticate(buildRegisterInfo()); + createDatabase(); + createTable(); + initTableWithFiles(); + } + + @AfterEach + public void clear() { + try { + optimizingService().listOptimizers() + .forEach(optimizer -> optimizingService().deleteOptimizer(optimizer.getGroupName(), + optimizer.getResourceId())); + dropTable(); + dropDatabase(); + } catch (Exception e) { + // ignore + } + } + + private void initTableWithFiles() { + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); + appendData(arcticTable.asUnkeyedTable(), 1); + appendData(arcticTable.asUnkeyedTable(), 2); + TableRuntime runtime = tableService().getRuntime(serverTableIdentifier()); + + runtime.refresh(tableService().loadTable(serverTableIdentifier())); + } + + private List appendData(UnkeyedTable table, int id) { + ArrayList newRecords = Lists.newArrayList( + MixedDataTestHelpers.createRecord(table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); + List dataFiles = MixedDataTestHelpers.writeBaseStore(table, 0L, newRecords, false); + AppendFiles appendFiles = table.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + return dataFiles; + } + + @Test + public void testPollWithoutAuth() { + // 1.poll task + clear(); + Assertions.assertThrows(PluginRetryAuthException.class, () -> + optimizingService().pollTask("whatever", 1)); + } + + @Test + public void testPollOnce() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + } + + @Test + public void testPollTaskTwice() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + optimizingService().touch(token); + + // 3.fail task + optimizingService().completeTask(token, + buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + + // 4.retry poll task + OptimizingTask task2 = optimizingService().pollTask(token, 1); + Assertions.assertEquals(task2, task); + } + + @Test + public void testPollTaskThreeTimes() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + + // 3.fail task + optimizingService().completeTask(token, + buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + + // 4.retry poll task + OptimizingTask task2 = optimizingService().pollTask(token, 1); + Assertions.assertEquals(task2, task); + + optimizingService().completeTask(token, + buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + + // retry again + OptimizingTask task3 = optimizingService().pollTask(token, 1); + Assertions.assertEquals(task3, task); + + //third time would be null + Assertions.assertNull(optimizingService().pollTask(token, 1)); + } + + @Test + public void testTouch() throws InterruptedException { + OptimizerInstance optimizer = optimizingService().listOptimizers().get(0); + long oldTouchTime = optimizer.getTouchTime(); + Thread.sleep(1); + optimizingService().touch(token); + Assertions.assertTrue(optimizer.getTouchTime() > oldTouchTime); + } + + @Test + public void testTouchTimeout() throws InterruptedException { + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + Thread.sleep(600); + Assertions.assertThrows(PluginRetryAuthException.class, () -> + optimizingService().touch(token)); + Assertions.assertThrows(PluginRetryAuthException.class, () -> + optimizingService().pollTask(token, 1)); + token = optimizingService().authenticate(buildRegisterInfo()); + OptimizingTask task2 = optimizingService().pollTask(token, 1); + Assertions.assertNotEquals(task2, task); + } + + @Test + public void testAckAndCompleteTask() { + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + Assertions.assertThrows(IllegalTaskStateException.class, () -> + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1))); + + optimizingService().ackTask(token, 1, task.getTaskId()); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + } + + @Test + public void testReloadScheduledTask() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + + reload(); + optimizingService().ackTask(token, 1, task.getTaskId()); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + } + + @Test + public void testReloadAckTask() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + optimizingService().ackTask(token, 1, task.getTaskId()); + + reload(); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), + TaskRuntime.Status.SUCCESS); + } + + @Test + public void testReloadCompletedTask() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + optimizingService().ackTask(token, 1, task.getTaskId()); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + + reload(); + Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), + TaskRuntime.Status.SUCCESS); + } + + @Test + public void testReloadFailedTask() { + // 1.poll task + OptimizingTask task = optimizingService().pollTask(token, 1); + Assertions.assertNotNull(task); + optimizingService().ackTask(token, 1, task.getTaskId()); + optimizingService().completeTask(token, + buildOptimizingTaskFailResult(task.getTaskId(), 1, "error")); + + reload(); + Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), + TaskRuntime.Status.FAILED); + Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getFailReason(), + "error"); + } + + private OptimizerRegisterInfo buildRegisterInfo() { + OptimizerRegisterInfo registerInfo = new OptimizerRegisterInfo(); + registerInfo.setThreadCount(1); + registerInfo.setMemoryMb(1024); + registerInfo.setGroupName(defaultResourceGroup().getName()); + registerInfo.setResourceId("1"); + registerInfo.setStartTime(System.currentTimeMillis()); + return registerInfo; + } + + private ResourceGroup defaultResourceGroup() { + return new ResourceGroup.Builder("test", "local").build(); + } + + private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, ResourceGroup resourceGroup) { + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); + TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); + tableRuntimeMeta.setCatalogName(serverTableIdentifier().getCatalog()); + tableRuntimeMeta.setDbName(serverTableIdentifier().getDatabase()); + tableRuntimeMeta.setTableName(serverTableIdentifier().getTableName()); + tableRuntimeMeta.setTableId(serverTableIdentifier().getId()); + tableRuntimeMeta.setTableStatus(status); + tableRuntimeMeta.setTableConfig(TableConfiguration.parseConfig(arcticTable.properties())); + tableRuntimeMeta.setOptimizerGroup(resourceGroup.getName()); + tableRuntimeMeta.constructTableRuntime(tableService()); + return tableRuntimeMeta; + } + + private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, int threadId) { + TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); + OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); + optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); + return optimizingTaskResult; + } + + private OptimizingTaskResult buildOptimizingTaskFailResult(OptimizingTaskId taskId, int threadId, + String errorMessage) { + TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); + OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); + optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); + optimizingTaskResult.setErrorMessage(errorMessage); + return optimizingTaskResult; + } +} diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index dafcedbff1..b723ac102b 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -2,8 +2,6 @@ import com.netease.arctic.BasicTableTestHelper; import com.netease.arctic.TableTestHelper; -import com.netease.arctic.ams.api.OptimizerRegisterInfo; -import com.netease.arctic.ams.api.OptimizingTask; import com.netease.arctic.ams.api.OptimizingTaskId; import com.netease.arctic.ams.api.OptimizingTaskResult; import com.netease.arctic.ams.api.TableFormat; @@ -13,9 +11,8 @@ import com.netease.arctic.io.MixedDataTestHelpers; import com.netease.arctic.optimizing.RewriteFilesOutput; import com.netease.arctic.optimizing.TableOptimizing; -import com.netease.arctic.server.persistence.PersistentBase; -import com.netease.arctic.server.persistence.mapper.TableMetaMapper; -import com.netease.arctic.server.resource.OptimizerInstance; +import com.netease.arctic.server.resource.OptimizerThread; +import com.netease.arctic.server.resource.QuotaProvider; import com.netease.arctic.server.table.AMSTableTestBase; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableRuntime; @@ -35,11 +32,23 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; @RunWith(Parameterized.class) public class TestOptimizingQueue extends AMSTableTestBase { - private final Persistency persistency = new Persistency(); + private final Executor planExecutor = Executors.newSingleThreadExecutor(); + private final QuotaProvider quotaProvider = resourceGroup -> 1; + private final long MAX_POLLING_TIME = 5000; + + private final OptimizerThread optimizerThread = new OptimizerThread(1, null) { + + @Override + public String getToken() { + return "aah"; + } + }; public TestOptimizingQueue(CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { @@ -53,20 +62,36 @@ public static Object[] parameters() { new BasicTableTestHelper(false, true)}}; } + private OptimizingQueue buildOptimizingGroupService(TableRuntimeMeta tableRuntimeMeta) { + return new OptimizingQueue( + tableService(), + defaultResourceGroup(), + quotaProvider, + planExecutor, + Collections.singletonList(tableRuntimeMeta), + 1); + } + + private OptimizingQueue buildOptimizingGroupService() { + return new OptimizingQueue( + tableService(), + defaultResourceGroup(), + quotaProvider, + planExecutor, + Collections.emptyList(), + 1); + } + @Test public void testPollNoTask() { TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingTask optimizingTask = queue.pollTask(authToken, 1); - Assert.assertNull(optimizingTask); + OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); + Assert.assertNull(queue.pollTask(0)); } @Test - public void testRefreshTable() { - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.emptyList(), Collections.emptyList(), 60000, 3000); + public void testRefreshAndReleaseTable() { + OptimizingQueue queue = buildOptimizingGroupService(); Assert.assertEquals(0, queue.getSchedulingPolicy().getTableRuntimeMap().size()); TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.IDLE, defaultResourceGroup()); queue.refreshTable(tableRuntimeMeta.getTableRuntime()); @@ -75,35 +100,66 @@ public void testRefreshTable() { queue.releaseTable(tableRuntimeMeta.getTableRuntime()); Assert.assertEquals(0, queue.getSchedulingPolicy().getTableRuntimeMap().size()); + + queue.refreshTable(tableRuntimeMeta.getTableRuntime()); + Assert.assertEquals(1, queue.getSchedulingPolicy().getTableRuntimeMap().size()); } @Test - public void testHandleTask() { + public void testPollTask() { TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); + OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); + // 1.poll task + TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); + + Assert.assertNotNull(task); + Assert.assertEquals(TaskRuntime.Status.PLANNED, task.getStatus()); + Assert.assertNull(queue.pollTask(0)); + } + + @Test + public void testRetryTask() { + TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); + OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); // 1.poll task - OptimizingTask task = pollTaskAndCheck(authToken, thread, queue); + TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); + Assert.assertNotNull(task); - // 2.ack task - ackTaskAndCheck(authToken, thread, queue, task); + queue.retryTask(task, false); + TaskRuntime retryTask = queue.pollTask(MAX_POLLING_TIME); + Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); - // 3.fail task - failTaskAndCheck(authToken, thread, queue, task); + retryTask.ack(optimizerThread); + Assert.assertEquals(TaskRuntime.Status.ACKED, task.getStatus()); + queue.retryTask(task, true); + retryTask = queue.pollTask(MAX_POLLING_TIME); + Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); - // 4.retry poll task - thread = new OptimizingQueue.OptimizingThread(authToken, 2); - task = pollTaskAndCheck(authToken, thread, queue); + retryTask.fail("error"); + Assert.assertEquals(TaskRuntime.Status.FAILED, task.getStatus()); + queue.retryTask(task, true); + Assert.assertEquals(TaskRuntime.Status.PLANNED, task.getStatus()); + retryTask = queue.pollTask(MAX_POLLING_TIME); + Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); + } - // 5.ackTask - ackTaskAndCheck(authToken, thread, queue, task); + @Test + public void testCommitTask() { + TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); + OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); + Assert.assertEquals(0, queue.collectRunningTasks().size()); + Assert.assertEquals(0, queue.collectTasks().size()); - // 6.succeed task - succeedTaskAndCheck(authToken, thread, queue, task); + TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); + Assert.assertEquals(1, queue.collectRunningTasks().size()); + Assert.assertEquals(1, queue.collectTasks().size()); + Assert.assertNotNull(task); + task.complete(optimizerThread, buildOptimizingTaskResult(task.getTaskId(), optimizerThread.getThreadId())); + Assert.assertEquals(0, queue.collectRunningTasks().size()); + Assert.assertEquals(1, queue.collectTasks().size()); + Assert.assertEquals(TaskRuntime.Status.SUCCESS, task.getStatus()); // 7.commit OptimizingProcess optimizingProcess = tableRuntimeMeta.getTableRuntime().getOptimizingProcess(); @@ -120,178 +176,20 @@ public void testHandleTask() { optimizingProcess.close(); Assert.assertEquals(OptimizingProcess.Status.CLOSED, optimizingProcess.getStatus()); + Assert.assertEquals(0, queue.collectTasks().size()); } @Test - public void testPollTwice() { - TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); - - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - // 1.poll task - pollTaskAndCheck(authToken, thread, queue); - - // 2.poll twice - Assert.assertNull(queue.pollTask(thread.getToken(), thread.getThreadId())); - } - - @Test - public void testCheckSuspendTask() { - TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); - - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - // 1.poll task - OptimizingTask task1 = pollTaskAndCheck(authToken, thread, queue); - queue.getOptimizers().forEach(optimizerInstance -> optimizerInstance.setTouchTime( - System.currentTimeMillis() - 60000 - 1)); - - // 2.check suspending and retry task - queue.checkSuspending(); - - // 3.poll again - Assert.assertEquals(0, queue.getExecutingTaskMap().size()); - String newAuthToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread newThread = new OptimizingQueue.OptimizingThread(newAuthToken, 1); - OptimizingTask task2 = pollTaskAndCheck(newAuthToken, newThread, queue); - Assert.assertEquals(task1, task2); - - } - - @Test - public void testReloadScheduledTask() { - TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); - - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - OptimizingTask task; - // 1.poll task - task = pollTaskAndCheck(authToken, thread, queue); - - // 2.reload from sysdb - List tableRuntimeMetas = persistency.selectTableRuntimeMetas(); - Assert.assertEquals(1, tableRuntimeMetas.size()); - tableRuntimeMetas.get(0).constructTableRuntime(tableService()); - queue = new OptimizingQueue(tableService(), defaultResourceGroup(), tableRuntimeMetas, queue.getOptimizers(), - 60000, 3000); - - Assert.assertEquals(1, queue.getExecutingTaskMap().size()); - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.SCHEDULED, thread); - - // 3.ack task - ackTaskAndCheck(authToken, thread, queue, task); - - // 4.succeed task - succeedTaskAndCheck(authToken, thread, queue, task); - } - - @Test - public void testReloadAckTask() { - TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); - - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - OptimizingTask task; - // 1.poll task - task = pollTaskAndCheck(authToken, thread, queue); - - // 2.ack task - ackTaskAndCheck(authToken, thread, queue, task); - - // 3.reload from sysdb - List tableRuntimeMetas = persistency.selectTableRuntimeMetas(); - Assert.assertEquals(1, tableRuntimeMetas.size()); - tableRuntimeMetas.get(0).constructTableRuntime(tableService()); - queue = new OptimizingQueue(tableService(), defaultResourceGroup(), tableRuntimeMetas, queue.getOptimizers(), - 60000, 3000); - - Assert.assertEquals(1, queue.getExecutingTaskMap().size()); - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.ACKED, thread); - - // 4.succeed task - succeedTaskAndCheck(authToken, thread, queue, task); - } - - @Test - public void testReloadCompleteTask() { + public void testCollectingTasks() { TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); + OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); + Assert.assertEquals(0, queue.collectTasks().size()); + Assert.assertEquals(0, queue.collectRunningTasks().size()); - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - OptimizingTask task; - // 1.poll task - task = pollTaskAndCheck(authToken, thread, queue); - - // 2.ack task - ackTaskAndCheck(authToken, thread, queue, task); - - // 3.succeed task - succeedTaskAndCheck(authToken, thread, queue, task); - - // 4.reload from sysdb - List tableRuntimeMetas = persistency.selectTableRuntimeMetas(); - Assert.assertEquals(1, tableRuntimeMetas.size()); - tableRuntimeMetas.get(0).constructTableRuntime(tableService()); - queue = new OptimizingQueue(tableService(), defaultResourceGroup(), tableRuntimeMetas, Collections.emptyList(), - 60000, 3000); - - Assert.assertEquals(0, queue.getExecutingTaskMap().size()); - } - - @Test - public void testReloadFailTask() { - TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); - - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.singletonList(tableRuntimeMeta), Collections.emptyList(), 60000, 3000); - - String authToken = queue.authenticate(buildRegisterInfo()); - OptimizingQueue.OptimizingThread thread = new OptimizingQueue.OptimizingThread(authToken, 1); - OptimizingTask task; - // 1.poll task - task = pollTaskAndCheck(authToken, thread, queue); - - // 2.ack task - ackTaskAndCheck(authToken, thread, queue, task); - - // 3.fail task - failTaskAndCheck(authToken, thread, queue, task); - - // 4.reload from sysdb - List tableRuntimeMetas = persistency.selectTableRuntimeMetas(); - Assert.assertEquals(1, tableRuntimeMetas.size()); - tableRuntimeMetas.get(0).constructTableRuntime(tableService()); - queue = new OptimizingQueue(tableService(), defaultResourceGroup(), tableRuntimeMetas, queue.getOptimizers(), - 60000, 3000); - - Assert.assertEquals(0, queue.getExecutingTaskMap().size()); - - // 5.retry poll task - thread = new OptimizingQueue.OptimizingThread(authToken, 2); - task = pollTaskAndCheck(authToken, thread, queue); - - // 6.ackTask - ackTaskAndCheck(authToken, thread, queue, task); - - // 7.succeed task - succeedTaskAndCheck(authToken, thread, queue, task); + TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); + Assert.assertNotNull(task); + Assert.assertEquals(1, queue.collectTasks().size()); + Assert.assertEquals(1, queue.collectRunningTasks().size()); } private TableRuntimeMeta initTableWithFiles() { @@ -305,85 +203,6 @@ private TableRuntimeMeta initTableWithFiles() { return tableRuntimeMeta; } - private void succeedTaskAndCheck(String authToken, OptimizingQueue.OptimizingThread thread, OptimizingQueue queue, - OptimizingTask task) { - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - queue.completeTask(authToken, buildOptimizingTaskResult(task.getTaskId(), thread.getThreadId())); - Assert.assertEquals(0, queue.getExecutingTaskMap().size()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.SUCCESS, null); - } - - private void failTaskAndCheck(String authToken, OptimizingQueue.OptimizingThread thread, OptimizingQueue queue, - OptimizingTask task) { - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - String errorMessage = "unknown error"; - queue.completeTask(authToken, buildOptimizingTaskFailResult(task.getTaskId(), thread.getThreadId(), errorMessage)); - Assert.assertEquals(0, queue.getExecutingTaskMap().size()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.PLANNED, null); // retry and change to PLANNED - Assert.assertEquals(1, taskRuntime.getRetry()); - Assert.assertEquals(errorMessage, taskRuntime.getFailReason()); - } - - private void ackTaskAndCheck(String authToken, - OptimizingQueue.OptimizingThread thread, - OptimizingQueue queue, - OptimizingTask task) { - queue.ackTask(authToken, thread.getThreadId(), task.getTaskId()); - Assert.assertEquals(1, queue.getExecutingTaskMap().size()); - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.ACKED, thread); - } - - private OptimizingTask pollTaskAndCheck(String authToken, OptimizingQueue.OptimizingThread thread, - OptimizingQueue queue) { - OptimizingTask task = queue.pollTask(authToken, thread.getThreadId()); - Assert.assertNotNull(task); - Assert.assertEquals(1, queue.getExecutingTaskMap().size()); - TaskRuntime taskRuntime = queue.getExecutingTaskMap().get(task.getTaskId()); - assertTaskRuntime(taskRuntime, TaskRuntime.Status.SCHEDULED, thread); - return task; - } - - @Test - public void testOptimizer() throws InterruptedException { - OptimizingQueue queue = new OptimizingQueue(tableService(), defaultResourceGroup(), - Collections.emptyList(), Collections.emptyList(), 60000, 3000); - OptimizerRegisterInfo registerInfo = buildRegisterInfo(); - - // authenticate - String authToken = queue.authenticate(registerInfo); - List optimizers = queue.getOptimizers(); - Assert.assertEquals(1, optimizers.size()); - OptimizerInstance optimizerInstance = optimizers.get(0); - Assert.assertEquals(authToken, optimizerInstance.getToken()); - - // touch - long oldTouchTime = optimizerInstance.getTouchTime(); - Thread.sleep(1); - queue.touch(authToken); - Assert.assertTrue(optimizerInstance.getTouchTime() > oldTouchTime); - - // remove - queue.removeOptimizer(registerInfo.getResourceId()); - Assert.assertEquals(0, queue.getOptimizers().size()); - } - - private OptimizerRegisterInfo buildRegisterInfo() { - OptimizerRegisterInfo registerInfo = new OptimizerRegisterInfo(); - registerInfo.setThreadCount(1); - registerInfo.setMemoryMb(1024); - registerInfo.setGroupName(defaultResourceGroup().getName()); - registerInfo.setResourceId("1"); - registerInfo.setStartTime(System.currentTimeMillis()); - return registerInfo; - } - - private void assertTaskRuntime(TaskRuntime taskRuntime, TaskRuntime.Status status, - OptimizingQueue.OptimizingThread thread) { - Assert.assertEquals(status, taskRuntime.getStatus()); - Assert.assertEquals(thread, taskRuntime.getOptimizingThread()); - } - private ResourceGroup defaultResourceGroup() { return new ResourceGroup.Builder("test", "local").build(); } @@ -402,14 +221,13 @@ private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, Resource return tableRuntimeMeta; } - private List appendData(UnkeyedTable table, int id) { + private void appendData(UnkeyedTable table, int id) { ArrayList newRecords = Lists.newArrayList( MixedDataTestHelpers.createRecord(table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); List dataFiles = MixedDataTestHelpers.writeBaseStore(table, 0L, newRecords, false); AppendFiles appendFiles = table.newAppend(); dataFiles.forEach(appendFiles::appendFile); appendFiles.commit(); - return dataFiles; } private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, int threadId) { @@ -418,19 +236,4 @@ private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); return optimizingTaskResult; } - - private OptimizingTaskResult buildOptimizingTaskFailResult(OptimizingTaskId taskId, int threadId, - String errorMessage) { - TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); - OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); - optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); - optimizingTaskResult.setErrorMessage(errorMessage); - return optimizingTaskResult; - } - - private static class Persistency extends PersistentBase { - public List selectTableRuntimeMetas() { - return getAs(TableMetaMapper.class, TableMetaMapper::selectTableRuntimeMetas); - } - } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java b/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java index 090e7f011e..e5583bc739 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java @@ -18,6 +18,8 @@ package com.netease.arctic.server.table; +import com.netease.arctic.server.ArcticManagementConf; +import com.netease.arctic.server.DefaultOptimizingService; import com.netease.arctic.server.utils.Configurations; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -29,10 +31,15 @@ public abstract class TableServiceTestBase { public static DerbyPersistence DERBY = new DerbyPersistence(); private static DefaultTableService TABLE_SERVICE = null; + private static DefaultOptimizingService OPTIMIZING_SERVICE = null; @BeforeClass public static void initTableService() { + Configurations configurations = new Configurations(); + configurations.set(ArcticManagementConf.OPTIMIZER_HB_TIMEOUT, 500L); TABLE_SERVICE = new DefaultTableService(new Configurations()); + OPTIMIZING_SERVICE = new DefaultOptimizingService(configurations, TABLE_SERVICE); + TABLE_SERVICE.addHandlerChain(OPTIMIZING_SERVICE.getTableRuntimeHandler()); TABLE_SERVICE.initialize(); } @@ -45,4 +52,12 @@ protected DefaultTableService tableService() { return TABLE_SERVICE; } + protected DefaultOptimizingService optimizingService() { + return OPTIMIZING_SERVICE; + } + + protected static void reload() { + disposeTableService(); + initTableService(); + } } From 56d333b49979bb9acd7ddc326c6d28a05549eafb Mon Sep 17 00:00:00 2001 From: majin1102 Date: Fri, 10 Nov 2023 17:24:44 +0800 Subject: [PATCH 02/23] [AMORO-1951] add unit test for OptimizingQueue and DefaultOptimizingService --- .../arctic/server/ArcticManagementConf.java | 11 +- .../arctic/server/ArcticServiceConstants.java | 2 - .../server/DefaultOptimizingService.java | 379 +++++++++++------- .../server/optimizing/OptimizingQueue.java | 314 +++++++++------ .../server/optimizing/SchedulingPolicy.java | 20 +- .../arctic/server/optimizing/TaskRuntime.java | 125 +++--- .../persistence/mapper/OptimizingMapper.java | 10 +- .../server/resource/OptimizerInstance.java | 14 +- .../server/resource/OptimizerThread.java | 8 +- .../arctic/server/resource/QuotaProvider.java | 3 - .../server/TestDefaultOptimizingService.java | 315 ++++++++++----- .../optimizing/TestOptimizingQueue.java | 75 ++-- .../server/table/TableServiceTestBase.java | 15 +- 13 files changed, 794 insertions(+), 497 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 49dc0d63db..74aab3c2ea 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -18,7 +18,6 @@ package com.netease.arctic.server; - import com.netease.arctic.server.utils.ConfigOption; import com.netease.arctic.server.utils.ConfigOptions; @@ -244,9 +243,13 @@ public class ArcticManagementConf { .defaultValue(1) .withDescription("Max planning parallelism in one optimizer group."); - /** - * config key prefix of terminal - */ + public static final ConfigOption OPTIMIZER_POLLING_TIMEOUT = + ConfigOptions.key("optimizer.polling-timeout") + .longType() + .defaultValue(1000L) + .withDescription("Optimizer polling task timeout."); + + /** config key prefix of terminal */ public static final String TERMINAL_PREFIX = "terminal."; public static final ConfigOption TERMINAL_BACKEND = diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticServiceConstants.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticServiceConstants.java index d3cc3f26e6..4336dc186e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticServiceConstants.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticServiceConstants.java @@ -2,8 +2,6 @@ public class ArcticServiceConstants { - public static final long OPTIMIZER_CHECK_INTERVAL = 10 * 1000; - public static final long INVALID_TIME = 0; public static final long QUOTA_LOOK_BACK_TIME = 60 * 60 * 1000; diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 4791c5d5e7..77ac29a021 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -59,58 +59,65 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.DelayQueue; +import java.util.concurrent.Delayed; import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; import java.util.stream.Collectors; /** - * DefaultOptimizingService is implementing the OptimizerManager Thrift service, which manages the optimization tasks - * for ArcticTable. It includes methods for authenticating optimizers, polling tasks from the optimizing queue, - * acknowledging tasks,and completing tasks. The code uses several data structures, including maps for optimizing queues - * ,task runtimes, and authenticated optimizers. - *

- * The code also includes a TimerTask for detecting and removing expired optimizers and suspending tasks. + * DefaultOptimizingService is implementing the OptimizerManager Thrift service, which manages the + * optimization tasks for ArcticTable. It includes methods for authenticating optimizers, polling + * tasks from the optimizing queue, acknowledging tasks,and completing tasks. The code uses several + * data structures, including maps for optimizing queues ,task runtimes, and authenticated + * optimizers. + * + *

The code also includes a TimerTask for detecting and removing expired optimizers and + * suspending tasks. */ -public class DefaultOptimizingService extends StatedPersistentBase implements OptimizingService.Iface, - OptimizerManager, QuotaProvider { +public class DefaultOptimizingService extends StatedPersistentBase + implements OptimizingService.Iface, OptimizerManager, QuotaProvider { private static final Logger LOG = LoggerFactory.getLogger(DefaultOptimizingService.class); - private static final long POLLING_TIMEOUT_MS = 10000; private final long optimizerTouchTimeout; private final long taskAckTimeout; private final int maxPlanningParallelism; + private final long pollingTimeout; private final Map optimizingQueueByGroup = new ConcurrentHashMap<>(); private final Map optimizingQueueByToken = new ConcurrentHashMap<>(); private final Map authOptimizers = new ConcurrentHashMap<>(); + private final SuspendingDetector suspendingDetector = new SuspendingDetector(); private final TableService tableService; private final RuntimeHandlerChain tableHandlerChain; private final Executor planExecutor; - private Timer optimizerMonitorTimer; public DefaultOptimizingService(Configurations serviceConfig, DefaultTableService tableService) { this.optimizerTouchTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_HB_TIMEOUT); this.taskAckTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_TASK_ACK_TIMEOUT); - this.maxPlanningParallelism = serviceConfig.getInteger(ArcticManagementConf.OPTIMIZER_MAX_PLANNING_PARALLELISM); + this.maxPlanningParallelism = + serviceConfig.getInteger(ArcticManagementConf.OPTIMIZER_MAX_PLANNING_PARALLELISM); + this.pollingTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_POLLING_TIMEOUT); this.tableService = tableService; this.tableHandlerChain = new TableRuntimeHandlerImpl(); - this.planExecutor = Executors.newCachedThreadPool(new ThreadFactory() { - private final AtomicInteger threadId = new AtomicInteger(0); - @Override - public Thread newThread(@NotNull Runnable r) { - Thread thread = new Thread(r, "plan-executor-thread-" + threadId.incrementAndGet()); - thread.setDaemon(true); - return thread; - } - }); + this.planExecutor = + Executors.newCachedThreadPool( + new ThreadFactory() { + private final AtomicInteger threadId = new AtomicInteger(0); + + @Override + public Thread newThread(@NotNull Runnable r) { + Thread thread = new Thread(r, "plan-executor-thread-" + threadId.incrementAndGet()); + thread.setDaemon(true); + return thread; + } + }); } public RuntimeHandlerChain getTableRuntimeHandler() { @@ -118,40 +125,50 @@ public RuntimeHandlerChain getTableRuntimeHandler() { } private void loadOptimizingQueues(List tableRuntimeMetaList) { - List optimizerGroups = getAs(ResourceMapper.class, ResourceMapper::selectResourceGroups); + List optimizerGroups = + getAs(ResourceMapper.class, ResourceMapper::selectResourceGroups); List optimizers = getAs(OptimizerMapper.class, OptimizerMapper::selectAll); - Map> groupToTableRuntimes = tableRuntimeMetaList.stream() - .collect(Collectors.groupingBy(TableRuntimeMeta::getOptimizerGroup)); - optimizerGroups.forEach(group -> { - String groupName = group.getName(); - List tableRuntimeMetas = groupToTableRuntimes.remove(groupName); - OptimizingQueue optimizingQueue = new OptimizingQueue( - tableService, - group, - this, - planExecutor, - Optional.ofNullable(tableRuntimeMetas).orElseGet(ArrayList::new), - maxPlanningParallelism); - optimizingQueueByGroup.put(groupName, optimizingQueue); - }); - optimizers.forEach(this::registerOptimizer); - groupToTableRuntimes.keySet().forEach(groupName -> LOG.warn("Unloaded task runtime in group " + groupName)); - } - - private void registerOptimizer(OptimizerInstance optimizer) { - String token = optimizer.getToken(); - authOptimizers.put(token, optimizer); - optimizingQueueByToken.put(token, optimizingQueueByGroup.get(optimizer.getGroupName())); + Map> groupToTableRuntimes = + tableRuntimeMetaList.stream() + .collect(Collectors.groupingBy(TableRuntimeMeta::getOptimizerGroup)); + optimizerGroups.forEach( + group -> { + String groupName = group.getName(); + List tableRuntimeMetas = groupToTableRuntimes.remove(groupName); + OptimizingQueue optimizingQueue = + new OptimizingQueue( + tableService, + group, + this, + planExecutor, + Optional.ofNullable(tableRuntimeMetas).orElseGet(ArrayList::new), + maxPlanningParallelism); + optimizingQueueByGroup.put(groupName, optimizingQueue); + }); + optimizers.forEach(optimizer -> registerOptimizer(optimizer, false)); + groupToTableRuntimes + .keySet() + .forEach(groupName -> LOG.warn("Unloaded task runtime in group " + groupName)); + } + + private void registerOptimizer(OptimizerInstance optimizer, boolean needPersistency) { + if (needPersistency) { + doAs(OptimizerMapper.class, mapper -> mapper.insertOptimizer(optimizer)); + } + authOptimizers.put(optimizer.getToken(), optimizer); + optimizingQueueByToken.put( + optimizer.getToken(), optimizingQueueByGroup.get(optimizer.getGroupName())); + suspendingDetector.detectTimeout(optimizer); } private void unregisterOptimizer(String token) { + doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token)); optimizingQueueByToken.remove(token); authOptimizers.remove(token); } @Override - public void ping() { - } + public void ping() {} public List listTasks(String optimizerGroup) { return getQueueByGroup(optimizerGroup).collectTasks(); @@ -167,8 +184,8 @@ public void touch(String authToken) { } private OptimizerInstance getAuthenticatedOptimizer(String authToken) { - org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument(authToken != null, - "authToken can not be null"); + org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument( + authToken != null, "authToken can not be null"); return Optional.ofNullable(authOptimizers.get(authToken)) .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); } @@ -179,15 +196,16 @@ public OptimizingTask pollTask(String authToken, int threadId) { LOG.debug("Optimizer {} (threadId {}) try polling task", authToken, threadId); } OptimizingQueue queue = getQueueByToken(authToken); - return Optional.ofNullable(queue.pollTask(POLLING_TIMEOUT_MS)) - .map(task -> extractOptimizingTask(task, - getAuthenticatedOptimizer(authToken).getThread(threadId), queue)) + return Optional.ofNullable(queue.pollTask(pollingTimeout)) + .map( + task -> + extractOptimizingTask( + task, getAuthenticatedOptimizer(authToken).getThread(threadId), queue)) .orElse(null); } - private OptimizingTask extractOptimizingTask(TaskRuntime task, - OptimizerThread optimizerThread, - OptimizingQueue queue) { + private OptimizingTask extractOptimizingTask( + TaskRuntime task, OptimizerThread optimizerThread, OptimizingQueue queue) { try { task.schedule(optimizerThread); LOG.info("OptimizerThread {} polled task {}", optimizerThread, task.getTaskId()); @@ -212,7 +230,8 @@ public void ackTask(String authToken, int threadId, OptimizingTaskId taskId) { public void completeTask(String authToken, OptimizingTaskResult taskResult) { LOG.info("Optimizer {} complete task {}", authToken, taskResult.getTaskId()); OptimizingQueue queue = getQueueByToken(authToken); - OptimizerThread thread = getAuthenticatedOptimizer(authToken).getThread(taskResult.getThreadId()); + OptimizerThread thread = + getAuthenticatedOptimizer(authToken).getThread(taskResult.getThreadId()); Optional.ofNullable(queue.getTask(taskResult.getTaskId())) .orElseThrow(() -> new TaskNotFoundException(taskResult.getTaskId())) .complete(thread, taskResult); @@ -223,8 +242,7 @@ public String authenticate(OptimizerRegisterInfo registerInfo) { LOG.info("Register optimizer {}.", registerInfo); OptimizingQueue queue = getQueueByGroup(registerInfo.getGroupName()); OptimizerInstance optimizer = new OptimizerInstance(registerInfo, queue.getContainerName()); - doAs(OptimizerMapper.class, mapper -> mapper.insertOptimizer(optimizer)); - registerOptimizer(optimizer); + registerOptimizer(optimizer, true); return optimizer.getToken(); } @@ -239,16 +257,12 @@ private OptimizingQueue getQueueByGroup(String optimizerGroup) { } private Optional getOptionalQueueByGroup(String optimizerGroup) { - Preconditions.checkArgument( - optimizerGroup != null, - "optimizerGroup can not be null"); + Preconditions.checkArgument(optimizerGroup != null, "optimizerGroup can not be null"); return Optional.ofNullable(optimizingQueueByGroup.get(optimizerGroup)); } private OptimizingQueue getQueueByToken(String token) { - Preconditions.checkArgument( - token != null, - "optimizer token can not be null"); + Preconditions.checkArgument(token != null, "optimizer token can not be null"); return Optional.ofNullable(optimizingQueueByToken.get(token)) .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); } @@ -269,26 +283,28 @@ public List listOptimizers(String group) { public void deleteOptimizer(String group, String resourceId) { List deleteOptimizers = getAs(OptimizerMapper.class, mapper -> mapper.selectByResourceId(resourceId)); - deleteOptimizers.forEach(optimizer -> { - String token = optimizer.getToken(); - doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token)); - unregisterOptimizer(token); - }); + deleteOptimizers.forEach( + optimizer -> { + String token = optimizer.getToken(); + unregisterOptimizer(token); + }); } @Override public void createResourceGroup(ResourceGroup resourceGroup) { - doAsTransaction(() -> { - doAs(ResourceMapper.class, mapper -> mapper.insertResourceGroup(resourceGroup)); - OptimizingQueue optimizingQueue = new OptimizingQueue( - tableService, - resourceGroup, - this, - planExecutor, - new ArrayList<>(), - maxPlanningParallelism); - optimizingQueueByGroup.put(resourceGroup.getName(), optimizingQueue); - }); + doAsTransaction( + () -> { + doAs(ResourceMapper.class, mapper -> mapper.insertResourceGroup(resourceGroup)); + OptimizingQueue optimizingQueue = + new OptimizingQueue( + tableService, + resourceGroup, + this, + planExecutor, + new ArrayList<>(), + maxPlanningParallelism); + optimizingQueueByGroup.put(resourceGroup.getName(), optimizingQueue); + }); } @Override @@ -297,8 +313,10 @@ public void deleteResourceGroup(String groupName) { doAs(ResourceMapper.class, mapper -> mapper.deleteResourceGroup(groupName)); optimizingQueueByGroup.remove(groupName); } else { - throw new RuntimeException(String.format("The resource group %s cannot be deleted because it is currently in " + - "use.", groupName)); + throw new RuntimeException( + String.format( + "The resource group %s cannot be deleted because it is currently in " + "use.", + groupName)); } } @@ -357,10 +375,12 @@ public void dispose() { public boolean canDeleteResourceGroup(String name) { for (CatalogMeta catalogMeta : tableService.listCatalogMetas()) { - if (catalogMeta.getCatalogProperties() != null && - catalogMeta.getCatalogProperties() + if (catalogMeta.getCatalogProperties() != null + && catalogMeta + .getCatalogProperties() .getOrDefault( - CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.SELF_OPTIMIZING_GROUP, + CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + + TableProperties.SELF_OPTIMIZING_GROUP, TableProperties.SELF_OPTIMIZING_GROUP_DEFAULT) .equals(name)) { return false; @@ -372,7 +392,8 @@ public boolean canDeleteResourceGroup(String name) { } } for (ServerTableIdentifier identifier : tableService.listManagedTables()) { - if (optimizingQueueByGroup.containsKey(name) && optimizingQueueByGroup.get(name).containsTable(identifier)) { + if (optimizingQueueByGroup.containsKey(name) + && optimizingQueueByGroup.get(name).containsTable(identifier)) { return false; } } @@ -392,7 +413,8 @@ private class TableRuntimeHandlerImpl extends RuntimeHandlerChain { @Override public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { if (!tableRuntime.getOptimizingStatus().isProcessing()) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()).ifPresent(q -> q.refreshTable(tableRuntime)); + getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + .ifPresent(q -> q.refreshTable(tableRuntime)); } } @@ -402,83 +424,170 @@ public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration or if (!tableRuntime.getOptimizerGroup().equals(originalGroup)) { getOptionalQueueByGroup(originalGroup).ifPresent(q -> q.releaseTable(tableRuntime)); } - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()).ifPresent(q -> q.refreshTable(tableRuntime)); + getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + .ifPresent(q -> q.refreshTable(tableRuntime)); } @Override public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()).ifPresent(q -> q.refreshTable(tableRuntime)); + getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + .ifPresent(q -> q.refreshTable(tableRuntime)); } @Override public void handleTableRemoved(TableRuntime tableRuntime) { - getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()).ifPresent(queue -> queue.releaseTable(tableRuntime)); + getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()) + .ifPresent(queue -> queue.releaseTable(tableRuntime)); } @Override protected void initHandler(List tableRuntimeMetaList) { LOG.info("OptimizerManagementService begin initializing"); loadOptimizingQueues(tableRuntimeMetaList); - optimizerMonitorTimer = new Timer("OptimizerMonitor", true); - optimizerMonitorTimer.schedule( - new SuspendingDetector(), - optimizerTouchTimeout, - ArcticServiceConstants.OPTIMIZER_CHECK_INTERVAL); - LOG.info("init SuspendingDetector for Optimizer with delay {} ms, interval {} ms", optimizerTouchTimeout, - ArcticServiceConstants.OPTIMIZER_CHECK_INTERVAL); + suspendingDetector.start(); + LOG.info("SuspendingDetector for Optimizer has been started."); LOG.info("OptimizerManagementService initializing has completed"); } @Override protected void doDispose() { - if (Objects.nonNull(optimizerMonitorTimer)) { - optimizerMonitorTimer.cancel(); - } + suspendingDetector.dispose(); + } + } + + private class SuspendingDetectTask implements Delayed { + + private final String token; + private final long checkDelayedTime; + + public SuspendingDetectTask(OptimizerInstance optimizer) { + this.token = optimizer.getToken(); + this.checkDelayedTime = optimizer.getTouchTime() + optimizerTouchTimeout; + } + + @Override + public long getDelay(@NotNull TimeUnit unit) { + return unit.convert(checkDelayedTime - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + } + + @Override + public int compareTo(@NotNull Delayed o) { + SuspendingDetectTask another = (SuspendingDetectTask) o; + return Long.compare(checkDelayedTime, another.checkDelayedTime); + } + + public String getToken() { + return token; } } - private class SuspendingDetector extends TimerTask { + private class SuspendingDetector implements Runnable { + + private volatile boolean stopped = false; + private final Thread thread = new Thread(this, "SuspendingDetector"); + private final DelayQueue suspendingQueue = new DelayQueue<>(); + + public SuspendingDetector() { + thread.setDaemon(true); + } + + public void detectTimeout(OptimizerInstance optimizerInstance) { + Preconditions.checkNotNull(optimizerInstance, "token can not be null"); + suspendingQueue.add(new SuspendingDetectTask(optimizerInstance)); + } + + public void start() { + thread.start(); + } + + public void dispose() { + stopped = true; + thread.interrupt(); + } @Override public void run() { - try { - long currentTime = System.currentTimeMillis(); - Set expiredTokens = authOptimizers.values().stream() - .filter(optimizer -> currentTime - optimizer.getTouchTime() > optimizerTouchTimeout) - .map(OptimizerInstance::getToken) - .collect(Collectors.toSet()); - - expiredTokens.forEach(authOptimizers.keySet()::remove); - if (!expiredTokens.isEmpty()) { - LOG.info("Expired optimizers: {}", expiredTokens); + while (!stopped) { + try { + SuspendingDetectTask detectTask = suspendingQueue.take(); + String token = detectTask.getToken(); + boolean isOptimzerExpired = + Optional.ofNullable(authOptimizers.get(token)) + .map( + optimizer -> + optimizer.getTouchTime() + optimizerTouchTimeout + < System.currentTimeMillis()) + .orElse(true); + OptimizingQueue queue = getQueueByToken(token); + queue + .collectTasks(buildSuspendingPredication(token, isOptimzerExpired)) + .forEach( + task -> { + LOG.info( + "Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", + task.getTaskId(), + task.getResourceDesc()); + // optimizing task of suspending optimizer would not be counted for retrying + queue.retryTask(task, false); + }); + if (isOptimzerExpired) { + LOG.info("Optimizer {} has been expired, unregister it", token); + unregisterOptimizer(token); + } else { + OptimizerInstance optimizer = authOptimizers.get(token); + detectTimeout(optimizer); + } + } catch (InterruptedException ignored) { + } catch (Throwable t) { + LOG.error("SuspendingDetector has encountered a problem.", t); } + } + } - for (OptimizingQueue queue : optimizingQueueByGroup.values()) { - queue.collectRunningTasks().stream() - .filter(task -> isTaskExpired(task, currentTime, expiredTokens, authOptimizers.keySet())) - .forEach(task -> { - LOG.info("Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", - task.getTaskId(), task.getResourceDesc()); - //optimizing task of suspending optimizer would not be counted for retrying - queue.retryTask(task, false); - }); - } + private boolean isOptimizerTimeout(String token) { + return Optional.ofNullable(authOptimizers.get(token)) + .map( + optimizer -> + optimizer.getTouchTime() + optimizerTouchTimeout < System.currentTimeMillis()) + .orElse(true); + } - expiredTokens.forEach(token -> { - doAs(OptimizerMapper.class, mapper -> mapper.deleteOptimizer(token)); - unregisterOptimizer(token); - }); - } catch (RuntimeException e) { - LOG.error("Update optimizer status abnormal failed. try next round", e); + private void dealSuspendingTasks(String token, boolean isOptimzerExpired) { + OptimizingQueue queue = getQueueByToken(token); + queue + .collectTasks(buildSuspendingPredication(token, isOptimzerExpired)) + .forEach( + task -> { + LOG.info( + "Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", + task.getTaskId(), + task.getResourceDesc()); + // optimizing task of suspending optimizer would not be counted for retrying + queue.retryTask(task, false); + }); + } + + private void dealOptimizer(String token, boolean isOptimzerExpired) { + if (isOptimzerExpired) { + LOG.info("Optimizer {} has been suspended, unregister it", token); + unregisterOptimizer(token); + } else { + OptimizerInstance optimizer = authOptimizers.get(token); + detectTimeout(optimizer); } } - private boolean isTaskExpired(TaskRuntime task, long currentTime, - Set expiredTokens, Set authTokens) { - return task.getStatus() == TaskRuntime.Status.SCHEDULED - && currentTime - task.getStartTime() > taskAckTimeout || - expiredTokens.contains(task.getToken()) || - !authTokens.contains(task.getToken()); + private Predicate buildSuspendingPredication( + String token, boolean isOptimzerExpired) { + return task -> { + if (isOptimzerExpired) { + return token.equals(task.getToken()); + } else { + return token.equals(task.getToken()) + && task.getStatus() == TaskRuntime.Status.SCHEDULED + && task.getStartTime() + taskAckTimeout < System.currentTimeMillis(); + } + }; } } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 61a9cc47e9..11b669d6f9 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -1,3 +1,21 @@ +/* + * 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 com.netease.arctic.server.optimizing; import com.google.common.collect.Maps; @@ -46,6 +64,7 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Predicate; import java.util.stream.Collectors; public class OptimizingQueue extends PersistentBase { @@ -85,16 +104,17 @@ public OptimizingQueue( private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { TableRuntime tableRuntime = tableRuntimeMeta.getTableRuntime(); - if (tableRuntime.getOptimizingStatus().isProcessing() && - tableRuntimeMeta.getOptimizingProcessId() != 0) { + if (tableRuntime.getOptimizingStatus().isProcessing() + && tableRuntimeMeta.getOptimizingProcessId() != 0) { tableRuntime.recover(new TableOptimizingProcess(tableRuntimeMeta)); } if (tableRuntime.isOptimizingEnabled()) { - //TODO: load task quotas - tableRuntime.resetTaskQuotas(System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); - if (tableRuntime.getOptimizingStatus() == OptimizingStatus.IDLE || - tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING) { + // TODO: load task quotas + tableRuntime.resetTaskQuotas( + System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); + if (tableRuntime.getOptimizingStatus() == OptimizingStatus.IDLE + || tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING) { schedulingPolicy.addTable(tableRuntime); } else if (tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { tableQueue.offer(new TableOptimizingProcess(tableRuntimeMeta)); @@ -113,15 +133,22 @@ public String getContainerName() { public void refreshTable(TableRuntime tableRuntime) { if (tableRuntime.isOptimizingEnabled() && !tableRuntime.getOptimizingStatus().isProcessing()) { - LOG.info("Bind queue {} success with table {}", optimizerGroup.getName(), tableRuntime.getTableIdentifier()); - tableRuntime.resetTaskQuotas(System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); + LOG.info( + "Bind queue {} success with table {}", + optimizerGroup.getName(), + tableRuntime.getTableIdentifier()); + tableRuntime.resetTaskQuotas( + System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); schedulingPolicy.addTable(tableRuntime); } } public void releaseTable(TableRuntime tableRuntime) { schedulingPolicy.removeTable(tableRuntime); - LOG.info("Release queue {} with table {}", optimizerGroup.getName(), tableRuntime.getTableIdentifier()); + LOG.info( + "Release queue {} with table {}", + optimizerGroup.getName(), + tableRuntime.getTableIdentifier()); } public boolean containsTable(ServerTableIdentifier identifier) { @@ -130,7 +157,8 @@ public boolean containsTable(ServerTableIdentifier identifier) { private void clearProcess(TableOptimizingProcess optimizingProcess) { tableQueue.removeIf(process -> process.getProcessId() == optimizingProcess.getProcessId()); - retryTaskQueue.removeIf(taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); + retryTaskQueue.removeIf( + taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); } public TaskRuntime pollTask(long maxWaitTime) { @@ -148,10 +176,9 @@ private long calculateDeadline(long maxWaitTime) { } private TaskRuntime fetchTask() { - return Optional.ofNullable(retryTaskQueue.poll()) - .orElse(Optional.ofNullable(tableQueue.peek()) - .map(TableOptimizingProcess::poll) - .orElse(null)); + return Optional.ofNullable(retryTaskQueue.poll()) + .orElse( + Optional.ofNullable(tableQueue.peek()).map(TableOptimizingProcess::poll).orElse(null)); } public TaskRuntime getTask(OptimizingTaskId taskId) { @@ -168,11 +195,10 @@ public List collectTasks() { .collect(Collectors.toList()); } - public List collectRunningTasks() { + public List collectTasks(Predicate predicate) { return tableQueue.stream() .flatMap(p -> p.getTaskMap().values().stream()) - .filter(task -> task.getStatus() == TaskRuntime.Status.ACKED || - task.getStatus() == TaskRuntime.Status.SCHEDULED) + .filter(predicate) .collect(Collectors.toList()); } @@ -194,8 +220,8 @@ private boolean waitTask(long waitDeadline) { try { long currentTime = System.currentTimeMillis(); scheduleTableIfNecessary(currentTime); - return waitDeadline > currentTime && - planningCompleted.await(waitDeadline - currentTime, TimeUnit.MILLISECONDS); + return waitDeadline > currentTime + && planningCompleted.await(waitDeadline - currentTime, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { LOG.error("Schedule table interrupted", e); return false; @@ -217,54 +243,62 @@ private void scheduleTableIfNecessary(long startTime) { } } - private void triggerAsyncPlanning(TableRuntime tableRuntime, Set skipTables, long startTime) { + private void triggerAsyncPlanning( + TableRuntime tableRuntime, Set skipTables, long startTime) { LOG.info("Trigger planning table {}", tableRuntime.getTableIdentifier()); planningTables.add(tableRuntime); - doPlanning(tableRuntime).whenComplete((process, throwable) -> { - long currentTime = System.currentTimeMillis(); - scheduleLock.lock(); - try { - plannedKeepingTables.put(tableRuntime, System.currentTimeMillis()); - planningTables.remove(tableRuntime); - if (process != null) { - tableQueue.offer(process); - LOG.info("{} completes planning {} tasks with a total cost of {} ms, skipping tables, {}", - optimizerGroup.getName(), - process.getTaskMap().size(), - currentTime - startTime, - skipTables); - } else { - if (throwable != null) { - LOG.error("Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); - } - LOG.info("{} skip planning table {} with a total cost of {} ms.", - optimizerGroup.getName(), - tableRuntime.getTableIdentifier(), - currentTime - startTime); - } - planningCompleted.signalAll(); - } finally { - scheduleLock.unlock(); - } - }); + doPlanning(tableRuntime) + .whenComplete( + (process, throwable) -> { + long currentTime = System.currentTimeMillis(); + scheduleLock.lock(); + try { + plannedKeepingTables.put(tableRuntime, System.currentTimeMillis()); + planningTables.remove(tableRuntime); + if (process != null) { + tableQueue.offer(process); + LOG.info( + "{} completes planning {} tasks with a total cost of {} ms, skipping tables, {}", + optimizerGroup.getName(), + process.getTaskMap().size(), + currentTime - startTime, + skipTables); + } else { + if (throwable != null) { + LOG.error( + "Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); + } + LOG.info( + "{} skip planning table {} with a total cost of {} ms.", + optimizerGroup.getName(), + tableRuntime.getTableIdentifier(), + currentTime - startTime); + } + planningCompleted.signalAll(); + } finally { + scheduleLock.unlock(); + } + }); } private CompletableFuture doPlanning(TableRuntime tableRuntime) { CompletableFuture future = new CompletableFuture<>(); - planExecutor.execute(() -> { - AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); - OptimizingPlanner planner = new OptimizingPlanner( - tableRuntime.refresh(table), - (ArcticTable) table.originalTable(), - getAvailableCore()); - if (planner.isNecessary()) { - TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); - future.complete(optimizingProcess); - } else { - tableRuntime.cleanPendingInput(); - future.complete(null); - } - }); + planExecutor.execute( + () -> { + AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + OptimizingPlanner planner = + new OptimizingPlanner( + tableRuntime.refresh(table), + (ArcticTable) table.originalTable(), + getAvailableCore()); + if (planner.isNecessary()) { + TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); + future.complete(optimizingProcess); + } else { + tableRuntime.cleanPendingInput(); + future.complete(null); + } + }); return future; } @@ -377,18 +411,23 @@ public void acceptResult(TaskRuntime taskRuntime) { try { try { tableRuntime.addTaskQuota(taskRuntime.getCurrentQuota()); - } catch (Throwable t) { - LOG.warn("{} failed to add task quota {}, ignore it", tableRuntime.getTableIdentifier(), - taskRuntime.getTaskId(), t); + } catch (Throwable throwable) { + LOG.warn( + "{} failed to add task quota {}, ignore it", + tableRuntime.getTableIdentifier(), + taskRuntime.getTaskId(), + throwable); } if (isClosed()) { throw new OptimizingClosedException(processId); } if (taskRuntime.getStatus() == TaskRuntime.Status.SUCCESS) { // the lock of TableOptimizingProcess makes it thread-safe - if (allTasksPrepared() && tableRuntime.getOptimizingStatus().isProcessing() && - tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { + if (allTasksPrepared() + && tableRuntime.getOptimizingStatus().isProcessing() + && tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { tableRuntime.beginCommitting(); + clearProcess(this); } } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { if (taskRuntime.getRetry() <= tableRuntime.getMaxExecuteRetryCount()) { @@ -420,9 +459,10 @@ public long getPlanTime() { @Override public long getDuration() { - long dur = endTime == ArcticServiceConstants.INVALID_TIME ? - System.currentTimeMillis() - planTime : - endTime - planTime; + long dur = + endTime == ArcticServiceConstants.INVALID_TIME + ? System.currentTimeMillis() - planTime + : endTime - planTime; return Math.max(0, dur); } @@ -463,8 +503,7 @@ private boolean allTasksPrepared() { */ @Override public long getRunningQuotaTime(long calculatingStartTime, long calculatingEndTime) { - return taskMap.values() - .stream() + return taskMap.values().stream() .filter(t -> !t.finished()) .mapToLong(task -> task.getQuotaTime(calculatingStartTime, calculatingEndTime)) .sum(); @@ -472,8 +511,11 @@ public long getRunningQuotaTime(long calculatingStartTime, long calculatingEndTi @Override public void commit() { - LOG.debug("{} get {} tasks of {} partitions to commit", tableRuntime.getTableIdentifier(), - taskMap.size(), taskMap.values()); + LOG.debug( + "{} get {} tasks of {} partitions to commit", + tableRuntime.getTableIdentifier(), + taskMap.size(), + taskMap.values()); lock.lock(); try { @@ -504,80 +546,122 @@ public MetricsSummary getSummary() { } private UnKeyedTableCommit buildCommit() { - ArcticTable table = (ArcticTable) tableManager.loadTable(tableRuntime.getTableIdentifier()).originalTable(); + ArcticTable table = + (ArcticTable) tableManager.loadTable(tableRuntime.getTableIdentifier()).originalTable(); if (table.isUnkeyedTable()) { return new UnKeyedTableCommit(targetSnapshotId, table, taskMap.values()); } else { - return new KeyedTableCommit(table, taskMap.values(), targetSnapshotId, - convertPartitionSequence(table, fromSequence), convertPartitionSequence(table, toSequence)); + return new KeyedTableCommit( + table, + taskMap.values(), + targetSnapshotId, + convertPartitionSequence(table, fromSequence), + convertPartitionSequence(table, toSequence)); } } - private StructLikeMap convertPartitionSequence(ArcticTable table, Map partitionSequence) { + private StructLikeMap convertPartitionSequence( + ArcticTable table, Map partitionSequence) { PartitionSpec spec = table.spec(); StructLikeMap results = StructLikeMap.create(spec.partitionType()); - partitionSequence.forEach((partition, sequence) -> { - if (spec.isUnpartitioned()) { - results.put(TablePropertyUtil.EMPTY_STRUCT, sequence); - } else { - StructLike partitionData = ArcticDataFiles.data(spec, partition); - results.put(partitionData, sequence); - } - }); + partitionSequence.forEach( + (partition, sequence) -> { + if (spec.isUnpartitioned()) { + results.put(TablePropertyUtil.EMPTY_STRUCT, sequence); + } else { + StructLike partitionData = ArcticDataFiles.data(spec, partition); + results.put(partitionData, sequence); + } + }); return results; } private void beginAndPersistProcess() { doAsTransaction( - () -> doAs(OptimizingMapper.class, mapper -> - mapper.insertOptimizingProcess(tableRuntime.getTableIdentifier(), - processId, targetSnapshotId, targetChangeSnapshotId, status, optimizingType, planTime, getSummary(), - fromSequence, toSequence)), - () -> doAs(OptimizingMapper.class, mapper -> - mapper.insertTaskRuntimes(Lists.newArrayList(taskMap.values()))), + () -> + doAs( + OptimizingMapper.class, + mapper -> + mapper.insertOptimizingProcess( + tableRuntime.getTableIdentifier(), + processId, + targetSnapshotId, + targetChangeSnapshotId, + status, + optimizingType, + planTime, + getSummary(), + fromSequence, + toSequence)), + () -> + doAs( + OptimizingMapper.class, + mapper -> mapper.insertTaskRuntimes(Lists.newArrayList(taskMap.values()))), () -> TaskFilesPersistence.persistTaskInputs(processId, taskMap.values()), - () -> tableRuntime.beginProcess(this) - ); + () -> tableRuntime.beginProcess(this)); } private void persistProcessCompleted(boolean success) { if (!success) { doAsTransaction( () -> taskMap.values().forEach(TaskRuntime::tryCanceling), - () -> doAs(OptimizingMapper.class, mapper -> - mapper.updateOptimizingProcess(tableRuntime.getTableIdentifier().getId(), processId, status, endTime, - getSummary(), getFailedReason())), - () -> tableRuntime.completeProcess(false) - ); + () -> + doAs( + OptimizingMapper.class, + mapper -> + mapper.updateOptimizingProcess( + tableRuntime.getTableIdentifier().getId(), + processId, + status, + endTime, + getSummary(), + getFailedReason())), + () -> tableRuntime.completeProcess(false)); } else { doAsTransaction( - () -> doAs(OptimizingMapper.class, mapper -> - mapper.updateOptimizingProcess(tableRuntime.getTableIdentifier().getId(), processId, status, endTime, - getSummary(), getFailedReason())), - () -> tableRuntime.completeProcess(true) - ); + () -> + doAs( + OptimizingMapper.class, + mapper -> + mapper.updateOptimizingProcess( + tableRuntime.getTableIdentifier().getId(), + processId, + status, + endTime, + getSummary(), + getFailedReason())), + () -> tableRuntime.completeProcess(true)); } } private void loadTaskRuntimes() { - List taskRuntimes = getAs( - OptimizingMapper.class, - mapper -> mapper.selectTaskRuntimes(tableRuntime.getTableIdentifier().getId(), processId)); + List taskRuntimes = + getAs( + OptimizingMapper.class, + mapper -> + mapper.selectTaskRuntimes(tableRuntime.getTableIdentifier().getId(), processId)); Map inputs = TaskFilesPersistence.loadTaskInputs(processId); - taskRuntimes.forEach(taskRuntime -> { - taskRuntime.claimOwnership(this); - taskRuntime.setInput(inputs.get(taskRuntime.getTaskId().getTaskId())); - taskMap.put(taskRuntime.getTaskId(), taskRuntime); - taskQueue.offer(taskRuntime); - }); + taskRuntimes.forEach( + taskRuntime -> { + taskRuntime.claimOwnership(this); + taskRuntime.setInput(inputs.get(taskRuntime.getTaskId().getTaskId())); + taskMap.put(taskRuntime.getTaskId(), taskRuntime); + taskQueue.offer(taskRuntime); + }); } private void loadTaskRuntimes(List taskDescriptors) { int taskId = 1; for (TaskDescriptor taskDescriptor : taskDescriptors) { - TaskRuntime taskRuntime = new TaskRuntime(new OptimizingTaskId(processId, taskId++), - taskDescriptor, taskDescriptor.properties()); - LOG.info("{} plan new task {}, summary {}", tableRuntime.getTableIdentifier(), taskRuntime.getTaskId(), + TaskRuntime taskRuntime = + new TaskRuntime( + new OptimizingTaskId(processId, taskId++), + taskDescriptor, + taskDescriptor.properties()); + LOG.info( + "{} plan new task {}, summary {}", + tableRuntime.getTableIdentifier(), + taskRuntime.getTaskId(), taskRuntime.getSummary()); taskMap.put(taskRuntime.getTaskId(), taskRuntime.claimOwnership(this)); taskQueue.offer(taskRuntime); diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 6b7b50e197..2d8a866636 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -4,18 +4,15 @@ import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.server.table.TableRuntimeMeta; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import java.util.Comparator; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; public class SchedulingPolicy { @@ -52,10 +49,12 @@ public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { - return tableRuntimeMap.values().stream() - .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) - .min(tableSorter) - .orElse(null); + TableRuntime t = + tableRuntimeMap.values().stream() + .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) + .min(tableSorter) + .orElse(null); + return t; } finally { tableLock.unlock(); } @@ -66,9 +65,10 @@ private boolean shouldSkip(Set skipSet, TableRuntime tableRuntime) } private boolean isTablePending(TableRuntime tableRuntime) { - return tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING && - (tableRuntime.getLastOptimizedSnapshotId() != tableRuntime.getCurrentSnapshotId() || - tableRuntime.getLastOptimizedChangeSnapshotId() != tableRuntime.getCurrentChangeSnapshotId()); + return tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING + && (tableRuntime.getLastOptimizedSnapshotId() != tableRuntime.getCurrentSnapshotId() + || tableRuntime.getLastOptimizedChangeSnapshotId() + != tableRuntime.getCurrentChangeSnapshotId()); } public void addTable(TableRuntime tableRuntime) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java index c1a0c77034..5a475fdd05 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java @@ -45,38 +45,25 @@ public class TaskRuntime extends StatedPersistentBase { private long tableId; private String partition; private OptimizingTaskId taskId; - @StateField - private Status status = Status.PLANNED; + @StateField private Status status = Status.PLANNED; private final TaskStatusMachine statusMachine = new TaskStatusMachine(); - @StateField - private int retry = 0; - @StateField - private long startTime = ArcticServiceConstants.INVALID_TIME; - @StateField - private long endTime = ArcticServiceConstants.INVALID_TIME; - @StateField - private long costTime = 0; - @StateField - private String token; - @StateField - private int threadId = -1; - @StateField - private String failReason; + @StateField private int retry = 0; + @StateField private long startTime = ArcticServiceConstants.INVALID_TIME; + @StateField private long endTime = ArcticServiceConstants.INVALID_TIME; + @StateField private long costTime = 0; + @StateField private String token; + @StateField private int threadId = -1; + @StateField private String failReason; private TaskOwner owner; private RewriteFilesInput input; - @StateField - private RewriteFilesOutput output; - @StateField - private MetricsSummary summary; + @StateField private RewriteFilesOutput output; + @StateField private MetricsSummary summary; private Map properties; - private TaskRuntime() { - } + private TaskRuntime() {} public TaskRuntime( - OptimizingTaskId taskId, - TaskDescriptor taskDescriptor, - Map properties) { + OptimizingTaskId taskId, TaskDescriptor taskDescriptor, Map properties) { this.taskId = taskId; this.partition = taskDescriptor.getPartition(); this.input = taskDescriptor.getInput(); @@ -86,17 +73,18 @@ public TaskRuntime( } public void complete(OptimizerThread thread, OptimizingTaskResult result) { - invokeConsisitency(() -> { - validThread(thread); - if (result.getErrorMessage() != null) { - fail(result.getErrorMessage()); - } else { - finish(TaskFilesPersistence.loadTaskOutput(result.getTaskOutput())); - } - owner.acceptResult(this); - token = null; - threadId = -1; - }); + invokeConsisitency( + () -> { + validThread(thread); + if (result.getErrorMessage() != null) { + fail(result.getErrorMessage()); + } else { + finish(TaskFilesPersistence.loadTaskOutput(result.getTaskOutput())); + } + owner.acceptResult(this); + token = null; + threadId = -1; + }); } private void finish(RewriteFilesOutput filesOutput) { @@ -138,23 +126,25 @@ void reset(boolean incRetryCount) { } public void schedule(OptimizerThread thread) { - invokeConsisitency(() -> { - statusMachine.accept(Status.SCHEDULED); - token = thread.getToken(); - threadId = thread.getThreadId(); - startTime = System.currentTimeMillis(); - persistTaskRuntime(this); - }); + invokeConsisitency( + () -> { + statusMachine.accept(Status.SCHEDULED); + token = thread.getToken(); + threadId = thread.getThreadId(); + startTime = System.currentTimeMillis(); + persistTaskRuntime(this); + }); } public void ack(OptimizerThread thread) { - invokeConsisitency(() -> { - validThread(thread); - statusMachine.accept(Status.ACKED); - startTime = System.currentTimeMillis(); - endTime = ArcticServiceConstants.INVALID_TIME; - persistTaskRuntime(this); - }); + invokeConsisitency( + () -> { + validThread(thread); + statusMachine.accept(Status.ACKED); + startTime = System.currentTimeMillis(); + endTime = ArcticServiceConstants.INVALID_TIME; + persistTaskRuntime(this); + }); } void tryCanceling() { @@ -310,6 +300,9 @@ public String toString() { } private void validThread(OptimizerThread thread) { + if (token == null) { + throw new IllegalStateException("Task not scheduled yet, taskId:" + taskId); + } if (!thread.getToken().equals(getToken()) || thread.getThreadId() != threadId) { throw new DuplicateRuntimeException("Task already acked by optimizer thread + " + thread); } @@ -331,38 +324,18 @@ public TaskQuota getCurrentQuota() { static { nextStatusMap.put( - Status.PLANNED, - Sets.newHashSet( - Status.PLANNED, - Status.SCHEDULED, - Status.CANCELED)); + Status.PLANNED, Sets.newHashSet(Status.PLANNED, Status.SCHEDULED, Status.CANCELED)); nextStatusMap.put( Status.SCHEDULED, - Sets.newHashSet( - Status.PLANNED, - Status.SCHEDULED, - Status.ACKED, - Status.CANCELED)); + Sets.newHashSet(Status.PLANNED, Status.SCHEDULED, Status.ACKED, Status.CANCELED)); nextStatusMap.put( Status.ACKED, Sets.newHashSet( - Status.PLANNED, - Status.ACKED, - Status.SUCCESS, - Status.FAILED, - Status.CANCELED)); - nextStatusMap.put( - Status.FAILED, - Sets.newHashSet( - Status.PLANNED, - Status.FAILED, - Status.CANCELED)); - nextStatusMap.put( - Status.SUCCESS, - Sets.newHashSet(Status.SUCCESS)); + Status.PLANNED, Status.ACKED, Status.SUCCESS, Status.FAILED, Status.CANCELED)); nextStatusMap.put( - Status.CANCELED, - Sets.newHashSet(Status.CANCELED)); + Status.FAILED, Sets.newHashSet(Status.PLANNED, Status.FAILED, Status.CANCELED)); + nextStatusMap.put(Status.SUCCESS, Sets.newHashSet(Status.SUCCESS)); + nextStatusMap.put(Status.CANCELED, Sets.newHashSet(Status.CANCELED)); } private class TaskStatusMachine { diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java index 858463d2fd..4914e74051 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java @@ -117,7 +117,7 @@ List selectOptimizingProcesses( + "#{taskRuntime.startTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + " #{taskRuntime.endTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}, " + "#{taskRuntime.status}, #{taskRuntime.failReason, jdbcType=VARCHAR}," - + " #{taskRuntime.optimizingThread.token, jdbcType=VARCHAR}, #{taskRuntime.optimizingThread.threadId, " + + " #{taskRuntime.token, jdbcType=VARCHAR}, #{taskRuntime.threadId, " + "jdbcType=INTEGER}, #{taskRuntime.output, jdbcType=BLOB, " + " typeHandler=com.netease.arctic.server.persistence.converter.Object2ByteArrayConvert}," + " #{taskRuntime.summary, typeHandler=com.netease.arctic.server.persistence.converter.JsonObjectConverter}," @@ -141,8 +141,8 @@ List selectOptimizingProcesses( @Result(property = "endTime", column = "end_time", typeHandler = Long2TsConverter.class), @Result(property = "status", column = "status"), @Result(property = "failReason", column = "fail_reason"), - @Result(property = "optimizingThread.token", column = "optimizer_token"), - @Result(property = "optimizingThread.threadId", column = "thread_id"), + @Result(property = "token", column = "optimizer_token"), + @Result(property = "threadId", column = "thread_id"), @Result( property = "output", column = "rewrite_output", @@ -195,8 +195,8 @@ List selectTaskRuntimes( + " typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + " cost_time = #{taskRuntime.costTime}, status = #{taskRuntime.status}," + " fail_reason = #{taskRuntime.failReason, jdbcType=VARCHAR}," - + " optimizer_token = #{taskRuntime.optimizingThread.token, jdbcType=VARCHAR}," - + " thread_id = #{taskRuntime.optimizingThread.threadId, jdbcType=INTEGER}," + + " optimizer_token = #{taskRuntime.token, jdbcType=VARCHAR}," + + " thread_id = #{taskRuntime.threadId, jdbcType=INTEGER}," + " rewrite_output = #{taskRuntime.output, jdbcType=BLOB," + " typeHandler=com.netease.arctic.server.persistence.converter.Object2ByteArrayConvert}," + " metrics_summary = #{taskRuntime.summary," diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java index f3b4b895c2..9047892b17 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java @@ -21,7 +21,6 @@ import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.resource.Resource; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import java.util.UUID; @@ -30,20 +29,14 @@ public class OptimizerInstance extends Resource { private String token; private long startTime; private long touchTime; - private OptimizerThread[] threads; - public OptimizerInstance() { - } + public OptimizerInstance() {} public OptimizerInstance(OptimizerRegisterInfo registerInfo, String containerName) { super(registerInfo, containerName); this.token = UUID.randomUUID().toString(); this.touchTime = System.currentTimeMillis(); this.startTime = registerInfo.getStartTime(); - this.threads = new OptimizerThread[registerInfo.getThreadCount()]; - for (int index = 0; index < threads.length; index++) { - threads[index] = new OptimizerThread(index, this); - } } public OptimizerInstance touch() { @@ -55,7 +48,6 @@ public String getToken() { return token; } - public long getTouchTime() { return touchTime; } @@ -69,9 +61,7 @@ public long getStartTime() { return startTime; } - public OptimizerThread getThread(int threadId) { - Preconditions.checkElementIndex(threadId, threads.length); - return threads[threadId]; + return new OptimizerThread(threadId, this); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java index e365241429..cef24a493a 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -1,10 +1,13 @@ package com.netease.arctic.server.resource; +import com.netease.arctic.server.optimizing.TaskRuntime; + import java.util.Objects; public class OptimizerThread { private final int threadId; private final OptimizerInstance optimizer; + private volatile TaskRuntime taskRuntime; protected OptimizerThread(int threadId, OptimizerInstance optimizer) { this.threadId = threadId; @@ -34,9 +37,6 @@ public int hashCode() { @Override public String toString() { - return "OptimizerThread{" + - "threadId=" + threadId + - ", optimizer=" + optimizer + - '}'; + return "OptimizerThread{" + "threadId=" + threadId + ", optimizer=" + optimizer + '}'; } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java index d0da963706..586935f9db 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java @@ -1,8 +1,5 @@ package com.netease.arctic.server.resource; -import com.netease.arctic.ams.api.resource.ResourceGroup; -import com.netease.arctic.server.table.TableRuntime; - public interface QuotaProvider { public int getTotalQuota(String resourceGroup); diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java index dc0690caed..62b9bf2ede 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java +++ b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java @@ -1,24 +1,26 @@ package com.netease.arctic.server; +import com.netease.arctic.BasicTableTestHelper; import com.netease.arctic.TableTestHelper; import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.OptimizingTask; import com.netease.arctic.ams.api.OptimizingTaskId; import com.netease.arctic.ams.api.OptimizingTaskResult; -import com.netease.arctic.ams.api.resource.ResourceGroup; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.catalog.BasicCatalogTestHelper; import com.netease.arctic.catalog.CatalogTestHelper; import com.netease.arctic.io.MixedDataTestHelpers; import com.netease.arctic.optimizing.RewriteFilesOutput; import com.netease.arctic.optimizing.TableOptimizing; import com.netease.arctic.server.exception.IllegalTaskStateException; import com.netease.arctic.server.exception.PluginRetryAuthException; +import com.netease.arctic.server.optimizing.OptimizingProcess; import com.netease.arctic.server.optimizing.OptimizingStatus; import com.netease.arctic.server.optimizing.TaskRuntime; import com.netease.arctic.server.resource.OptimizerInstance; import com.netease.arctic.server.table.AMSTableTestBase; -import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.server.table.TableRuntimeMeta; +import com.netease.arctic.server.table.executor.TableRuntimeRefreshExecutor; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.SerializationUtil; @@ -26,46 +28,70 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.util.ArrayList; import java.util.List; +@RunWith(Parameterized.class) public class TestDefaultOptimizingService extends AMSTableTestBase { + private final int THREAD_ID = 0; private String token; + private Toucher toucher; - public TestDefaultOptimizingService(CatalogTestHelper catalogTestHelper, - TableTestHelper tableTestHelper) { + @Parameterized.Parameters(name = "{0}, {1}") + public static Object[] parameters() { + return new Object[][] { + {new BasicCatalogTestHelper(TableFormat.ICEBERG), new BasicTableTestHelper(false, true)} + }; + } + + public TestDefaultOptimizingService( + CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { super(catalogTestHelper, tableTestHelper, false); } - @BeforeEach + @Before public void prepare() { token = optimizingService().authenticate(buildRegisterInfo()); + toucher = new Toucher(); createDatabase(); createTable(); initTableWithFiles(); + TableRuntimeRefresher refresher = new TableRuntimeRefresher(); + refresher.refreshPending(); + refresher.dispose(); } - @AfterEach + @After public void clear() { try { - optimizingService().listOptimizers() - .forEach(optimizer -> optimizingService().deleteOptimizer(optimizer.getGroupName(), - optimizer.getResourceId())); + optimizingService() + .listOptimizers() + .forEach( + optimizer -> + optimizingService() + .deleteOptimizer(optimizer.getGroupName(), optimizer.getResourceId())); dropTable(); dropDatabase(); + if (toucher != null) { + toucher.stop(); + toucher = null; + } } catch (Exception e) { // ignore } } private void initTableWithFiles() { - ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); + ArcticTable arcticTable = + (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); appendData(arcticTable.asUnkeyedTable(), 1); appendData(arcticTable.asUnkeyedTable(), 2); TableRuntime runtime = tableService().getRuntime(serverTableIdentifier()); @@ -73,70 +99,97 @@ private void initTableWithFiles() { runtime.refresh(tableService().loadTable(serverTableIdentifier())); } - private List appendData(UnkeyedTable table, int id) { - ArrayList newRecords = Lists.newArrayList( - MixedDataTestHelpers.createRecord(table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); + private void appendData(UnkeyedTable table, int id) { + ArrayList newRecords = + Lists.newArrayList( + MixedDataTestHelpers.createRecord( + table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); List dataFiles = MixedDataTestHelpers.writeBaseStore(table, 0L, newRecords, false); AppendFiles appendFiles = table.newAppend(); dataFiles.forEach(appendFiles::appendFile); appendFiles.commit(); - return dataFiles; } @Test public void testPollWithoutAuth() { // 1.poll task clear(); - Assertions.assertThrows(PluginRetryAuthException.class, () -> - optimizingService().pollTask("whatever", 1)); + Assertions.assertThrows( + PluginRetryAuthException.class, () -> optimizingService().pollTask("whatever", THREAD_ID)); } @Test public void testPollOnce() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); + assertTaskStatus(TaskRuntime.Status.SCHEDULED); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + assertTaskStatus(TaskRuntime.Status.ACKED); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test public void testPollTaskTwice() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - optimizingService().touch(token); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); // 3.fail task - optimizingService().completeTask(token, - buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "unknown error")); + assertTaskStatus(TaskRuntime.Status.PLANNED); // 4.retry poll task - OptimizingTask task2 = optimizingService().pollTask(token, 1); + OptimizingTask task2 = optimizingService().pollTask(token, THREAD_ID); Assertions.assertEquals(task2, task); + assertTaskStatus(TaskRuntime.Status.SCHEDULED); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + assertTaskStatus(TaskRuntime.Status.ACKED); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test public void testPollTaskThreeTimes() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); // 3.fail task - optimizingService().completeTask(token, - buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "unknown error")); // 4.retry poll task - OptimizingTask task2 = optimizingService().pollTask(token, 1); + OptimizingTask task2 = optimizingService().pollTask(token, THREAD_ID); Assertions.assertEquals(task2, task); - optimizingService().completeTask(token, - buildOptimizingTaskFailResult(task.getTaskId(), 1, "unknown error")); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "unknown error")); // retry again - OptimizingTask task3 = optimizingService().pollTask(token, 1); + OptimizingTask task3 = optimizingService().pollTask(token, THREAD_ID); Assertions.assertEquals(task3, task); - - //third time would be null - Assertions.assertNull(optimizingService().pollTask(token, 1)); + assertTaskStatus(TaskRuntime.Status.SCHEDULED); + // third time would be null + Assertions.assertNull(optimizingService().pollTask(token, THREAD_ID)); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + assertTaskStatus(TaskRuntime.Status.ACKED); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test @@ -150,79 +203,110 @@ public void testTouch() throws InterruptedException { @Test public void testTouchTimeout() throws InterruptedException { - OptimizingTask task = optimizingService().pollTask(token, 1); + toucher.stop(); + toucher = null; + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - Thread.sleep(600); - Assertions.assertThrows(PluginRetryAuthException.class, () -> - optimizingService().touch(token)); - Assertions.assertThrows(PluginRetryAuthException.class, () -> - optimizingService().pollTask(token, 1)); + Thread.sleep(1000); + Assertions.assertThrows(PluginRetryAuthException.class, () -> optimizingService().touch(token)); + Assertions.assertThrows( + PluginRetryAuthException.class, () -> optimizingService().pollTask(token, THREAD_ID)); + assertTaskStatus(TaskRuntime.Status.PLANNED); token = optimizingService().authenticate(buildRegisterInfo()); - OptimizingTask task2 = optimizingService().pollTask(token, 1); - Assertions.assertNotEquals(task2, task); + OptimizingTask task2 = optimizingService().pollTask(token, THREAD_ID); + Assertions.assertEquals(task2, task); } @Test public void testAckAndCompleteTask() { - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - Assertions.assertThrows(IllegalTaskStateException.class, () -> - optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1))); + Assertions.assertThrows( + IllegalTaskStateException.class, + () -> optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId()))); - optimizingService().ackTask(token, 1, task.getTaskId()); - optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test public void testReloadScheduledTask() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); reload(); - optimizingService().ackTask(token, 1, task.getTaskId()); - optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + assertTaskStatus(TaskRuntime.Status.SCHEDULED); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test public void testReloadAckTask() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - optimizingService().ackTask(token, 1, task.getTaskId()); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); reload(); - optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); - Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), - TaskRuntime.Status.SUCCESS); + assertTaskStatus(TaskRuntime.Status.ACKED); + + TaskRuntime taskRuntime = + optimizingService().listTasks(defaultResourceGroup().getName()).get(0); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); + assertTaskCompleted(taskRuntime); } @Test public void testReloadCompletedTask() { - // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + // THREAD_ID.poll task + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - optimizingService().ackTask(token, 1, task.getTaskId()); - optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId(), 1)); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + optimizingService().completeTask(token, buildOptimizingTaskResult(task.getTaskId())); reload(); - Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), - TaskRuntime.Status.SUCCESS); + assertTaskCompleted(null); + Assertions.assertNull(optimizingService().pollTask(token, THREAD_ID)); } @Test public void testReloadFailedTask() { // 1.poll task - OptimizingTask task = optimizingService().pollTask(token, 1); + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); - optimizingService().ackTask(token, 1, task.getTaskId()); - optimizingService().completeTask(token, - buildOptimizingTaskFailResult(task.getTaskId(), 1, "error")); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "error")); + + reload(); + assertTaskStatus(TaskRuntime.Status.PLANNED); + + OptimizingTask task2 = optimizingService().pollTask(token, THREAD_ID); + Assertions.assertEquals(task2.getTaskId(), task.getTaskId()); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "error")); reload(); - Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus(), - TaskRuntime.Status.FAILED); - Assertions.assertEquals(optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getFailReason(), + assertTaskStatus(TaskRuntime.Status.PLANNED); + + OptimizingTask task3 = optimizingService().pollTask(token, THREAD_ID); + Assertions.assertEquals(task3.getTaskId(), task.getTaskId()); + optimizingService().ackTask(token, THREAD_ID, task.getTaskId()); + optimizingService() + .completeTask(token, buildOptimizingTaskFailResult(task.getTaskId(), "error")); + + Assertions.assertEquals( + optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getFailReason(), "error"); } @@ -236,37 +320,82 @@ private OptimizerRegisterInfo buildRegisterInfo() { return registerInfo; } - private ResourceGroup defaultResourceGroup() { - return new ResourceGroup.Builder("test", "local").build(); - } - - private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, ResourceGroup resourceGroup) { - ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); - TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); - tableRuntimeMeta.setCatalogName(serverTableIdentifier().getCatalog()); - tableRuntimeMeta.setDbName(serverTableIdentifier().getDatabase()); - tableRuntimeMeta.setTableName(serverTableIdentifier().getTableName()); - tableRuntimeMeta.setTableId(serverTableIdentifier().getId()); - tableRuntimeMeta.setTableStatus(status); - tableRuntimeMeta.setTableConfig(TableConfiguration.parseConfig(arcticTable.properties())); - tableRuntimeMeta.setOptimizerGroup(resourceGroup.getName()); - tableRuntimeMeta.constructTableRuntime(tableService()); - return tableRuntimeMeta; - } - - private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, int threadId) { + private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId) { TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); - OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); + OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, THREAD_ID); optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); return optimizingTaskResult; } - private OptimizingTaskResult buildOptimizingTaskFailResult(OptimizingTaskId taskId, int threadId, - String errorMessage) { + private OptimizingTaskResult buildOptimizingTaskFailResult( + OptimizingTaskId taskId, String errorMessage) { TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); - OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); + OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, THREAD_ID); optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); optimizingTaskResult.setErrorMessage(errorMessage); return optimizingTaskResult; } + + private void assertTaskStatus(TaskRuntime.Status expectedStatus) { + Assertions.assertEquals( + expectedStatus, + optimizingService().listTasks(defaultResourceGroup().getName()).get(0).getStatus()); + } + + private void assertTaskCompleted(TaskRuntime taskRuntime) { + if (taskRuntime != null) { + Assertions.assertEquals(TaskRuntime.Status.SUCCESS, taskRuntime.getStatus()); + } + Assertions.assertEquals( + 0, optimizingService().listTasks(defaultResourceGroup().getName()).size()); + Assertions.assertEquals( + OptimizingProcess.Status.RUNNING, + tableService().getRuntime(serverTableIdentifier()).getOptimizingProcess().getStatus()); + Assertions.assertEquals( + OptimizingStatus.COMMITTING, + tableService().getRuntime(serverTableIdentifier()).getOptimizingStatus()); + } + + protected static void reload() { + disposeTableService(); + initTableService(); + } + + private class TableRuntimeRefresher extends TableRuntimeRefreshExecutor { + + public TableRuntimeRefresher() { + super(tableService(), 1, Integer.MAX_VALUE); + } + + void refreshPending() { + execute(tableService().getRuntime(serverTableIdentifier())); + } + } + + private class Toucher implements Runnable { + + private volatile boolean stop = false; + private final Thread thread = new Thread(this); + + public Toucher() { + thread.setDaemon(true); + thread.start(); + } + + public void stop() { + stop = true; + thread.interrupt(); + } + + @Override + public void run() { + while (!stop) { + try { + optimizingService().touch(token); + Thread.sleep(300); + } catch (Throwable ignored) { + } + } + } + } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index b723ac102b..63d6f7382d 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -42,24 +42,24 @@ public class TestOptimizingQueue extends AMSTableTestBase { private final QuotaProvider quotaProvider = resourceGroup -> 1; private final long MAX_POLLING_TIME = 5000; - private final OptimizerThread optimizerThread = new OptimizerThread(1, null) { + private final OptimizerThread optimizerThread = + new OptimizerThread(1, null) { - @Override - public String getToken() { - return "aah"; - } - }; + @Override + public String getToken() { + return "aah"; + } + }; - public TestOptimizingQueue(CatalogTestHelper catalogTestHelper, - TableTestHelper tableTestHelper) { + public TestOptimizingQueue(CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { super(catalogTestHelper, tableTestHelper, true); } @Parameterized.Parameters(name = "{0}, {1}") public static Object[] parameters() { - return new Object[][]{ - {new BasicCatalogTestHelper(TableFormat.ICEBERG), - new BasicTableTestHelper(false, true)}}; + return new Object[][] { + {new BasicCatalogTestHelper(TableFormat.ICEBERG), new BasicTableTestHelper(false, true)} + }; } private OptimizingQueue buildOptimizingGroupService(TableRuntimeMeta tableRuntimeMeta) { @@ -84,7 +84,8 @@ private OptimizingQueue buildOptimizingGroupService() { @Test public void testPollNoTask() { - TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); + TableRuntimeMeta tableRuntimeMeta = + buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); Assert.assertNull(queue.pollTask(0)); } @@ -93,10 +94,12 @@ public void testPollNoTask() { public void testRefreshAndReleaseTable() { OptimizingQueue queue = buildOptimizingGroupService(); Assert.assertEquals(0, queue.getSchedulingPolicy().getTableRuntimeMap().size()); - TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.IDLE, defaultResourceGroup()); + TableRuntimeMeta tableRuntimeMeta = + buildTableRuntimeMeta(OptimizingStatus.IDLE, defaultResourceGroup()); queue.refreshTable(tableRuntimeMeta.getTableRuntime()); Assert.assertEquals(1, queue.getSchedulingPolicy().getTableRuntimeMap().size()); - Assert.assertTrue(queue.getSchedulingPolicy().getTableRuntimeMap().containsKey(serverTableIdentifier())); + Assert.assertTrue( + queue.getSchedulingPolicy().getTableRuntimeMap().containsKey(serverTableIdentifier())); queue.releaseTable(tableRuntimeMeta.getTableRuntime()); Assert.assertEquals(0, queue.getSchedulingPolicy().getTableRuntimeMap().size()); @@ -131,12 +134,15 @@ public void testRetryTask() { TaskRuntime retryTask = queue.pollTask(MAX_POLLING_TIME); Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); + retryTask.schedule(optimizerThread); retryTask.ack(optimizerThread); Assert.assertEquals(TaskRuntime.Status.ACKED, task.getStatus()); queue.retryTask(task, true); retryTask = queue.pollTask(MAX_POLLING_TIME); Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); + retryTask.schedule(optimizerThread); + retryTask.ack(optimizerThread); retryTask.fail("error"); Assert.assertEquals(TaskRuntime.Status.FAILED, task.getStatus()); queue.retryTask(task, true); @@ -149,16 +155,17 @@ public void testRetryTask() { public void testCommitTask() { TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); - Assert.assertEquals(0, queue.collectRunningTasks().size()); Assert.assertEquals(0, queue.collectTasks().size()); TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); - Assert.assertEquals(1, queue.collectRunningTasks().size()); - Assert.assertEquals(1, queue.collectTasks().size()); + task.schedule(optimizerThread); + task.ack(optimizerThread); + Assert.assertEquals( + 1, queue.collectTasks(t -> t.getStatus() == TaskRuntime.Status.ACKED).size()); Assert.assertNotNull(task); - task.complete(optimizerThread, buildOptimizingTaskResult(task.getTaskId(), optimizerThread.getThreadId())); - Assert.assertEquals(0, queue.collectRunningTasks().size()); - Assert.assertEquals(1, queue.collectTasks().size()); + task.complete( + optimizerThread, + buildOptimizingTaskResult(task.getTaskId(), optimizerThread.getThreadId())); Assert.assertEquals(TaskRuntime.Status.SUCCESS, task.getStatus()); // 7.commit @@ -184,36 +191,38 @@ public void testCollectingTasks() { TableRuntimeMeta tableRuntimeMeta = initTableWithFiles(); OptimizingQueue queue = buildOptimizingGroupService(tableRuntimeMeta); Assert.assertEquals(0, queue.collectTasks().size()); - Assert.assertEquals(0, queue.collectRunningTasks().size()); TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); Assert.assertNotNull(task); + task.schedule(optimizerThread); Assert.assertEquals(1, queue.collectTasks().size()); - Assert.assertEquals(1, queue.collectRunningTasks().size()); + Assert.assertEquals( + 1, queue.collectTasks(t -> t.getStatus() == TaskRuntime.Status.SCHEDULED).size()); } private TableRuntimeMeta initTableWithFiles() { - ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); + ArcticTable arcticTable = + (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); appendData(arcticTable.asUnkeyedTable(), 1); appendData(arcticTable.asUnkeyedTable(), 2); - TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); + TableRuntimeMeta tableRuntimeMeta = + buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); TableRuntime runtime = tableRuntimeMeta.getTableRuntime(); runtime.refresh(tableService().loadTable(serverTableIdentifier())); return tableRuntimeMeta; } - private ResourceGroup defaultResourceGroup() { - return new ResourceGroup.Builder("test", "local").build(); - } - - private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, ResourceGroup resourceGroup) { - ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); + private TableRuntimeMeta buildTableRuntimeMeta( + OptimizingStatus status, ResourceGroup resourceGroup) { + ArcticTable arcticTable = + (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); tableRuntimeMeta.setCatalogName(serverTableIdentifier().getCatalog()); tableRuntimeMeta.setDbName(serverTableIdentifier().getDatabase()); tableRuntimeMeta.setTableName(serverTableIdentifier().getTableName()); tableRuntimeMeta.setTableId(serverTableIdentifier().getId()); + tableRuntimeMeta.setFormat(TableFormat.ICEBERG); tableRuntimeMeta.setTableStatus(status); tableRuntimeMeta.setTableConfig(TableConfiguration.parseConfig(arcticTable.properties())); tableRuntimeMeta.setOptimizerGroup(resourceGroup.getName()); @@ -222,8 +231,10 @@ private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, Resource } private void appendData(UnkeyedTable table, int id) { - ArrayList newRecords = Lists.newArrayList( - MixedDataTestHelpers.createRecord(table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); + ArrayList newRecords = + Lists.newArrayList( + MixedDataTestHelpers.createRecord( + table.schema(), id, "111", 0L, "2022-01-01T12:00:00")); List dataFiles = MixedDataTestHelpers.writeBaseStore(table, 0L, newRecords, false); AppendFiles appendFiles = table.newAppend(); dataFiles.forEach(appendFiles::appendFile); diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java b/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java index e5583bc739..051726c917 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/TableServiceTestBase.java @@ -18,6 +18,7 @@ package com.netease.arctic.server.table; +import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.server.ArcticManagementConf; import com.netease.arctic.server.DefaultOptimizingService; import com.netease.arctic.server.utils.Configurations; @@ -27,8 +28,7 @@ public abstract class TableServiceTestBase { - @ClassRule - public static DerbyPersistence DERBY = new DerbyPersistence(); + @ClassRule public static DerbyPersistence DERBY = new DerbyPersistence(); private static DefaultTableService TABLE_SERVICE = null; private static DefaultOptimizingService OPTIMIZING_SERVICE = null; @@ -36,11 +36,15 @@ public abstract class TableServiceTestBase { @BeforeClass public static void initTableService() { Configurations configurations = new Configurations(); - configurations.set(ArcticManagementConf.OPTIMIZER_HB_TIMEOUT, 500L); + configurations.set(ArcticManagementConf.OPTIMIZER_HB_TIMEOUT, 800L); TABLE_SERVICE = new DefaultTableService(new Configurations()); OPTIMIZING_SERVICE = new DefaultOptimizingService(configurations, TABLE_SERVICE); TABLE_SERVICE.addHandlerChain(OPTIMIZING_SERVICE.getTableRuntimeHandler()); TABLE_SERVICE.initialize(); + try { + OPTIMIZING_SERVICE.createResourceGroup(defaultResourceGroup()); + } catch (Throwable ignored) { + } } @AfterClass @@ -56,8 +60,7 @@ protected DefaultOptimizingService optimizingService() { return OPTIMIZING_SERVICE; } - protected static void reload() { - disposeTableService(); - initTableService(); + protected static ResourceGroup defaultResourceGroup() { + return new ResourceGroup.Builder("default", "local").build(); } } From ba5344d0657840b6ac4cf86e6fd88fad751d9fcd Mon Sep 17 00:00:00 2001 From: majin1102 Date: Fri, 10 Nov 2023 20:20:03 +0800 Subject: [PATCH 03/23] [AMORO-1951] optimize default parameters --- .../arctic/server/ArcticManagementConf.java | 8 +- .../server/DefaultOptimizingService.java | 107 +++++++----------- .../server/optimizing/OptimizingQueue.java | 8 +- .../optimizing/TestOptimizingQueue.java | 2 + 4 files changed, 53 insertions(+), 72 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 74aab3c2ea..0e76a639c2 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -122,6 +122,12 @@ public class ArcticManagementConf { .defaultValue(60000L) .withDescription("Interval for refreshing table metadata."); + public static final ConfigOption GLOBAL_MIN_PLANNING_INTERVAL = + ConfigOptions.key("refresh-tables.interval") + .longType() + .defaultValue(60000L) + .withDescription("Minimum interval for planning on a single table."); + public static final ConfigOption BLOCKER_TIMEOUT = ConfigOptions.key("blocker.timeout") .longType() @@ -246,7 +252,7 @@ public class ArcticManagementConf { public static final ConfigOption OPTIMIZER_POLLING_TIMEOUT = ConfigOptions.key("optimizer.polling-timeout") .longType() - .defaultValue(1000L) + .defaultValue(3000L) .withDescription("Optimizer polling task timeout."); /** config key prefix of terminal */ diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 77ac29a021..2773d1726d 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -52,6 +52,7 @@ import com.netease.arctic.server.utils.Configurations; import com.netease.arctic.table.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,15 +60,14 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.DelayQueue; import java.util.concurrent.Delayed; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -90,6 +90,7 @@ public class DefaultOptimizingService extends StatedPersistentBase private final long taskAckTimeout; private final int maxPlanningParallelism; private final long pollingTimeout; + private final long minPlanningInterval; private final Map optimizingQueueByGroup = new ConcurrentHashMap<>(); private final Map optimizingQueueByToken = new ConcurrentHashMap<>(); private final Map authOptimizers = new ConcurrentHashMap<>(); @@ -104,20 +105,16 @@ public DefaultOptimizingService(Configurations serviceConfig, DefaultTableServic this.maxPlanningParallelism = serviceConfig.getInteger(ArcticManagementConf.OPTIMIZER_MAX_PLANNING_PARALLELISM); this.pollingTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_POLLING_TIMEOUT); + this.minPlanningInterval = + serviceConfig.getLong(ArcticManagementConf.GLOBAL_MIN_PLANNING_INTERVAL); this.tableService = tableService; this.tableHandlerChain = new TableRuntimeHandlerImpl(); this.planExecutor = Executors.newCachedThreadPool( - new ThreadFactory() { - private final AtomicInteger threadId = new AtomicInteger(0); - - @Override - public Thread newThread(@NotNull Runnable r) { - Thread thread = new Thread(r, "plan-executor-thread-" + threadId.incrementAndGet()); - thread.setDaemon(true); - return thread; - } - }); + new ThreadFactoryBuilder() + .setNameFormat("plan-executor-thread-%d") + .setDaemon(true) + .build()); } public RuntimeHandlerChain getTableRuntimeHandler() { @@ -142,7 +139,8 @@ private void loadOptimizingQueues(List tableRuntimeMetaList) { this, planExecutor, Optional.ofNullable(tableRuntimeMetas).orElseGet(ArrayList::new), - maxPlanningParallelism); + maxPlanningParallelism, + minPlanningInterval); optimizingQueueByGroup.put(groupName, optimizingQueue); }); optimizers.forEach(optimizer -> registerOptimizer(optimizer, false)); @@ -302,7 +300,8 @@ public void createResourceGroup(ResourceGroup resourceGroup) { this, planExecutor, new ArrayList<>(), - maxPlanningParallelism); + maxPlanningParallelism, + minPlanningInterval); optimizingQueueByGroup.put(resourceGroup.getName(), optimizingQueue); }); } @@ -457,11 +456,11 @@ protected void doDispose() { private class SuspendingDetectTask implements Delayed { - private final String token; + private final OptimizerInstance optimizerInstance; private final long checkDelayedTime; public SuspendingDetectTask(OptimizerInstance optimizer) { - this.token = optimizer.getToken(); + this.optimizerInstance = optimizer; this.checkDelayedTime = optimizer.getTouchTime() + optimizerTouchTimeout; } @@ -477,7 +476,11 @@ public int compareTo(@NotNull Delayed o) { } public String getToken() { - return token; + return optimizerInstance.getToken(); + } + + public OptimizerInstance getOptimizer() { + return optimizerInstance; } } @@ -511,31 +514,18 @@ public void run() { try { SuspendingDetectTask detectTask = suspendingQueue.take(); String token = detectTask.getToken(); - boolean isOptimzerExpired = - Optional.ofNullable(authOptimizers.get(token)) - .map( - optimizer -> - optimizer.getTouchTime() + optimizerTouchTimeout - < System.currentTimeMillis()) - .orElse(true); - OptimizingQueue queue = getQueueByToken(token); - queue - .collectTasks(buildSuspendingPredication(token, isOptimzerExpired)) - .forEach( - task -> { - LOG.info( - "Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", - task.getTaskId(), - task.getResourceDesc()); - // optimizing task of suspending optimizer would not be counted for retrying - queue.retryTask(task, false); - }); - if (isOptimzerExpired) { + boolean isExpired = checkOptimizerTimeout(detectTask.getOptimizer()); + Optional.ofNullable(optimizingQueueByToken.get(token)) + .ifPresent( + queue -> + queue + .collectTasks(buildSuspendingPredication(token, isExpired)) + .forEach(task -> retryTask(task, queue))); + if (isExpired) { LOG.info("Optimizer {} has been expired, unregister it", token); unregisterOptimizer(token); } else { - OptimizerInstance optimizer = authOptimizers.get(token); - detectTimeout(optimizer); + detectTimeout(detectTask.getOptimizer()); } } catch (InterruptedException ignored) { } catch (Throwable t) { @@ -544,37 +534,18 @@ public void run() { } } - private boolean isOptimizerTimeout(String token) { - return Optional.ofNullable(authOptimizers.get(token)) - .map( - optimizer -> - optimizer.getTouchTime() + optimizerTouchTimeout < System.currentTimeMillis()) - .orElse(true); - } - - private void dealSuspendingTasks(String token, boolean isOptimzerExpired) { - OptimizingQueue queue = getQueueByToken(token); - queue - .collectTasks(buildSuspendingPredication(token, isOptimzerExpired)) - .forEach( - task -> { - LOG.info( - "Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", - task.getTaskId(), - task.getResourceDesc()); - // optimizing task of suspending optimizer would not be counted for retrying - queue.retryTask(task, false); - }); + private void retryTask(TaskRuntime task, OptimizingQueue queue) { + LOG.info( + "Task {} is suspending, since it's optimizer is expired, put it to retry queue, optimizer {}", + task.getTaskId(), + task.getResourceDesc()); + // optimizing task of suspending optimizer would not be counted for retrying + queue.retryTask(task, false); } - private void dealOptimizer(String token, boolean isOptimzerExpired) { - if (isOptimzerExpired) { - LOG.info("Optimizer {} has been suspended, unregister it", token); - unregisterOptimizer(token); - } else { - OptimizerInstance optimizer = authOptimizers.get(token); - detectTimeout(optimizer); - } + private boolean checkOptimizerTimeout(OptimizerInstance optimizer) { + return Objects.equals(optimizer, authOptimizers.get(optimizer.getToken())) + && optimizer.getTouchTime() + optimizerTouchTimeout > System.currentTimeMillis(); } private Predicate buildSuspendingPredication( diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 11b669d6f9..5c14262e9d 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -70,7 +70,6 @@ public class OptimizingQueue extends PersistentBase { private static final Logger LOG = LoggerFactory.getLogger(OptimizingQueue.class); - private static final long TABLE_PLANNING_MIN_INTERVAL = 60 * 1000L; private final QuotaProvider quotaProvider; private final Queue tableQueue = new LinkedTransferQueue<>(); @@ -83,6 +82,7 @@ public class OptimizingQueue extends PersistentBase { private final Lock scheduleLock = new ReentrantLock(); private final Condition planningCompleted = scheduleLock.newCondition(); private final int maxPlanningParallelism; + private final long minPlanningInterval; private ResourceGroup optimizerGroup; public OptimizingQueue( @@ -91,7 +91,8 @@ public OptimizingQueue( QuotaProvider quotaProvider, Executor planExecutor, List tableRuntimeMetaList, - int maxPlanningParallelism) { + int maxPlanningParallelism, + long minPlanningInterval) { Preconditions.checkNotNull(optimizerGroup, "Optimizer group can not be null"); this.planExecutor = planExecutor; this.optimizerGroup = optimizerGroup; @@ -99,6 +100,7 @@ public OptimizingQueue( this.schedulingPolicy = new SchedulingPolicy(optimizerGroup); this.tableManager = tableManager; this.maxPlanningParallelism = maxPlanningParallelism; + this.minPlanningInterval = minPlanningInterval; tableRuntimeMetaList.forEach(this::initTableRuntime); } @@ -234,7 +236,7 @@ private void scheduleTableIfNecessary(long startTime) { if (planningTables.size() < maxPlanningParallelism) { Set skipTables = new HashSet<>(planningTables); plannedKeepingTables.entrySet().stream() - .filter(entry -> startTime - entry.getValue() < TABLE_PLANNING_MIN_INTERVAL) + .filter(entry -> startTime - entry.getValue() < minPlanningInterval) .filter(entry -> isOptimizingBlocked(entry.getKey())) .map(Map.Entry::getKey) .forEach(skipTables::add); diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index 63d6f7382d..508fb5c28b 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -69,6 +69,7 @@ private OptimizingQueue buildOptimizingGroupService(TableRuntimeMeta tableRuntim quotaProvider, planExecutor, Collections.singletonList(tableRuntimeMeta), + 1, 1); } @@ -79,6 +80,7 @@ private OptimizingQueue buildOptimizingGroupService() { quotaProvider, planExecutor, Collections.emptyList(), + 1, 1); } From 9a94284f331509a7937a0e9ed0b697f7e21b6504 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Tue, 21 Nov 2023 16:50:58 +0800 Subject: [PATCH 04/23] fix bugs --- .../arctic/server/ArcticManagementConf.java | 6 -- .../server/DefaultOptimizingService.java | 62 +++++++------ .../server/optimizing/OptimizingQueue.java | 18 +++- .../arctic/server/optimizing/TaskRuntime.java | 64 ++++++------- .../server/persistence/PersistentBase.java | 6 +- .../persistence/StatedPersistentBase.java | 92 +++++++++---------- .../persistence/mapper/OptimizingMapper.java | 6 +- .../server/resource/OptimizerInstance.java | 9 ++ .../server/TestDefaultOptimizingService.java | 52 ++++++++--- .../optimizing/TestOptimizingQueue.java | 36 +++++--- 10 files changed, 187 insertions(+), 164 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 0e76a639c2..7f16f9a708 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -318,12 +318,6 @@ public class ArcticManagementConf { public static final String CONTAINER_IMPL = "container-impl"; public static final String CONTAINER_PROPERTIES = "properties"; - // optimizer config - public static final String OPTIMIZER_GROUP_LIST = "optimizer_groups"; - public static final String OPTIMIZER_GROUP_NAME = "name"; - public static final String OPTIMIZER_GROUP_CONTAINER = "container"; - public static final String OPTIMIZER_GROUP_PROPERTIES = "properties"; - public static final String DB_TYPE_DERBY = "derby"; public static final String DB_TYPE_MYSQL = "mysql"; public static final String DB_TYPE_POSTGRES = "postgres"; diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 2773d1726d..a3c6c7b1bc 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -94,7 +94,7 @@ public class DefaultOptimizingService extends StatedPersistentBase private final Map optimizingQueueByGroup = new ConcurrentHashMap<>(); private final Map optimizingQueueByToken = new ConcurrentHashMap<>(); private final Map authOptimizers = new ConcurrentHashMap<>(); - private final SuspendingDetector suspendingDetector = new SuspendingDetector(); + private final OptimizerKeeper optimizerKeeper = new OptimizerKeeper(); private final TableService tableService; private final RuntimeHandlerChain tableHandlerChain; private final Executor planExecutor; @@ -156,7 +156,7 @@ private void registerOptimizer(OptimizerInstance optimizer, boolean needPersiste authOptimizers.put(optimizer.getToken(), optimizer); optimizingQueueByToken.put( optimizer.getToken(), optimizingQueueByGroup.get(optimizer.getGroupName())); - suspendingDetector.detectTimeout(optimizer); + optimizerKeeper.keepInTouch(optimizer); } private void unregisterOptimizer(String token) { @@ -210,7 +210,7 @@ private OptimizingTask extractOptimizingTask( return task.getOptimizingTask(); } catch (Throwable throwable) { LOG.error("Schedule task {} failed, put it to retry queue", task.getTaskId(), throwable); - queue.retryTask(task, false); + queue.retryTask(task); return null; } } @@ -443,36 +443,42 @@ public void handleTableRemoved(TableRuntime tableRuntime) { protected void initHandler(List tableRuntimeMetaList) { LOG.info("OptimizerManagementService begin initializing"); loadOptimizingQueues(tableRuntimeMetaList); - suspendingDetector.start(); + optimizerKeeper.start(); LOG.info("SuspendingDetector for Optimizer has been started."); LOG.info("OptimizerManagementService initializing has completed"); } @Override protected void doDispose() { - suspendingDetector.dispose(); + optimizerKeeper.dispose(); } } - private class SuspendingDetectTask implements Delayed { + private class OptimizerKeepingTask implements Delayed { private final OptimizerInstance optimizerInstance; - private final long checkDelayedTime; + private final long lastTouchTime; - public SuspendingDetectTask(OptimizerInstance optimizer) { + public OptimizerKeepingTask(OptimizerInstance optimizer) { this.optimizerInstance = optimizer; - this.checkDelayedTime = optimizer.getTouchTime() + optimizerTouchTimeout; + this.lastTouchTime = optimizer.getTouchTime(); + } + + public boolean tryKeeping() { + return Objects.equals(optimizerInstance, authOptimizers.get(optimizerInstance.getToken())) && + lastTouchTime != optimizerInstance.getTouchTime(); } @Override public long getDelay(@NotNull TimeUnit unit) { - return unit.convert(checkDelayedTime - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + return unit.convert(lastTouchTime + optimizerTouchTimeout - System.currentTimeMillis(), + TimeUnit.MILLISECONDS); } @Override public int compareTo(@NotNull Delayed o) { - SuspendingDetectTask another = (SuspendingDetectTask) o; - return Long.compare(checkDelayedTime, another.checkDelayedTime); + OptimizerKeepingTask another = (OptimizerKeepingTask) o; + return Long.compare(lastTouchTime, another.lastTouchTime); } public String getToken() { @@ -484,19 +490,19 @@ public OptimizerInstance getOptimizer() { } } - private class SuspendingDetector implements Runnable { + private class OptimizerKeeper implements Runnable { private volatile boolean stopped = false; private final Thread thread = new Thread(this, "SuspendingDetector"); - private final DelayQueue suspendingQueue = new DelayQueue<>(); + private final DelayQueue suspendingQueue = new DelayQueue<>(); - public SuspendingDetector() { + public OptimizerKeeper() { thread.setDaemon(true); } - public void detectTimeout(OptimizerInstance optimizerInstance) { + public void keepInTouch(OptimizerInstance optimizerInstance) { Preconditions.checkNotNull(optimizerInstance, "token can not be null"); - suspendingQueue.add(new SuspendingDetectTask(optimizerInstance)); + suspendingQueue.add(new OptimizerKeepingTask(optimizerInstance)); } public void start() { @@ -512,9 +518,9 @@ public void dispose() { public void run() { while (!stopped) { try { - SuspendingDetectTask detectTask = suspendingQueue.take(); - String token = detectTask.getToken(); - boolean isExpired = checkOptimizerTimeout(detectTask.getOptimizer()); + OptimizerKeepingTask keepingTask = suspendingQueue.take(); + String token = keepingTask.getToken(); + boolean isExpired = !keepingTask.tryKeeping(); Optional.ofNullable(optimizingQueueByToken.get(token)) .ifPresent( queue -> @@ -522,14 +528,17 @@ public void run() { .collectTasks(buildSuspendingPredication(token, isExpired)) .forEach(task -> retryTask(task, queue))); if (isExpired) { - LOG.info("Optimizer {} has been expired, unregister it", token); + LOG.info("Optimizer {} has been expired, unregister it", keepingTask.getOptimizer()); unregisterOptimizer(token); } else { - detectTimeout(detectTask.getOptimizer()); + if (LOG.isDebugEnabled()) { + LOG.debug("Optimizer {} is being touched, keep it", keepingTask.getOptimizer()); + } + keepInTouch(keepingTask.getOptimizer()); } } catch (InterruptedException ignored) { } catch (Throwable t) { - LOG.error("SuspendingDetector has encountered a problem.", t); + LOG.error("OptimizerKeeper has encountered a problem.", t); } } } @@ -540,12 +549,7 @@ private void retryTask(TaskRuntime task, OptimizingQueue queue) { task.getTaskId(), task.getResourceDesc()); // optimizing task of suspending optimizer would not be counted for retrying - queue.retryTask(task, false); - } - - private boolean checkOptimizerTimeout(OptimizerInstance optimizer) { - return Objects.equals(optimizer, authOptimizers.get(optimizer.getToken())) - && optimizer.getTouchTime() + optimizerTouchTimeout > System.currentTimeMillis(); + queue.retryTask(task); } private Predicate buildSuspendingPredication( diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 5c14262e9d..98e0198a05 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -204,8 +204,8 @@ public List collectTasks(Predicate predicate) { .collect(Collectors.toList()); } - public void retryTask(TaskRuntime taskRuntime, boolean incRetryCount) { - taskRuntime.reset(incRetryCount); + public void retryTask(TaskRuntime taskRuntime) { + taskRuntime.reset(); retryTaskQueue.offer(taskRuntime); } @@ -432,14 +432,18 @@ public void acceptResult(TaskRuntime taskRuntime) { clearProcess(this); } } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { - if (taskRuntime.getRetry() <= tableRuntime.getMaxExecuteRetryCount()) { - retryTask(taskRuntime, true); + if (taskRuntime.getRetry() < tableRuntime.getMaxExecuteRetryCount()) { + System.out.println("/n/n #### getMaxExecuteRetryCount {}" + tableRuntime.getMaxExecuteRetryCount()); + System.out.println("/n/n #### retry task times {}" + (taskRuntime.getRunTimes() - 1)); + retryTask(taskRuntime); + System.out.println("/n/n #### task status {}" + taskRuntime.getStatus()); } else { clearProcess(this); this.failedReason = taskRuntime.getFailReason(); this.status = OptimizingProcess.Status.FAILED; this.endTime = taskRuntime.getEndTime(); persistProcessCompleted(false); + System.out.println("/n/n #### task status {}" + taskRuntime.getStatus()); } } } catch (Exception e) { @@ -648,7 +652,11 @@ private void loadTaskRuntimes() { taskRuntime.claimOwnership(this); taskRuntime.setInput(inputs.get(taskRuntime.getTaskId().getTaskId())); taskMap.put(taskRuntime.getTaskId(), taskRuntime); - taskQueue.offer(taskRuntime); + if (taskRuntime.getStatus() == TaskRuntime.Status.PLANNED) { + taskQueue.offer(taskRuntime); + } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { + retryTask(taskRuntime); + } }); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java index 5a475fdd05..47618f6696 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java @@ -47,7 +47,7 @@ public class TaskRuntime extends StatedPersistentBase { private OptimizingTaskId taskId; @StateField private Status status = Status.PLANNED; private final TaskStatusMachine statusMachine = new TaskStatusMachine(); - @StateField private int retry = 0; + @StateField private int runTimes = 0; @StateField private long startTime = ArcticServiceConstants.INVALID_TIME; @StateField private long endTime = ArcticServiceConstants.INVALID_TIME; @StateField private long costTime = 0; @@ -63,7 +63,9 @@ public class TaskRuntime extends StatedPersistentBase { private TaskRuntime() {} public TaskRuntime( - OptimizingTaskId taskId, TaskDescriptor taskDescriptor, Map properties) { + OptimizingTaskId taskId, + TaskDescriptor taskDescriptor, + Map properties) { this.taskId = taskId; this.partition = taskDescriptor.getPartition(); this.input = taskDescriptor.getInput(); @@ -77,49 +79,31 @@ public void complete(OptimizerThread thread, OptimizingTaskResult result) { () -> { validThread(thread); if (result.getErrorMessage() != null) { - fail(result.getErrorMessage()); + statusMachine.accept(Status.FAILED); + failReason = result.getErrorMessage(); + endTime = System.currentTimeMillis(); + costTime += endTime - startTime; } else { - finish(TaskFilesPersistence.loadTaskOutput(result.getTaskOutput())); + statusMachine.accept(Status.SUCCESS); + RewriteFilesOutput filesOutput = + TaskFilesPersistence.loadTaskOutput(result.getTaskOutput()); + summary.setNewFileCnt(OptimizingUtil.getFileCount(filesOutput)); + summary.setNewFileSize(OptimizingUtil.getFileSize(filesOutput)); + endTime = System.currentTimeMillis(); + costTime += endTime - startTime; + output = filesOutput; } + runTimes += 1; + persistTaskRuntime(this); owner.acceptResult(this); token = null; threadId = -1; }); } - private void finish(RewriteFilesOutput filesOutput) { + void reset() { invokeConsisitency( () -> { - statusMachine.accept(Status.SUCCESS); - summary.setNewFileCnt(OptimizingUtil.getFileCount(filesOutput)); - summary.setNewFileSize(OptimizingUtil.getFileSize(filesOutput)); - endTime = System.currentTimeMillis(); - costTime += endTime - startTime; - output = filesOutput; - persistTaskRuntime(this); - }); - } - - void fail(String errorMessage) { - invokeConsisitency( - () -> { - statusMachine.accept(Status.FAILED); - failReason = errorMessage; - endTime = System.currentTimeMillis(); - costTime += endTime - startTime; - persistTaskRuntime(this); - }); - } - - void reset(boolean incRetryCount) { - invokeConsisitency( - () -> { - if (!incRetryCount && status == Status.PLANNED) { - return; - } - if (incRetryCount) { - retry++; - } statusMachine.accept(Status.PLANNED); doAs(OptimizingMapper.class, mapper -> mapper.updateTaskStatus(this, Status.PLANNED)); }); @@ -226,8 +210,12 @@ public Status getStatus() { return status; } + public int getRunTimes() { + return runTimes; + } + public int getRetry() { - return retry; + return runTimes - 1; } public MetricsSummary getMetricsSummary() { @@ -288,7 +276,7 @@ public String toString() { .add("partition", partition) .add("taskId", taskId.getTaskId()) .add("status", status) - .add("retry", retry) + .add("runTimes", runTimes) .add("startTime", startTime) .add("endTime", endTime) .add("costTime", costTime) @@ -333,7 +321,7 @@ public TaskQuota getCurrentQuota() { Sets.newHashSet( Status.PLANNED, Status.ACKED, Status.SUCCESS, Status.FAILED, Status.CANCELED)); nextStatusMap.put( - Status.FAILED, Sets.newHashSet(Status.PLANNED, Status.FAILED, Status.CANCELED)); + Status.FAILED, Sets.newHashSet(Status.PLANNED, Status.FAILED)); nextStatusMap.put(Status.SUCCESS, Sets.newHashSet(Status.SUCCESS)); nextStatusMap.put(Status.CANCELED, Sets.newHashSet(Status.CANCELED)); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java index 691a9508e9..12479c5c73 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java @@ -24,6 +24,7 @@ import org.apache.ibatis.session.TransactionIsolationLevel; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import java.util.Arrays; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -57,9 +58,8 @@ protected final void doAs(Class mapperClz, Consumer consumer) { protected final void doAsTransaction(Runnable... operations) { try (NestedSqlSession session = beginSession()) { try { - for (Runnable runnable : operations) { - runnable.run(); - } + Arrays.stream(operations) + .forEach(Runnable::run); session.commit(); } catch (Throwable t) { session.rollback(); diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java index b14dbe6a64..1bd4fc41c1 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java @@ -7,8 +7,10 @@ import java.lang.annotation.RetentionPolicy; import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Arrays; 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.function.Supplier; @@ -16,22 +18,23 @@ public abstract class StatedPersistentBase extends PersistentBase { - private static final Map, List> stateMetaCache = + private static final Map, Field[]> metaCache = Maps.newConcurrentMap(); + private static final Object NULL_VALUE = new Object(); private final Lock stateLock = new ReentrantLock(); - private List states = Lists.newArrayList(); + private final Field[] consistentFields; protected StatedPersistentBase() { - initStateFields(); + consistentFields = getOrCreateConsistentFields(); } protected final void invokeConsisitency(Runnable runnable) { stateLock.lock(); + Map states = retainStates(); try { - retainStates(); - runnable.run(); + doAsTransaction(runnable); } catch (Throwable throwable) { - restoreStates(); + restoreStates(states); throw throwable; } finally { stateLock.unlock(); @@ -40,11 +43,11 @@ protected final void invokeConsisitency(Runnable runnable) { protected final T invokeConsisitency(Supplier supplier) { stateLock.lock(); + Map states = retainStates(); try { - retainStates(); return supplier.get(); } catch (Throwable throwable) { - restoreStates(); + restoreStates(states); throw throwable; } finally { stateLock.unlock(); @@ -71,59 +74,46 @@ protected final T invokeInStateLock(Supplier supplier) { } } - private void initStateFields() { - states = - stateMetaCache + Map retainStates() { + return Arrays.stream(consistentFields) + .collect(Collectors.toMap(field -> field, this::getValue)); + } + + void restoreStates(Map statesMap) { + statesMap.forEach(this::setValue); + } + + private Field[] getOrCreateConsistentFields() { + return metaCache .computeIfAbsent( getClass(), clz -> { - List states = new ArrayList<>(); + List fields = new ArrayList<>(); while (clz != PersistentBase.class) { - for (Field field : clz.getDeclaredFields()) { - if (field.isAnnotationPresent(StateField.class)) { - states.add(new State(field)); - } - } + Arrays.stream(clz.getDeclaredFields()) + .filter(field -> field.isAnnotationPresent(StateField.class)) + .forEach(fields::add); clz = clz.getSuperclass().asSubclass(PersistentBase.class); } - return states; - }) - .stream() - .map(state -> new State(state.field)) - .collect(Collectors.toList()); - } - - private void retainStates() { - states.forEach(State::retain); + return fields.toArray(new Field[0]); + }); } - private void restoreStates() { - states.forEach(State::restore); - } - - private class State { - private Object value; - private final Field field; - - State(Field field) { - this.field = field; - } - - void retain() { - try { - field.setAccessible(true); - value = field.get(StatedPersistentBase.this); - } catch (IllegalAccessException e) { - throw new IllegalStateException(e); - } + private Object getValue(Field field) { + try { + field.setAccessible(true); + return Optional.ofNullable(field.get(StatedPersistentBase.this)) + .orElse(NULL_VALUE); + } catch (IllegalAccessException e) { + throw new IllegalStateException(e); } + } - void restore() { - try { - field.set(StatedPersistentBase.this, value); - } catch (IllegalAccessException e) { - throw new IllegalStateException(e); - } + void setValue(Field field, Object value) { + try { + field.set(StatedPersistentBase.this, NULL_VALUE.equals(value) ? null : value); + } catch (IllegalAccessException e) { + throw new IllegalStateException(e); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java index 4914e74051..50a5059698 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/OptimizingMapper.java @@ -112,7 +112,7 @@ List selectOptimizingProcesses( + "end_time, status, fail_reason, optimizer_token, thread_id, rewrite_output, metrics_summary, properties) " + "VALUES ", "", - "(#{taskRuntime.taskId.processId}, #{taskRuntime.taskId.taskId}, #{taskRuntime.retry}," + "(#{taskRuntime.taskId.processId}, #{taskRuntime.taskId.taskId}, #{taskRuntime.runTimes}," + " #{taskRuntime.tableId}, #{taskRuntime.partition}, " + "#{taskRuntime.startTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + " #{taskRuntime.endTime, typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}, " @@ -134,7 +134,7 @@ List selectOptimizingProcesses( @Results({ @Result(property = "taskId.processId", column = "process_id"), @Result(property = "taskId.taskId", column = "task_id"), - @Result(property = "retry", column = "retry_num"), + @Result(property = "runTimes", column = "retry_num"), @Result(property = "tableId", column = "table_id"), @Result(property = "partition", column = "partition_data"), @Result(property = "startTime", column = "start_time", typeHandler = Long2TsConverter.class), @@ -188,7 +188,7 @@ List selectTaskRuntimes( List selectOptimizeTaskMetas(@Param("processIds") List processIds); @Update( - "UPDATE task_runtime SET retry_num = #{taskRuntime.retry}, " + "UPDATE task_runtime SET retry_num = #{taskRuntime.runTimes}, " + "start_time = #{taskRuntime.startTime," + " typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + " end_time = #{taskRuntime.endTime," diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java index 9047892b17..0be548c20a 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java @@ -64,4 +64,13 @@ public long getStartTime() { public OptimizerThread getThread(int threadId) { return new OptimizerThread(threadId, this); } + + @Override + public String toString() { + return "OptimizerInstance{" + + "token='" + token + '\'' + + ", startTime=" + startTime + + ", touchTime=" + touchTime + + '}'; + } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java index 62b9bf2ede..74ece52a61 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java +++ b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java @@ -37,6 +37,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.locks.LockSupport; @RunWith(Parameterized.class) public class TestDefaultOptimizingService extends AMSTableTestBase { @@ -59,7 +60,6 @@ public TestDefaultOptimizingService( @Before public void prepare() { - token = optimizingService().authenticate(buildRegisterInfo()); toucher = new Toucher(); createDatabase(); createTable(); @@ -72,6 +72,10 @@ public void prepare() { @After public void clear() { try { + if (toucher != null) { + toucher.stop(); + toucher = null; + } optimizingService() .listOptimizers() .forEach( @@ -80,10 +84,6 @@ public void clear() { .deleteOptimizer(optimizer.getGroupName(), optimizer.getResourceId())); dropTable(); dropDatabase(); - if (toucher != null) { - toucher.stop(); - toucher = null; - } } catch (Exception e) { // ignore } @@ -203,10 +203,10 @@ public void testTouch() throws InterruptedException { @Test public void testTouchTimeout() throws InterruptedException { - toucher.stop(); - toucher = null; OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); + toucher.stop(); + toucher = null; Thread.sleep(1000); Assertions.assertThrows(PluginRetryAuthException.class, () -> optimizingService().touch(token)); Assertions.assertThrows( @@ -234,7 +234,7 @@ public void testAckAndCompleteTask() { } @Test - public void testReloadScheduledTask() { + public void testReloadScheduledTask() throws InterruptedException { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -250,7 +250,7 @@ public void testReloadScheduledTask() { } @Test - public void testReloadAckTask() { + public void testReloadAckTask() throws InterruptedException { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -266,7 +266,7 @@ public void testReloadAckTask() { } @Test - public void testReloadCompletedTask() { + public void testReloadCompletedTask() throws InterruptedException { // THREAD_ID.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -279,7 +279,7 @@ public void testReloadCompletedTask() { } @Test - public void testReloadFailedTask() { + public void testReloadFailedTask() throws InterruptedException { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -356,9 +356,11 @@ private void assertTaskCompleted(TaskRuntime taskRuntime) { tableService().getRuntime(serverTableIdentifier()).getOptimizingStatus()); } - protected static void reload() { + protected void reload() { disposeTableService(); + toucher.suspend(); initTableService(); + toucher.goOn(); } private class TableRuntimeRefresher extends TableRuntimeRefreshExecutor { @@ -375,25 +377,45 @@ void refreshPending() { private class Toucher implements Runnable { private volatile boolean stop = false; + private volatile boolean suspend = false; private final Thread thread = new Thread(this); public Toucher() { + token = optimizingService().authenticate(buildRegisterInfo()); thread.setDaemon(true); thread.start(); } - public void stop() { + public synchronized void stop() throws InterruptedException { stop = true; thread.interrupt(); + thread.join(); + } + + public synchronized void suspend() { + suspend = true; + thread.interrupt(); + } + + public synchronized void goOn() { + suspend = false; + thread.interrupt(); } @Override public void run() { while (!stop) { try { - optimizingService().touch(token); Thread.sleep(300); - } catch (Throwable ignored) { + synchronized (this) { + if (!suspend) { + optimizingService().touch(token); + } + } + } catch (PluginRetryAuthException e) { + e.printStackTrace(); + } catch (Exception ignore) { + // ignore } } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index 508fb5c28b..752727607d 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -18,6 +18,7 @@ import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.TableRuntimeMeta; import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.SerializationUtil; import org.apache.iceberg.AppendFiles; @@ -132,25 +133,25 @@ public void testRetryTask() { TaskRuntime task = queue.pollTask(MAX_POLLING_TIME); Assert.assertNotNull(task); - queue.retryTask(task, false); + for (int i = 0; i < TableProperties.SELF_OPTIMIZING_EXECUTE_RETRY_NUMBER_DEFAULT; i++) { + queue.retryTask(task); + TaskRuntime retryTask = queue.pollTask(MAX_POLLING_TIME); + Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); + retryTask.schedule(optimizerThread); + retryTask.ack(optimizerThread); + retryTask.complete(optimizerThread, buildOptimizingTaskFailed(task.getTaskId(), + optimizerThread.getThreadId())); + Assert.assertEquals(TaskRuntime.Status.PLANNED, task.getStatus()); + } + + queue.retryTask(task); TaskRuntime retryTask = queue.pollTask(MAX_POLLING_TIME); Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); - - retryTask.schedule(optimizerThread); - retryTask.ack(optimizerThread); - Assert.assertEquals(TaskRuntime.Status.ACKED, task.getStatus()); - queue.retryTask(task, true); - retryTask = queue.pollTask(MAX_POLLING_TIME); - Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); - retryTask.schedule(optimizerThread); retryTask.ack(optimizerThread); - retryTask.fail("error"); + retryTask.complete(optimizerThread, buildOptimizingTaskFailed(task.getTaskId(), + optimizerThread.getThreadId())); Assert.assertEquals(TaskRuntime.Status.FAILED, task.getStatus()); - queue.retryTask(task, true); - Assert.assertEquals(TaskRuntime.Status.PLANNED, task.getStatus()); - retryTask = queue.pollTask(MAX_POLLING_TIME); - Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); } @Test @@ -249,4 +250,11 @@ private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, optimizingTaskResult.setTaskOutput(SerializationUtil.simpleSerialize(output)); return optimizingTaskResult; } + + private OptimizingTaskResult buildOptimizingTaskFailed(OptimizingTaskId taskId, int threadId) { + TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); + OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); + optimizingTaskResult.setErrorMessage("error"); + return optimizingTaskResult; + } } From 33042540df60b88a67c6b9281d9154cf7c61cb42 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Tue, 21 Nov 2023 20:09:56 +0800 Subject: [PATCH 05/23] fix warnings and spotless issues --- .../server/DefaultOptimizingService.java | 7 ++-- .../server/optimizing/OptimizingQueue.java | 4 +- .../arctic/server/optimizing/TaskRuntime.java | 16 ++++---- .../server/persistence/PersistentBase.java | 6 +-- .../persistence/StatedPersistentBase.java | 40 ++++++------------- .../server/resource/OptimizerInstance.java | 20 +++++----- .../server/TestDefaultOptimizingService.java | 27 ++++++++++--- .../optimizing/TestOptimizingQueue.java | 29 +++++++++++--- 8 files changed, 85 insertions(+), 64 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index a3c6c7b1bc..84247b87e0 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -465,13 +465,14 @@ public OptimizerKeepingTask(OptimizerInstance optimizer) { } public boolean tryKeeping() { - return Objects.equals(optimizerInstance, authOptimizers.get(optimizerInstance.getToken())) && - lastTouchTime != optimizerInstance.getTouchTime(); + return Objects.equals(optimizerInstance, authOptimizers.get(optimizerInstance.getToken())) + && lastTouchTime != optimizerInstance.getTouchTime(); } @Override public long getDelay(@NotNull TimeUnit unit) { - return unit.convert(lastTouchTime + optimizerTouchTimeout - System.currentTimeMillis(), + return unit.convert( + lastTouchTime + optimizerTouchTimeout - System.currentTimeMillis(), TimeUnit.MILLISECONDS); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 98e0198a05..c3ce25b647 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -112,7 +112,6 @@ private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { } if (tableRuntime.isOptimizingEnabled()) { - // TODO: load task quotas tableRuntime.resetTaskQuotas( System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); if (tableRuntime.getOptimizingStatus() == OptimizingStatus.IDLE @@ -433,7 +432,8 @@ public void acceptResult(TaskRuntime taskRuntime) { } } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { if (taskRuntime.getRetry() < tableRuntime.getMaxExecuteRetryCount()) { - System.out.println("/n/n #### getMaxExecuteRetryCount {}" + tableRuntime.getMaxExecuteRetryCount()); + System.out.println( + "/n/n #### getMaxExecuteRetryCount {}" + tableRuntime.getMaxExecuteRetryCount()); System.out.println("/n/n #### retry task times {}" + (taskRuntime.getRunTimes() - 1)); retryTask(taskRuntime); System.out.println("/n/n #### task status {}" + taskRuntime.getStatus()); diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java index 47618f6696..ff9c5f8d8e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/TaskRuntime.java @@ -63,9 +63,7 @@ public class TaskRuntime extends StatedPersistentBase { private TaskRuntime() {} public TaskRuntime( - OptimizingTaskId taskId, - TaskDescriptor taskDescriptor, - Map properties) { + OptimizingTaskId taskId, TaskDescriptor taskDescriptor, Map properties) { this.taskId = taskId; this.partition = taskDescriptor.getPartition(); this.input = taskDescriptor.getInput(); @@ -87,8 +85,13 @@ public void complete(OptimizerThread thread, OptimizingTaskResult result) { statusMachine.accept(Status.SUCCESS); RewriteFilesOutput filesOutput = TaskFilesPersistence.loadTaskOutput(result.getTaskOutput()); - summary.setNewFileCnt(OptimizingUtil.getFileCount(filesOutput)); - summary.setNewFileSize(OptimizingUtil.getFileSize(filesOutput)); + summary.setNewDataFileCnt(OptimizingUtil.getFileCount(filesOutput.getDataFiles())); + summary.setNewDataSize(OptimizingUtil.getFileSize(filesOutput.getDataFiles())); + summary.setNewDataRecordCnt(OptimizingUtil.getRecordCnt(filesOutput.getDataFiles())); + summary.setNewDeleteFileCnt(OptimizingUtil.getFileCount(filesOutput.getDeleteFiles())); + summary.setNewDeleteSize(OptimizingUtil.getFileSize(filesOutput.getDeleteFiles())); + summary.setNewDeleteRecordCnt( + OptimizingUtil.getRecordCnt(filesOutput.getDeleteFiles())); endTime = System.currentTimeMillis(); costTime += endTime - startTime; output = filesOutput; @@ -320,8 +323,7 @@ public TaskQuota getCurrentQuota() { Status.ACKED, Sets.newHashSet( Status.PLANNED, Status.ACKED, Status.SUCCESS, Status.FAILED, Status.CANCELED)); - nextStatusMap.put( - Status.FAILED, Sets.newHashSet(Status.PLANNED, Status.FAILED)); + nextStatusMap.put(Status.FAILED, Sets.newHashSet(Status.PLANNED, Status.FAILED)); nextStatusMap.put(Status.SUCCESS, Sets.newHashSet(Status.SUCCESS)); nextStatusMap.put(Status.CANCELED, Sets.newHashSet(Status.CANCELED)); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java index 12479c5c73..d2ec10e8ba 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/PersistentBase.java @@ -58,8 +58,7 @@ protected final void doAs(Class mapperClz, Consumer consumer) { protected final void doAsTransaction(Runnable... operations) { try (NestedSqlSession session = beginSession()) { try { - Arrays.stream(operations) - .forEach(Runnable::run); + Arrays.stream(operations).forEach(Runnable::run); session.commit(); } catch (Throwable t) { session.rollback(); @@ -72,8 +71,7 @@ protected final R getAs(Class mapperClz, Function func) { try (NestedSqlSession session = beginSession()) { try { T mapper = getMapper(session, mapperClz); - R result = func.apply(mapper); - return result; + return func.apply(mapper); } catch (Throwable t) { throw ArcticRuntimeException.wrap(t, PersistenceException::new); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java index 1bd4fc41c1..9a54443f4f 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/StatedPersistentBase.java @@ -1,6 +1,5 @@ package com.netease.arctic.server.persistence; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import java.lang.annotation.Retention; @@ -63,17 +62,6 @@ protected final void invokeInStateLock(Runnable runnable) { } } - protected final T invokeInStateLock(Supplier supplier) { - stateLock.lock(); - try { - return supplier.get(); - } catch (Throwable throwable) { - throw throwable; - } finally { - stateLock.unlock(); - } - } - Map retainStates() { return Arrays.stream(consistentFields) .collect(Collectors.toMap(field -> field, this::getValue)); @@ -84,26 +72,24 @@ void restoreStates(Map statesMap) { } private Field[] getOrCreateConsistentFields() { - return metaCache - .computeIfAbsent( - getClass(), - clz -> { - List fields = new ArrayList<>(); - while (clz != PersistentBase.class) { - Arrays.stream(clz.getDeclaredFields()) - .filter(field -> field.isAnnotationPresent(StateField.class)) - .forEach(fields::add); - clz = clz.getSuperclass().asSubclass(PersistentBase.class); - } - return fields.toArray(new Field[0]); - }); + return metaCache.computeIfAbsent( + getClass(), + clz -> { + List fields = new ArrayList<>(); + while (clz != PersistentBase.class) { + Arrays.stream(clz.getDeclaredFields()) + .filter(field -> field.isAnnotationPresent(StateField.class)) + .forEach(fields::add); + clz = clz.getSuperclass().asSubclass(PersistentBase.class); + } + return fields.toArray(new Field[0]); + }); } private Object getValue(Field field) { try { field.setAccessible(true); - return Optional.ofNullable(field.get(StatedPersistentBase.this)) - .orElse(NULL_VALUE); + return Optional.ofNullable(field.get(StatedPersistentBase.this)).orElse(NULL_VALUE); } catch (IllegalAccessException e) { throw new IllegalStateException(e); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java index 0be548c20a..e7379316cb 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java @@ -20,7 +20,6 @@ import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.resource.Resource; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import java.util.UUID; @@ -52,11 +51,6 @@ public long getTouchTime() { return touchTime; } - @VisibleForTesting - public void setTouchTime(long touchTime) { - this.touchTime = touchTime; - } - public long getStartTime() { return startTime; } @@ -67,10 +61,14 @@ public OptimizerThread getThread(int threadId) { @Override public String toString() { - return "OptimizerInstance{" + - "token='" + token + '\'' + - ", startTime=" + startTime + - ", touchTime=" + touchTime + - '}'; + return "OptimizerInstance{" + + "token='" + + token + + '\'' + + ", startTime=" + + startTime + + ", touchTime=" + + touchTime + + '}'; } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java index 74ece52a61..8ed85772ca 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java +++ b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java @@ -1,3 +1,21 @@ +/* + * 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 com.netease.arctic.server; import com.netease.arctic.BasicTableTestHelper; @@ -37,7 +55,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.locks.LockSupport; @RunWith(Parameterized.class) public class TestDefaultOptimizingService extends AMSTableTestBase { @@ -234,7 +251,7 @@ public void testAckAndCompleteTask() { } @Test - public void testReloadScheduledTask() throws InterruptedException { + public void testReloadScheduledTask() { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -250,7 +267,7 @@ public void testReloadScheduledTask() throws InterruptedException { } @Test - public void testReloadAckTask() throws InterruptedException { + public void testReloadAckTask() { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -266,7 +283,7 @@ public void testReloadAckTask() throws InterruptedException { } @Test - public void testReloadCompletedTask() throws InterruptedException { + public void testReloadCompletedTask() { // THREAD_ID.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); @@ -279,7 +296,7 @@ public void testReloadCompletedTask() throws InterruptedException { } @Test - public void testReloadFailedTask() throws InterruptedException { + public void testReloadFailedTask() { // 1.poll task OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); Assertions.assertNotNull(task); diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index 752727607d..4dc4112cb6 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -1,3 +1,21 @@ +/* + * 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 com.netease.arctic.server.optimizing; import com.netease.arctic.BasicTableTestHelper; @@ -139,8 +157,9 @@ public void testRetryTask() { Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); retryTask.schedule(optimizerThread); retryTask.ack(optimizerThread); - retryTask.complete(optimizerThread, buildOptimizingTaskFailed(task.getTaskId(), - optimizerThread.getThreadId())); + retryTask.complete( + optimizerThread, + buildOptimizingTaskFailed(task.getTaskId(), optimizerThread.getThreadId())); Assert.assertEquals(TaskRuntime.Status.PLANNED, task.getStatus()); } @@ -149,8 +168,9 @@ public void testRetryTask() { Assert.assertEquals(retryTask.getTaskId(), task.getTaskId()); retryTask.schedule(optimizerThread); retryTask.ack(optimizerThread); - retryTask.complete(optimizerThread, buildOptimizingTaskFailed(task.getTaskId(), - optimizerThread.getThreadId())); + retryTask.complete( + optimizerThread, + buildOptimizingTaskFailed(task.getTaskId(), optimizerThread.getThreadId())); Assert.assertEquals(TaskRuntime.Status.FAILED, task.getStatus()); } @@ -252,7 +272,6 @@ private OptimizingTaskResult buildOptimizingTaskResult(OptimizingTaskId taskId, } private OptimizingTaskResult buildOptimizingTaskFailed(OptimizingTaskId taskId, int threadId) { - TableOptimizing.OptimizingOutput output = new RewriteFilesOutput(null, null, null); OptimizingTaskResult optimizingTaskResult = new OptimizingTaskResult(taskId, threadId); optimizingTaskResult.setErrorMessage("error"); return optimizingTaskResult; From 990d0ef3d545b83dabb5410370fa5e665a2ba9f5 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 23 Nov 2023 21:17:37 +0800 Subject: [PATCH 06/23] merge from #2290 --- .../server/optimizing/OptimizingQueue.java | 45 +++++++++---------- 1 file changed, 22 insertions(+), 23 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index c3ce25b647..1a50c13050 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -114,8 +114,7 @@ private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { if (tableRuntime.isOptimizingEnabled()) { tableRuntime.resetTaskQuotas( System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); - if (tableRuntime.getOptimizingStatus() == OptimizingStatus.IDLE - || tableRuntime.getOptimizingStatus() == OptimizingStatus.PENDING) { + if (!tableRuntime.getOptimizingStatus().isProcessing()) { schedulingPolicy.addTable(tableRuntime); } else if (tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { tableQueue.offer(new TableOptimizingProcess(tableRuntimeMeta)); @@ -264,11 +263,7 @@ private void triggerAsyncPlanning( process.getTaskMap().size(), currentTime - startTime, skipTables); - } else { - if (throwable != null) { - LOG.error( - "Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); - } + } else if (throwable == null){ LOG.info( "{} skip planning table {} with a total cost of {} ms.", optimizerGroup.getName(), @@ -286,18 +281,26 @@ private CompletableFuture doPlanning(TableRuntime tableR CompletableFuture future = new CompletableFuture<>(); planExecutor.execute( () -> { - AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); - OptimizingPlanner planner = - new OptimizingPlanner( - tableRuntime.refresh(table), - (ArcticTable) table.originalTable(), - getAvailableCore()); - if (planner.isNecessary()) { - TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); - future.complete(optimizingProcess); - } else { - tableRuntime.cleanPendingInput(); - future.complete(null); + tableRuntime.beginPlanning(); + try { + AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + OptimizingPlanner planner = + new OptimizingPlanner( + tableRuntime.refresh(table), + (ArcticTable) table.originalTable(), + getAvailableCore()); + if (planner.isNecessary()) { + TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); + future.complete(optimizingProcess); + } else { + tableRuntime.cleanPendingInput(); + future.complete(null); + } + } catch (Throwable throwable) { + tableRuntime.planFailed(); + LOG.error( + "Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); + future.completeExceptionally(throwable); } }); return future; @@ -432,11 +435,7 @@ public void acceptResult(TaskRuntime taskRuntime) { } } else if (taskRuntime.getStatus() == TaskRuntime.Status.FAILED) { if (taskRuntime.getRetry() < tableRuntime.getMaxExecuteRetryCount()) { - System.out.println( - "/n/n #### getMaxExecuteRetryCount {}" + tableRuntime.getMaxExecuteRetryCount()); - System.out.println("/n/n #### retry task times {}" + (taskRuntime.getRunTimes() - 1)); retryTask(taskRuntime); - System.out.println("/n/n #### task status {}" + taskRuntime.getStatus()); } else { clearProcess(this); this.failedReason = taskRuntime.getFailReason(); From 829dea2bef265e9a8b851ad26508e3dac179add8 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 23 Nov 2023 22:02:57 +0800 Subject: [PATCH 07/23] add apache license and fix spotless --- .../server/optimizing/OptimizingQueue.java | 49 +++++++++---------- .../arctic/server/resource/QuotaProvider.java | 21 +++++++- 2 files changed, 42 insertions(+), 28 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 1a50c13050..5ee3c9409c 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -263,7 +263,7 @@ private void triggerAsyncPlanning( process.getTaskMap().size(), currentTime - startTime, skipTables); - } else if (throwable == null){ + } else if (throwable == null) { LOG.info( "{} skip planning table {} with a total cost of {} ms.", optimizerGroup.getName(), @@ -278,32 +278,27 @@ private void triggerAsyncPlanning( } private CompletableFuture doPlanning(TableRuntime tableRuntime) { - CompletableFuture future = new CompletableFuture<>(); - planExecutor.execute( - () -> { - tableRuntime.beginPlanning(); - try { - AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); - OptimizingPlanner planner = - new OptimizingPlanner( - tableRuntime.refresh(table), - (ArcticTable) table.originalTable(), - getAvailableCore()); - if (planner.isNecessary()) { - TableOptimizingProcess optimizingProcess = new TableOptimizingProcess(planner); - future.complete(optimizingProcess); - } else { - tableRuntime.cleanPendingInput(); - future.complete(null); - } - } catch (Throwable throwable) { - tableRuntime.planFailed(); - LOG.error( - "Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); - future.completeExceptionally(throwable); - } - }); - return future; + return CompletableFuture.supplyAsync(() -> planInternal(tableRuntime), planExecutor); + } + + private TableOptimizingProcess planInternal(TableRuntime tableRuntime) { + tableRuntime.beginPlanning(); + try { + AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + OptimizingPlanner planner = + new OptimizingPlanner( + tableRuntime.refresh(table), (ArcticTable) table.originalTable(), getAvailableCore()); + if (planner.isNecessary()) { + return new TableOptimizingProcess(planner); + } else { + tableRuntime.cleanPendingInput(); + return null; + } + } catch (Throwable throwable) { + tableRuntime.planFailed(); + LOG.error("Planning table {} failed", tableRuntime.getTableIdentifier(), throwable); + throw throwable; + } } private boolean isOptimizingBlocked(TableRuntime tableRuntime) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java index 586935f9db..9c32d63524 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java @@ -1,6 +1,25 @@ +/* + * 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 com.netease.arctic.server.resource; +/** Quota information provider, used for optimizing plan */ public interface QuotaProvider { - public int getTotalQuota(String resourceGroup); + int getTotalQuota(String resourceGroup); } From 372c7eff9a05d5309cd56b421defdf28c0bebe86 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Fri, 24 Nov 2023 19:49:46 +0800 Subject: [PATCH 08/23] fix config error --- .../java/com/netease/arctic/server/ArcticManagementConf.java | 2 +- .../netease/arctic/server/optimizing/SchedulingPolicy.java | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 7f16f9a708..c0bdba525b 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -244,7 +244,7 @@ public class ArcticManagementConf { /** config key prefix of terminal */ public static final ConfigOption OPTIMIZER_MAX_PLANNING_PARALLELISM = - ConfigOptions.key("optimizer.task-ack-timeout") + ConfigOptions.key("optimizer.max-planning-parallelism") .intType() .defaultValue(1) .withDescription("Max planning parallelism in one optimizer group."); diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 2d8a866636..03c79bdf06 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -49,12 +49,10 @@ public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { - TableRuntime t = - tableRuntimeMap.values().stream() + return tableRuntimeMap.values().stream() .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) .min(tableSorter) .orElse(null); - return t; } finally { tableLock.unlock(); } From 927fdb6554d9e00a3ece86f29da984d4605cd4ce Mon Sep 17 00:00:00 2001 From: JinMat Date: Thu, 30 Nov 2023 16:00:17 +0800 Subject: [PATCH 09/23] Update ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java Co-authored-by: ZhouJinsong --- .../com/netease/arctic/server/DefaultOptimizingService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 84247b87e0..247de28b6f 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -554,7 +554,7 @@ private void retryTask(TaskRuntime task, OptimizingQueue queue) { } private Predicate buildSuspendingPredication( - String token, boolean isOptimzerExpired) { + String token, boolean isOptimizerExpired) { return task -> { if (isOptimzerExpired) { return token.equals(task.getToken()); From 3c1cacaba3e05d1452d9dbeee8cd1ec272cd9118 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 30 Nov 2023 21:25:00 +0800 Subject: [PATCH 10/23] add annotations --- .../ams/api/resource/ResourceManager.java | 1 + .../arctic/server/ArcticManagementConf.java | 2 +- .../server/DefaultOptimizingService.java | 5 +- .../server/optimizing/OptimizingQueue.java | 57 +++++++++---------- .../server/optimizing/SchedulingPolicy.java | 25 ++++---- .../server/resource/OptimizerInstance.java | 15 ++--- .../server/resource/OptimizerThread.java | 3 - .../arctic/server/resource/QuotaProvider.java | 6 ++ 8 files changed, 60 insertions(+), 54 deletions(-) diff --git a/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java b/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java index cd3af0eefb..8bca5ae4ec 100644 --- a/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java +++ b/ams/api/src/main/java/com/netease/arctic/ams/api/resource/ResourceManager.java @@ -24,5 +24,6 @@ public interface ResourceManager { Resource getResource(String resourceId); + /** close this resource manager */ void dispose(); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index c0bdba525b..fbb15dca28 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -123,7 +123,7 @@ public class ArcticManagementConf { .withDescription("Interval for refreshing table metadata."); public static final ConfigOption GLOBAL_MIN_PLANNING_INTERVAL = - ConfigOptions.key("refresh-tables.interval") + ConfigOptions.key("planning.min-interval") .longType() .defaultValue(60000L) .withDescription("Minimum interval for planning on a single table."); diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 84247b87e0..584b5ff984 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -366,6 +366,7 @@ public Resource getResource(String resourceId) { @Override public void dispose() { + optimizerKeeper.dispose(); tableHandlerChain.dispose(); optimizingQueueByGroup.clear(); optimizingQueueByToken.clear(); @@ -450,7 +451,7 @@ protected void initHandler(List tableRuntimeMetaList) { @Override protected void doDispose() { - optimizerKeeper.dispose(); + DefaultOptimizingService.this.dispose(); } } @@ -494,7 +495,7 @@ public OptimizerInstance getOptimizer() { private class OptimizerKeeper implements Runnable { private volatile boolean stopped = false; - private final Thread thread = new Thread(this, "SuspendingDetector"); + private final Thread thread = new Thread(this, "optimizer-keeper-thread"); private final DelayQueue suspendingQueue = new DelayQueue<>(); public OptimizerKeeper() { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 5ee3c9409c..7bb8f0c0bc 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -74,11 +74,11 @@ public class OptimizingQueue extends PersistentBase { private final QuotaProvider quotaProvider; private final Queue tableQueue = new LinkedTransferQueue<>(); private final Queue retryTaskQueue = new LinkedTransferQueue<>(); - private final SchedulingPolicy schedulingPolicy; + private final SchedulingPolicy scheduler; private final TableManager tableManager; private final Executor planExecutor; - private final Map plannedKeepingTables = new HashMap<>(); - private final Set planningTables = new HashSet<>(); + private final Map plannedKeepingTables = new HashMap<>(); + private final Set planningTables = new HashSet<>(); private final Lock scheduleLock = new ReentrantLock(); private final Condition planningCompleted = scheduleLock.newCondition(); private final int maxPlanningParallelism; @@ -97,7 +97,7 @@ public OptimizingQueue( this.planExecutor = planExecutor; this.optimizerGroup = optimizerGroup; this.quotaProvider = quotaProvider; - this.schedulingPolicy = new SchedulingPolicy(optimizerGroup); + this.scheduler = new SchedulingPolicy(optimizerGroup); this.tableManager = tableManager; this.maxPlanningParallelism = maxPlanningParallelism; this.minPlanningInterval = minPlanningInterval; @@ -115,7 +115,7 @@ private void initTableRuntime(TableRuntimeMeta tableRuntimeMeta) { tableRuntime.resetTaskQuotas( System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); if (!tableRuntime.getOptimizingStatus().isProcessing()) { - schedulingPolicy.addTable(tableRuntime); + scheduler.addTable(tableRuntime); } else if (tableRuntime.getOptimizingStatus() != OptimizingStatus.COMMITTING) { tableQueue.offer(new TableOptimizingProcess(tableRuntimeMeta)); } @@ -139,12 +139,12 @@ public void refreshTable(TableRuntime tableRuntime) { tableRuntime.getTableIdentifier()); tableRuntime.resetTaskQuotas( System.currentTimeMillis() - ArcticServiceConstants.QUOTA_LOOK_BACK_TIME); - schedulingPolicy.addTable(tableRuntime); + scheduler.addTable(tableRuntime); } } public void releaseTable(TableRuntime tableRuntime) { - schedulingPolicy.removeTable(tableRuntime); + scheduler.removeTable(tableRuntime); LOG.info( "Release queue {} with table {}", optimizerGroup.getName(), @@ -152,7 +152,7 @@ public void releaseTable(TableRuntime tableRuntime) { } public boolean containsTable(ServerTableIdentifier identifier) { - return this.schedulingPolicy.containsTable(identifier); + return scheduler.getTableRuntime(identifier) != null; } private void clearProcess(TableOptimizingProcess optimizingProcess) { @@ -212,7 +212,7 @@ public void updateOptimizerGroup(ResourceGroup optimizerGroup) { this.optimizerGroup.getName().equals(optimizerGroup.getName()), "optimizer group name mismatch"); this.optimizerGroup = optimizerGroup; - schedulingPolicy.setTableSorterIfNeeded(optimizerGroup); + scheduler.setTableSorterIfNeeded(optimizerGroup); } private boolean waitTask(long waitDeadline) { @@ -232,41 +232,43 @@ private boolean waitTask(long waitDeadline) { private void scheduleTableIfNecessary(long startTime) { if (planningTables.size() < maxPlanningParallelism) { - Set skipTables = new HashSet<>(planningTables); + Set skipTables = new HashSet<>(planningTables); plannedKeepingTables.entrySet().stream() - .filter(entry -> startTime - entry.getValue() < minPlanningInterval) - .filter(entry -> isOptimizingBlocked(entry.getKey())) + .filter( + entry -> + startTime - entry.getValue() < minPlanningInterval + || isOptimizingBlocked(entry.getKey())) .map(Map.Entry::getKey) .forEach(skipTables::add); - Optional.ofNullable(schedulingPolicy.scheduleTable(skipTables)) + Optional.ofNullable(scheduler.scheduleTable(skipTables)) .ifPresent(tableRuntime -> triggerAsyncPlanning(tableRuntime, skipTables, startTime)); } } private void triggerAsyncPlanning( - TableRuntime tableRuntime, Set skipTables, long startTime) { + TableRuntime tableRuntime, Set skipTables, long startTime) { LOG.info("Trigger planning table {}", tableRuntime.getTableIdentifier()); - planningTables.add(tableRuntime); + planningTables.add(tableRuntime.getTableIdentifier()); doPlanning(tableRuntime) .whenComplete( (process, throwable) -> { long currentTime = System.currentTimeMillis(); scheduleLock.lock(); try { - plannedKeepingTables.put(tableRuntime, System.currentTimeMillis()); - planningTables.remove(tableRuntime); + plannedKeepingTables.put( + tableRuntime.getTableIdentifier(), System.currentTimeMillis()); + planningTables.remove(tableRuntime.getTableIdentifier()); if (process != null) { tableQueue.offer(process); LOG.info( - "{} completes planning {} tasks with a total cost of {} ms, skipping tables, {}", - optimizerGroup.getName(), + "Completed planning on table {} with {} tasks with a total cost of {} ms, skipping tables {}", + tableRuntime.getTableIdentifier(), process.getTaskMap().size(), currentTime - startTime, skipTables); } else if (throwable == null) { LOG.info( - "{} skip planning table {} with a total cost of {} ms.", - optimizerGroup.getName(), + "Skip planning table {} with a total cost of {} ms.", tableRuntime.getTableIdentifier(), currentTime - startTime); } @@ -301,13 +303,10 @@ private TableOptimizingProcess planInternal(TableRuntime tableRuntime) { } } - private boolean isOptimizingBlocked(TableRuntime tableRuntime) { - if (tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) { - LOG.info("{} optimize is blocked, continue", tableRuntime.getTableIdentifier()); - return false; - } else { - return true; - } + private boolean isOptimizingBlocked(ServerTableIdentifier tableIdentifier) { + return Optional.ofNullable(scheduler.getTableRuntime(tableIdentifier)) + .map(tableRuntime -> tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) + .orElse(false); } private double getAvailableCore() { @@ -317,7 +316,7 @@ private double getAvailableCore() { @VisibleForTesting SchedulingPolicy getSchedulingPolicy() { - return schedulingPolicy; + return scheduler; } private class TableOptimizingProcess implements OptimizingProcess, TaskRuntime.TaskOwner { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 03c79bdf06..34afa7fed7 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -46,20 +46,29 @@ public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { } } - public TableRuntime scheduleTable(Set skipSet) { + public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { return tableRuntimeMap.values().stream() - .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) - .min(tableSorter) - .orElse(null); + .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) + .min(tableSorter) + .orElse(null); } finally { tableLock.unlock(); } } - private boolean shouldSkip(Set skipSet, TableRuntime tableRuntime) { - return skipSet.contains(tableRuntime) || !isTablePending(tableRuntime); + public TableRuntime getTableRuntime(ServerTableIdentifier tableIdentifier) { + tableLock.lock(); + try { + return tableRuntimeMap.get(tableIdentifier); + } finally { + tableLock.unlock(); + } + } + + private boolean shouldSkip(Set skipSet, TableRuntime tableRuntime) { + return skipSet.contains(tableRuntime.getTableIdentifier()) || !isTablePending(tableRuntime); } private boolean isTablePending(TableRuntime tableRuntime) { @@ -87,10 +96,6 @@ public void removeTable(TableRuntime tableRuntime) { } } - public boolean containsTable(ServerTableIdentifier tableIdentifier) { - return tableRuntimeMap.containsKey(tableIdentifier); - } - @VisibleForTesting Map getTableRuntimeMap() { return tableRuntimeMap; diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java index e7379316cb..3379be2d5e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerInstance.java @@ -20,6 +20,7 @@ import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.resource.Resource; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import java.util.UUID; @@ -61,14 +62,10 @@ public OptimizerThread getThread(int threadId) { @Override public String toString() { - return "OptimizerInstance{" - + "token='" - + token - + '\'' - + ", startTime=" - + startTime - + ", touchTime=" - + touchTime - + '}'; + return MoreObjects.toStringHelper(this) + .add("token", token) + .add("startTime", startTime) + .add("touchTime", touchTime) + .toString(); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java index cef24a493a..4697e4253f 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -1,13 +1,10 @@ package com.netease.arctic.server.resource; -import com.netease.arctic.server.optimizing.TaskRuntime; - import java.util.Objects; public class OptimizerThread { private final int threadId; private final OptimizerInstance optimizer; - private volatile TaskRuntime taskRuntime; protected OptimizerThread(int threadId, OptimizerInstance optimizer) { this.threadId = threadId; diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java index 9c32d63524..c2c86beffa 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/QuotaProvider.java @@ -21,5 +21,11 @@ /** Quota information provider, used for optimizing plan */ public interface QuotaProvider { + /** + * Get the total available quota of a specific resource group + * + * @param resourceGroup resource group name + * @return quota of the resource group + */ int getTotalQuota(String resourceGroup); } From 9488cd554ef2b9577a1dd6947f3d03a19535cb41 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 30 Nov 2023 21:27:51 +0800 Subject: [PATCH 11/23] fix compile errors --- .../com/netease/arctic/server/DefaultOptimizingService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 1d599c8199..1b1e3d73e3 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -557,7 +557,7 @@ private void retryTask(TaskRuntime task, OptimizingQueue queue) { private Predicate buildSuspendingPredication( String token, boolean isOptimizerExpired) { return task -> { - if (isOptimzerExpired) { + if (isOptimizerExpired) { return token.equals(task.getToken()); } else { return token.equals(task.getToken()) From 5b03bc821291288bfeccaadb54249cd536e29f43 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 30 Nov 2023 21:32:32 +0800 Subject: [PATCH 12/23] fix import problem --- .../com/netease/arctic/server/DefaultOptimizingService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 1b1e3d73e3..a53a61aab1 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -18,7 +18,6 @@ package com.netease.arctic.server; -import com.google.common.base.Preconditions; import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.OptimizerRegisterInfo; @@ -51,6 +50,7 @@ import com.netease.arctic.server.table.TableService; import com.netease.arctic.server.utils.Configurations; import com.netease.arctic.table.TableProperties; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.jetbrains.annotations.NotNull; @@ -182,7 +182,7 @@ public void touch(String authToken) { } private OptimizerInstance getAuthenticatedOptimizer(String authToken) { - org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument( + Preconditions.checkArgument( authToken != null, "authToken can not be null"); return Optional.ofNullable(authOptimizers.get(authToken)) .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); From 4820eb55d043676c53301d0dee2ba06dee894465 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 30 Nov 2023 21:33:35 +0800 Subject: [PATCH 13/23] remove isDebugEnabled() --- .../arctic/server/DefaultOptimizingService.java | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index a53a61aab1..cc3e517c1b 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -175,9 +175,7 @@ public List listTasks(String optimizerGroup) { @Override public void touch(String authToken) { OptimizerInstance optimizer = getAuthenticatedOptimizer(authToken).touch(); - if (LOG.isDebugEnabled()) { - LOG.debug("Optimizer {} touch time: {}", optimizer.getToken(), optimizer.getTouchTime()); - } + LOG.debug("Optimizer {} touch time: {}", optimizer.getToken(), optimizer.getTouchTime()); doAs(OptimizerMapper.class, mapper -> mapper.updateTouchTime(optimizer.getToken())); } @@ -190,9 +188,7 @@ private OptimizerInstance getAuthenticatedOptimizer(String authToken) { @Override public OptimizingTask pollTask(String authToken, int threadId) { - if (LOG.isDebugEnabled()) { - LOG.debug("Optimizer {} (threadId {}) try polling task", authToken, threadId); - } + LOG.debug("Optimizer {} (threadId {}) try polling task", authToken, threadId); OptimizingQueue queue = getQueueByToken(authToken); return Optional.ofNullable(queue.pollTask(pollingTimeout)) .map( @@ -533,9 +529,7 @@ public void run() { LOG.info("Optimizer {} has been expired, unregister it", keepingTask.getOptimizer()); unregisterOptimizer(token); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Optimizer {} is being touched, keep it", keepingTask.getOptimizer()); - } + LOG.debug("Optimizer {} is being touched, keep it", keepingTask.getOptimizer()); keepInTouch(keepingTask.getOptimizer()); } } catch (InterruptedException ignored) { From c745b473458bd8229b81f5a01f800b3ca0855c54 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Thu, 30 Nov 2023 21:36:01 +0800 Subject: [PATCH 14/23] spotless apply --- .../com/netease/arctic/server/DefaultOptimizingService.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index cc3e517c1b..02ba890a92 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -180,8 +180,7 @@ public void touch(String authToken) { } private OptimizerInstance getAuthenticatedOptimizer(String authToken) { - Preconditions.checkArgument( - authToken != null, "authToken can not be null"); + Preconditions.checkArgument(authToken != null, "authToken can not be null"); return Optional.ofNullable(authOptimizers.get(authToken)) .orElseThrow(() -> new PluginRetryAuthException("Optimizer has not been authenticated")); } From 89c0ce66ece3e0b7e480b10eab1d94f3a2aee8a1 Mon Sep 17 00:00:00 2001 From: JinMat Date: Fri, 1 Dec 2023 12:02:14 +0800 Subject: [PATCH 15/23] Update ArcticManagementConf.java --- .../java/com/netease/arctic/server/ArcticManagementConf.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 16a9c0c4b3..7e84172879 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -141,7 +141,7 @@ public class ArcticManagementConf { .withDescription("Interval for refreshing table metadata."); public static final ConfigOption GLOBAL_MIN_PLANNING_INTERVAL = - ConfigOptions.key("planning.min-interval") + ConfigOptions.key("self-optimizing.min-plan-interval") .longType() .defaultValue(60000L) .withDescription("Minimum interval for planning on a single table."); From 3d80143e1ea3b61f34e0bf8dd4b293efa683140a Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 16:27:13 +0800 Subject: [PATCH 16/23] fix reboot bug and supply document content --- .../server/DefaultOptimizingService.java | 15 ++- .../server/optimizing/OptimizingConfig.java | 19 ++- .../server/optimizing/OptimizingQueue.java | 116 ++++++++---------- .../server/optimizing/SchedulingPolicy.java | 16 ++- .../server/resource/OptimizerThread.java | 18 +++ .../arctic/server/table/TableRuntime.java | 9 ++ .../server/TestDefaultOptimizingService.java | 22 ++++ .../optimizing/TestOptimizingQueue.java | 2 - .../netease/arctic/table/TableProperties.java | 4 + docs/user-guides/configurations.md | 31 ++--- 10 files changed, 157 insertions(+), 95 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 02ba890a92..806a94af5f 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -90,7 +90,6 @@ public class DefaultOptimizingService extends StatedPersistentBase private final long taskAckTimeout; private final int maxPlanningParallelism; private final long pollingTimeout; - private final long minPlanningInterval; private final Map optimizingQueueByGroup = new ConcurrentHashMap<>(); private final Map optimizingQueueByToken = new ConcurrentHashMap<>(); private final Map authOptimizers = new ConcurrentHashMap<>(); @@ -105,8 +104,6 @@ public DefaultOptimizingService(Configurations serviceConfig, DefaultTableServic this.maxPlanningParallelism = serviceConfig.getInteger(ArcticManagementConf.OPTIMIZER_MAX_PLANNING_PARALLELISM); this.pollingTimeout = serviceConfig.getLong(ArcticManagementConf.OPTIMIZER_POLLING_TIMEOUT); - this.minPlanningInterval = - serviceConfig.getLong(ArcticManagementConf.GLOBAL_MIN_PLANNING_INTERVAL); this.tableService = tableService; this.tableHandlerChain = new TableRuntimeHandlerImpl(); this.planExecutor = @@ -139,8 +136,7 @@ private void loadOptimizingQueues(List tableRuntimeMetaList) { this, planExecutor, Optional.ofNullable(tableRuntimeMetas).orElseGet(ArrayList::new), - maxPlanningParallelism, - minPlanningInterval); + maxPlanningParallelism); optimizingQueueByGroup.put(groupName, optimizingQueue); }); optimizers.forEach(optimizer -> registerOptimizer(optimizer, false)); @@ -295,8 +291,7 @@ public void createResourceGroup(ResourceGroup resourceGroup) { this, planExecutor, new ArrayList<>(), - maxPlanningParallelism, - minPlanningInterval); + maxPlanningParallelism); optimizingQueueByGroup.put(resourceGroup.getName(), optimizingQueue); }); } @@ -482,6 +477,10 @@ public String getToken() { return optimizerInstance.getToken(); } + public OptimizingQueue getQueue() { + return optimizingQueueByGroup.get(optimizerInstance.getGroupName()); + } + public OptimizerInstance getOptimizer() { return optimizerInstance; } @@ -518,7 +517,7 @@ public void run() { OptimizerKeepingTask keepingTask = suspendingQueue.take(); String token = keepingTask.getToken(); boolean isExpired = !keepingTask.tryKeeping(); - Optional.ofNullable(optimizingQueueByToken.get(token)) + Optional.ofNullable(keepingTask.getQueue()) .ifPresent( queue -> queue diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingConfig.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingConfig.java index 2cf7fdd1d1..726b65b296 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingConfig.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingConfig.java @@ -67,6 +67,9 @@ public class OptimizingConfig { // base.hive.refresh-interval private long hiveRefreshInterval; + // self-optimizing.min-plan-interval + private long minPlanInterval; + public OptimizingConfig() {} public boolean isEnabled() { @@ -87,6 +90,15 @@ public OptimizingConfig setTargetQuota(double targetQuota) { return this; } + public OptimizingConfig setMinPlanInterval(long minPlanInterval) { + this.minPlanInterval = minPlanInterval; + return this; + } + + public long getMinPlanInterval() { + return minPlanInterval; + } + public String getOptimizerGroup() { return optimizerGroup; } @@ -388,6 +400,11 @@ public static OptimizingConfig parseOptimizingConfig(Map propert PropertyUtil.propertyAsLong( properties, HiveTableProperties.REFRESH_HIVE_INTERVAL, - HiveTableProperties.REFRESH_HIVE_INTERVAL_DEFAULT)); + HiveTableProperties.REFRESH_HIVE_INTERVAL_DEFAULT)) + .setMinPlanInterval( + PropertyUtil.propertyAsLong( + properties, + TableProperties.SELF_OPTIMIZING_MIN_PLAN_INTERVAL, + TableProperties.SELF_OPTIMIZING_MIN_PLAN_INTERVAL_DEFAULT)); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 7bb8f0c0bc..3019ffd6e6 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -20,7 +20,6 @@ import com.google.common.collect.Maps; import com.netease.arctic.AmoroTable; -import com.netease.arctic.ams.api.BlockableOperation; import com.netease.arctic.ams.api.OptimizingTaskId; import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.optimizing.RewriteFilesInput; @@ -49,11 +48,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Queue; import java.util.Set; @@ -77,12 +76,11 @@ public class OptimizingQueue extends PersistentBase { private final SchedulingPolicy scheduler; private final TableManager tableManager; private final Executor planExecutor; - private final Map plannedKeepingTables = new HashMap<>(); + // Keep all planning table identifiers private final Set planningTables = new HashSet<>(); private final Lock scheduleLock = new ReentrantLock(); private final Condition planningCompleted = scheduleLock.newCondition(); private final int maxPlanningParallelism; - private final long minPlanningInterval; private ResourceGroup optimizerGroup; public OptimizingQueue( @@ -91,8 +89,7 @@ public OptimizingQueue( QuotaProvider quotaProvider, Executor planExecutor, List tableRuntimeMetaList, - int maxPlanningParallelism, - long minPlanningInterval) { + int maxPlanningParallelism) { Preconditions.checkNotNull(optimizerGroup, "Optimizer group can not be null"); this.planExecutor = planExecutor; this.optimizerGroup = optimizerGroup; @@ -100,7 +97,6 @@ public OptimizingQueue( this.scheduler = new SchedulingPolicy(optimizerGroup); this.tableManager = tableManager; this.maxPlanningParallelism = maxPlanningParallelism; - this.minPlanningInterval = minPlanningInterval; tableRuntimeMetaList.forEach(this::initTableRuntime); } @@ -158,7 +154,7 @@ public boolean containsTable(ServerTableIdentifier identifier) { private void clearProcess(TableOptimizingProcess optimizingProcess) { tableQueue.removeIf(process -> process.getProcessId() == optimizingProcess.getProcessId()); retryTaskQueue.removeIf( - taskRuntime -> taskRuntime.getProcessId() == optimizingProcess.getProcessId()); + taskRuntime -> taskRuntime.getTaskId().getProcessId() == optimizingProcess.getProcessId()); } public TaskRuntime pollTask(long maxWaitTime) { @@ -175,46 +171,6 @@ private long calculateDeadline(long maxWaitTime) { return deadline <= 0 ? Long.MAX_VALUE : deadline; } - private TaskRuntime fetchTask() { - return Optional.ofNullable(retryTaskQueue.poll()) - .orElse( - Optional.ofNullable(tableQueue.peek()).map(TableOptimizingProcess::poll).orElse(null)); - } - - public TaskRuntime getTask(OptimizingTaskId taskId) { - return tableQueue.stream() - .filter(p -> p.getProcessId() == taskId.getProcessId()) - .findFirst() - .map(p -> p.getTaskMap().get(taskId)) - .orElse(null); - } - - public List collectTasks() { - return tableQueue.stream() - .flatMap(p -> p.getTaskMap().values().stream()) - .collect(Collectors.toList()); - } - - public List collectTasks(Predicate predicate) { - return tableQueue.stream() - .flatMap(p -> p.getTaskMap().values().stream()) - .filter(predicate) - .collect(Collectors.toList()); - } - - public void retryTask(TaskRuntime taskRuntime) { - taskRuntime.reset(); - retryTaskQueue.offer(taskRuntime); - } - - public void updateOptimizerGroup(ResourceGroup optimizerGroup) { - Preconditions.checkArgument( - this.optimizerGroup.getName().equals(optimizerGroup.getName()), - "optimizer group name mismatch"); - this.optimizerGroup = optimizerGroup; - scheduler.setTableSorterIfNeeded(optimizerGroup); - } - private boolean waitTask(long waitDeadline) { scheduleLock.lock(); try { @@ -230,16 +186,21 @@ private boolean waitTask(long waitDeadline) { } } + private TaskRuntime fetchTask() { + return Optional.ofNullable(retryTaskQueue.poll()).orElse(fetchScheduledTask()); + } + + private TaskRuntime fetchScheduledTask() { + return tableQueue.stream() + .map(TableOptimizingProcess::poll) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + private void scheduleTableIfNecessary(long startTime) { if (planningTables.size() < maxPlanningParallelism) { Set skipTables = new HashSet<>(planningTables); - plannedKeepingTables.entrySet().stream() - .filter( - entry -> - startTime - entry.getValue() < minPlanningInterval - || isOptimizingBlocked(entry.getKey())) - .map(Map.Entry::getKey) - .forEach(skipTables::add); Optional.ofNullable(scheduler.scheduleTable(skipTables)) .ifPresent(tableRuntime -> triggerAsyncPlanning(tableRuntime, skipTables, startTime)); } @@ -249,14 +210,13 @@ private void triggerAsyncPlanning( TableRuntime tableRuntime, Set skipTables, long startTime) { LOG.info("Trigger planning table {}", tableRuntime.getTableIdentifier()); planningTables.add(tableRuntime.getTableIdentifier()); - doPlanning(tableRuntime) + CompletableFuture.supplyAsync(() -> planInternal(tableRuntime), planExecutor) .whenComplete( (process, throwable) -> { long currentTime = System.currentTimeMillis(); scheduleLock.lock(); try { - plannedKeepingTables.put( - tableRuntime.getTableIdentifier(), System.currentTimeMillis()); + tableRuntime.setLastPlanTime(currentTime); planningTables.remove(tableRuntime.getTableIdentifier()); if (process != null) { tableQueue.offer(process); @@ -279,10 +239,6 @@ private void triggerAsyncPlanning( }); } - private CompletableFuture doPlanning(TableRuntime tableRuntime) { - return CompletableFuture.supplyAsync(() -> planInternal(tableRuntime), planExecutor); - } - private TableOptimizingProcess planInternal(TableRuntime tableRuntime) { tableRuntime.beginPlanning(); try { @@ -303,10 +259,38 @@ private TableOptimizingProcess planInternal(TableRuntime tableRuntime) { } } - private boolean isOptimizingBlocked(ServerTableIdentifier tableIdentifier) { - return Optional.ofNullable(scheduler.getTableRuntime(tableIdentifier)) - .map(tableRuntime -> tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) - .orElse(false); + public TaskRuntime getTask(OptimizingTaskId taskId) { + return tableQueue.stream() + .filter(p -> p.getProcessId() == taskId.getProcessId()) + .findFirst() + .map(p -> p.getTaskMap().get(taskId)) + .orElse(null); + } + + public List collectTasks() { + return tableQueue.stream() + .flatMap(p -> p.getTaskMap().values().stream()) + .collect(Collectors.toList()); + } + + public List collectTasks(Predicate predicate) { + return tableQueue.stream() + .flatMap(p -> p.getTaskMap().values().stream()) + .filter(predicate) + .collect(Collectors.toList()); + } + + public void retryTask(TaskRuntime taskRuntime) { + taskRuntime.reset(); + retryTaskQueue.offer(taskRuntime); + } + + public void updateOptimizerGroup(ResourceGroup optimizerGroup) { + Preconditions.checkArgument( + this.optimizerGroup.getName().equals(optimizerGroup.getName()), + "optimizer group name mismatch"); + this.optimizerGroup = optimizerGroup; + scheduler.setTableSorterIfNeeded(optimizerGroup); } private double getAvailableCore() { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 34afa7fed7..70c9e5bce8 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -1,6 +1,7 @@ package com.netease.arctic.server.optimizing; import com.google.common.collect.Maps; +import com.netease.arctic.ams.api.BlockableOperation; import com.netease.arctic.ams.api.resource.ResourceGroup; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableRuntime; @@ -49,8 +50,9 @@ public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { + fillSkipSet(skipSet); return tableRuntimeMap.values().stream() - .filter(tableRuntime -> !shouldSkip(skipSet, tableRuntime)) + .filter(tableRuntime -> !skipSet.contains(tableRuntime.getTableIdentifier())) .min(tableSorter) .orElse(null); } finally { @@ -67,8 +69,16 @@ public TableRuntime getTableRuntime(ServerTableIdentifier tableIdentifier) { } } - private boolean shouldSkip(Set skipSet, TableRuntime tableRuntime) { - return skipSet.contains(tableRuntime.getTableIdentifier()) || !isTablePending(tableRuntime); + private void fillSkipSet(Set originalSet) { + long currentTime = System.currentTimeMillis(); + tableRuntimeMap.values().stream() + .filter( + tableRuntime -> + !isTablePending(tableRuntime) + || tableRuntime.isBlocked(BlockableOperation.OPTIMIZE) + || currentTime - tableRuntime.getLastPlanTime() + < tableRuntime.getOptimizingConfig().getMinPlanInterval()) + .forEach(tableRuntime -> originalSet.add(tableRuntime.getTableIdentifier())); } private boolean isTablePending(TableRuntime tableRuntime) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java index 4697e4253f..1644459bae 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -1,3 +1,21 @@ +/* + * 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 com.netease.arctic.server.resource; import java.util.Objects; diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java index 59450ff11f..9f730f7ccf 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java @@ -89,6 +89,7 @@ public class TableRuntime extends StatedPersistentBase { @StateField private volatile TableConfiguration tableConfiguration; @StateField private volatile long processId; @StateField private volatile OptimizingEvaluator.PendingInput pendingInput; + private volatile long lastPlanTime; private final ReentrantLock blockerLock = new ReentrantLock(); @@ -431,6 +432,14 @@ public long getNewestProcessId() { return processId; } + public long getLastPlanTime() { + return lastPlanTime; + } + + public void setLastPlanTime(long lastPlanTime) { + this.lastPlanTime = lastPlanTime; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java index 8ed85772ca..26fa8cc7d7 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java +++ b/ams/server/src/test/java/com/netease/arctic/server/TestDefaultOptimizingService.java @@ -234,6 +234,20 @@ public void testTouchTimeout() throws InterruptedException { Assertions.assertEquals(task2, task); } + @Test + public void testRebootAndPoll() throws InterruptedException { + OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); + Assertions.assertNotNull(task); + reboot(); + + // wait for last optimizer expiring + Thread.sleep(1000); + assertTaskStatus(TaskRuntime.Status.PLANNED); + OptimizingTask task2 = optimizingService().pollTask(token, THREAD_ID); + Assertions.assertNotNull(task2); + Assertions.assertEquals(task2.getTaskId(), task.getTaskId()); + } + @Test public void testAckAndCompleteTask() { OptimizingTask task = optimizingService().pollTask(token, THREAD_ID); @@ -380,6 +394,14 @@ protected void reload() { toucher.goOn(); } + protected void reboot() throws InterruptedException { + disposeTableService(); + toucher.stop(); + toucher = null; + initTableService(); + toucher = new Toucher(); + } + private class TableRuntimeRefresher extends TableRuntimeRefreshExecutor { public TableRuntimeRefresher() { diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index 4dc4112cb6..063c1d6686 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -88,7 +88,6 @@ private OptimizingQueue buildOptimizingGroupService(TableRuntimeMeta tableRuntim quotaProvider, planExecutor, Collections.singletonList(tableRuntimeMeta), - 1, 1); } @@ -99,7 +98,6 @@ private OptimizingQueue buildOptimizingGroupService() { quotaProvider, planExecutor, Collections.emptyList(), - 1, 1); } diff --git a/core/src/main/java/com/netease/arctic/table/TableProperties.java b/core/src/main/java/com/netease/arctic/table/TableProperties.java index cef4fc2fa3..fc4e39abd1 100644 --- a/core/src/main/java/com/netease/arctic/table/TableProperties.java +++ b/core/src/main/java/com/netease/arctic/table/TableProperties.java @@ -31,6 +31,10 @@ public class TableProperties { private TableProperties() {} + public static final String SELF_OPTIMIZING_MIN_PLAN_INTERVAL = + "self-optimizing.min-plan-interval"; + public static final long SELF_OPTIMIZING_MIN_PLAN_INTERVAL_DEFAULT = 60000; + public static final String TABLE_PARTITION_PROPERTIES = "table.partition-properties"; public static final String BASE_TABLE_MAX_TRANSACTION_ID = "base.table.max-transaction-id"; diff --git a/docs/user-guides/configurations.md b/docs/user-guides/configurations.md index ed51491fe4..2c26ec6502 100644 --- a/docs/user-guides/configurations.md +++ b/docs/user-guides/configurations.md @@ -27,21 +27,22 @@ modified through [Alter Table](../using-tables/#modify-table) operations. Self-optimizing configurations are applicable to both Iceberg Format and Mixed streaming Format. -| Key | Default | Description | -|-----------------------------------------------------|------------------|---------------------------------------------------| -| self-optimizing.enabled | true | Enables Self-optimizing | -| self-optimizing.group | default | Optimizer group for Self-optimizing | -| self-optimizing.quota | 0.1 | Quota for Self-optimizing, indicating the CPU resource the table can take up | -| self-optimizing.execute.num-retries | 5 | Number of retries after failure of Self-optimizing | -| self-optimizing.target-size | 134217728(128MB) | Target size for Self-optimizing | -| self-optimizing.max-file-count | 10000 | Maximum number of files processed by a Self-optimizing process | | -| self-optimizing.max-task-size-bytes | 134217728(128MB) | Maximum file size bytes in a single task for splitting tasks | | -| self-optimizing.fragment-ratio | 8 | The fragment file size threshold. We could divide self-optimizing.target-size by this ratio to get the actual fragment file size | -| self-optimizing.minor.trigger.file-count | 12 | The minimum numbers of fragment files to trigger minor optimizing | -| self-optimizing.minor.trigger.interval | 3600000(1 hour) | The time interval in milliseconds to trigger minor optimizing | -| self-optimizing.major.trigger.duplicate-ratio | 0.1 | The ratio of duplicate data of segment files to trigger major optimizing | -| self-optimizing.full.trigger.interval | -1(closed) | The time interval in milliseconds to trigger full optimizing -| self-optimizing.full.rewrite-all-files | true | Whether full optimizing rewrites all files or skips files that do not need to be optimized | +| Key | Default | Description | +|-----------------------------------------------|------------------|----------------------------------------------------------------------------------------------------------------------------------| +| self-optimizing.enabled | true | Enables Self-optimizing | +| self-optimizing.group | default | Optimizer group for Self-optimizing | +| self-optimizing.quota | 0.1 | Quota for Self-optimizing, indicating the CPU resource the table can take up | +| self-optimizing.execute.num-retries | 5 | Number of retries after failure of Self-optimizing | +| self-optimizing.target-size | 134217728(128MB) | Target size for Self-optimizing | +| self-optimizing.max-file-count | 10000 | Maximum number of files processed by a Self-optimizing process | +| self-optimizing.max-task-size-bytes | 134217728(128MB) | Maximum file size bytes in a single task for splitting tasks | +| self-optimizing.fragment-ratio | 8 | The fragment file size threshold. We could divide self-optimizing.target-size by this ratio to get the actual fragment file size | +| self-optimizing.minor.trigger.file-count | 12 | The minimum numbers of fragment files to trigger minor optimizing | +| self-optimizing.minor.trigger.interval | 3600000(1 hour) | The time interval in milliseconds to trigger minor optimizing | +| self-optimizing.major.trigger.duplicate-ratio | 0.1 | The ratio of duplicate data of segment files to trigger major optimizing | +| self-optimizing.full.trigger.interval | -1(closed) | The time interval in milliseconds to trigger full optimizing | +| self-optimizing.full.rewrite-all-files | true | Whether full optimizing rewrites all files or skips files that do not need to be optimized | +| self-optimizing.min-plan-interval | 60000 | The minimum time interval between two self-optimizing planning action | ## Data-cleaning configurations From 28f501ff26ca0ea4d4d91b54397892fde556158e Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 16:37:09 +0800 Subject: [PATCH 17/23] use MoreObjects.toStringHelper for OptimizerThread.java --- .../netease/arctic/server/resource/OptimizerThread.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java index 1644459bae..4f786e0750 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -18,6 +18,9 @@ package com.netease.arctic.server.resource; + +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + import java.util.Objects; public class OptimizerThread { @@ -52,6 +55,9 @@ public int hashCode() { @Override public String toString() { - return "OptimizerThread{" + "threadId=" + threadId + ", optimizer=" + optimizer + '}'; + return MoreObjects.toStringHelper(this) + .add("threadId", threadId) + .add("optimizer", optimizer) + .toString(); } } From 93e7e2fdc2c7bf4580198585ade5643ded6db56f Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 17:58:18 +0800 Subject: [PATCH 18/23] Merged from [AMORO-2376] Print right log info after calculating and sorting tables --- .../server/optimizing/OptimizingQueue.java | 2 +- .../server/optimizing/SchedulingPolicy.java | 35 ++++++++++++------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 3019ffd6e6..f3fa626020 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -208,7 +208,7 @@ private void scheduleTableIfNecessary(long startTime) { private void triggerAsyncPlanning( TableRuntime tableRuntime, Set skipTables, long startTime) { - LOG.info("Trigger planning table {}", tableRuntime.getTableIdentifier()); + LOG.info("Trigger planning table {} by policy {}", tableRuntime.getTableIdentifier(), scheduler.name()); planningTables.add(tableRuntime.getTableIdentifier()); CompletableFuture.supplyAsync(() -> planInternal(tableRuntime), planExecutor) .whenComplete( diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 70c9e5bce8..471bda221a 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -22,6 +22,7 @@ public class SchedulingPolicy { private static final String BALANCED = "balanced"; private final Map tableRuntimeMap = new HashMap<>(); + private volatile String policyName; private Comparator tableSorter; private final Lock tableLock = new ReentrantLock(); @@ -30,23 +31,31 @@ public SchedulingPolicy(ResourceGroup group) { } public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { - String schedulingPolicy = - Optional.ofNullable(optimizerGroup.getProperties()) - .orElseGet(Maps::newHashMap) - .getOrDefault(SCHEDULING_POLICY_PROPERTY_NAME, QUOTA); - if (schedulingPolicy.equalsIgnoreCase(QUOTA)) { - if (tableSorter == null || !(tableSorter instanceof QuotaOccupySorter)) { - tableSorter = new QuotaOccupySorter(); - } - } else if (schedulingPolicy.equalsIgnoreCase(BALANCED)) { - if (tableSorter == null || !(tableSorter instanceof BalancedSorter)) { - tableSorter = new BalancedSorter(); + tableLock.lock(); + try { + policyName = Optional.ofNullable(optimizerGroup.getProperties()) + .orElseGet(Maps::newHashMap) + .getOrDefault(SCHEDULING_POLICY_PROPERTY_NAME, QUOTA); + if (policyName.equalsIgnoreCase(QUOTA)) { + if (tableSorter == null || !(tableSorter instanceof QuotaOccupySorter)) { + tableSorter = new QuotaOccupySorter(); + } + } else if (policyName.equalsIgnoreCase(BALANCED)) { + if (tableSorter == null || !(tableSorter instanceof BalancedSorter)) { + tableSorter = new BalancedSorter(); + } + } else { + throw new IllegalArgumentException("Illegal scheduling policy: " + policyName); } - } else { - throw new IllegalArgumentException("Illegal scheduling policy: " + schedulingPolicy); + } finally { + tableLock.unlock(); } } + public String name() { + return policyName; + } + public TableRuntime scheduleTable(Set skipSet) { tableLock.lock(); try { From ebe5edca909d69d1a95824be5d552839946e7a8f Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 17:59:38 +0800 Subject: [PATCH 19/23] fix import problem --- .../com/netease/arctic/server/optimizing/OptimizingQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index f3fa626020..46a382562f 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -18,7 +18,6 @@ package com.netease.arctic.server.optimizing; -import com.google.common.collect.Maps; import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.OptimizingTaskId; import com.netease.arctic.ams.api.resource.ResourceGroup; @@ -44,6 +43,7 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.StructLikeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From f5caae523d55b079147ad127e834e9a1a20426f9 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 18:43:01 +0800 Subject: [PATCH 20/23] remove unused codes --- .../com/netease/arctic/server/ArcticManagementConf.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index 7e84172879..c675ec70ee 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -140,12 +140,6 @@ public class ArcticManagementConf { .defaultValue(60000L) .withDescription("Interval for refreshing table metadata."); - public static final ConfigOption GLOBAL_MIN_PLANNING_INTERVAL = - ConfigOptions.key("self-optimizing.min-plan-interval") - .longType() - .defaultValue(60000L) - .withDescription("Minimum interval for planning on a single table."); - public static final ConfigOption BLOCKER_TIMEOUT = ConfigOptions.key("blocker.timeout") .longType() From 2b58c90ff69f14fd91afa3ad29b478768a1528fd Mon Sep 17 00:00:00 2001 From: majin1102 Date: Mon, 4 Dec 2023 19:20:13 +0800 Subject: [PATCH 21/23] spotless --- .../netease/arctic/server/optimizing/OptimizingQueue.java | 5 ++++- .../netease/arctic/server/optimizing/SchedulingPolicy.java | 7 ++++--- .../netease/arctic/server/resource/OptimizerThread.java | 1 - 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 46a382562f..683433c9d4 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -208,7 +208,10 @@ private void scheduleTableIfNecessary(long startTime) { private void triggerAsyncPlanning( TableRuntime tableRuntime, Set skipTables, long startTime) { - LOG.info("Trigger planning table {} by policy {}", tableRuntime.getTableIdentifier(), scheduler.name()); + LOG.info( + "Trigger planning table {} by policy {}", + tableRuntime.getTableIdentifier(), + scheduler.name()); planningTables.add(tableRuntime.getTableIdentifier()); CompletableFuture.supplyAsync(() -> planInternal(tableRuntime), planExecutor) .whenComplete( diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java index 471bda221a..7c9be00178 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/SchedulingPolicy.java @@ -33,9 +33,10 @@ public SchedulingPolicy(ResourceGroup group) { public void setTableSorterIfNeeded(ResourceGroup optimizerGroup) { tableLock.lock(); try { - policyName = Optional.ofNullable(optimizerGroup.getProperties()) - .orElseGet(Maps::newHashMap) - .getOrDefault(SCHEDULING_POLICY_PROPERTY_NAME, QUOTA); + policyName = + Optional.ofNullable(optimizerGroup.getProperties()) + .orElseGet(Maps::newHashMap) + .getOrDefault(SCHEDULING_POLICY_PROPERTY_NAME, QUOTA); if (policyName.equalsIgnoreCase(QUOTA)) { if (tableSorter == null || !(tableSorter instanceof QuotaOccupySorter)) { tableSorter = new QuotaOccupySorter(); diff --git a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java index 4f786e0750..01d906948c 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java +++ b/ams/server/src/main/java/com/netease/arctic/server/resource/OptimizerThread.java @@ -18,7 +18,6 @@ package com.netease.arctic.server.resource; - import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import java.util.Objects; From d8464779b750934caf6187349987aa4e48c47012 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Tue, 5 Dec 2023 10:59:59 +0800 Subject: [PATCH 22/23] remove incorrect comments --- ams/dist/src/main/arctic-bin/conf/config.yaml | 1 + .../java/com/netease/arctic/server/ArcticManagementConf.java | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/ams/dist/src/main/arctic-bin/conf/config.yaml b/ams/dist/src/main/arctic-bin/conf/config.yaml index 9ec09c3105..2826aaa829 100644 --- a/ams/dist/src/main/arctic-bin/conf/config.yaml +++ b/ams/dist/src/main/arctic-bin/conf/config.yaml @@ -32,6 +32,7 @@ ams: optimizer: heart-beat-timeout: 60000 # 1min task-ack-timeout: 30000 # 30s + polling-timeout: 3000 # 3s blocker: timeout: 60000 # 1min diff --git a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java index c675ec70ee..a59f575095 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java +++ b/ams/server/src/main/java/com/netease/arctic/server/ArcticManagementConf.java @@ -254,7 +254,6 @@ public class ArcticManagementConf { .defaultValue(30000L) .withDescription("Timeout duration for task acknowledgment."); - /** config key prefix of terminal */ public static final ConfigOption OPTIMIZER_MAX_PLANNING_PARALLELISM = ConfigOptions.key("optimizer.max-planning-parallelism") .intType() From e6efd9785de2db23b727a2df5366f01f72e8b901 Mon Sep 17 00:00:00 2001 From: majin1102 Date: Tue, 5 Dec 2023 11:24:49 +0800 Subject: [PATCH 23/23] add max-planning-parallelism to config --- ams/dist/src/main/arctic-bin/conf/config.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/ams/dist/src/main/arctic-bin/conf/config.yaml b/ams/dist/src/main/arctic-bin/conf/config.yaml index 2826aaa829..e876b72937 100644 --- a/ams/dist/src/main/arctic-bin/conf/config.yaml +++ b/ams/dist/src/main/arctic-bin/conf/config.yaml @@ -33,6 +33,7 @@ ams: heart-beat-timeout: 60000 # 1min task-ack-timeout: 30000 # 30s polling-timeout: 3000 # 3s + max-planning-parallelism: 1 # default 1 blocker: timeout: 60000 # 1min