From a2e1ad3d4be16b6266c2ccec28b961686c6c6163 Mon Sep 17 00:00:00 2001 From: Whatfor <31770280+CrazyMountain@users.noreply.github.com> Date: Thu, 28 Sep 2023 15:53:13 +0800 Subject: [PATCH] [ISSUE-183] Support Async Shuffle --- .../config/keys/ExecutionConfigKeys.java | 27 +- .../geaflow/common/metric/CycleMetrics.java | 28 +++ .../geaflow/common/metric/EventMetrics.java | 184 +++++++++----- .../common/metric/ShuffleReadMetrics.java | 5 + .../common/serialize/impl/KryoSerializer.java | 4 +- .../geaflow/common/thread/Executors.java | 50 +++- .../geaflow/common/utils/LoggerFormatter.java | 32 ++- .../geaflow/io/AbstractMessageBuffer.java | 63 +++++ .../antgroup/geaflow/io/IMessageBuffer.java} | 29 ++- .../collector/AbstractEmitterRequest.java | 37 +++ .../collector/AbstractPipelineCollector.java | 58 +---- .../AbstractPipelineOutputCollector.java | 9 +- .../collector/ClearEmitterRequest.java | 30 +++ .../collector/CloseEmitterRequest.java | 17 +- .../cluster/collector/CollectorFactory.java | 8 +- .../cluster/collector/EmitterRunner.java | 28 +-- .../cluster/collector/EmitterService.java | 13 +- .../cluster/collector/IEmitterRequest.java | 18 +- .../collector/IOutputMessageBuffer.java | 55 +++++ .../collector/InitCollectEmitterRequest.java | 39 --- .../cluster/collector/InitEmitterRequest.java | 70 ++++-- .../collector/PipelineOutputEmitter.java | 228 +++++++++++++++++ .../cluster/collector/RequestType.java | 10 + .../collector/UpdateEmitterRequest.java | 54 ++++ .../geaflow/cluster/container/Container.java | 2 +- ...Listener.java => IInputMessageBuffer.java} | 14 +- .../cluster/fetcher/InitFetchRequest.java | 6 +- .../cluster/fetcher/PipelineInputFetcher.java | 24 +- .../cluster/protocol/AbstractMessage.java | 34 +++ .../geaflow/cluster/protocol/IMessage.java | 10 +- .../{Message.java => InputMessage.java} | 28 +-- .../cluster/protocol/OutputMessage.java | 53 ++++ .../pipeline/channel/LocalInputChannel.java | 2 +- .../shuffle/api/reader/AbstractFetcher.java | 19 +- .../shuffle/api/reader/IShuffleReader.java | 8 + .../shuffle/api/reader/PipelineFetcher.java | 9 +- .../shuffle/api/reader/PipelineReader.java | 6 + .../shuffle/api/writer/IShuffleWriter.java | 20 ++ .../api/writer/PipelineShardBuffer.java | 6 +- .../shuffle/api/writer/PipelineWriter.java | 12 + .../shuffle/api/writer/ShardBuffer.java | 59 ++--- .../api/writer/SpillableShardBuffer.java | 29 +-- .../geaflow/shuffle/config/ShuffleConfig.java | 59 ++++- .../netty/SliceRequestServerHandler.java | 1 - .../api/writer/SpillableShardBufferTest.java | 2 +- .../operator/base/AbstractOperator.java | 4 +- .../geaflow/core/graph/ExecutionTask.java | 28 ++- .../core/context/DefaultRuntimeContext.java | 16 +- .../runtime/core/context/EventContext.java | 4 - .../protocol/AbstractExecutableCommand.java | 13 +- .../core/protocol/AbstractInitCommand.java | 147 +++++++++-- .../AbstractIterationComputeCommand.java | 3 + .../core/protocol/CleanCycleEvent.java | 27 +- .../runtime/core/protocol/CleanEnvEvent.java | 12 +- .../core/protocol/CleanStashEnvEvent.java | 6 +- .../runtime/core/protocol/DoneEvent.java | 20 +- .../core/protocol/FinishIterationEvent.java | 5 +- .../core/protocol/InitCollectCycleEvent.java | 24 +- .../runtime/core/protocol/InitCycleEvent.java | 56 +---- .../core/protocol/InitIterationEvent.java | 15 +- ...IterationExecutionComputeWithAggEvent.java | 18 +- .../core/protocol/LaunchSourceEvent.java | 16 +- .../runtime/core/protocol/PopWorkerEvent.java | 31 +-- .../core/protocol/StashWorkerEvent.java | 27 +- .../scheduler/AbstractCycleScheduler.java | 5 +- .../scheduler/ExecutionCycleTaskAssigner.java | 1 + .../ExecutionGraphCycleScheduler.java | 61 ++--- .../scheduler/PipelineCycleScheduler.java | 230 ++++++++++-------- .../core/scheduler/SchedulerEventBuilder.java | 8 +- .../AbstractCycleSchedulerContext.java | 2 +- .../core/worker/AbstractAlignedWorker.java | 83 +++---- .../runtime/core/worker/InputReader.java | 40 ++- .../runtime/core/worker/OutputWriter.java | 95 +++++++- .../worker/context/AbstractWorkerContext.java | 30 ++- .../core/worker/context/WorkerContext.java | 6 +- .../core/worker/fetch/FetchListenerImpl.java | 43 ---- .../core/worker/impl/ComputeWorker.java | 35 ++- .../scheduler/BaseCycleSchedulerTest.java | 4 +- .../geaflow/utils/BlockingListBenchmark.java | 70 ++++++ 79 files changed, 1819 insertions(+), 865 deletions(-) create mode 100644 geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/AbstractMessageBuffer.java rename geaflow/geaflow-core/{geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineMaster.java => geaflow-core-common/src/main/java/com/antgroup/geaflow/io/IMessageBuffer.java} (53%) create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractEmitterRequest.java create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/ClearEmitterRequest.java create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IOutputMessageBuffer.java delete mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitCollectEmitterRequest.java create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/PipelineOutputEmitter.java create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/UpdateEmitterRequest.java rename geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/{FetchListener.java => IInputMessageBuffer.java} (67%) create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/AbstractMessage.java rename geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/{Message.java => InputMessage.java} (62%) create mode 100644 geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/OutputMessage.java delete mode 100644 geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/fetch/FetchListenerImpl.java create mode 100644 geaflow/geaflow-utils/src/test/java/com/antgroup/geaflow/utils/BlockingListBenchmark.java diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/config/keys/ExecutionConfigKeys.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/config/keys/ExecutionConfigKeys.java index 946ca85c4..ee69a11a9 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/config/keys/ExecutionConfigKeys.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/config/keys/ExecutionConfigKeys.java @@ -384,6 +384,11 @@ public class ExecutionConfigKeys implements Serializable { .defaultValue(600000) .description("shuffle fetch timeout in milliseconds"); + public static final ConfigKey SHUFFLE_FETCH_QUEUE_SIZE = ConfigKeys + .key("geaflow.shuffle.fetch.queue.size") + .defaultValue(1) + .description("size of shuffle fetch queue"); + /** shuffle write config. */ public static final ConfigKey SHUFFLE_SPILL_RECORDS = ConfigKeys @@ -396,15 +401,25 @@ public class ExecutionConfigKeys implements Serializable { .defaultValue(1610612736L) // 1.5G .description("max size of each spill per slice in Bytes"); - public static final ConfigKey SHUFFLE_WRITE_BUFFER_SIZE = ConfigKeys - .key("geaflow.shuffle.write.buffer.size") - .defaultValue(15360) + public static final ConfigKey SHUFFLE_WRITE_BUFFER_SIZE_BYTES = ConfigKeys + .key("geaflow.shuffle.write.buffer.size.bytes") + .defaultValue(128 * 1024) .description("size of shuffle write buffer"); - public static final ConfigKey SHUFFLE_FLUSH_BUFFER_TIMEOUT = ConfigKeys - .key("geaflow.shuffle.flush.buffer.timeout") + public static final ConfigKey SHUFFLE_EMIT_BUFFER_SIZE = ConfigKeys + .key("geaflow.shuffle.emit.buffer.size") + .defaultValue(1024) + .description("size of shuffle emit buffer of java object"); + + public static final ConfigKey SHUFFLE_EMIT_QUEUE_SIZE = ConfigKeys + .key("geaflow.shuffle.emit.queue.size") + .defaultValue(1) + .description("size of shuffle emit queue"); + + public static final ConfigKey SHUFFLE_FLUSH_BUFFER_TIMEOUT_MS = ConfigKeys + .key("geaflow.shuffle.flush.buffer.timeout.ms") .defaultValue(100) - .description("shuffle flush buffer timeout"); + .description("shuffle flush buffer timeout ms"); public static final ConfigKey SHUFFLE_CACHE_SPILL_THRESHOLD = ConfigKeys .key("geaflow.shuffle.cache.spill.threshold") diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/CycleMetrics.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/CycleMetrics.java index 6be5031e5..43d03cee7 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/CycleMetrics.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/CycleMetrics.java @@ -144,6 +144,34 @@ public void setOutputKb(long outputKb) { this.outputKb = outputKb; } + public static CycleMetrics build(String metricName, + String pipelineName, + String opName, + int taskNum, + int slowestTask, + long startTime, + long duration, + long totalExecuteTime, + long totalGcTime, + long slowestTaskExecuteTime, + long totalInputRecords, + long totalInputBytes, + long totalOutputRecords, + long totalOutputBytes) { + CycleMetrics cycleMetrics = new CycleMetrics(metricName, pipelineName, opName); + cycleMetrics.setStartTime(startTime); + cycleMetrics.setTotalTasks(taskNum); + cycleMetrics.setSlowestTask(slowestTask); + cycleMetrics.setDuration(duration); + cycleMetrics.setAvgExecuteTime(totalExecuteTime / taskNum); + cycleMetrics.setAvgGcTime(totalGcTime / taskNum); + cycleMetrics.setSlowestTaskExecuteTime(slowestTaskExecuteTime); + cycleMetrics.setInputRecords(totalInputRecords); + cycleMetrics.setInputKb(totalInputBytes / 1024); + cycleMetrics.setOutputRecords(totalOutputRecords); + cycleMetrics.setOutputKb(totalOutputBytes / 1024); + return cycleMetrics; + } @Override public String toString() { diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/EventMetrics.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/EventMetrics.java index 03a8ff803..a0aa40455 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/EventMetrics.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/EventMetrics.java @@ -14,114 +14,188 @@ package com.antgroup.geaflow.common.metric; +import com.antgroup.geaflow.common.utils.GcUtil; import java.io.Serializable; public class EventMetrics implements Serializable { /** - * execute compute event time cost. + * Meta. */ - private long executeTime; + private final int vertexId; + private final int parallelism; + private final int index; /** - * gc time in total during current task execution. - */ - private long gcTime; - - /** - * finish time of process certain windowId. + * Execution. */ + private long startTime; private long finishTime; + private long executeCostMs; + private long processCostMs; + private long gcCostMs; /** - * total input records. - */ - private long inputRecords; - private long inputBytes; - - /** - * total output records. + * Shuffle. */ - private long outputRecords; - private long outputBytes; + private long shuffleReadRecords; + private long shuffleReadBytes; + private long shuffleReadCostMs; - private transient long startTs; + private long shuffleWriteRecords; + private long shuffleWriteBytes; + private long shuffleWriteCostMs; private transient long startGcTs; - public EventMetrics() { + public EventMetrics(int vertexId, int parallelism, int index) { + this.vertexId = vertexId; + this.parallelism = parallelism; + this.index = index; + this.startTime = System.currentTimeMillis(); + this.startGcTs = GcUtil.computeCurrentTotalGcTime(); } - public long getExecuteTime() { - return executeTime; + public int getVertexId() { + return this.vertexId; } - public void setExecuteTime(long executeTime) { - this.executeTime = executeTime; + public int getParallelism() { + return this.parallelism; } - public long getGcTime() { - return gcTime; + public int getIndex() { + return this.index; } - public void setGcTime(long gcTime) { - this.gcTime = gcTime; + public long getStartTime() { + return this.startTime; } - public long getStartTs() { - return startTs; + public void setStartTime(long startTime) { + this.startTime = startTime; } - public void setStartTs(long startTs) { - this.startTs = startTs; + public long getFinishTime() { + return this.finishTime; } - public long getStartGcTs() { - return startGcTs; + public void setFinishTime(long finishTime) { + this.finishTime = finishTime; + this.executeCostMs = this.finishTime - this.startTime; } - public void setStartGcTs(long startGcTs) { - this.startGcTs = startGcTs; + public long getExecuteCostMs() { + return this.executeCostMs; } - public long getInputRecords() { - return inputRecords; + public long getProcessCostMs() { + return this.processCostMs; } - public void setInputRecords(long inputRecords) { - this.inputRecords = inputRecords; + public void setProcessCostMs(long processCostMs) { + this.processCostMs = processCostMs; } - public long getFinishTime() { - return finishTime; + public void addProcessCostMs(long processCostMs) { + this.processCostMs += processCostMs; } - public void setFinishTime(long finishTime) { - this.finishTime = finishTime; + public long getGcCostMs() { + return this.gcCostMs; + } + + public long getShuffleReadRecords() { + return this.shuffleReadRecords; + } + + public void setShuffleReadRecords(long shuffleReadRecords) { + this.shuffleReadRecords = shuffleReadRecords; + } + + public void addShuffleReadRecords(long shuffleReadRecords) { + this.shuffleReadRecords += shuffleReadRecords; + } + + public long getShuffleReadBytes() { + return this.shuffleReadBytes; + } + + public void setShuffleReadBytes(long shuffleReadBytes) { + this.shuffleReadBytes = shuffleReadBytes; + } + + public void addShuffleReadBytes(long shuffleReadBytes) { + this.shuffleReadBytes += shuffleReadBytes; + } + + public long getShuffleReadCostMs() { + return this.shuffleReadCostMs; + } + + public void setShuffleReadCostMs(long shuffleReadCostMs) { + this.shuffleReadCostMs = shuffleReadCostMs; + } + + public void addShuffleReadCostMs(long shuffleReadCostMs) { + this.shuffleReadCostMs += shuffleReadCostMs; } - public long getOutputRecords() { - return outputRecords; + public long getShuffleWriteRecords() { + return this.shuffleWriteRecords; } - public void setOutputRecords(long outputRecords) { - this.outputRecords = outputRecords; + public void setShuffleWriteRecords(long shuffleWriteRecords) { + this.shuffleWriteRecords = shuffleWriteRecords; } - public long getInputBytes() { - return inputBytes; + public void addShuffleWriteRecords(long shuffleWriteRecords) { + this.shuffleWriteRecords += shuffleWriteRecords; } - public void setInputBytes(long inputBytes) { - this.inputBytes = inputBytes; + public long getShuffleWriteBytes() { + return this.shuffleWriteBytes; } - public long getOutputBytes() { - return outputBytes; + public void setShuffleWriteBytes(long shuffleWriteBytes) { + this.shuffleWriteBytes = shuffleWriteBytes; + } + + public void addShuffleWriteBytes(long shuffleWriteBytes) { + this.shuffleWriteBytes += shuffleWriteBytes; + } + + public long getShuffleWriteCostMs() { + return this.shuffleWriteCostMs; + } + + public void addShuffleWriteCostMs(long shuffleWriteCostMs) { + this.shuffleWriteCostMs += shuffleWriteCostMs; + } + + public void setStartGcTs(long startGcTs) { + this.startGcTs = startGcTs; } - public void setOutputBytes(long outputBytes) { - this.outputBytes = outputBytes; + public void setFinishGcTs(long finishGcTs) { + this.gcCostMs = finishGcTs - this.startGcTs; + } + + @Override + public String toString() { + return "EventMetrics{" + + "startTime=" + startTime + + ", finishTime=" + finishTime + + ", executeCostMs=" + executeCostMs + + ", processCostMs=" + processCostMs + + ", gcCostMs=" + gcCostMs + + ", shuffleReadRecords=" + shuffleReadRecords + + ", shuffleReadBytes=" + shuffleReadBytes + + ", shuffleReadCostMs=" + shuffleReadCostMs + + ", shuffleWriteRecords=" + shuffleWriteRecords + + ", shuffleWriteBytes=" + shuffleWriteBytes + + ", shuffleWriteCostMs=" + shuffleWriteCostMs + + '}'; } } diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/ShuffleReadMetrics.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/ShuffleReadMetrics.java index 432e17786..21e27952d 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/ShuffleReadMetrics.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/metric/ShuffleReadMetrics.java @@ -63,6 +63,11 @@ public void setDecodeBytes(long decodeBytes) { this.decodeBytes = decodeBytes; } + public void increaseDecodeBytes(long decodeBytes) { + this.decodeBytes += decodeBytes; + } + + public long getFetchWaitMs() { return fetchWaitMs; } diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/serialize/impl/KryoSerializer.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/serialize/impl/KryoSerializer.java index 34fc2b8fb..c514da1dc 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/serialize/impl/KryoSerializer.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/serialize/impl/KryoSerializer.java @@ -196,7 +196,7 @@ protected Kryo initialValue() { private void registerClass(Kryo kryo, String className, int kryoId) { try { - LOGGER.info("register class:{} id:{}", className, kryoId); + LOGGER.debug("register class:{} id:{}", className, kryoId); Class clazz = ClassUtil.classForName(className); kryo.register(clazz, kryoId); } catch (GeaflowRuntimeException e) { @@ -211,7 +211,7 @@ private void registerClass(Kryo kryo, String className, int kryoId) { private void registerClass(Kryo kryo, String className, String serializerClassName, int kryoId) { try { - LOGGER.info("register class:{} id:{}", className, kryoId); + LOGGER.debug("register class:{} id:{}", className, kryoId); Class clazz = ClassUtil.classForName(className); Class serializerClazz = ClassUtil.classForName(serializerClassName); Serializer serializer = (Serializer) serializerClazz.newInstance(); diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/thread/Executors.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/thread/Executors.java index 7ae88bf89..f61c28a63 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/thread/Executors.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/thread/Executors.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.StringUtils; @@ -30,8 +31,9 @@ public class Executors { private static final int DEFAULT_QUEUE_CAPACITY = 1024; private static final int DEFAULT_MAGNIFICATION = 2; - private static Map boundedExecutors = new HashMap<>(); - private static int CORE_NUM = Runtime.getRuntime().availableProcessors(); + private static final Map BOUNDED_EXECUTORS = new HashMap<>(); + private static final Map UNBOUNDED_EXECUTORS = new HashMap<>(); + private static final int CORE_NUM = Runtime.getRuntime().availableProcessors(); private static String getKey(String type, int bound, int capacity, long keepAliveTime, TimeUnit unit) { @@ -42,12 +44,12 @@ public static synchronized ExecutorService getBoundedService(int bound, int capa long keepAliveTime, TimeUnit unit) { String key = getKey("bound", bound, capacity, keepAliveTime, unit); - if (boundedExecutors.get(key) == null) { + if (BOUNDED_EXECUTORS.get(key) == null) { BoundedExecutor boundedExecutor = new BoundedExecutor(bound, capacity, keepAliveTime, unit); - boundedExecutors.put(key, boundedExecutor); + BOUNDED_EXECUTORS.put(key, boundedExecutor); } - return boundedExecutors.get(key); + return BOUNDED_EXECUTORS.get(key); } public static synchronized ExecutorService getMaxCoreBoundedService() { @@ -63,12 +65,12 @@ public static synchronized ExecutorService getMaxCoreBoundedService(int magnific public static synchronized ExecutorService getService(int bound, int capacity, long keepAliveTime, TimeUnit unit) { String key = getKey("normal", bound, capacity, keepAliveTime, unit); - if (boundedExecutors.get(key) == null) { + if (BOUNDED_EXECUTORS.get(key) == null) { ExecutorService boundedExecutor = new ThreadPoolExecutor(bound, bound, keepAliveTime, unit, new LinkedBlockingQueue<>(capacity)); - boundedExecutors.put(key, boundedExecutor); + BOUNDED_EXECUTORS.put(key, boundedExecutor); } - return boundedExecutors.get(key); + return BOUNDED_EXECUTORS.get(key); } public static synchronized ExecutorService getMultiCoreExecutorService(int maxMultiple, @@ -122,7 +124,7 @@ private static synchronized ExecutorService getNamedService(int bound, int capac String threadFormat, Thread.UncaughtExceptionHandler handler) { String key = getKey(threadFormat, bound, capacity, keepAliveTime, unit); - if (boundedExecutors.get(key) == null || boundedExecutors.get(key).isShutdown()) { + if (BOUNDED_EXECUTORS.get(key) == null || BOUNDED_EXECUTORS.get(key).isShutdown()) { ThreadFactoryBuilder builder = new ThreadFactoryBuilder() .setNameFormat(threadFormat) .setDaemon(true); @@ -131,9 +133,35 @@ private static synchronized ExecutorService getNamedService(int bound, int capac } ExecutorService boundedExecutor = new ThreadPoolExecutor(bound, bound, keepAliveTime, unit, new LinkedBlockingQueue<>(capacity), builder.build()); - boundedExecutors.put(key, boundedExecutor); + BOUNDED_EXECUTORS.put(key, boundedExecutor); } - return boundedExecutors.get(key); + return BOUNDED_EXECUTORS.get(key); + } + + public static synchronized ExecutorService getUnboundedExecutorService(String name, + long keepAliveTime, + TimeUnit unit, + String threadFormat, + Thread.UncaughtExceptionHandler handler) { + ExecutorService cached = UNBOUNDED_EXECUTORS.get(name); + if (cached != null && !cached.isShutdown()) { + return cached; + } + + ThreadFactoryBuilder builder = new ThreadFactoryBuilder() + .setDaemon(true); + if (threadFormat != null) { + builder.setNameFormat(threadFormat); + } + if (handler != null) { + builder.setUncaughtExceptionHandler(handler); + } + ThreadPoolExecutor pool = new ThreadPoolExecutor( + 0, Integer.MAX_VALUE, keepAliveTime, unit, + new SynchronousQueue<>(), builder.build()); + UNBOUNDED_EXECUTORS.put(name, pool); + return pool; + } } diff --git a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/utils/LoggerFormatter.java b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/utils/LoggerFormatter.java index 80ff8fd0a..993b9a394 100644 --- a/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/utils/LoggerFormatter.java +++ b/geaflow/geaflow-common/src/main/java/com/antgroup/geaflow/common/utils/LoggerFormatter.java @@ -21,28 +21,40 @@ public class LoggerFormatter { public LoggerFormatter() { } - public static String getTaskLog(String pipelineName, int cycleId, long batchId, int taskId) { - return String.format("%s task#%s", getCycleTag(pipelineName, cycleId, batchId), taskId); + public static String getCycleName(int cycleId) { + return String.format("cycle#%s", cycleId); } - public static String getTaskLog(String pipelineName, int cycleId, int taskId) { - return String.format("%s task#%s", getCycleTag(pipelineName, cycleId), taskId); + public static String getCycleName(int cycleId, long windowId) { + return String.format("cycle#%s-%s", cycleId, windowId); } public static String getCycleTag(String pipelineName, int cycleId) { return String.format("%s %s", pipelineName, getCycleName(cycleId)); } - public static String getCycleTag(String pipelineName, int cycleId, Object label) { - return String.format("%s %s", pipelineName, getCycleName(cycleId, label)); + public static String getCycleTag(String pipelineName, int cycleId, long windowId) { + return String.format("%s %s", pipelineName, getCycleName(cycleId, windowId)); } - public static String getCycleName(int cycleId) { - return String.format("cycle#%s", cycleId); + public static String getCycleMetricName(int cycleId, int vertexId) { + return String.format("%s[%d]", getCycleName(cycleId), vertexId); + } + + public static String getCycleMetricName(int cycleId, long windowId, int vertexId) { + return String.format("%s[%d]", getCycleName(cycleId, windowId), vertexId); + } + + public static String getTaskTag(String pipelineName, int cycleId, + int taskId, int vertexId, int index, int parallelism) { + return String.format("%s task#%d [%d-%d/%d]", getCycleTag(pipelineName, cycleId), + taskId, vertexId, index, parallelism); } - public static String getCycleName(int cycleId, Object label) { - return String.format("cycle#%s-%s", cycleId, label); + public static String getTaskTag(String pipelineName, int cycleId, long windowId, + int taskId, int vertexId, int index, int parallelism) { + return String.format("%s task#%d [%d-%d/%d]", getCycleTag(pipelineName, cycleId, windowId), + taskId, vertexId, index, parallelism); } /** diff --git a/geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/AbstractMessageBuffer.java b/geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/AbstractMessageBuffer.java new file mode 100644 index 000000000..f15780550 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/AbstractMessageBuffer.java @@ -0,0 +1,63 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.io; + +import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; +import com.antgroup.geaflow.common.metric.EventMetrics; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +public abstract class AbstractMessageBuffer implements IMessageBuffer { + + private static final int DEFAULT_TIMEOUT_MS = 100; + + private final LinkedBlockingQueue queue; + protected volatile EventMetrics eventMetrics; + + public AbstractMessageBuffer(int capacity) { + this.queue = new LinkedBlockingQueue<>(capacity); + } + + @Override + public void offer(R record) { + while (true) { + try { + if (this.queue.offer(record, DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + break; + } + } catch (InterruptedException e) { + throw new GeaflowRuntimeException(e); + } + } + } + + @Override + public R poll(long timeout, TimeUnit unit) { + try { + return this.queue.poll(timeout, unit); + } catch (InterruptedException e) { + throw new GeaflowRuntimeException(e); + } + } + + public void setEventMetrics(EventMetrics eventMetrics) { + this.eventMetrics = eventMetrics; + } + + public EventMetrics getEventMetrics() { + return this.eventMetrics; + } + +} diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineMaster.java b/geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/IMessageBuffer.java similarity index 53% rename from geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineMaster.java rename to geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/IMessageBuffer.java index 51cc4fa93..561fb1603 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineMaster.java +++ b/geaflow/geaflow-core/geaflow-core-common/src/main/java/com/antgroup/geaflow/io/IMessageBuffer.java @@ -12,23 +12,26 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. */ -package com.antgroup.geaflow.runtime.core.scheduler; +package com.antgroup.geaflow.io; -import com.antgroup.geaflow.cluster.protocol.IEvent; -import com.antgroup.geaflow.cluster.rpc.RpcClient; +import java.util.concurrent.TimeUnit; -public class PipelineMaster { +public interface IMessageBuffer { - private String driverId; + /** + * Push a message to this pipe. + * + * @param message message + */ + void offer(M message); - public PipelineMaster(String driverId) { - this.driverId = driverId; - } - /** - * Send event to scheduler in worker. + * Pull a record from this pipe with timeout. + * + * @param timeout timeout number + * @param unit timeout unit + * @return message */ - public void send(IEvent event) { - RpcClient.getInstance().processPipeline(driverId, event); - } + M poll(long timeout, TimeUnit unit); + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractEmitterRequest.java new file mode 100644 index 000000000..83fef1bb6 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractEmitterRequest.java @@ -0,0 +1,37 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.collector; + +public abstract class AbstractEmitterRequest implements IEmitterRequest { + + private final int taskId; + private final long windowId; + + public AbstractEmitterRequest(int taskId, long windowId) { + this.taskId = taskId; + this.windowId = windowId; + } + + @Override + public int getTaskId() { + return this.taskId; + } + + @Override + public long getWindowId() { + return this.windowId; + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineCollector.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineCollector.java index 4c1809f19..7983546b8 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineCollector.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineCollector.java @@ -17,25 +17,19 @@ import com.antgroup.geaflow.api.context.RuntimeContext; import com.antgroup.geaflow.collector.AbstractCollector; import com.antgroup.geaflow.collector.ICollector; -import com.antgroup.geaflow.common.encoder.IEncoder; import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; import com.antgroup.geaflow.partitioner.IPartitioner; import com.antgroup.geaflow.partitioner.impl.KeyPartitioner; import com.antgroup.geaflow.selector.ISelector; import com.antgroup.geaflow.selector.impl.ChannelSelector; import com.antgroup.geaflow.shuffle.ForwardOutputDesc; -import com.antgroup.geaflow.shuffle.api.writer.IShuffleWriter; -import com.antgroup.geaflow.shuffle.api.writer.IWriterContext; -import com.antgroup.geaflow.shuffle.api.writer.WriterContext; -import com.antgroup.geaflow.shuffle.service.ShuffleManager; -import java.io.IOException; import java.util.List; import java.util.stream.IntStream; public abstract class AbstractPipelineCollector extends AbstractCollector implements ICollector { - protected transient IShuffleWriter pipeRecordWriter; + protected transient IOutputMessageBuffer outputBuffer; protected transient ISelector recordISelector; protected ForwardOutputDesc outputDesc; protected long windowId; @@ -55,26 +49,6 @@ public void setUp(RuntimeContext runtimeContext) { } this.recordISelector = new ChannelSelector(targetTaskIds.size(), partitioner); - - this.pipeRecordWriter = ShuffleManager.getInstance().loadShuffleWriter(); - - IEncoder encoder = this.outputDesc.getEncoder(); - if (encoder != null) { - encoder.init(runtimeContext.getConfiguration()); - } - IWriterContext writerContext = WriterContext.newBuilder() - .setPipelineId(runtimeContext.getPipelineId()) - .setPipelineName(runtimeContext.getPipelineName()) - .setVertexId(id) - .setEdgeId(outputDesc.getEdgeId()) - .setTaskId(runtimeContext.getTaskArgs().getTaskId()) - .setTaskIndex(runtimeContext.getTaskArgs().getTaskIndex()) - .setTaskName(runtimeContext.getTaskArgs().getTaskName()) - .setChannelNum(targetTaskIds.size()) - .setConfig(runtimeContext.getConfiguration()) - .setShuffleDescriptor(outputDesc.getShuffleDescriptor()) - .setEncoder(encoder); - this.pipeRecordWriter.init(writerContext); } @Override @@ -82,6 +56,10 @@ public int getId() { return id; } + public void setOutputBuffer(IOutputMessageBuffer outputBuffer) { + this.outputBuffer = outputBuffer; + } + public long getWindowId() { return windowId; } @@ -95,9 +73,9 @@ public void broadcast(T value) { List targetTaskIds = outputDesc.getTargetTaskIndices(); int[] channels = IntStream.rangeClosed(0, targetTaskIds.size() - 1).toArray(); try { - pipeRecordWriter.emit(windowId, value, false, channels); + this.outputBuffer.emit(this.windowId, value, false, channels); this.outputMeter.mark(); - } catch (IOException e) { + } catch (Exception e) { throw new GeaflowRuntimeException(e); } } @@ -115,19 +93,12 @@ public void partition(KEY key, T value) { @Override public void finish() { try { - pipeRecordWriter.flush(windowId); - } catch (IOException e) { + this.outputBuffer.finish(this.windowId); + } catch (Exception e) { throw new GeaflowRuntimeException(e); } } - @Override - public void close() { - if (pipeRecordWriter != null) { - pipeRecordWriter.close(); - } - } - /** * Shuffle data with value itself. */ @@ -135,9 +106,9 @@ protected void shuffle(T value, boolean isRetract) { int[] targetChannels = this.recordISelector.selectChannels(value); try { - pipeRecordWriter.emit(windowId, value, isRetract, targetChannels); + this.outputBuffer.emit(this.windowId, value, isRetract, targetChannels); this.outputMeter.mark(); - } catch (IOException e) { + } catch (Exception e) { throw new GeaflowRuntimeException(e); } } @@ -149,14 +120,11 @@ protected void shuffle(KEY key, T value, boolean isRetract) { int[] targetChannels = this.recordISelector.selectChannels(key); try { - pipeRecordWriter.emit(windowId, value, isRetract, targetChannels); + this.outputBuffer.emit(this.windowId, value, isRetract, targetChannels); this.outputMeter.mark(); - } catch (IOException e) { + } catch (Exception e) { throw new GeaflowRuntimeException(e); } } - public void setOutputDesc(ForwardOutputDesc outputDesc) { - this.outputDesc = outputDesc; - } } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineOutputCollector.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineOutputCollector.java index 206d90652..1cf13a7da 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineOutputCollector.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/AbstractPipelineOutputCollector.java @@ -21,8 +21,6 @@ import com.antgroup.geaflow.io.CollectType; import com.antgroup.geaflow.shuffle.ForwardOutputDesc; import com.antgroup.geaflow.shuffle.message.Shard; -import java.io.IOException; -import java.util.Optional; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,11 +54,8 @@ public void setUp(RuntimeContext runtimeContext) { @Override public void finish() { try { - Optional result = this.pipeRecordWriter.flush(this.windowId); - if (result.isPresent()) { - shard = (Shard) result.get(); - } - } catch (IOException e) { + this.shard = (Shard) this.outputBuffer.finish(this.windowId); + } catch (Exception e) { throw new GeaflowRuntimeException(e); } } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/ClearEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/ClearEmitterRequest.java new file mode 100644 index 000000000..615ca07ac --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/ClearEmitterRequest.java @@ -0,0 +1,30 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.collector; + +public class ClearEmitterRequest extends AbstractEmitterRequest { + + public static final ClearEmitterRequest INSTANCE = new ClearEmitterRequest(); + + private ClearEmitterRequest() { + super(-1, -1); + } + + @Override + public RequestType getRequestType() { + return RequestType.CLEAR; + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CloseEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CloseEmitterRequest.java index e0b744790..9eae6d413 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CloseEmitterRequest.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CloseEmitterRequest.java @@ -14,24 +14,15 @@ package com.antgroup.geaflow.cluster.collector; -import com.antgroup.geaflow.collector.ICollector; -import java.util.List; +public class CloseEmitterRequest extends AbstractEmitterRequest { -public class CloseEmitterRequest implements IEmitterRequest { - - /** - * Close output collector. - */ - public void closeEmitter(List collectors) { - if (collectors != null) { - for (ICollector collector : collectors) { - collector.close(); - } - } + public CloseEmitterRequest(int taskId, long windowId) { + super(taskId, windowId); } @Override public RequestType getRequestType() { return RequestType.CLOSE; } + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CollectorFactory.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CollectorFactory.java index 5f46ff302..618b6d738 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CollectorFactory.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/CollectorFactory.java @@ -22,14 +22,14 @@ public class CollectorFactory { - public static ICollector create(IOutputDesc outputDesc) { + public static ICollector create(IOutputDesc outputDesc) { switch (outputDesc.getType()) { case FORWARD: - return new ForwardOutputCollector((ForwardOutputDesc) outputDesc); + return new ForwardOutputCollector<>((ForwardOutputDesc) outputDesc); case LOOP: - return new IterationOutputCollector((ForwardOutputDesc) outputDesc); + return new IterationOutputCollector<>((ForwardOutputDesc) outputDesc); case RESPONSE: - return new CollectResponseCollector((ResponseOutputDesc) outputDesc); + return new CollectResponseCollector<>((ResponseOutputDesc) outputDesc); default: throw new GeaflowRuntimeException("not support output type {}" + outputDesc.getType()); diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterRunner.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterRunner.java index 69c1dbaab..de7366e6a 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterRunner.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterRunner.java @@ -15,39 +15,37 @@ package com.antgroup.geaflow.cluster.collector; import com.antgroup.geaflow.cluster.task.runner.AbstractTaskRunner; -import com.antgroup.geaflow.collector.ICollector; +import com.antgroup.geaflow.common.config.Configuration; import com.antgroup.geaflow.common.errorcode.RuntimeErrors; import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; -import java.util.ArrayList; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class EmitterRunner extends AbstractTaskRunner { - private static final Logger LOGGER = LoggerFactory.getLogger(EmitterRunner.class); - - private final List collectors; + private final PipelineOutputEmitter outputEmitter; - public EmitterRunner() { - this.collectors = new ArrayList<>(); + public EmitterRunner(Configuration configuration, int index) { + this.outputEmitter = new PipelineOutputEmitter(configuration, index); } @Override protected void process(IEmitterRequest request) { switch (request.getRequestType()) { case INIT: - InitEmitterRequest initEmitRequest = (InitEmitterRequest) request; - initEmitRequest.initEmitter(this.collectors); + this.outputEmitter.init((InitEmitterRequest) request); + break; + case UPDATE: + this.outputEmitter.update((UpdateEmitterRequest) request); break; case CLOSE: - CloseEmitterRequest closeEmitterRequest = (CloseEmitterRequest) request; - closeEmitterRequest.closeEmitter(this.collectors); + this.outputEmitter.close((CloseEmitterRequest) request); + break; + case CLEAR: + this.outputEmitter.clear(); break; default: throw new GeaflowRuntimeException( RuntimeErrors.INST.requestTypeNotSupportError(request.getRequestType().name())); } } + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterService.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterService.java index 98e76866c..d7607da55 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterService.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/EmitterService.java @@ -15,29 +15,28 @@ package com.antgroup.geaflow.cluster.collector; import com.antgroup.geaflow.cluster.task.service.AbstractTaskService; +import com.antgroup.geaflow.common.config.Configuration; import com.google.common.base.Preconditions; import java.io.Serializable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class EmitterService extends AbstractTaskService implements Serializable { - private static final Logger LOGGER = LoggerFactory.getLogger(EmitterService.class); - private static final String EMITTER_FORMAT = "geaflow-emitter-%d"; - private int slots; + private final int slots; + private final Configuration configuration; - public EmitterService(int slots) { + public EmitterService(int slots, Configuration configuration) { super(EMITTER_FORMAT); this.slots = slots; + this.configuration = configuration; } protected EmitterRunner[] buildTaskRunner() { Preconditions.checkArgument(slots > 0, "fetcher pool should be larger than 0"); EmitterRunner[] emitterRunners = new EmitterRunner[slots]; for (int i = 0; i < slots; i++) { - EmitterRunner runner = new EmitterRunner(); + EmitterRunner runner = new EmitterRunner(this.configuration, i); emitterRunners[i] = runner; } return emitterRunners; diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IEmitterRequest.java index 803e662d3..23a508391 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IEmitterRequest.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IEmitterRequest.java @@ -19,7 +19,23 @@ public interface IEmitterRequest extends Serializable { /** - * Returns the request type. + * Return the task id of the emitter request. + * + * @return task id. + */ + int getTaskId(); + + /** + * Return the window id of the emitter request. + * + * @return window id. + */ + long getWindowId(); + + /** + * Return the request type of the emitter request. + * + * @return request type. */ RequestType getRequestType(); } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IOutputMessageBuffer.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IOutputMessageBuffer.java new file mode 100644 index 000000000..c3bfc9d88 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/IOutputMessageBuffer.java @@ -0,0 +1,55 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.collector; + +import com.antgroup.geaflow.cluster.protocol.OutputMessage; +import com.antgroup.geaflow.io.IMessageBuffer; + +public interface IOutputMessageBuffer extends IMessageBuffer> { + + /** + * Emit a record. + * + * @param windowId window id + * @param data data + * @param isRetract if this data is retract + * @param targetChannels target channels + */ + void emit(long windowId, T data, boolean isRetract, int[] targetChannels); + + /** + * For the consumer to finish. + * + * @param windowId window id + * @param result finish result + */ + void setResult(long windowId, R result); + + /** + * For the producer to call finish. + * + * @param windowId window id + * @return finish result + */ + R finish(long windowId); + + /** + * Meet error. + * + * @param t error + */ + void error(Throwable t); + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitCollectEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitCollectEmitterRequest.java deleted file mode 100644 index bcc731546..000000000 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitCollectEmitterRequest.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright 2023 AntGroup CO., Ltd. - * - * Licensed 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. - */ - -package com.antgroup.geaflow.cluster.collector; - -import com.antgroup.geaflow.collector.ICollector; -import com.antgroup.geaflow.io.ResponseOutputDesc; -import java.util.List; - -public class InitCollectEmitterRequest extends InitEmitterRequest { - - private ResponseOutputDesc outputDesc; - - public InitCollectEmitterRequest(ResponseOutputDesc outputDesc) { - super(null); - this.outputDesc = outputDesc; - } - - /** - * Init output collectors. - */ - public void initEmitter(List collectors) { - collectors.clear(); - collectors.add(new CollectResponseCollector(outputDesc)); - this.collectors = collectors; - } -} - diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitEmitterRequest.java index b55f01c14..924e005ae 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitEmitterRequest.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/InitEmitterRequest.java @@ -14,48 +14,64 @@ package com.antgroup.geaflow.cluster.collector; -import com.antgroup.geaflow.collector.ICollector; -import com.antgroup.geaflow.shuffle.IOutputDesc; +import com.antgroup.geaflow.common.config.Configuration; +import com.antgroup.geaflow.common.task.TaskArgs; import com.antgroup.geaflow.shuffle.OutputDescriptor; +import com.antgroup.geaflow.shuffle.message.Shard; import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public class InitEmitterRequest implements IEmitterRequest { +public class InitEmitterRequest extends AbstractEmitterRequest { - private static final Logger LOGGER = LoggerFactory.getLogger(InitEmitterRequest.class); + private final Configuration configuration; + private final long pipelineId; + private final String pipelineName; + private final TaskArgs taskArgs; + private final OutputDescriptor outputDescriptor; + protected final List> outputBuffers; - private OutputDescriptor outputDescriptor; - protected List collectors; - - public InitEmitterRequest(OutputDescriptor outputDescriptor) { + public InitEmitterRequest(Configuration configuration, + long windowId, + long pipelineId, + String pipelineName, + TaskArgs taskArgs, + OutputDescriptor outputDescriptor, + List> outputBuffers) { + super(taskArgs.getTaskId(), windowId); + this.configuration = configuration; + this.pipelineId = pipelineId; + this.pipelineName = pipelineName; + this.taskArgs = taskArgs; this.outputDescriptor = outputDescriptor; + this.outputBuffers = outputBuffers; } - /** - * Init output collectors. - */ - public void initEmitter(List collectors) { - if (outputDescriptor != null && outputDescriptor.getOutputDescList() != null) { - collectors.clear(); - for (IOutputDesc outputDesc : outputDescriptor.getOutputDescList()) { - ICollector collector = CollectorFactory.create(outputDesc); - collectors.add(collector); - } - } - this.collectors = collectors; + public Configuration getConfiguration() { + return this.configuration; } + public long getPipelineId() { + return this.pipelineId; + } - public List getCollectors() { - while (collectors == null) { - LOGGER.debug("wait init request done"); - } - return collectors; + public String getPipelineName() { + return this.pipelineName; + } + + public TaskArgs getTaskArgs() { + return this.taskArgs; + } + + public OutputDescriptor getOutputDescriptor() { + return this.outputDescriptor; + } + + public List> getOutputBuffers() { + return this.outputBuffers; } @Override public RequestType getRequestType() { return RequestType.INIT; } + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/PipelineOutputEmitter.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/PipelineOutputEmitter.java new file mode 100644 index 000000000..2c12063b1 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/PipelineOutputEmitter.java @@ -0,0 +1,228 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.collector; + +import com.antgroup.geaflow.cluster.protocol.OutputMessage; +import com.antgroup.geaflow.common.config.Configuration; +import com.antgroup.geaflow.common.encoder.IEncoder; +import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; +import com.antgroup.geaflow.common.metric.EventMetrics; +import com.antgroup.geaflow.common.metric.ShuffleWriteMetrics; +import com.antgroup.geaflow.common.task.TaskArgs; +import com.antgroup.geaflow.common.thread.Executors; +import com.antgroup.geaflow.io.AbstractMessageBuffer; +import com.antgroup.geaflow.io.CollectType; +import com.antgroup.geaflow.model.record.RecordArgs; +import com.antgroup.geaflow.shuffle.ForwardOutputDesc; +import com.antgroup.geaflow.shuffle.IOutputDesc; +import com.antgroup.geaflow.shuffle.OutputDescriptor; +import com.antgroup.geaflow.shuffle.api.writer.IShuffleWriter; +import com.antgroup.geaflow.shuffle.api.writer.IWriterContext; +import com.antgroup.geaflow.shuffle.api.writer.WriterContext; +import com.antgroup.geaflow.shuffle.message.Shard; +import com.antgroup.geaflow.shuffle.service.ShuffleManager; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PipelineOutputEmitter { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipelineOutputEmitter.class); + + private static final ExecutorService EMIT_EXECUTOR = Executors.getUnboundedExecutorService( + PipelineOutputEmitter.class.getSimpleName(), 60, TimeUnit.SECONDS, null, null); + + private static final int DEFAULT_TIMEOUT_MS = 100; + + private final Configuration configuration; + private final int index; + private final Map initRequestCache = new HashMap<>(); + private final Map runningFlags = new HashMap<>(); + + public PipelineOutputEmitter(Configuration configuration, int index) { + this.configuration = configuration; + this.index = index; + } + + public void init(InitEmitterRequest request) { + this.initRequestCache.put(request.getTaskId(), request); + UpdateEmitterRequest updateEmitterRequest = new UpdateEmitterRequest( + request.getTaskId(), + request.getWindowId(), + request.getPipelineId(), + request.getPipelineName(), + request.getOutputBuffers()); + this.update(updateEmitterRequest); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public void update(UpdateEmitterRequest request) { + int taskId = request.getTaskId(); + if (!this.initRequestCache.containsKey(taskId)) { + throw new GeaflowRuntimeException("init emitter request not found for task " + taskId); + } + InitEmitterRequest initEmitterRequest = this.initRequestCache.get(taskId); + OutputDescriptor outputDescriptor = initEmitterRequest.getOutputDescriptor(); + List> outputBuffers = request.getOutputBuffers(); + List outputDescList = outputDescriptor.getOutputDescList(); + + int outputNum = outputDescList.size(); + AtomicBoolean[] flags = new AtomicBoolean[outputNum]; + for (int i = 0; i < outputNum; i++) { + IOutputDesc outputDesc = outputDescList.get(i); + if (outputDesc.getType() == CollectType.RESPONSE) { + continue; + } + ForwardOutputDesc forwardOutputDesc = (ForwardOutputDesc) outputDesc; + IShuffleWriter pipeRecordWriter = ShuffleManager.getInstance().loadShuffleWriter(); + IEncoder encoder = forwardOutputDesc.getEncoder(); + if (encoder != null) { + encoder.init(initEmitterRequest.getConfiguration()); + } + TaskArgs taskArgs = initEmitterRequest.getTaskArgs(); + IWriterContext writerContext = WriterContext.newBuilder() + .setPipelineId(request.getPipelineId()) + .setPipelineName(request.getPipelineName()) + .setVertexId(forwardOutputDesc.getPartitioner().getOpId()) + .setEdgeId(forwardOutputDesc.getEdgeId()) + .setTaskId(taskArgs.getTaskId()) + .setTaskIndex(taskArgs.getTaskIndex()) + .setTaskName(taskArgs.getTaskName()) + .setChannelNum(forwardOutputDesc.getTargetTaskIndices().size()) + .setConfig(this.configuration) + .setShuffleDescriptor(forwardOutputDesc.getShuffleDescriptor()) + .setEncoder(encoder); + pipeRecordWriter.init(writerContext); + + AtomicBoolean flag = new AtomicBoolean(true); + flags[i] = flag; + String emitterId = String.format("%d[%d/%d]", taskId, taskArgs.getTaskIndex(), taskArgs.getParallelism()); + EmitterTask emitterTask = new EmitterTask( + pipeRecordWriter, + outputBuffers.get(i), + flag, + request.getWindowId(), + this.index, + forwardOutputDesc.getEdgeName(), + emitterId); + EMIT_EXECUTOR.execute(emitterTask); + } + this.runningFlags.put(taskId, flags); + } + + public void close(CloseEmitterRequest request) { + int taskId = request.getTaskId(); + if (!this.runningFlags.containsKey(taskId)) { + return; + } + for (AtomicBoolean flag : this.runningFlags.remove(taskId)) { + if (flag != null) { + flag.set(false); + } + } + } + + public Configuration getConfiguration() { + return this.configuration; + } + + public void clear() { + LOGGER.info("clear emitter cache of task {}", this.initRequestCache.keySet()); + this.initRequestCache.clear(); + } + + private static class EmitterTask implements Runnable { + + private static final String WRITER_NAME_PATTERN = "shuffle-writer-%d-%s"; + + private final IShuffleWriter writer; + private final IOutputMessageBuffer pipe; + private final AtomicBoolean running; + private final long windowId; + private final String name; + private final String emitterId; + private final boolean isMessage; + + public EmitterTask(IShuffleWriter writer, + IOutputMessageBuffer pipe, + AtomicBoolean running, + long windowId, + int workerIndex, + String edgeName, + String emitterId) { + this.writer = writer; + this.pipe = pipe; + this.running = running; + this.windowId = windowId; + this.name = String.format(WRITER_NAME_PATTERN, workerIndex, edgeName); + this.emitterId = emitterId; + this.isMessage = edgeName.equals(RecordArgs.GraphRecordNames.Message.name()); + } + + @Override + public void run() { + Thread.currentThread().setName(this.name); + try { + this.execute(); + } catch (Throwable t) { + this.pipe.error(t); + LOGGER.error("emitter task err in window id {} {}", this.windowId, this.emitterId, t); + } + LOGGER.info("emitter task finish window id {} {}", this.windowId, this.emitterId); + } + + private void execute() throws Exception { + while (this.running.get()) { + OutputMessage record = this.pipe.poll(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS); + if (record == null) { + continue; + } + long windowId = record.getWindowId(); + if (record.isBarrier()) { + Optional result = this.writer.flush(windowId); + this.handleMetrics(); + this.pipe.setResult(windowId, result.orElse(null)); + } else { + this.writer.emit(windowId, record.getMessage(), false, record.getTargetChannel()); + } + } + this.writer.close(); + } + + @SuppressWarnings("unchecked") + private void handleMetrics() { + ShuffleWriteMetrics shuffleWriteMetrics = this.writer.getShuffleWriteMetrics(); + EventMetrics eventMetrics = ((AbstractMessageBuffer) this.pipe).getEventMetrics(); + if (this.isMessage) { + // When send message, all iteration share the same context and writer, just set the total metric. + eventMetrics.setShuffleWriteRecords(shuffleWriteMetrics.getWrittenRecords()); + eventMetrics.setShuffleWriteBytes(shuffleWriteMetrics.getEncodedSize()); + } else { + // In FINISH iteration or other case, just add output metric. + eventMetrics.addShuffleWriteRecords(shuffleWriteMetrics.getWrittenRecords()); + eventMetrics.addShuffleWriteBytes(shuffleWriteMetrics.getEncodedSize()); + } + + } + + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/RequestType.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/RequestType.java index 7f15a0bb2..d23ac12fc 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/RequestType.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/RequestType.java @@ -21,8 +21,18 @@ public enum RequestType { */ INIT, + /** + * Update request. + */ + UPDATE, + /** * Close request. */ CLOSE, + /** + * Clear the init emitter request in cache. + */ + CLEAR + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/UpdateEmitterRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/UpdateEmitterRequest.java new file mode 100644 index 000000000..b3ae6ca2f --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/collector/UpdateEmitterRequest.java @@ -0,0 +1,54 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.collector; + +import com.antgroup.geaflow.shuffle.message.Shard; +import java.util.List; + +public class UpdateEmitterRequest extends AbstractEmitterRequest { + + private final long pipelineId; + private final String pipelineName; + private final List> outputBuffers; + + public UpdateEmitterRequest(int taskId, + long windowId, + long pipelineId, + String pipelineName, + List> outputBuffers) { + super(taskId, windowId); + this.pipelineId = pipelineId; + this.pipelineName = pipelineName; + this.outputBuffers = outputBuffers; + } + + public long getPipelineId() { + return this.pipelineId; + } + + public String getPipelineName() { + return this.pipelineName; + } + + public List> getOutputBuffers() { + return this.outputBuffers; + } + + @Override + public RequestType getRequestType() { + return RequestType.UPDATE; + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/container/Container.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/container/Container.java index 44949f01a..956a9a216 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/container/Container.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/container/Container.java @@ -81,7 +81,7 @@ public OpenContainerResponseEvent open(OpenContainerEvent event) { LOGGER.info("open container {} with {} executors", name, num); this.fetcherService = new FetcherService(num, configuration); - this.emitterService = new EmitterService(num); + this.emitterService = new EmitterService(num, configuration); this.workerService = new TaskService(id, num, configuration, metricGroup, fetcherService, emitterService); this.dispatcher = new Dispatcher(workerService); diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/FetchListener.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/IInputMessageBuffer.java similarity index 67% rename from geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/FetchListener.java rename to geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/IInputMessageBuffer.java index 836286672..0e2e80db0 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/FetchListener.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/IInputMessageBuffer.java @@ -14,18 +14,14 @@ package com.antgroup.geaflow.cluster.fetcher; +import com.antgroup.geaflow.cluster.protocol.InputMessage; +import com.antgroup.geaflow.io.IMessageBuffer; import com.antgroup.geaflow.shuffle.message.PipelineMessage; -public interface FetchListener { +public interface IInputMessageBuffer extends IMessageBuffer> { - /** - * Trigger processor to process message. - */ - void onMessage(PipelineMessage message); + void onMessage(PipelineMessage message); - /** - * Trigger processor to process barrier. - */ - void onCompleted(long windowId, long windowCount); + void onBarrier(long windowId, long windowCount); } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/InitFetchRequest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/InitFetchRequest.java index 7855886c9..10ddfce9e 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/InitFetchRequest.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/InitFetchRequest.java @@ -35,7 +35,7 @@ public class InitFetchRequest implements IFetchRequest { private int vertexId; private ShuffleDescriptor descriptor; - private List fetchListeners; + private List> fetchListeners; private Map inputStreamMap; private Map> inputSlices; private Map> encoders; @@ -79,7 +79,7 @@ public ShuffleDescriptor getDescriptor() { return descriptor; } - public List getFetchListeners() { + public List> getFetchListeners() { return fetchListeners; } @@ -91,7 +91,7 @@ public int getTotalSliceNum() { return totalSliceNum; } - public void addListener(FetchListener listener) { + public void addListener(IInputMessageBuffer listener) { fetchListeners.add(listener); } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/PipelineInputFetcher.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/PipelineInputFetcher.java index 0ecc18509..efdb61a03 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/PipelineInputFetcher.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/fetcher/PipelineInputFetcher.java @@ -17,6 +17,9 @@ import com.antgroup.geaflow.common.config.Configuration; import com.antgroup.geaflow.common.encoder.IEncoder; import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; +import com.antgroup.geaflow.common.metric.EventMetrics; +import com.antgroup.geaflow.common.metric.ShuffleReadMetrics; +import com.antgroup.geaflow.io.AbstractMessageBuffer; import com.antgroup.geaflow.shuffle.api.reader.IShuffleReader; import com.antgroup.geaflow.shuffle.message.FetchRequest; import com.antgroup.geaflow.shuffle.message.PipelineBarrier; @@ -35,7 +38,7 @@ public class PipelineInputFetcher { private IShuffleReader shuffleReader; private InitFetchRequest initRequest; - private List fetchListeners; + private List> fetchListeners; private BarrierHandler barrierHandler; private long pipelineId; @@ -132,7 +135,7 @@ protected void fetch(FetchRequest request) { if (event != null) { if (event instanceof PipelineMessage) { PipelineMessage message = (PipelineMessage) event; - for (FetchListener listener : fetchListeners) { + for (IInputMessageBuffer listener : fetchListeners) { listener.onMessage(message); } } else { @@ -140,19 +143,28 @@ protected void fetch(FetchRequest request) { if (barrierHandler.checkCompleted(barrier)) { long windowId = barrier.getWindowId(); long windowCount = barrierHandler.getTotalWindowCount(); - for (FetchListener listener : fetchListeners) { - listener.onCompleted(windowId, windowCount); + this.handleMetrics(); + for (IInputMessageBuffer listener : fetchListeners) { + listener.onBarrier(windowId, windowCount); } } } } } - LOGGER.info("task {} worker reader finish fetch by reader {} fetch windowId {}", - request.getTaskId(), shuffleReader, request.getTargetBatchId()); + LOGGER.info("task {} worker reader finish fetch windowId {}", + request.getTaskId(), request.getTargetBatchId()); } catch (Throwable e) { LOGGER.error("fetcher encounters unexpected exception: {}", e.getMessage(), e); throw new GeaflowRuntimeException(e); } } + private void handleMetrics() { + ShuffleReadMetrics shuffleReadMetrics = this.shuffleReader.getShuffleReadMetrics(); + for (IInputMessageBuffer listener : this.fetchListeners) { + EventMetrics eventMetrics = ((AbstractMessageBuffer) listener).getEventMetrics(); + eventMetrics.addShuffleReadBytes(shuffleReadMetrics.getDecodeBytes()); + } + } + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/AbstractMessage.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/AbstractMessage.java new file mode 100644 index 000000000..5a203fee4 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/AbstractMessage.java @@ -0,0 +1,34 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.protocol; + +public abstract class AbstractMessage implements IMessage { + + private final long windowId; + + public AbstractMessage(long windowId) { + this.windowId = windowId; + } + + public long getWindowId() { + return this.windowId; + } + + @Override + public EventType getEventType() { + return EventType.MESSAGE; + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/IMessage.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/IMessage.java index 04f64633a..e58a9d649 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/IMessage.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/IMessage.java @@ -17,5 +17,13 @@ /** * A message is the event of data flow among cycle scheduling. */ -public interface IMessage extends IEvent { +public interface IMessage extends IEvent { + + /** + * Get the message content. + * + * @return message + */ + T getMessage(); + } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/Message.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/InputMessage.java similarity index 62% rename from geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/Message.java rename to geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/InputMessage.java index 61c9362de..22403e9b8 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/Message.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/InputMessage.java @@ -19,27 +19,25 @@ /** * A message which is processed by worker. */ -public class Message implements IMessage { +public class InputMessage extends AbstractMessage> { - private final long windowId; - private PipelineMessage message; - private long windowCount; + private final PipelineMessage message; + private final long windowCount; - public Message(long windowId, PipelineMessage message) { - this.windowId = windowId; + public InputMessage(PipelineMessage message) { + super(message.getWindowId()); this.message = message; + this.windowCount = -1; } - public Message(long windowId, long windowCount) { - this.windowId = windowId; + public InputMessage(long windowId, long windowCount) { + super(windowId); + this.message = null; this.windowCount = windowCount; } - public long getWindowId() { - return windowId; - } - - public PipelineMessage getMessage() { + @Override + public PipelineMessage getMessage() { return message; } @@ -47,8 +45,4 @@ public long getWindowCount() { return windowCount; } - @Override - public EventType getEventType() { - return EventType.MESSAGE; - } } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/OutputMessage.java b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/OutputMessage.java new file mode 100644 index 000000000..1de721364 --- /dev/null +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-cluster/src/main/java/com/antgroup/geaflow/cluster/protocol/OutputMessage.java @@ -0,0 +1,53 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.cluster.protocol; + +import java.util.List; + +public class OutputMessage extends AbstractMessage> { + + private final int targetChannel; + private final List data; + private final boolean isBarrier; + + public OutputMessage(long windowId, int targetChannel, List data) { + super(windowId); + this.targetChannel = targetChannel; + this.data = data; + this.isBarrier = data == null; + } + + @Override + public List getMessage() { + return this.data; + } + + public int getTargetChannel() { + return this.targetChannel; + } + + public boolean isBarrier() { + return this.isBarrier; + } + + public static OutputMessage data(long windowId, int targetChannel, List data) { + return new OutputMessage<>(windowId, targetChannel, data); + } + + public static OutputMessage barrier(long windowId) { + return new OutputMessage<>(windowId, -1, null); + } + +} diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/pipeline/channel/LocalInputChannel.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/pipeline/channel/LocalInputChannel.java index bb72db4e3..72dc52c14 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/pipeline/channel/LocalInputChannel.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/pipeline/channel/LocalInputChannel.java @@ -68,7 +68,7 @@ public void requestSlice(long batchId) throws IOException { if (increaseBackoff()) { retriggerRequest = true; } else { - LOGGER.error("not found slice:{}", inputSliceId); + LOGGER.warn("not found slice:{}", inputSliceId); throw notFound; } } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/AbstractFetcher.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/AbstractFetcher.java index ac82b5559..bff0109f6 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/AbstractFetcher.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/AbstractFetcher.java @@ -24,6 +24,7 @@ import com.antgroup.geaflow.shuffle.message.FetchRequest; import com.antgroup.geaflow.shuffle.message.ISliceMeta; import com.antgroup.geaflow.shuffle.network.IConnectionManager; +import com.antgroup.geaflow.shuffle.serialize.AbstractMessageIterator; import com.antgroup.geaflow.shuffle.serialize.EncoderMessageIterator; import com.antgroup.geaflow.shuffle.serialize.IMessageIterator; import com.antgroup.geaflow.shuffle.serialize.MessageIterator; @@ -54,7 +55,7 @@ public AbstractFetcher() { public void setup(IConnectionManager connectionManager, Configuration config) { this.maxBytesInFlight = config.getLong(SHUFFLE_MAX_BYTES_IN_FLIGHT); - this.shuffleConfig = ShuffleConfig.getInstance(config); + this.shuffleConfig = ShuffleConfig.getInstance(); this.totalSliceNum = 0; } @@ -84,18 +85,18 @@ public void close() { protected IMessageIterator getMessageIterator(int edgeId, OutBuffer outBuffer) { IEncoder encoder = this.encoders.get(edgeId); - if (encoder == null) { - return new MessageIterator<>(outBuffer); - } - return new EncoderMessageIterator<>(outBuffer, encoder); + AbstractMessageIterator messageIterator = encoder == null + ? new MessageIterator<>(outBuffer) + : new EncoderMessageIterator<>(outBuffer, encoder); + return messageIterator; } protected IMessageIterator getMessageIterator(int edgeId, InputStream inputStream) { IEncoder encoder = this.encoders.get(edgeId); - if (encoder == null) { - return new MessageIterator<>(inputStream); - } - return new EncoderMessageIterator<>(inputStream, encoder); + AbstractMessageIterator messageIterator = encoder == null + ? new MessageIterator<>(inputStream) + : new EncoderMessageIterator<>(inputStream, encoder); + return messageIterator; } } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/IShuffleReader.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/IShuffleReader.java index 2db5cad2a..d364e7b8c 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/IShuffleReader.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/IShuffleReader.java @@ -14,6 +14,7 @@ package com.antgroup.geaflow.shuffle.api.reader; +import com.antgroup.geaflow.common.metric.ShuffleReadMetrics; import com.antgroup.geaflow.common.shuffle.DataExchangeMode; import com.antgroup.geaflow.shuffle.message.FetchRequest; import com.antgroup.geaflow.shuffle.message.PipelineEvent; @@ -55,6 +56,13 @@ public interface IShuffleReader extends Serializable { */ DataExchangeMode getExchangeMode(); + /** + * Get read metrics. + * + * @return read metrics. + */ + ShuffleReadMetrics getShuffleReadMetrics(); + /** * Close. */ diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineFetcher.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineFetcher.java index 36b20da83..35942457d 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineFetcher.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineFetcher.java @@ -98,6 +98,7 @@ public PipelineEvent next() { return barrier; } else { int edgeId = buffer.getSliceId().getEdgeId(); + this.readMetrics.increaseDecodeBytes(buffer.getBufferSize()); IMessageIterator msgIterator = this.getMessageIterator(edgeId, buffer.getBuffer()); return new PipelineMessage<>(buffer.getBatchId(), buffer.getStreamName(), msgIterator); } @@ -183,12 +184,4 @@ private boolean checkInputUnchanged(Map> inputSlices) return false; } - public long getInputQueueSize() { - long inputQueueSize = 0; - if (inputFetcher != null) { - inputQueueSize = inputFetcher.getNumberOfQueuedBuffers(); - } - return inputQueueSize; - } - } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineReader.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineReader.java index 019dfdd86..b1968402a 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineReader.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/reader/PipelineReader.java @@ -14,6 +14,7 @@ package com.antgroup.geaflow.shuffle.api.reader; +import com.antgroup.geaflow.common.metric.ShuffleReadMetrics; import com.antgroup.geaflow.common.shuffle.DataExchangeMode; import com.antgroup.geaflow.shuffle.message.FetchRequest; import com.antgroup.geaflow.shuffle.message.PipelineEvent; @@ -62,6 +63,11 @@ public PipelineEvent next() { return shuffleFetcher.next(); } + @Override + public ShuffleReadMetrics getShuffleReadMetrics() { + return this.shuffleFetcher.getReadMetrics(); + } + @Override public void close() { try { diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/IShuffleWriter.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/IShuffleWriter.java index fad1e328a..2fbab26a2 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/IShuffleWriter.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/IShuffleWriter.java @@ -14,7 +14,9 @@ package com.antgroup.geaflow.shuffle.api.writer; +import com.antgroup.geaflow.common.metric.ShuffleWriteMetrics; import java.io.IOException; +import java.util.List; import java.util.Optional; public interface IShuffleWriter { @@ -34,6 +36,17 @@ public interface IShuffleWriter { */ void emit(long batchId, T value, boolean isRetract, int[] channels) throws IOException; + /** + * Emit values to output channels. + * + * @param batchId batch id + * @param value data list + * @param isRetract if retract + * @param channels output channels + * @throws IOException err + */ + void emit(long batchId, List value, boolean isRetract, int channels) throws IOException; + /** * Flush buffered data. * @@ -42,6 +55,13 @@ public interface IShuffleWriter { */ Optional flush(long batchId) throws IOException; + /** + * Get write metrics. + * + * @return write metrics. + */ + ShuffleWriteMetrics getShuffleWriteMetrics(); + /** * Close. */ diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineShardBuffer.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineShardBuffer.java index 17e67659e..f3f0d990b 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineShardBuffer.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineShardBuffer.java @@ -14,9 +14,6 @@ package com.antgroup.geaflow.shuffle.api.writer; -import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_FLUSH_BUFFER_TIMEOUT; -import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_WRITE_BUFFER_SIZE; - import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.OutBuffer.BufferBuilder; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.PipelineShard; @@ -46,11 +43,10 @@ public PipelineShardBuffer() { public void init(IWriterContext writerContext) { super.init(writerContext); - this.maxBufferSize = config.getInteger(SHUFFLE_WRITE_BUFFER_SIZE); initResultSlices(targetChannels); String threadName = "OutputFlusher-" + Thread.currentThread().getName(); - int flushTimeout = config.getInteger(SHUFFLE_FLUSH_BUFFER_TIMEOUT); + int flushTimeout = this.shuffleConfig.getFlushBufferTimeoutMs(); this.outputFlusher = new OutputFlusher(threadName, flushTimeout); this.outputFlusher.start(); } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineWriter.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineWriter.java index 269ee8788..b5878fa89 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineWriter.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/PipelineWriter.java @@ -14,9 +14,11 @@ package com.antgroup.geaflow.shuffle.api.writer; +import com.antgroup.geaflow.common.metric.ShuffleWriteMetrics; import com.antgroup.geaflow.common.shuffle.ShuffleDescriptor; import com.antgroup.geaflow.shuffle.network.IConnectionManager; import java.io.IOException; +import java.util.List; import java.util.Optional; public class PipelineWriter implements IShuffleWriter { @@ -41,11 +43,21 @@ public void emit(long batchId, T value, boolean isRetract, int[] channels) throw shardBuffer.emit(batchId, value, isRetract, channels); } + @Override + public void emit(long batchId, List data, boolean isRetract, int channel) throws IOException { + this.shardBuffer.emit(batchId, data, channel); + } + @Override public Optional flush(long batchId) throws IOException { return shardBuffer.finish(batchId); } + @Override + public ShuffleWriteMetrics getShuffleWriteMetrics() { + return this.shardBuffer.getShuffleWriteMetrics(); + } + @Override public void close() { if (shardBuffer != null) { diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/ShardBuffer.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/ShardBuffer.java index 1993515ab..a2377b763 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/ShardBuffer.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/ShardBuffer.java @@ -14,13 +14,14 @@ package com.antgroup.geaflow.shuffle.api.writer; -import com.antgroup.geaflow.common.config.Configuration; import com.antgroup.geaflow.common.encoder.IEncoder; +import com.antgroup.geaflow.common.metric.ShuffleWriteMetrics; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.HeapBuffer.HeapBufferBuilder; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.OutBuffer; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.OutBuffer.BufferBuilder; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.PipeBuffer; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.PipelineSlice; +import com.antgroup.geaflow.shuffle.config.ShuffleConfig; import com.antgroup.geaflow.shuffle.memory.ShuffleMemoryTracker; import com.antgroup.geaflow.shuffle.message.PipelineBarrier; import com.antgroup.geaflow.shuffle.serialize.EncoderRecordSerializer; @@ -30,11 +31,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public abstract class ShardBuffer { - private static final Logger LOGGER = LoggerFactory.getLogger(ShardBuffer.class); + + protected ShuffleConfig shuffleConfig; protected long pipelineId; protected int edgeId; @@ -43,18 +43,19 @@ public abstract class ShardBuffer { protected int targetChannels; protected String taskLogTag; - protected Configuration config; protected List buffers; protected PipelineSlice[] resultSlices; protected int[] batchCounter; protected long[] bytesCounter; protected ShuffleMemoryTracker memoryTracker; + protected ShuffleWriteMetrics writeMetrics; protected long maxBufferSize; protected IRecordSerializer recordSerializer; public void init(IWriterContext writerContext) { - this.config = writerContext.getConfig(); + this.shuffleConfig = ShuffleConfig.getInstance(); this.memoryTracker = ShuffleMemoryTracker.getInstance(); + this.writeMetrics = new ShuffleWriteMetrics(); this.targetChannels = writerContext.getTargetChannelNum(); this.pipelineId = writerContext.getPipelineInfo().getPipelineId(); @@ -66,6 +67,7 @@ public void init(IWriterContext writerContext) { this.batchCounter = new int[targetChannels]; this.bytesCounter = new long[targetChannels]; + this.maxBufferSize = this.shuffleConfig.getWriteBufferSizeBytes(); buildBufferBuilder(targetChannels); } @@ -91,6 +93,18 @@ public void emit(long batchId, T value, boolean isRetract, int[] channels) } } + public void emit(long batchId, List data, int channel) { + BufferBuilder outBuffer = this.buffers.get(channel); + int size = data.size(); + for (int i = 0; i < size; i++) { + this.recordSerializer.serialize(data.get(i), false, outBuffer); + this.batchCounter[channel]++; + } + if (outBuffer.getBufferSize() >= maxBufferSize) { + send(channel, outBuffer.build(), batchId); + } + } + protected void send(int selectChannel, OutBuffer outBuffer, long batchId) { sendBuffer(selectChannel, outBuffer, batchId); this.bytesCounter[selectChannel] += outBuffer.getBufferSize(); @@ -103,6 +117,10 @@ protected void sendBuffer(int sliceIndex, OutBuffer buffer, long batchId) { public abstract Optional finish(long batchId) throws IOException; + public ShuffleWriteMetrics getShuffleWriteMetrics() { + return this.writeMetrics; + } + public void close() { } @@ -114,10 +132,13 @@ protected void notify(PipelineBarrier barrier) throws IOException { protected void notify(PipelineBarrier barrier, int channel) { long batchId = barrier.getWindowId(); - int counter = batchCounter[channel]; - batchCounter[channel] = 0; - bytesCounter[channel] = 0; - sendBarrier(channel, batchId, counter, barrier.isFinish()); + int recordCount = this.batchCounter[channel]; + sendBarrier(channel, batchId, recordCount, barrier.isFinish()); + + this.writeMetrics.increaseRecords(recordCount); + this.writeMetrics.increaseEncodedSize(this.bytesCounter[channel]); + this.batchCounter[channel] = 0; + this.bytesCounter[channel] = 0; } protected void sendBarrier(int sliceIndex, long batchId, int count, boolean isFinish) { @@ -125,24 +146,6 @@ protected void sendBarrier(int sliceIndex, long batchId, int count, boolean isFi resultSlice.add(new PipeBuffer(batchId, count, false, isFinish)); } - public long getOutputQueueSize() { - PipelineSlice[] slices = resultSlices; - long queueSize = 0; - if (slices != null) { - for (int retry = 0; retry < 3; retry++) { - try { - for (int i = 0; i < slices.length; i++) { - queueSize += slices[i].getNumberOfBuffers(); - } - } catch (Throwable e) { - LOGGER.warn("get slice buffer number failed", e); - // ignore - } - } - } - return queueSize; - } - @SuppressWarnings("unchecked") private static IRecordSerializer getRecordSerializer(IWriterContext writerContext) { IEncoder encoder = writerContext.getEncoder(); diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBuffer.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBuffer.java index 90ae4c6f5..9063a6028 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBuffer.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBuffer.java @@ -14,13 +14,9 @@ package com.antgroup.geaflow.shuffle.api.writer; -import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_CACHE_SPILL_THRESHOLD; - -import com.antgroup.geaflow.common.metric.ShuffleWriteMetrics; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.OutBuffer.BufferBuilder; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.PipelineShard; import com.antgroup.geaflow.shuffle.api.pipeline.buffer.PipelineSlice; -import com.antgroup.geaflow.shuffle.config.ShuffleConfig; import com.antgroup.geaflow.shuffle.memory.ShuffleDataManager; import com.antgroup.geaflow.shuffle.message.ISliceMeta; import com.antgroup.geaflow.shuffle.message.PipelineBarrier; @@ -30,7 +26,6 @@ import com.antgroup.geaflow.shuffle.message.SliceId; import com.antgroup.geaflow.shuffle.message.WriterId; import com.antgroup.geaflow.shuffle.network.IConnectionManager; -import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -47,8 +42,6 @@ public class SpillableShardBuffer extends ShardBuffer { protected WriterId writerId; protected ShuffleId shuffleId; - protected ShuffleWriteMetrics writeMetrics; - protected ShuffleConfig shuffleConfig; protected IWriterContext writerContext; protected IConnectionManager connectionManager; protected int taskId; @@ -70,9 +63,6 @@ public void init(IWriterContext writerContext) { this.writerContext = writerContext; this.taskId = writerContext.getTaskId(); - this.shuffleConfig = ShuffleConfig.getInstance(config); - - this.writeMetrics = new ShuffleWriteMetrics(); this.cacheEnabled = writerContext.getShuffleDescriptor().isCacheEnabled(); if (cacheEnabled) { @@ -85,7 +75,7 @@ public void init(IWriterContext writerContext) { int refCount = cacheEnabled ? Integer.MAX_VALUE : 1; initResultSlices(channels, refCount); - this.cacheSpillThreshold = config.getDouble(SHUFFLE_CACHE_SPILL_THRESHOLD); + this.cacheSpillThreshold = this.shuffleConfig.getCacheSpillThreshold(); } private void initResultSlices(int channels, int refCount) { @@ -110,9 +100,7 @@ public Optional finish(long batchId) throws IOException { for (int i = 0; i < slices.size(); i++) { ISliceMeta sliceMeta = slices.get(i); if (sliceMeta.getRecordNum() > 0) { - writeMetrics.increaseWrittenChannels(); - writeMetrics.increaseRecords(sliceMeta.getRecordNum()); - writeMetrics.increaseEncodedSize(sliceMeta.getEncodedSize()); + this.writeMetrics.increaseWrittenChannels(); if (sliceMeta.getEncodedSize() > maxSliceSize) { maxSliceSize = sliceMeta.getEncodedSize(); } @@ -120,12 +108,12 @@ public Optional finish(long batchId) throws IOException { buffers.get(i).close(); } - writeMetrics.setMaxSliceKB(maxSliceSize / 1024); - writeMetrics.setNumChannels(slices.size()); + this.writeMetrics.setMaxSliceKB(maxSliceSize / 1024); + this.writeMetrics.setNumChannels(slices.size()); long flushTime = System.currentTimeMillis() - beginTime; - writeMetrics.setFlushMs(flushTime); + this.writeMetrics.setFlushMs(flushTime); LOGGER.info("taskId {} {} flush batchId:{} useTime:{}ms {}", taskId, taskLogTag, batchId, - flushTime, writeMetrics); + flushTime, this.writeMetrics); buffers.clear(); buffers = null; @@ -161,11 +149,6 @@ private List buildSliceMeta(long batchId) { return slices; } - @VisibleForTesting - public ShuffleWriteMetrics getWriteMetrics() { - return writeMetrics; - } - @Override public void close() { } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/config/ShuffleConfig.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/config/ShuffleConfig.java index eca891c7c..8e64abb4c 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/config/ShuffleConfig.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/config/ShuffleConfig.java @@ -28,13 +28,21 @@ import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.NETTY_SERVER_PORT; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.NETTY_SERVER_THREADS_NUM; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.NETTY_THREAD_CACHE_ENABLE; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_CACHE_SPILL_THRESHOLD; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_COMPRESSION_ENABLE; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_EMIT_BUFFER_SIZE; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_EMIT_QUEUE_SIZE; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_FETCH_QUEUE_SIZE; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_FETCH_TIMEOUT_MS; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_FLUSH_BUFFER_TIMEOUT_MS; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_MEMORY_POOL_ENABLE; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_SLICE_MAX_SPILL_SIZE; import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_STORAGE_TYPE; +import static com.antgroup.geaflow.common.config.keys.ExecutionConfigKeys.SHUFFLE_WRITE_BUFFER_SIZE_BYTES; import com.antgroup.geaflow.common.config.Configuration; import com.antgroup.geaflow.common.shuffle.StorageLevel; +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +63,14 @@ public class ShuffleConfig { private final int clientThreads; private final long maxSpillSizePerSlice; + private final boolean memoryPoolEnable; + private final int fetchQueueSize; + private final int emitQueueSize; + private final int emitBufferSize; + private final int writeBufferSizeBytes; + private final int flushBufferTimeoutMs; + private final double cacheSpillThreshold; + private final Configuration configuration; private boolean compressionEnabled; @@ -74,14 +90,20 @@ private ShuffleConfig(Configuration config) { this.serverThreads = config.getInteger(NETTY_SERVER_THREADS_NUM); this.clientThreads = config.getInteger(NETTY_CLIENT_THREADS_NUM); + this.memoryPoolEnable = config.getBoolean(SHUFFLE_MEMORY_POOL_ENABLE); + this.fetchQueueSize = config.getInteger(SHUFFLE_FETCH_QUEUE_SIZE); + this.emitQueueSize = config.getInteger(SHUFFLE_EMIT_QUEUE_SIZE); + this.emitBufferSize = config.getInteger(SHUFFLE_EMIT_BUFFER_SIZE); + this.writeBufferSizeBytes = config.getInteger(SHUFFLE_WRITE_BUFFER_SIZE_BYTES); + this.flushBufferTimeoutMs = config.getInteger(SHUFFLE_FLUSH_BUFFER_TIMEOUT_MS); + this.cacheSpillThreshold = config.getDouble(SHUFFLE_CACHE_SPILL_THRESHOLD); + this.compressionEnabled = config.getBoolean(SHUFFLE_COMPRESSION_ENABLE); this.configuration = config; LOGGER.info("init shuffle config: {}", config); } - - public static synchronized ShuffleConfig getInstance(Configuration config) { if (INSTANCE == null) { INSTANCE = new ShuffleConfig(config); @@ -89,6 +111,11 @@ public static synchronized ShuffleConfig getInstance(Configuration config) { return INSTANCE; } + @VisibleForTesting + public static synchronized void reset(Configuration config) { + INSTANCE = new ShuffleConfig(config); + } + public static ShuffleConfig getInstance() { return INSTANCE; } @@ -163,6 +190,34 @@ public int getConnectMaxBackoffMs() { return configuration.getInteger(NETTY_CONNECT_MAX_BACKOFF_MS); } + public boolean isMemoryPoolEnable() { + return this.memoryPoolEnable; + } + + public int getFetchQueueSize() { + return this.fetchQueueSize; + } + + public int getEmitQueueSize() { + return this.emitQueueSize; + } + + public int getEmitBufferSize() { + return this.emitBufferSize; + } + + public int getWriteBufferSizeBytes() { + return this.writeBufferSizeBytes; + } + + public int getFlushBufferTimeoutMs() { + return this.flushBufferTimeoutMs; + } + + public double getCacheSpillThreshold() { + return this.cacheSpillThreshold; + } + public Configuration getConfig() { return configuration; } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/network/netty/SliceRequestServerHandler.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/network/netty/SliceRequestServerHandler.java index a0be458c8..5de0b52fc 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/network/netty/SliceRequestServerHandler.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/main/java/com/antgroup/geaflow/shuffle/network/netty/SliceRequestServerHandler.java @@ -89,7 +89,6 @@ private void respondWithError(ChannelHandlerContext ctx, Throwable error) { private void respondWithError(ChannelHandlerContext ctx, Throwable error, ChannelId sourceId) { - LOGGER.error("Responding with error: {}.", error.getClass()); ctx.writeAndFlush(new ErrorResponse(sourceId, error)); } diff --git a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/test/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBufferTest.java b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/test/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBufferTest.java index fb78963cc..4474e3ae5 100644 --- a/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/test/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBufferTest.java +++ b/geaflow/geaflow-core/geaflow-engine/geaflow-shuffle/src/test/java/com/antgroup/geaflow/shuffle/api/writer/SpillableShardBufferTest.java @@ -57,7 +57,7 @@ public void testEmit() throws IOException { Shard shard = optional.get(); Assert.assertNotNull(shard); - ShuffleWriteMetrics writeMetrics = shardBuffer.getWriteMetrics(); + ShuffleWriteMetrics writeMetrics = shardBuffer.getShuffleWriteMetrics(); Assert.assertEquals(writeMetrics.getSpillNum(), 0); Assert.assertEquals(writeMetrics.getSpillDisk(), 0); long usedMemory = ShuffleMemoryTracker.getInstance().getUsedMemory(); diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-operator/src/main/java/com/antgroup/geaflow/operator/base/AbstractOperator.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-operator/src/main/java/com/antgroup/geaflow/operator/base/AbstractOperator.java index 16840e070..00fa4c4ee 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-operator/src/main/java/com/antgroup/geaflow/operator/base/AbstractOperator.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-operator/src/main/java/com/antgroup/geaflow/operator/base/AbstractOperator.java @@ -104,14 +104,14 @@ public void open(OpContext opContext) { for (Operator subOperator : subOperatorList) { OpContext subOpContext = new DefaultOpContext(opContext.getCollectors(), opContext.getRuntimeContext()); subOperator.open(subOpContext); - IChainCollector chainCollector = new OpChainCollector(opArgs.getOpId(), subOperator); + IChainCollector chainCollector = new OpChainCollector<>(opArgs.getOpId(), subOperator); this.collectors.add(chainCollector); } this.collectors.addAll(opContext.getCollectors().stream().filter(collector -> collector.getId() == opArgs.getOpId()) .collect(Collectors.toList())); for (int i = 0, size = this.collectors.size(); i < size; i++) { - ICollector collector = this.collectors.get(i); + ICollector collector = this.collectors.get(i); collector.setUp(this.runtimeContext); if (collector instanceof AbstractCollector) { ((AbstractCollector) collector).setOutputMetric(this.opOutputMeter); diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-plan/src/main/java/com/antgroup/geaflow/core/graph/ExecutionTask.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-plan/src/main/java/com/antgroup/geaflow/core/graph/ExecutionTask.java index dbf3068f4..af1da05f5 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-plan/src/main/java/com/antgroup/geaflow/core/graph/ExecutionTask.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-plan/src/main/java/com/antgroup/geaflow/core/graph/ExecutionTask.java @@ -15,6 +15,8 @@ package com.antgroup.geaflow.core.graph; import com.antgroup.geaflow.cluster.resourcemanager.WorkerInfo; +import com.antgroup.geaflow.common.task.TaskArgs; +import com.antgroup.geaflow.common.utils.LoggerFormatter; import com.antgroup.geaflow.processor.Processor; import java.io.Serializable; @@ -30,6 +32,7 @@ public class ExecutionTask implements Serializable { private WorkerInfo workerInfo; private Processor processor; private ExecutionTaskType executionTaskType; + private boolean iterative; private long startTime; private long duration; @@ -126,12 +129,33 @@ public void setExecutionTaskType(ExecutionTaskType executionTaskType) { this.executionTaskType = executionTaskType; } + public void setIterative(boolean iterative) { + this.iterative = iterative; + } + + public boolean isIterative() { + return this.iterative; + } + public String getTaskName() { return taskName; } - public void setTaskName(String taskName) { - this.taskName = taskName; + public void buildTaskName(String pipelineName, int cycleId, long windowId) { + this.taskName = this.iterative + ? LoggerFormatter.getTaskTag( + pipelineName, cycleId, windowId, this.taskId, this.vertexId, this.index, this.parallelism) + : LoggerFormatter.getTaskTag( + pipelineName, cycleId, this.taskId, this.vertexId, this.index, this.parallelism); + } + + public TaskArgs buildTaskArgs() { + return new TaskArgs( + this.taskId, + this.index, + this.taskName, + this.parallelism, + this.maxParallelism); } @Override diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/DefaultRuntimeContext.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/DefaultRuntimeContext.java index ef7e8f5a0..c3d1e31e9 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/DefaultRuntimeContext.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/DefaultRuntimeContext.java @@ -18,18 +18,13 @@ import com.antgroup.geaflow.common.config.Configuration; import com.antgroup.geaflow.common.task.TaskArgs; import com.antgroup.geaflow.context.AbstractRuntimeContext; -import com.antgroup.geaflow.core.graph.ExecutionTask; import com.antgroup.geaflow.metrics.common.api.MetricGroup; import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; import java.util.HashMap; import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class DefaultRuntimeContext extends AbstractRuntimeContext { - private static final Logger LOGGER = LoggerFactory.getLogger(DefaultRuntimeContext.class); - private long pipelineId; private String pipelineName; private TaskArgs taskArgs; @@ -66,9 +61,7 @@ public TaskArgs getTaskArgs() { } public DefaultRuntimeContext setTaskArgs(TaskArgs taskArgs) { - this.taskArgs = new TaskArgs(taskArgs.getTaskId(), - taskArgs.getTaskIndex(), taskArgs.getTaskName(), - taskArgs.getParallelism(), taskArgs.getMaxParallelism()); + this.taskArgs = taskArgs; return this; } @@ -77,13 +70,6 @@ public Configuration getConfiguration() { return this.jobConfig; } - public DefaultRuntimeContext setExecutionTask(ExecutionTask task) { - this.taskArgs = new TaskArgs(task.getTaskId(), task.getIndex(), - task.getTaskName(), task.getParallelism(), - task.getMaxParallelism()); - return this; - } - public DefaultRuntimeContext setIoDescriptor(IoDescriptor ioDescriptor) { this.ioDescriptor = ioDescriptor; return this; diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/EventContext.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/EventContext.java index 87a066ca0..b70bdd437 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/EventContext.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/context/EventContext.java @@ -15,7 +15,6 @@ package com.antgroup.geaflow.runtime.core.context; import com.antgroup.geaflow.cluster.protocol.IEventContext; -import com.antgroup.geaflow.common.utils.LoggerFormatter; import com.antgroup.geaflow.core.graph.ExecutionTask; import com.antgroup.geaflow.processor.Processor; import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; @@ -133,9 +132,6 @@ public EventContextBuilder withPipelineId(long pipelineId) { public EventContextBuilder withPipelineName(String pipelineName) { this.pipelineName = pipelineName; - String taskName = LoggerFormatter.getTaskLog(pipelineName, cycleId, - currentWindowId, executionTask.getIndex()); - executionTask.setTaskName(taskName); return this; } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractExecutableCommand.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractExecutableCommand.java index 831c94373..42e6c7330 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractExecutableCommand.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractExecutableCommand.java @@ -18,9 +18,11 @@ import com.antgroup.geaflow.cluster.fetcher.FetcherRunner; import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.protocol.IExecutableCommand; +import com.antgroup.geaflow.cluster.rpc.RpcClient; import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.cluster.worker.IWorker; import com.antgroup.geaflow.cluster.worker.IWorkerContext; +import com.antgroup.geaflow.common.metric.EventMetrics; import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,9 +85,12 @@ public void interrupt() { * @param windowId * @param eventType */ - protected void sendDoneEvent(int cycleId, long windowId, EventType eventType) { - AbstractWorkerContext workerContext = (AbstractWorkerContext) context; - DoneEvent doneEvent = new DoneEvent(cycleId, windowId, workerContext.getTaskId(), eventType); - workerContext.getPipelineMaster().send(doneEvent); + protected void sendDoneEvent(String driverId, EventType sourceEventType, T result, boolean sendMetrics) { + AbstractWorkerContext workerContext = (AbstractWorkerContext) this.context; + int taskId = workerContext.getTaskId(); + EventMetrics eventMetrics = sendMetrics ? workerContext.getEventMetrics() : null; + DoneEvent doneEvent = new DoneEvent<>(this.cycleId, this.windowId, taskId, sourceEventType, result, eventMetrics); + RpcClient.getInstance().processPipeline(driverId, doneEvent); } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractInitCommand.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractInitCommand.java index 4179a926b..a75772eae 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractInitCommand.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractInitCommand.java @@ -14,38 +14,72 @@ package com.antgroup.geaflow.runtime.core.protocol; +import com.antgroup.geaflow.api.context.RuntimeContext; +import com.antgroup.geaflow.cluster.collector.AbstractPipelineCollector; +import com.antgroup.geaflow.cluster.collector.CollectorFactory; +import com.antgroup.geaflow.cluster.collector.IOutputMessageBuffer; +import com.antgroup.geaflow.cluster.collector.InitEmitterRequest; +import com.antgroup.geaflow.cluster.collector.UpdateEmitterRequest; import com.antgroup.geaflow.cluster.fetcher.InitFetchRequest; +import com.antgroup.geaflow.collector.ICollector; import com.antgroup.geaflow.common.encoder.IEncoder; +import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; import com.antgroup.geaflow.common.shuffle.ShuffleDescriptor; import com.antgroup.geaflow.core.graph.ExecutionTask; +import com.antgroup.geaflow.core.graph.util.ExecutionTaskUtils; +import com.antgroup.geaflow.io.CollectType; import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; -import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; -import com.antgroup.geaflow.runtime.core.worker.fetch.FetchListenerImpl; +import com.antgroup.geaflow.runtime.core.worker.InputReader; +import com.antgroup.geaflow.runtime.core.worker.OutputWriter; +import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; import com.antgroup.geaflow.runtime.io.IInputDesc; import com.antgroup.geaflow.runtime.shuffle.InputDescriptor; import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; import com.antgroup.geaflow.runtime.shuffle.ShardInputDesc; +import com.antgroup.geaflow.shuffle.ForwardOutputDesc; +import com.antgroup.geaflow.shuffle.IOutputDesc; +import com.antgroup.geaflow.shuffle.OutputDescriptor; import com.antgroup.geaflow.shuffle.message.Shard; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; public abstract class AbstractInitCommand extends AbstractExecutableCommand { - public AbstractInitCommand(int workerId, int cycleId, long windowId) { + protected final long pipelineId; + protected final String pipelineName; + protected IoDescriptor ioDescriptor; + + public AbstractInitCommand(int workerId, int cycleId, long windowId, long pipelineId, String pipelineName) { super(workerId, cycleId, windowId); + this.pipelineId = pipelineId; + this.pipelineName = pipelineName; + } + + public void setIoDescriptor(IoDescriptor ioDescriptor) { + this.ioDescriptor = ioDescriptor; } - private InitFetchRequest buildInitFetchRequest(long pipelineId, - String pipelineName, - ExecutionTask executionTask, - IoDescriptor ioDescriptor) { - InputDescriptor inputDescriptor = ioDescriptor.getInputDescriptor(); + protected void initFetcher() { + WorkerContext workerContext = (WorkerContext) this.context; + if (!(ExecutionTaskUtils.isCycleHead(workerContext.getExecutionTask()) + && this.ioDescriptor.getInputTaskNum() == 0)) { + InitFetchRequest request = this.buildInitFetchRequest(workerContext.getExecutionTask()); + InputReader inputReader = ((AbstractAlignedWorker) this.worker).getInputReader(); + inputReader.setEventMetrics(workerContext.getEventMetrics()); + request.addListener(inputReader); + this.fetcherRunner.add(request); + } + } + + private InitFetchRequest buildInitFetchRequest(ExecutionTask executionTask) { + InputDescriptor inputDescriptor = this.ioDescriptor.getInputDescriptor(); Map> inputShardMap = new HashMap<>(); Map streamId2NameMap = new HashMap<>(); Map> edgeId2EncoderMap = new HashMap<>(); ShuffleDescriptor shuffleDescriptor = null; - for (IInputDesc inputDesc : inputDescriptor.getInputDescMap().values()) { + for (IInputDesc inputDesc : inputDescriptor.getInputDescMap().values()) { if (inputDesc.getInputType() == IInputDesc.InputType.META) { inputShardMap.put(inputDesc.getEdgeId(), inputDesc.getInput()); edgeId2EncoderMap.put(inputDesc.getEdgeId(), ((ShardInputDesc) inputDesc).getEncoder()); @@ -53,7 +87,7 @@ private InitFetchRequest buildInitFetchRequest(long pipelineId, shuffleDescriptor = ((ShardInputDesc) inputDesc).getShuffleDescriptor(); } } - return new InitFetchRequest.InitRequestBuilder(pipelineId, pipelineName) + return new InitFetchRequest.InitRequestBuilder(this.pipelineId, this.pipelineName) .setInputShardMap(inputShardMap) .setInputStreamMap(streamId2NameMap) .setEncoders(edgeId2EncoderMap) @@ -64,14 +98,89 @@ private InitFetchRequest buildInitFetchRequest(long pipelineId, .setVertexId(executionTask.getVertexId()); } - /** - * Init input fetcher. - */ - protected void initFetchRequest(IoDescriptor ioDescriptor, long pipelineId, String pipelineName) { - AbstractWorkerContext workerContext = (AbstractWorkerContext) context; - InitFetchRequest request = buildInitFetchRequest(pipelineId, - pipelineName, workerContext.getExecutionTask(), ioDescriptor); - request.addListener(new FetchListenerImpl(((AbstractAlignedWorker) worker).getInputReader())); - fetcherRunner.add(request); + protected void initEmitter() { + OutputDescriptor outputDescriptor = this.ioDescriptor.getOutputDescriptor(); + if (outputDescriptor == null) { + return; + } + InitEmitterRequest request = this.buildInitEmitterRequest(outputDescriptor); + this.emitterRunner.add(request); + + List> collectors = this.buildCollectors(outputDescriptor, request); + ((WorkerContext) this.context).setCollectors(collectors); + } + + protected List> buildCollectors(OutputDescriptor outputDescriptor, InitEmitterRequest request) { + List outputDescList = outputDescriptor.getOutputDescList(); + int outputNum = outputDescList.size(); + List> collectors = new ArrayList<>(outputNum); + List> outputBuffers = request.getOutputBuffers(); + for (int i = 0; i < outputNum; i++) { + IOutputDesc outputDesc = outputDescList.get(i); + IOutputMessageBuffer outputBuffer = outputBuffers.get(i); + ICollector collector = CollectorFactory.create(outputDesc); + if (outputDesc.getType() != CollectType.RESPONSE) { + ((AbstractPipelineCollector) collector).setOutputBuffer(outputBuffer); + } + collectors.add(collector); + } + return collectors; + } + + private InitEmitterRequest buildInitEmitterRequest(OutputDescriptor outputDescriptor) { + List> outputBuffers = this.getOutputBuffers(outputDescriptor.getOutputDescList()); + RuntimeContext runtimeContext = ((WorkerContext) this.context).getRuntimeContext(); + return new InitEmitterRequest( + runtimeContext.getConfiguration(), + this.windowId, + runtimeContext.getPipelineId(), + runtimeContext.getPipelineName(), + runtimeContext.getTaskArgs(), + outputDescriptor, + outputBuffers); } + + protected void updateEmitter() { + OutputDescriptor outputDescriptor = ioDescriptor.getOutputDescriptor(); + if (outputDescriptor == null) { + return; + } + + WorkerContext workerContext = (WorkerContext) this.context; + List outputDescList = outputDescriptor.getOutputDescList(); + int outputNum = outputDescList.size(); + List> collectors = workerContext.getCollectors(); + if (collectors.size() != outputNum) { + throw new GeaflowRuntimeException(String.format("collector num %d not match output desc num %d", collectors.size(), outputNum)); + } + + List> outputBuffers = this.getOutputBuffers(outputDescList); + for (int i = 0; i < outputNum; i++) { + if (collectors.get(i) instanceof AbstractPipelineCollector) { + AbstractPipelineCollector collector = (AbstractPipelineCollector) collectors.get(i); + IOutputMessageBuffer outputBuffer = outputBuffers.get(i); + collector.setOutputBuffer(outputBuffer); + } + } + + UpdateEmitterRequest updateEmitterRequest = + new UpdateEmitterRequest(workerContext.getTaskId(), this.windowId, this.pipelineId, this.pipelineName, outputBuffers); + this.emitterRunner.add(updateEmitterRequest); + } + + private List> getOutputBuffers(List outputDescList) { + int outputNum = outputDescList.size(); + List> outputBuffers = new ArrayList<>(outputNum); + for (IOutputDesc outputDesc : outputDescList) { + OutputWriter outputBuffer = null; + if (outputDesc.getType() != CollectType.RESPONSE) { + int bucketNum = ((ForwardOutputDesc) outputDesc).getTargetTaskIndices().size(); + outputBuffer = new OutputWriter<>(outputDesc.getEdgeId(), bucketNum); + outputBuffer.setEventMetrics(((WorkerContext) this.context).getEventMetrics()); + } + outputBuffers.add(outputBuffer); + } + return outputBuffers; + } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractIterationComputeCommand.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractIterationComputeCommand.java index 7b33915fd..28ccb7ab3 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractIterationComputeCommand.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/AbstractIterationComputeCommand.java @@ -17,6 +17,7 @@ import com.antgroup.geaflow.cluster.fetcher.ReFetchRequest; import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; +import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; import com.antgroup.geaflow.shuffle.message.PipelineMessage; import java.util.HashMap; import java.util.List; @@ -43,11 +44,13 @@ public AbstractIterationComputeCommand(int workerId, int cycleId, long windowId, @Override public void execute(ITaskContext taskContext) { + final long start = System.currentTimeMillis(); super.execute(taskContext); AbstractAlignedWorker alignedWorker = (AbstractAlignedWorker) worker; alignedWorker.init(windowId); fetcherRunner.add(new ReFetchRequest(fetchWindowId, fetchCount)); alignedWorker.alignedProcess(fetchCount); + ((AbstractWorkerContext) this.context).getEventMetrics().addProcessCostMs(System.currentTimeMillis() - start); } } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanCycleEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanCycleEvent.java index 1f2a66753..e322fc315 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanCycleEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanCycleEvent.java @@ -17,6 +17,13 @@ import com.antgroup.geaflow.cluster.collector.CloseEmitterRequest; import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.task.ITaskContext; +import com.antgroup.geaflow.common.metric.EventMetrics; +import com.antgroup.geaflow.common.utils.GcUtil; +import com.antgroup.geaflow.core.graph.ExecutionTask; +import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; +import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Clean worker runtime execution env, e.g. close shuffle reader/writer and close processor. @@ -24,6 +31,8 @@ */ public class CleanCycleEvent extends AbstractCleanCommand { + private static final Logger LOGGER = LoggerFactory.getLogger(CleanCycleEvent.class); + public CleanCycleEvent(int workerId, int cycleId, long windowId) { super(workerId, cycleId, windowId); } @@ -31,9 +40,21 @@ public CleanCycleEvent(int workerId, int cycleId, long windowId) { @Override public void execute(ITaskContext taskContext) { super.execute(taskContext); - emitterRunner.add(new CloseEmitterRequest()); - worker.close(); - sendDoneEvent(cycleId, windowId, EventType.CLEAN_CYCLE); + WorkerContext workerContext = (WorkerContext) this.context; + ExecutionTask executionTask = workerContext.getExecutionTask(); + this.emitterRunner.add(new CloseEmitterRequest(executionTask.getTaskId(), this.windowId)); + this.worker.close(); + EventMetrics eventMetrics = ((AbstractWorkerContext) this.context).getEventMetrics(); + eventMetrics.setFinishTime(System.currentTimeMillis()); + eventMetrics.setFinishGcTs(GcUtil.computeCurrentTotalGcTime()); + LOGGER.info("clean task {} {}/{} of {} {} : {}", + executionTask.getTaskId(), + executionTask.getIndex(), + executionTask.getParallelism(), + executionTask.getVertexId(), + executionTask.getProcessor().toString(), + eventMetrics); + this.sendDoneEvent(workerContext.getDriverId(), EventType.CLEAN_CYCLE, null, true); } @Override diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanEnvEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanEnvEvent.java index 6495d4d67..878f2ec7a 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanEnvEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanEnvEvent.java @@ -14,6 +14,7 @@ package com.antgroup.geaflow.runtime.core.protocol; +import com.antgroup.geaflow.cluster.collector.ClearEmitterRequest; import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.rpc.RpcClient; import com.antgroup.geaflow.cluster.task.ITaskContext; @@ -25,8 +26,8 @@ public class CleanEnvEvent extends AbstractCleanCommand { private final long pipelineId; private final String driverId; - public CleanEnvEvent(int workerId, int cycleId, long iterationId, long pipelineId, String driverId) { - super(workerId, cycleId, iterationId); + public CleanEnvEvent(int workerId, int cycleId, long windowId, long pipelineId, String driverId) { + super(workerId, cycleId, windowId); this.pipelineId = pipelineId; this.driverId = driverId; } @@ -36,7 +37,8 @@ public void execute(ITaskContext taskContext) { super.execute(taskContext); ShuffleDataManager.getInstance().release(pipelineId); WorkerContextManager.clear(); - sendDoneEvent(cycleId, windowId, EventType.CLEAN_ENV); + this.emitterRunner.add(ClearEmitterRequest.INSTANCE); + this.sendDoneEvent(this.driverId, EventType.CLEAN_ENV, null, false); } @Override @@ -58,8 +60,8 @@ public void setIterationId(int iterationId) { } @Override - protected void sendDoneEvent(int cycleId, long windowId, EventType eventType) { - DoneEvent doneEvent = new DoneEvent(cycleId, windowId, 0, eventType); + protected void sendDoneEvent(String driverId, EventType sourceEventType, T result, boolean sendMetrics) { + DoneEvent doneEvent = new DoneEvent<>(this.cycleId, this.windowId, 0, sourceEventType, result); RpcClient.getInstance().processPipeline(driverId, doneEvent); } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanStashEnvEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanStashEnvEvent.java index b395c0c8f..7fac3f073 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanStashEnvEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/CleanStashEnvEvent.java @@ -35,7 +35,7 @@ public CleanStashEnvEvent(int workerId, int cycleId, long iterationId, long pipe public void execute(ITaskContext taskContext) { super.execute(taskContext); ShuffleDataManager.getInstance().release(pipelineId); - sendDoneEvent(cycleId, windowId, EventType.CLEAN_ENV); + this.sendDoneEvent(this.driverId, EventType.CLEAN_ENV, null, false); } @Override @@ -57,8 +57,8 @@ public void setIterationId(int iterationId) { } @Override - protected void sendDoneEvent(int cycleId, long windowId, EventType eventType) { - DoneEvent doneEvent = new DoneEvent(cycleId, windowId, 0, eventType); + protected void sendDoneEvent(String driverId, EventType sourceEventType, T result, boolean sendMetrics) { + DoneEvent doneEvent = new DoneEvent<>(this.cycleId, this.windowId, 0, sourceEventType, result); RpcClient.getInstance().processPipeline(driverId, doneEvent); } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/DoneEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/DoneEvent.java index 6896cdfd2..1e9c07bda 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/DoneEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/DoneEvent.java @@ -42,20 +42,24 @@ public class DoneEvent implements ICycleResponseEvent { private EventMetrics eventMetrics; public DoneEvent(int cycleId, long windowId, int tailTaskId, EventType sourceEvent) { - this.cycleId = cycleId; - this.windowId = windowId; - this.taskId = tailTaskId; - this.sourceEvent = sourceEvent; + this(cycleId, windowId, tailTaskId, sourceEvent, null, null); } public DoneEvent(int cycleId, long windowId, int tailTaskId, EventType sourceEvent, T result) { - this(cycleId, windowId, tailTaskId, sourceEvent); - this.result = result; + this(cycleId, windowId, tailTaskId, sourceEvent, result, null); } - public DoneEvent(int cycleId, long windowId, int tailTaskId, EventType sourceEvent, T result, + public DoneEvent(int cycleId, + long windowId, + int tailTaskId, + EventType sourceEvent, + T result, EventMetrics eventMetrics) { - this(cycleId, windowId, tailTaskId, sourceEvent, result); + this.cycleId = cycleId; + this.windowId = windowId; + this.taskId = tailTaskId; + this.sourceEvent = sourceEvent; + this.result = result; this.eventMetrics = eventMetrics; } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/FinishIterationEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/FinishIterationEvent.java index 05cb3f554..cf4883596 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/FinishIterationEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/FinishIterationEvent.java @@ -16,6 +16,7 @@ import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.task.ITaskContext; +import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -23,7 +24,7 @@ public class FinishIterationEvent extends AbstractExecutableCommand { private static final Logger LOGGER = LoggerFactory.getLogger(FinishIterationEvent.class); - private static final long END_OF_ITERATION_ID = 0; + public static final long END_OF_ITERATION_ID = 0; private int taskId; @@ -34,9 +35,11 @@ public FinishIterationEvent(int workerId, long windowId, int cycleId, int taskId @Override public void execute(ITaskContext taskContext) { + final long start = System.currentTimeMillis(); super.execute(taskContext); worker.init(windowId); worker.finish(END_OF_ITERATION_ID); + ((AbstractWorkerContext) this.context).getEventMetrics().addProcessCostMs(System.currentTimeMillis() - start); } @Override diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCollectCycleEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCollectCycleEvent.java index 9fa3d3166..be5c47ebd 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCollectCycleEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCollectCycleEvent.java @@ -14,15 +14,15 @@ package com.antgroup.geaflow.runtime.core.protocol; -import com.antgroup.geaflow.cluster.collector.InitCollectEmitterRequest; +import com.antgroup.geaflow.cluster.collector.CollectResponseCollector; import com.antgroup.geaflow.cluster.collector.InitEmitterRequest; import com.antgroup.geaflow.collector.ICollector; import com.antgroup.geaflow.core.graph.ExecutionTask; import com.antgroup.geaflow.ha.runtime.HighAvailableLevel; import com.antgroup.geaflow.io.ResponseOutputDesc; -import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; import com.antgroup.geaflow.shuffle.OutputDescriptor; import com.google.common.base.Preconditions; +import java.util.Collections; import java.util.List; /** @@ -31,24 +31,20 @@ */ public class InitCollectCycleEvent extends InitCycleEvent { + private static final int COLLECT_BUCKET_NUM = 1; + public InitCollectCycleEvent(int workerId, int cycleId, long iterationId, long pipelineId, String pipelineName, ExecutionTask task, HighAvailableLevel haLevel, long nestedWindowId) { super(workerId, cycleId, iterationId, pipelineId, pipelineName, task, haLevel, nestedWindowId); } - /** - * Init output emitter. - */ - protected List initEmitterRequest(OutputDescriptor outputDescriptor) { - - Preconditions.checkArgument(outputDescriptor.getOutputDescList().size() == 1, + @Override + protected List> buildCollectors(OutputDescriptor outputDescriptor, InitEmitterRequest request) { + Preconditions.checkArgument(outputDescriptor.getOutputDescList().size() == COLLECT_BUCKET_NUM, "only support one collect output info yet"); - InitEmitterRequest request = new InitCollectEmitterRequest( - (ResponseOutputDesc) outputDescriptor.getOutputDescList().get(0)); - emitterRunner.add(request); - ((AbstractAlignedWorker) worker).getOutputWriter() - .setCollectors(request.getCollectors()); - return request.getCollectors(); + ResponseOutputDesc outputDesc = (ResponseOutputDesc) outputDescriptor.getOutputDescList().get(0); + return Collections.singletonList(new CollectResponseCollector<>(outputDesc)); } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCycleEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCycleEvent.java index e3b7579d4..5a381efb6 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCycleEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitCycleEvent.java @@ -14,25 +14,14 @@ package com.antgroup.geaflow.runtime.core.protocol; -import com.antgroup.geaflow.cluster.collector.InitEmitterRequest; import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.protocol.IEventContext; import com.antgroup.geaflow.cluster.protocol.IHighAvailableEvent; import com.antgroup.geaflow.cluster.task.ITaskContext; -import com.antgroup.geaflow.collector.ICollector; import com.antgroup.geaflow.core.graph.ExecutionTask; -import com.antgroup.geaflow.core.graph.util.ExecutionTaskUtils; import com.antgroup.geaflow.ha.runtime.HighAvailableLevel; import com.antgroup.geaflow.runtime.core.context.EventContext; -import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; -import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; -import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; -import com.antgroup.geaflow.shuffle.OutputDescriptor; -import java.util.ArrayList; -import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * An assign event provides some runtime execution information for worker to build the cycle pipeline. @@ -40,12 +29,7 @@ */ public class InitCycleEvent extends AbstractInitCommand implements IHighAvailableEvent { - private static final Logger LOGGER = LoggerFactory.getLogger(InitCycleEvent.class); - - private long pipelineId; - private String pipelineName; private ExecutionTask task; - private IoDescriptor ioDescriptor; private String driverId; private HighAvailableLevel haLevel; private long iterationWindowId; @@ -54,9 +38,7 @@ public InitCycleEvent(int workerId, int cycleId, long iterationId, long pipelineId, String pipelineName, ExecutionTask task, HighAvailableLevel haLevel, long windowId) { - super(workerId, cycleId, iterationId); - this.pipelineId = pipelineId; - this.pipelineName = pipelineName; + super(workerId, cycleId, iterationId, pipelineId, pipelineName); this.task = task; this.haLevel = haLevel; this.iterationWindowId = windowId; @@ -65,8 +47,9 @@ public InitCycleEvent(int workerId, int cycleId, long iterationId, @Override public void execute(ITaskContext taskContext) { super.execute(taskContext); - AbstractWorkerContext workerContext = new WorkerContext(taskContext); + WorkerContext workerContext = new WorkerContext(taskContext); context = workerContext; + this.task.buildTaskName(this.pipelineName, this.cycleId, this.windowId); IEventContext eventContext = EventContext.builder() .withExecutionTask(task) .withDriverId(driverId) @@ -78,14 +61,9 @@ public void execute(ITaskContext taskContext) { .withWindowId(iterationWindowId) .build(); workerContext.init(eventContext); - if (!(ExecutionTaskUtils.isCycleHead(workerContext.getExecutionTask()) - && ioDescriptor.getInputTaskNum() == 0)) { - initFetchRequest(ioDescriptor, pipelineId, pipelineName); - } - if (ioDescriptor.getOutputDescriptor() != null) { - List collectors = initEmitterRequest(ioDescriptor.getOutputDescriptor()); - workerContext.setCollectors(collectors); - } + + this.initFetcher(); + this.initEmitter(); worker.open(context); } @@ -104,10 +82,6 @@ public HighAvailableLevel getHaLevel() { return haLevel; } - public int getCycleId() { - return cycleId; - } - public long getIterationWindowId() { return this.iterationWindowId; } @@ -120,27 +94,10 @@ public ExecutionTask getTask() { return task; } - public void setIoDescriptor(IoDescriptor ioDescriptor) { - this.ioDescriptor = ioDescriptor; - } - public void setDriverId(String driverId) { this.driverId = driverId; } - /** - * Init output emitter. - */ - protected List initEmitterRequest(OutputDescriptor outputIoDescriptor) { - - InitEmitterRequest request = new InitEmitterRequest(outputIoDescriptor); - emitterRunner.add(request); - List collectors = new ArrayList<>(request.getCollectors()); - ((AbstractAlignedWorker) worker).getOutputWriter() - .setCollectors(collectors); - return collectors; - } - @Override public String toString() { return "InitCycleEvent{" @@ -151,4 +108,5 @@ public String toString() { + ", pipelineName=" + pipelineName + '}'; } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitIterationEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitIterationEvent.java index e84ccc868..c705cd5c3 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitIterationEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/InitIterationEvent.java @@ -16,7 +16,6 @@ import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.task.ITaskContext; -import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; /** * An assign event provides some runtime execution information for worker to build the cycle pipeline. @@ -24,20 +23,14 @@ */ public class InitIterationEvent extends AbstractInitCommand { - private IoDescriptor ioDescriptor; - private final long pipelineId; - private final String pipelineName; - public InitIterationEvent(int workerId, int cycleId, long iterationId, long pipelineId, String pipelineName) { - super(workerId, cycleId, iterationId); - this.pipelineId = pipelineId; - this.pipelineName = pipelineName; + super(workerId, cycleId, iterationId, pipelineId, pipelineName); } @Override public void execute(ITaskContext taskContext) { super.execute(taskContext); - initFetchRequest(ioDescriptor, pipelineId, pipelineName); + this.initFetcher(); } @Override @@ -50,10 +43,6 @@ public EventType getEventType() { return EventType.INIT_ITERATION; } - public void setIoDescriptor(IoDescriptor ioDescriptor) { - this.ioDescriptor = ioDescriptor; - } - @Override public String toString() { return "InitIterationEvent{" diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/IterationExecutionComputeWithAggEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/IterationExecutionComputeWithAggEvent.java index 6fad6a4fa..70e1e0fc3 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/IterationExecutionComputeWithAggEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/IterationExecutionComputeWithAggEvent.java @@ -15,7 +15,6 @@ package com.antgroup.geaflow.runtime.core.protocol; import com.antgroup.geaflow.cluster.protocol.EventType; -import com.antgroup.geaflow.cluster.protocol.Message; import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.model.record.RecordArgs; import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; @@ -32,7 +31,7 @@ */ public class IterationExecutionComputeWithAggEvent extends AbstractIterationComputeCommand { - private InputDescriptor inputDescriptor; + private final InputDescriptor inputDescriptor; public IterationExecutionComputeWithAggEvent(int workerId, int cycleId, long windowId, long fetchWindowId, long fetchCount, @@ -43,7 +42,7 @@ public IterationExecutionComputeWithAggEvent(int workerId, int cycleId, long win @Override public void execute(ITaskContext taskContext) { - ((AbstractAlignedWorker) taskContext.getWorker()).getInputReader().add(fetchAggResult()); + ((AbstractAlignedWorker) taskContext.getWorker()).getInputReader().onMessage(fetchAggResult()); super.execute(taskContext); } @@ -57,20 +56,19 @@ public EventType getEventType() { return EventType.ITERATIVE_COMPUTE_WITH_AGGREGATE; } - private Message fetchAggResult() { - List aggRecords = new ArrayList(); + private PipelineMessage fetchAggResult() { + List aggRecords = new ArrayList<>(); for (IInputDesc inputDesc : inputDescriptor.getInputDescMap().values()) { aggRecords.addAll(inputDesc.getInput()); } - Message message = new Message(fetchWindowId, - new PipelineMessage(fetchWindowId, RecordArgs.GraphRecordNames.Aggregate.name(), - new DataMessageIterator(aggRecords))); - return message; + return new PipelineMessage<>(this.fetchWindowId, + RecordArgs.GraphRecordNames.Aggregate.name(), + new DataMessageIterator<>(aggRecords)); } private class DataMessageIterator implements IMessageIterator { - private Iterator iterator; + private final Iterator iterator; private long size = 0; public DataMessageIterator(List data) { diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/LaunchSourceEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/LaunchSourceEvent.java index 89c9d4ec4..cefb3b4ac 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/LaunchSourceEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/LaunchSourceEvent.java @@ -16,7 +16,7 @@ import com.antgroup.geaflow.cluster.protocol.EventType; import com.antgroup.geaflow.cluster.task.ITaskContext; -import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; +import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,14 +33,17 @@ public LaunchSourceEvent(int workerId, int cycleId, long windowId) { @Override public void execute(ITaskContext taskContext) { + final long start = System.currentTimeMillis(); super.execute(taskContext); worker.init(windowId); boolean hasData = (boolean) worker.process(null); + WorkerContext workerContext = (WorkerContext) this.context; if (!hasData) { LOGGER.info("source is finished"); - sendDoneEvent(cycleId, windowId, EventType.LAUNCH_SOURCE, false); + this.sendDoneEvent(workerContext.getDriverId(), EventType.LAUNCH_SOURCE, false, false); } worker.finish(windowId); + workerContext.getEventMetrics().addProcessCostMs(System.currentTimeMillis() - start); } @Override @@ -61,15 +64,6 @@ public EventType getEventType() { return EventType.LAUNCH_SOURCE; } - /** - * Finish compute and tell scheduler finish. - */ - protected void sendDoneEvent(int cycleId, long batchId, EventType eventType, T result) { - AbstractWorkerContext workerContext = (AbstractWorkerContext) context; - DoneEvent doneEvent = new DoneEvent(cycleId, batchId, workerContext.getTaskId(), eventType, result); - workerContext.getPipelineMaster().send(doneEvent); - } - @Override public String toString() { return "LaunchSourceEvent{" diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/PopWorkerEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/PopWorkerEvent.java index 200a67565..3812955a1 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/PopWorkerEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/PopWorkerEvent.java @@ -14,35 +14,26 @@ package com.antgroup.geaflow.runtime.core.protocol; -import com.antgroup.geaflow.cluster.collector.CloseEmitterRequest; import com.antgroup.geaflow.cluster.protocol.EventType; -import com.antgroup.geaflow.cluster.protocol.IExecutableCommand; import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.cluster.worker.IAffinityWorker; -import com.antgroup.geaflow.core.graph.util.ExecutionTaskUtils; import com.antgroup.geaflow.runtime.core.worker.context.AbstractWorkerContext; import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; -import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Pop worker from cache and reuse context. */ -public class PopWorkerEvent extends AbstractInitCommand implements IExecutableCommand { +public class PopWorkerEvent extends AbstractInitCommand { private static final Logger LOGGER = LoggerFactory.getLogger(PopWorkerEvent.class); - private long pipelineId; - private String pipelineName; - private IoDescriptor ioDescriptor; - private int taskId; + private final int taskId; public PopWorkerEvent(int workerId, int cycleId, long windowId, long pipelineId, String pipelineName, int taskId) { - super(workerId, cycleId, windowId); - this.pipelineId = pipelineId; - this.pipelineName = pipelineName; + super(workerId, cycleId, windowId, pipelineId, pipelineName); this.taskId = taskId; } @@ -58,13 +49,9 @@ public void execute(ITaskContext taskContext) { ((IAffinityWorker) worker).pop(popWorkerContext); context = worker.getWorkerContext(); - emitterRunner.add(new CloseEmitterRequest()); - AbstractWorkerContext workerContext = (AbstractWorkerContext) context; - if (!(ExecutionTaskUtils.isCycleHead(workerContext.getExecutionTask()) - && ioDescriptor.getInputTaskNum() == 0)) { - initFetchRequest(ioDescriptor, pipelineId, pipelineName); - } + this.initFetcher(); + this.updateEmitter(); } @Override @@ -77,10 +64,6 @@ public EventType getEventType() { return EventType.POP_WORKER; } - public int getCycleId() { - return cycleId; - } - public long getWindowId() { return windowId; } @@ -89,10 +72,6 @@ public long getPipelineId() { return pipelineId; } - public void setIoDescriptor(IoDescriptor ioDescriptor) { - this.ioDescriptor = ioDescriptor; - } - @Override public String toString() { return "PopWorkerEvent{" diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/StashWorkerEvent.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/StashWorkerEvent.java index 51adbb072..fcc1bf518 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/StashWorkerEvent.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/protocol/StashWorkerEvent.java @@ -14,10 +14,11 @@ package com.antgroup.geaflow.runtime.core.protocol; +import com.antgroup.geaflow.cluster.collector.CloseEmitterRequest; import com.antgroup.geaflow.cluster.protocol.EventType; -import com.antgroup.geaflow.cluster.protocol.IExecutableCommand; import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.cluster.worker.IAffinityWorker; +import com.antgroup.geaflow.core.graph.ExecutionTask; import com.antgroup.geaflow.runtime.core.worker.context.WorkerContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,25 +26,39 @@ /** * Stash worker. */ -public class StashWorkerEvent extends AbstractExecutableCommand implements IExecutableCommand { +public class StashWorkerEvent extends AbstractCleanCommand { private static final Logger LOGGER = LoggerFactory.getLogger(StashWorkerEvent.class); - public StashWorkerEvent(int workerId, int cycleId, long iterationId) { - super(workerId, cycleId, iterationId); + private final int taskId; + + public StashWorkerEvent(int workerId, int cycleId, long windowId, int taskId) { + super(workerId, cycleId, windowId); + this.taskId = taskId; } @Override public void execute(ITaskContext taskContext) { super.execute(taskContext); + WorkerContext workerContext = (WorkerContext) this.context; + ExecutionTask executionTask = workerContext.getExecutionTask(); + workerContext.getEventMetrics().setFinishTime(System.currentTimeMillis()); + LOGGER.info("stash task {} {}/{} of {} {} : {}", + executionTask.getTaskId(), + executionTask.getIndex(), + executionTask.getParallelism(), + executionTask.getVertexId(), + executionTask.getProcessor().toString(), + workerContext.getEventMetrics()); + // Stash worker context. ((IAffinityWorker) worker).stash(); + this.emitterRunner.add(new CloseEmitterRequest(this.taskId, this.windowId)); worker.close(); LOGGER.info("stash worker context, taskId {}", ((WorkerContext) context).getTaskId()); - sendDoneEvent(cycleId, windowId, EventType.CLEAN_CYCLE); - + this.sendDoneEvent(workerContext.getDriverId(), EventType.CLEAN_CYCLE, null, true); } @Override diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/AbstractCycleScheduler.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/AbstractCycleScheduler.java index 2724e6433..f37a69c7d 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/AbstractCycleScheduler.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/AbstractCycleScheduler.java @@ -14,6 +14,7 @@ package com.antgroup.geaflow.runtime.core.scheduler; +import com.antgroup.geaflow.common.utils.LoggerFormatter; import com.antgroup.geaflow.runtime.core.scheduler.context.ICycleSchedulerContext; import com.antgroup.geaflow.runtime.core.scheduler.cycle.IExecutionCycle; import com.antgroup.geaflow.runtime.core.scheduler.result.ExecutionResult; @@ -27,17 +28,16 @@ public abstract class AbstractCycleScheduler implements ICycleScheduler execute() { + String cycleLogTag = LoggerFormatter.getCycleTag(cycle.getPipelineName(), cycle.getCycleId()); try { long finishedIterationId = 0; while (!context.isCycleFinished()) { @@ -45,7 +45,6 @@ public IExecutionResult execute() { // Submit tasks to cycle head until run out of on the fly permits. while (context.hasNextIteration()) { long iterationId = context.getNextIterationId(); - LOGGER.info("{} execute iterationId:{}", cycleLogTag, iterationId); execute(iterationId); } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionCycleTaskAssigner.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionCycleTaskAssigner.java index 90a4b41a3..deaf24be6 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionCycleTaskAssigner.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionCycleTaskAssigner.java @@ -43,6 +43,7 @@ public static Map> assign(ExecutionGraph executionG for (int i = 0; i < vertex.getParallelism(); i++) { ExecutionTask task = new ExecutionTask(taskId.getAndIncrement(), i, vertex.getParallelism(), vertex.getMaxParallelism(), vertex.getNumPartitions(), vertex.getVertexId()); + task.setIterative(vertexGroup.getCycleGroupMeta().isIterative()); tasks.add(task); taskIds.add(task.getTaskId()); } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionGraphCycleScheduler.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionGraphCycleScheduler.java index b74907a37..afe10363c 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionGraphCycleScheduler.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/ExecutionGraphCycleScheduler.java @@ -27,6 +27,7 @@ import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; import com.antgroup.geaflow.common.metric.PipelineMetrics; import com.antgroup.geaflow.common.utils.FutureUtil; +import com.antgroup.geaflow.common.utils.LoggerFormatter; import com.antgroup.geaflow.core.graph.ExecutionEdge; import com.antgroup.geaflow.runtime.core.protocol.CleanEnvEvent; import com.antgroup.geaflow.runtime.core.protocol.CleanStashEnvEvent; @@ -82,19 +83,18 @@ public void init(ICycleSchedulerContext context) { this.dispatcher = new GraphCycleEventDispatcher(); this.resultManager = context.getResultManager(); this.finishedCycles = new HashSet<>(); + this.results = new ArrayList<>(); } @Override public void execute(long iterationId) { - IScheduleStrategy scheduleStrategy = new TopologicalOrderScheduleStrategy(context.getConfig()); scheduleStrategy.init(cycle); pipelineId = ExecutionIdGenerator.getInstance().generateId(); pipelineName = getPipelineName(iterationId); - cycleLogTag = pipelineName; this.sourceCycleNum = getSourceCycleNum(cycle); dispatcher.registerListener(cycle.getCycleId(), new ExecutionGraphCycleEventListener()); - LOGGER.info("{} execute iterationId {}, executionId {}", cycleLogTag, iterationId, pipelineId); + LOGGER.info("{} execute iterationId {}, executionId {}", pipelineName, iterationId, pipelineId); this.pipelineMetrics = new PipelineMetrics(pipelineName); this.pipelineMetrics.setStartTime(System.currentTimeMillis()); StatsCollectorFactory.getInstance().getPipelineStatsCollector().reportPipelineMetrics(pipelineMetrics); @@ -115,9 +115,9 @@ public void execute(long iterationId) { ExecutionNodeCycle cycle = (ExecutionNodeCycle) nextCycle; cycle.setPipelineName(pipelineName); cycle.setPipelineId(pipelineId); - LOGGER.info("{} start schedule cycle {}, total task num {}, head task num {}, tail " + LOGGER.info("{} start schedule {}, total task num {}, head task num {}, tail " + "task num {} type:{}", - cycleLogTag, cycle.getCycleId(), cycle.getTasks().size(), + pipelineName, LoggerFormatter.getCycleName(cycle.getCycleId(), iterationId), cycle.getTasks().size(), cycle.getCycleHeads().size(), cycle.getCycleTails().size(), cycle.getType()); // Schedule cycle. @@ -133,18 +133,19 @@ public void execute(long iterationId) { final long start = System.currentTimeMillis(); IExecutionResult result = cycleScheduler.execute(); if (!result.isSuccess()) { - throw new GeaflowRuntimeException(String.format("schedule execute %s failed ", - getPipelineName(iterationId))) ; + throw new GeaflowRuntimeException(String.format("%s schedule execute %s failed ", + pipelineName, cycle.getCycleId())) ; } scheduleStrategy.finish(cycle); - LOGGER.info("{} iterationId {} finished, cost {}ms", - cycleLogTag, iterationId, System.currentTimeMillis() - start); + LOGGER.info("{} schedule {} finished, cost {}ms", + pipelineName, LoggerFormatter.getCycleName(cycle.getCycleId(), iterationId), + System.currentTimeMillis() - start); if (cycleScheduler instanceof IEventListener) { dispatcher.removeListener(cycle.getCycleId()); } } catch (Throwable e) { throw new GeaflowRuntimeException(String.format("%s schedule iterationId %s failed ", - cycleLogTag, iterationId), e) ; + pipelineName, iterationId), e) ; } finally { context.release(cycle); cycleScheduler.close(); @@ -156,24 +157,30 @@ public void execute(long iterationId) { @Override public void finish(long iterationId) { - LOGGER.info("{} finish", cycleLogTag); + String cycleLogTag = LoggerFormatter.getCycleTag(pipelineName, cycle.getCycleId(), iterationId); // Clean shuffle data for all used workers. - context.getSchedulerWorkerManager().clean(usedWorkers -> cleanEnv(usedWorkers, iterationId, false)); + context.getSchedulerWorkerManager().clean(usedWorkers -> + cleanEnv(usedWorkers, cycleLogTag, iterationId, false)); // Clear last iteration shard meta. resultManager.clear(); this.pipelineMetrics.setDuration(System.currentTimeMillis() - pipelineMetrics.getStartTime()); StatsCollectorFactory.getInstance().getPipelineStatsCollector().reportPipelineMetrics(pipelineMetrics); - LOGGER.info("{} iterationId {} clean and finish done, {}", pipelineName, iterationId, pipelineMetrics); + LOGGER.info("{} finished {}", cycleLogTag, pipelineMetrics); } @Override protected R finish() { // Clean shuffle data for all used workers. - context.getSchedulerWorkerManager().clean(usedWorkers -> cleanEnv(usedWorkers, cycle.getIterationCount(), true)); + String cycleLogTag = LoggerFormatter.getCycleTag(pipelineName, cycle.getCycleId()); + context.getSchedulerWorkerManager().clean(usedWorkers -> cleanEnv(usedWorkers, cycleLogTag, + cycle.getIterationCount(), true)); return (R) results; } - private void cleanEnv(Set usedWorkers, long iterationId, boolean needCleanWorkerContext) { + private void cleanEnv(Set usedWorkers, + String cycleLogTag, + long iterationId, + boolean needCleanWorkerContext) { cleanEnvWaitingResponse = new CountDownLatch(usedWorkers.size()); LOGGER.info("{} start wait {} clean env response for iteration {}, need clean worker context {}", cycleLogTag, usedWorkers.size(), iterationId, needCleanWorkerContext); @@ -198,7 +205,6 @@ private void cleanEnv(Set usedWorkers, long iterationId, boolean nee } catch (InterruptedException e) { throw new GeaflowRuntimeException("exception when wait all clean event finish", e); } finally { - LOGGER.info("{} clean shuffle master", cycleLogTag); IShuffleMaster shuffleMaster = ShuffleManager.getInstance().getShuffleMaster(); if (shuffleMaster != null) { shuffleMaster.clean(new PipelineInfo(pipelineId, pipelineName)); @@ -208,17 +214,13 @@ private void cleanEnv(Set usedWorkers, long iterationId, boolean nee } private String getPipelineName(long iterationId) { - if (cycle.getIterationCount() > 1) { - return String.format("%s-%s", cycle.getPipelineName(), iterationId); - } else { - return cycle.getPipelineName(); - } + return String.format("%s-%s", cycle.getPipelineName(), iterationId); } @Override public void close() { context.close(); - LOGGER.info("{} scheduler closed", cycleLogTag); + LOGGER.info("{} closed", cycle.getPipelineName()); } @Override @@ -266,12 +268,10 @@ public void dispatch(IEvent event) throws GeaflowDispatchException { } public void registerListener(int cycleId, IEventListener eventListener) { - LOGGER.info("{} register scheduler {}", cycle.getPipelineName(), cycleId); cycleIdToScheduler.put(cycleId, eventListener); } public void removeListener(int cycleId) { - LOGGER.info("{} remove scheduler {}", cycle.getPipelineName(), cycleId); cycleIdToScheduler.remove(cycleId); } @@ -288,12 +288,12 @@ public void handleEvent(IEvent event) { case LAUNCH_SOURCE: if (!finishedCycles.contains(doneEvent.getResult())) { finishedCycles.add((Integer) doneEvent.getResult()); - LOGGER.info("cycle {} source finished at iteration {}", - doneEvent.getResult(), doneEvent.getWindowId()); + LOGGER.info("{} cycle {} source finished at iteration {}", + cycle.getPipelineName(), doneEvent.getResult(), doneEvent.getWindowId()); if (finishedCycles.size() == sourceCycleNum) { ((AbstractCycleSchedulerContext) context).setTerminateIterationId( doneEvent.getWindowId()); - LOGGER.info("all source cycle finished"); + LOGGER.info("{} all source cycle finished", cycle.getPipelineName()); } } break; @@ -301,17 +301,18 @@ public void handleEvent(IEvent event) { if (event.getEventType() == EventType.DONE) { cleanEnvWaitingResponse.countDown(); } - LOGGER.info("received clean env event {}", cleanEnvWaitingResponse.getCount()); + LOGGER.info("{} received clean env event {}", + getPipelineName(doneEvent.getWindowId()), cleanEnvWaitingResponse.getCount()); break; default: throw new GeaflowRuntimeException(String.format("%s not support handle done event %s", - cycleLogTag, event)); + getPipelineName(doneEvent.getWindowId()), event)); } break; default: throw new GeaflowRuntimeException(String.format("%s not support handle event %s", - cycleLogTag, event)); + cycle.getPipelineName(), event)); } } } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineCycleScheduler.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineCycleScheduler.java index 49853b31c..04ff4aa32 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineCycleScheduler.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/PipelineCycleScheduler.java @@ -24,10 +24,12 @@ import com.antgroup.geaflow.cluster.rpc.RpcClient; import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; import com.antgroup.geaflow.common.metric.CycleMetrics; +import com.antgroup.geaflow.common.metric.EventMetrics; import com.antgroup.geaflow.common.shuffle.DataExchangeMode; import com.antgroup.geaflow.common.utils.FutureUtil; import com.antgroup.geaflow.common.utils.LoggerFormatter; import com.antgroup.geaflow.core.graph.ExecutionTask; +import com.antgroup.geaflow.core.graph.ExecutionVertex; import com.antgroup.geaflow.io.CollectType; import com.antgroup.geaflow.runtime.core.protocol.ComposeEvent; import com.antgroup.geaflow.runtime.core.protocol.DoneEvent; @@ -44,6 +46,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Future; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -57,18 +60,19 @@ public class PipelineCycleScheduler private static final Logger LOGGER = LoggerFactory.getLogger(PipelineCycleScheduler.class); - private static final String FINISH_TAG = "FINISH"; - private ExecutionNodeCycle nodeCycle; private CycleResultManager resultManager; private boolean enableSchedulerDebug = false; private CycleResponseEventPool responseEventPool; private HashMap> iterationIdToFinishedTasks; - private HashMap iterationIdToMetrics; + private Map vertexIdToMetrics; private Map cycleTasks; private Map taskIdToFinishedSourceIds; + private String pipelineName; private long pipelineId; + private int cycleId; + private long scheduleStartTime; private boolean isIteration; private DataExchangeMode inputExchangeMode; @@ -76,6 +80,7 @@ public class PipelineCycleScheduler private SchedulerEventBuilder eventBuilder; private SchedulerGraphAggregateProcessor aggregator; + private String cycleLogTag; public PipelineCycleScheduler() { } @@ -85,14 +90,16 @@ public void init(ICycleSchedulerContext context) { super.init(context); this.responseEventPool = new CycleResponseEventPool<>(); this.iterationIdToFinishedTasks = new HashMap<>(); - this.iterationIdToMetrics = new HashMap<>(); this.taskIdToFinishedSourceIds = new HashMap<>(); this.nodeCycle = (ExecutionNodeCycle) context.getCycle(); - this.cycleLogTag = LoggerFormatter.getCycleTag(nodeCycle.getPipelineName(), nodeCycle.getCycleId()); this.cycleTasks = nodeCycle.getTasks().stream().collect(Collectors.toMap(t -> t.getTaskId(), t -> t)); this.isIteration = nodeCycle.getVertexGroup().getCycleGroupMeta().isIterative(); + this.pipelineName = nodeCycle.getPipelineName(); this.pipelineId = nodeCycle.getPipelineId(); + this.cycleId = nodeCycle.getCycleId(); this.resultManager = context.getResultManager(); + this.cycleLogTag = LoggerFormatter.getCycleTag(this.pipelineName, this.cycleId); + this.initMetrics(); inputExchangeMode = DataExchangeMode.BATCH; if (nodeCycle.isPipelineDataLoop()) { @@ -112,6 +119,17 @@ public void init(ICycleSchedulerContext context) { } } + private void initMetrics() { + this.scheduleStartTime = System.currentTimeMillis(); + this.vertexIdToMetrics = new HashMap<>(); + Map vertexMap = this.nodeCycle.getVertexGroup().getVertexMap(); + for (Map.Entry entry : vertexMap.entrySet()) { + Integer vertexId = entry.getKey(); + ExecutionVertex vertex = entry.getValue(); + this.vertexIdToMetrics.put(vertexId, new EventMetrics[vertex.getParallelism()]); + } + } + @Override public void close() { super.close(); @@ -124,7 +142,7 @@ public void close() { if (((AbstractCycleSchedulerContext) context).getParentContext() == null) { context.close(); } - LOGGER.info("cycle {} closed", cycleLogTag); + LOGGER.info("{} closed", cycleLogTag); } @Override @@ -145,7 +163,7 @@ public void handleEvent(IEvent event) { ICycleSchedulerContext parentContext = ((AbstractCycleSchedulerContext) context).getParentContext(); if (parentContext != null) { - DoneEvent cycleDoneEvent = new DoneEvent(parentContext.getCycle().getCycleId(), + DoneEvent cycleDoneEvent = new DoneEvent<>(parentContext.getCycle().getCycleId(), doneEvent.getWindowId(), doneEvent.getTaskId(), EventType.LAUNCH_SOURCE, doneEvent.getCycleId()); RpcClient.getInstance().processPipeline(cycle.getDriverId(), cycleDoneEvent); @@ -170,11 +188,6 @@ protected void execute(long iterationId) { String iterationLogTag = getCycleIterationTag(iterationId); LOGGER.info("{} execute", iterationLogTag); iterationIdToFinishedTasks.put(iterationId, new ArrayList<>()); - CycleMetrics cycleMetrics = new CycleMetrics(LoggerFormatter.getCycleName(cycle.getCycleId(), iterationId), - cycle.getPipelineName(), - nodeCycle.getName()); - cycleMetrics.setStartTime(System.currentTimeMillis()); - iterationIdToMetrics.put(iterationId, cycleMetrics); List schedulerStates = context.getSchedulerState(iterationId); Map events; if (schedulerStates == null) { @@ -188,10 +201,16 @@ protected void execute(long iterationId) { for (Map.Entry entry : events.entrySet()) { ExecutionTask task = cycleTasks.get(entry.getKey()); + String taskTag = this.isIteration + ? LoggerFormatter.getTaskTag(this.pipelineName, this.cycleId, iterationId, + task.getTaskId(), task.getVertexId(), task.getIndex(), task.getParallelism()) + : LoggerFormatter.getTaskTag(this.pipelineName, this.cycleId, task.getTaskId(), + task.getVertexId(), task.getIndex(), task.getParallelism()); LOGGER.info("{} submit event {} on worker {} host {} process {}", - getCycleTaskLogTag(iterationId, task.getIndex()), + taskTag, entry.getValue(), - task.getWorkerInfo().getWorkerIndex(), task.getWorkerInfo().getHost(), + task.getWorkerInfo().getWorkerIndex(), + task.getWorkerInfo().getHost(), task.getWorkerInfo().getProcessId()); Future future = RpcClient.getInstance() @@ -232,14 +251,6 @@ protected void finish(long iterationId) { iterationIdToFinishedTasks.keySet())); } iterationIdToFinishedTasks.get(currentTaskIterationId).add(response); - LOGGER.info("{} receive event {}, iterationId {}, current response iterationId {}, " - + "received response {}/{}, duration {}ms", - cycleLogTag, event, - iterationId, - currentTaskIterationId, - iterationIdToFinishedTasks.get(currentTaskIterationId).size(), - expectedResponseSize, - System.currentTimeMillis() - iterationIdToMetrics.get(iterationId).getStartTime()); } // Get current iteration result. @@ -248,16 +259,16 @@ protected void finish(long iterationId) { registerResults((DoneEvent) e); } - CycleMetrics cycleMetrics = iterationIdToMetrics.remove(iterationId); - collectEventMetrics(cycleMetrics, responses); - StatsCollectorFactory.getInstance().getPipelineStatsCollector().reportCycleMetrics(cycleMetrics); - LOGGER.info("{} finished iterationId {}, {}", iterationLogTag, iterationId, cycleMetrics); + if (this.isIteration) { + this.collectEventMetrics(responses, iterationId); + } + LOGGER.info("{} finished iterationId {}", iterationLogTag, iterationId); } protected List finish() { - String finishLogTag = getCycleFinishTag(); - CycleMetrics cycleMetrics = new CycleMetrics(getCycleFinishName(), cycle.getPipelineName(), nodeCycle.getName()); - cycleMetrics.setStartTime(System.currentTimeMillis()); + long finishIterationId = this.isIteration + ? this.context.getFinishIterationId() + 1 : this.context.getFinishIterationId(); + String finishLogTag = this.getCycleIterationTag(finishIterationId); Map events = eventBuilder.build(ICycleSchedulerContext.SchedulerState.FINISH, context.getCurrentIterationId()); @@ -274,15 +285,17 @@ protected List finish() { // Need receive all tail responses. int responseCount = 0; - List responses = new ArrayList<>(); + List resultResponses = new ArrayList<>(this.cycleTasks.size()); + List metricResponses = new ArrayList<>(this.cycleTasks.size()); while (true) { IEvent e = responseEventPool.waitEvent(); DoneEvent> event = (DoneEvent) e; switch (event.getSourceEvent()) { case EXECUTE_COMPUTE: - responses.add(event); + resultResponses.add(event); break; default: + metricResponses.add(event); responseCount++; break; } @@ -291,14 +304,14 @@ protected List finish() { break; } } - if (!responses.isEmpty()) { - for (IEvent e : responses) { + if (!resultResponses.isEmpty()) { + for (IEvent e : resultResponses) { registerResults((DoneEvent) e); } - - collectEventMetrics(cycleMetrics, responses); - StatsCollectorFactory.getInstance().getPipelineStatsCollector().reportCycleMetrics(cycleMetrics); - LOGGER.info("{} finished {}", finishLogTag, cycleMetrics); + } + if (!metricResponses.isEmpty()) { + this.collectEventMetrics(metricResponses, finishIterationId); + LOGGER.info("{} finished", finishLogTag); } return context.getResultManager().getDataResponse(); @@ -320,43 +333,83 @@ private void registerResults(DoneEvent> event) { } } - private void collectEventMetrics(CycleMetrics cycleMetrics, List responses) { - - final long duration = System.currentTimeMillis() - cycleMetrics.getStartTime(); - final int totalTaskNum = responses.size(); - long totalExecuteTime = 0; - long totalGcTime = 0; - long slowestTaskExecuteTime = 0; - int slowestTaskId = 0; - long totalInputRecords = 0; - long totalInputBytes = 0; - long totalOutputRecords = 0; - long totalOutputBytes = 0; - - for (IEvent response : responses) { - DoneEvent event = (DoneEvent) response; - if (event.getEventMetrics() != null) { - totalExecuteTime += event.getEventMetrics().getExecuteTime(); - totalExecuteTime += event.getEventMetrics().getGcTime(); - totalGcTime += event.getEventMetrics().getGcTime(); - if (event.getEventMetrics().getExecuteTime() > slowestTaskExecuteTime) { - slowestTaskExecuteTime = event.getEventMetrics().getExecuteTime(); - slowestTaskId = event.getTaskId(); + private void collectEventMetrics(List responses, long windowId) { + Map> vertexId2metrics = responses.stream() + .map(e -> ((DoneEvent) e).getEventMetrics()) + .filter(Objects::nonNull) + .collect(Collectors.groupingBy(EventMetrics::getVertexId)); + + long duration = System.currentTimeMillis() - this.scheduleStartTime; + for (Map.Entry> entry : vertexId2metrics.entrySet()) { + Integer vertexId = entry.getKey(); + List metrics = entry.getValue(); + EventMetrics[] previousMetrics = this.vertexIdToMetrics.get(vertexId); + + int taskNum = previousMetrics.length; + int slowestTask = 0; + long executeCostMs = 0; + long totalExecuteTime = 0; + long totalGcTime = 0; + long slowestTaskExecuteTime = 0; + long totalInputRecords = 0; + long totalInputBytes = 0; + long totalOutputRecords = 0; + long totalOutputBytes = 0; + + for (EventMetrics eventMetrics : metrics) { + int index = eventMetrics.getIndex(); + EventMetrics previous = previousMetrics[index]; + if (previous == null) { + executeCostMs = eventMetrics.getProcessCostMs(); + totalExecuteTime += executeCostMs; + totalGcTime += eventMetrics.getGcCostMs(); + totalInputRecords += eventMetrics.getShuffleReadRecords(); + totalInputBytes += eventMetrics.getShuffleReadBytes(); + totalOutputRecords += eventMetrics.getShuffleWriteRecords(); + totalOutputBytes += eventMetrics.getShuffleWriteBytes(); + } else { + executeCostMs = eventMetrics.getProcessCostMs() - previous.getProcessCostMs(); + totalExecuteTime += executeCostMs; + totalGcTime += eventMetrics.getGcCostMs() - previous.getGcCostMs(); + totalInputRecords += eventMetrics.getShuffleReadRecords() - previous.getShuffleReadRecords(); + totalInputBytes += eventMetrics.getShuffleReadBytes() - previous.getShuffleReadBytes(); + totalOutputRecords += eventMetrics.getShuffleWriteRecords() - previous.getShuffleWriteRecords(); + totalOutputBytes += eventMetrics.getShuffleWriteBytes() - previous.getShuffleWriteBytes(); + } + if (executeCostMs > slowestTaskExecuteTime) { + slowestTaskExecuteTime = executeCostMs; + slowestTask = index; + } + if (this.isIteration) { + previousMetrics[index] = eventMetrics; } - totalInputRecords += event.getEventMetrics().getInputRecords(); - totalInputBytes += event.getEventMetrics().getInputBytes(); - totalOutputRecords += event.getEventMetrics().getOutputRecords(); - totalOutputBytes += event.getEventMetrics().getOutputBytes(); } + + String metricName = this.isIteration + ? LoggerFormatter.getCycleMetricName(this.cycleId, windowId, vertexId) + : LoggerFormatter.getCycleMetricName(this.cycleId, vertexId); + String opName = this.nodeCycle.getVertexGroup().getVertexMap().get(vertexId).getName(); + CycleMetrics cycleMetrics = CycleMetrics.build( + metricName, + this.pipelineName, + opName, + taskNum, + slowestTask, + this.scheduleStartTime, + duration, + totalExecuteTime, + totalGcTime, + slowestTaskExecuteTime, + totalInputRecords, + totalInputBytes, + totalOutputRecords, + totalOutputBytes + ); + LOGGER.info("collect metric {} {}", metricName, cycleMetrics); + StatsCollectorFactory.getInstance().getPipelineStatsCollector().reportCycleMetrics(cycleMetrics); } - cycleMetrics.setTotalTasks(responses.size()); - cycleMetrics.setDuration(duration); - cycleMetrics.setAvgGcTime(totalGcTime / totalTaskNum); - cycleMetrics.setAvgExecuteTime(totalExecuteTime / totalTaskNum); - cycleMetrics.setSlowestTaskExecuteTime(slowestTaskExecuteTime); - cycleMetrics.setSlowestTask(slowestTaskId); - cycleMetrics.setOutputRecords(totalOutputRecords); - cycleMetrics.setOutputKb(totalOutputBytes / 1024); + + this.scheduleStartTime = System.currentTimeMillis(); } private Map mergeEvents(List> list) { @@ -380,38 +433,9 @@ private Map mergeEvents(List> list) { return result; } - private String getCycleName(long iterationId) { - return LoggerFormatter.getCycleTag(nodeCycle.getPipelineName(), nodeCycle.getCycleId(), iterationId); - } - - private String getCycleTaskName(int vertexId, int taskIndex) { - return LoggerFormatter.getTaskLog(nodeCycle.getPipelineName(), nodeCycle.getCycleId(), vertexId, taskIndex); - } - - private String getCycleTaskLogTag(long iterationId, int taskIndex) { - if (isIteration) { - return LoggerFormatter.getTaskLog(nodeCycle.getPipelineName(), - nodeCycle.getCycleId(), iterationId, taskIndex); - } else { - return LoggerFormatter.getTaskLog(nodeCycle.getPipelineName(), - nodeCycle.getCycleId(), taskIndex); - } - } - private String getCycleIterationTag(long iterationId) { - if (!isIteration) { - return cycleLogTag; - } else { - return LoggerFormatter.getCycleTag(cycle.getPipelineName(), cycle.getCycleId(), iterationId); - - } - } - - private String getCycleFinishTag() { - return LoggerFormatter.getCycleTag(cycle.getPipelineName(), cycle.getCycleId(), FINISH_TAG); - } - - private String getCycleFinishName() { - return LoggerFormatter.getCycleName(cycle.getCycleId(), FINISH_TAG); + return this.isIteration + ? LoggerFormatter.getCycleTag(this.pipelineName, this.cycleId, iterationId) + : LoggerFormatter.getCycleTag(this.pipelineName, this.cycleId); } } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/SchedulerEventBuilder.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/SchedulerEventBuilder.java index 35124bd3f..0e367b2d5 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/SchedulerEventBuilder.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/SchedulerEventBuilder.java @@ -136,9 +136,9 @@ private IEvent buildInitOrPopEvent(ExecutionTask task, IoDescriptor ioDescriptor } } - private IEvent buildCleanOrStashEvent(int workerId) { + private IEvent buildCleanOrStashEvent(int workerId, int taskId) { if (enableAffinity) { - return new StashWorkerEvent(workerId, cycle.getCycleId(), cycle.getIterationCount()); + return new StashWorkerEvent(workerId, cycle.getCycleId(), cycle.getIterationCount(), taskId); } else { return new CleanCycleEvent(workerId, cycle.getCycleId(), cycle.getIterationCount()); } @@ -233,7 +233,7 @@ private Map finishPipeline() { int workerId = task.getWorkerInfo().getWorkerIndex(); IEvent cleanEvent; if (enableAffinity) { - cleanEvent = new StashWorkerEvent(workerId, cycle.getCycleId(), cycle.getIterationCount()); + cleanEvent = new StashWorkerEvent(workerId, cycle.getCycleId(), cycle.getIterationCount(), task.getTaskId()); } else { cleanEvent = new CleanCycleEvent(workerId, cycle.getCycleId(), cycle.getIterationCount()); } @@ -257,7 +257,7 @@ private Map finishIteration() { // finish iteration FinishIterationEvent iterationFinishEvent = new FinishIterationEvent(workerId, context.getInitialIterationId(), cycle.getCycleId(), task.getTaskId()); - IEvent cleanEvent = buildCleanOrStashEvent(workerId); + IEvent cleanEvent = buildCleanOrStashEvent(workerId, task.getTaskId()); ComposeEvent composeEvent = new ComposeEvent(task.getWorkerInfo().getWorkerIndex(), diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/context/AbstractCycleSchedulerContext.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/context/AbstractCycleSchedulerContext.java index c9f28fbb2..fd1ed69da 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/context/AbstractCycleSchedulerContext.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/scheduler/context/AbstractCycleSchedulerContext.java @@ -94,7 +94,7 @@ public void init(long startIterationId) { this.workerManager.init(cycle); - LOGGER.info("init cycle {} context onTheFlyThreshold {}, currentIterationId {}, " + LOGGER.info("{} init cycle context onTheFlyThreshold {}, currentIterationId {}, " + "iterationCount {}, finishIterationId {}, initialIterationId {}", cycle.getPipelineName(), cycle.getFlyingCount(), this.currentIterationId, cycle.getIterationCount(), this.finishIterationId, this.initialIterationId); diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/AbstractAlignedWorker.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/AbstractAlignedWorker.java index b77340e3f..493471b64 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/AbstractAlignedWorker.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/AbstractAlignedWorker.java @@ -17,17 +17,16 @@ import com.antgroup.geaflow.api.trait.CancellableTrait; import com.antgroup.geaflow.cluster.collector.AbstractPipelineOutputCollector; import com.antgroup.geaflow.cluster.protocol.EventType; -import com.antgroup.geaflow.cluster.protocol.Message; +import com.antgroup.geaflow.cluster.protocol.InputMessage; import com.antgroup.geaflow.cluster.response.IResult; -import com.antgroup.geaflow.cluster.response.ShardResult; +import com.antgroup.geaflow.cluster.rpc.RpcClient; import com.antgroup.geaflow.cluster.worker.IWorker; import com.antgroup.geaflow.cluster.worker.IWorkerContext; import com.antgroup.geaflow.collector.ICollector; import com.antgroup.geaflow.collector.IResultCollector; import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; -import com.antgroup.geaflow.common.utils.GcUtil; +import com.antgroup.geaflow.common.metric.EventMetrics; import com.antgroup.geaflow.core.graph.util.ExecutionTaskUtils; -import com.antgroup.geaflow.io.CollectType; import com.antgroup.geaflow.model.record.BatchRecord; import com.antgroup.geaflow.model.record.RecordArgs; import com.antgroup.geaflow.runtime.core.protocol.DoneEvent; @@ -42,22 +41,22 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractAlignedWorker implements IWorker { +public abstract class AbstractAlignedWorker implements IWorker, O> { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractAlignedWorker.class); + private static final int DEFAULT_TIMEOUT_MS = 100; + protected AbstractWorkerContext context; - protected InputReader inputReader; - protected OutputWriter outputWriter; + protected InputReader inputReader; protected Map windowCount; - protected Map> windowMessageCache; + protected Map>> windowMessageCache; protected volatile boolean running; public AbstractAlignedWorker() { - this.inputReader = new InputReader(); - this.outputWriter = new OutputWriter(); + this.inputReader = new InputReader<>(); this.windowCount = new HashMap<>(); - this.windowMessageCache = new HashMap(); + this.windowMessageCache = new HashMap<>(); this.running = false; } @@ -72,29 +71,30 @@ public IWorkerContext getWorkerContext() { return context; } - public InputReader getInputReader() { + public InputReader getInputReader() { return inputReader; } - public OutputWriter getOutputWriter() { - return outputWriter; - } - /** * Fetch message from input queue and trigger aligned compute all the time, * and finish until total batch count has fetched. */ public void alignedProcess(long totalWindowCount) { long processedWindowCount = 0; + long fetchCost = 0; while (processedWindowCount < totalWindowCount && running) { try { - Message input = inputReader.poll(100, TimeUnit.MILLISECONDS); + long fetchStart = System.currentTimeMillis(); + InputMessage input = this.inputReader.poll(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS); + fetchCost += System.currentTimeMillis() - fetchStart; if (input != null) { long windowId = input.getWindowId(); if (input.getMessage() != null) { - PipelineMessage message = input.getMessage(); + PipelineMessage message = input.getMessage(); processMessage(windowId, message); } else { + this.context.getEventMetrics().addShuffleReadCostMs(fetchCost); + fetchCost = 0; long totalCount = input.getWindowCount(); processBarrier(windowId, totalCount); processedWindowCount++; @@ -114,40 +114,31 @@ public void alignedProcess(long totalWindowCount) { * Tell scheduler finish and send back response to scheduler. */ protected void finishWindow(long windowId) { - Map results = new HashMap<>(); - List collectors = outputWriter.getCollectors(); + Map> results = new HashMap<>(); + List> collectors = this.context.getCollectors(); if (ExecutionTaskUtils.isCycleTail(context.getExecutionTask())) { - long outputRecordNum = 0; - long outputBytes = 0; for (int i = 0; i < collectors.size(); i++) { - IResultCollector responseCollector = (IResultCollector) collectors.get(i); - IResult result = (IResult) responseCollector.collectResult(); + IResultCollector responseCollector = (IResultCollector) collectors.get(i); + IResult result = (IResult) responseCollector.collectResult(); if (result != null) { results.put(result.getId(), result); - if (result.getType() == CollectType.FORWARD || result.getType() == CollectType.LOOP) { - outputRecordNum += ((ShardResult) result).getRecordNum(); - outputBytes += ((ShardResult) result).getRecordBytes(); - } } } - context.getEventMetrics().setOutputRecords(outputRecordNum); - context.getEventMetrics().setOutputBytes(outputBytes); // Tell scheduler finish or response. - DoneEvent done = new DoneEvent(context.getCycleId(), windowId, context.getTaskId(), - EventType.EXECUTE_COMPUTE, results, context.getEventMetrics()); - context.getPipelineMaster().send(done); + EventMetrics eventMetrics = this.context.isIterativeTask() ? this.context.getEventMetrics() : null; + DoneEvent done = new DoneEvent<>(context.getCycleId(), windowId, context.getTaskId(), + EventType.EXECUTE_COMPUTE, results, eventMetrics); + RpcClient.getInstance().processPipeline(context.getDriverId(), done); } } protected void updateWindowId(long windowId) { - context.getEventMetrics().setStartTs(System.currentTimeMillis()); - context.getEventMetrics().setStartGcTs(GcUtil.computeCurrentTotalGcTime()); context.setCurrentWindowId(windowId); - for (ICollector collector : outputWriter.getCollectors()) { + for (ICollector collector : this.context.getCollectors()) { if (collector instanceof AbstractPipelineOutputCollector) { - ((AbstractPipelineOutputCollector) collector).setWindowId(windowId); + ((AbstractPipelineOutputCollector) collector).setWindowId(windowId); } } } @@ -155,12 +146,12 @@ protected void updateWindowId(long windowId) { /** * Trigger worker to process message. */ - private void processMessage(long windowId, PipelineMessage message) { + private void processMessage(long windowId, PipelineMessage message) { if (windowId > context.getCurrentWindowId()) { if (windowMessageCache.containsKey(windowId)) { windowMessageCache.get(windowId).add(message); } else { - List cache = new ArrayList(); + List> cache = new ArrayList<>(); cache.add(message); windowMessageCache.put(windowId, cache); } @@ -172,7 +163,7 @@ private void processMessage(long windowId, PipelineMessage message) { /** * Trigger worker to process buffered message. */ - private void processBarrier( long windowId, long totalCount) { + private void processBarrier(long windowId, long totalCount) { processBufferedMessages(windowId); long processCount = 0; @@ -186,7 +177,7 @@ private void processBarrier( long windowId, long totalCount) { throw new GeaflowRuntimeException(String.format("taskId %s mismatch, TotalCount:%s != ProcessCount:%s", context.getTaskId(), totalCount, processCount)); } - context.getEventMetrics().setInputRecords(totalCount); + context.getEventMetrics().addShuffleReadRecords(totalCount); long currentWindowId = context.getCurrentWindowId(); finish(currentWindowId); @@ -196,15 +187,15 @@ private void processBarrier( long windowId, long totalCount) { /** * Process message event and trigger worker to process. */ - private void processMessageEvent(long windowId, PipelineMessage message) { - IMessageIterator messageIterator = message.getMessageIterator(); + private void processMessageEvent(long windowId, PipelineMessage message) { + IMessageIterator messageIterator = message.getMessageIterator(); process(new BatchRecord<>(message.getRecordArgs(), messageIterator)); long count = messageIterator.getSize(); messageIterator.close(); // Aggregate message not take into account when check message count. - if (message.getRecordArgs().getName() != RecordArgs.GraphRecordNames.Aggregate.name()) { + if (!message.getRecordArgs().getName().equals(RecordArgs.GraphRecordNames.Aggregate.name())) { if (!windowCount.containsKey(windowId)) { windowCount.put(windowId, count); } else { @@ -219,8 +210,8 @@ private void processMessageEvent(long windowId, PipelineMessage message) { */ private void processBufferedMessages(long windowId) { if (windowMessageCache.containsKey(windowId)) { - List cacheMessages = windowMessageCache.get(windowId); - for (PipelineMessage message : cacheMessages) { + List> cacheMessages = windowMessageCache.get(windowId); + for (PipelineMessage message : cacheMessages) { processMessageEvent(windowId, message); } windowMessageCache.remove(windowId); diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/InputReader.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/InputReader.java index b792b137b..8b7fd7b35 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/InputReader.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/InputReader.java @@ -14,38 +14,26 @@ package com.antgroup.geaflow.runtime.core.worker; -import com.antgroup.geaflow.cluster.protocol.Message; -import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; +import com.antgroup.geaflow.cluster.fetcher.IInputMessageBuffer; +import com.antgroup.geaflow.cluster.protocol.InputMessage; +import com.antgroup.geaflow.io.AbstractMessageBuffer; +import com.antgroup.geaflow.shuffle.config.ShuffleConfig; +import com.antgroup.geaflow.shuffle.message.PipelineMessage; -import java.io.Serializable; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; - -public class InputReader implements Serializable { - - private LinkedBlockingQueue inputQueue; +public class InputReader extends AbstractMessageBuffer> implements IInputMessageBuffer { public InputReader() { - this.inputQueue = new LinkedBlockingQueue<>(); + super(ShuffleConfig.getInstance().getFetchQueueSize()); } - /** - * Add message into input queue. - */ - public void add(Message message) { - inputQueue.add(message); + @Override + public void onMessage(PipelineMessage message) { + this.offer(new InputMessage<>(message)); } - /** - * Returns message from input queue. - */ - public Message poll(long timeout, TimeUnit unit) { - Message message; - try { - message = inputQueue.poll(timeout, unit); - } catch (Throwable t) { - throw new GeaflowRuntimeException(t); - } - return message; + @Override + public void onBarrier(long windowId, long windowCount) { + this.offer(new InputMessage<>(windowId, windowCount)); } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/OutputWriter.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/OutputWriter.java index fced4b853..1baee6546 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/OutputWriter.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/OutputWriter.java @@ -14,20 +14,97 @@ package com.antgroup.geaflow.runtime.core.worker; -import com.antgroup.geaflow.collector.ICollector; +import com.antgroup.geaflow.cluster.collector.IOutputMessageBuffer; +import com.antgroup.geaflow.cluster.protocol.OutputMessage; +import com.antgroup.geaflow.common.exception.GeaflowRuntimeException; +import com.antgroup.geaflow.io.AbstractMessageBuffer; +import com.antgroup.geaflow.shuffle.config.ShuffleConfig; +import com.antgroup.geaflow.shuffle.message.Shard; -import java.io.Serializable; -import java.util.List; +import java.util.ArrayList; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; -public class OutputWriter implements Serializable { +public class OutputWriter extends AbstractMessageBuffer> + implements IOutputMessageBuffer { - private List collectors; + private CompletableFuture resultFuture = new CompletableFuture<>(); - public List getCollectors() { - return collectors; + private final int edgeId; + private final int bufferSize; + private final ArrayList[] buffers; + private final AtomicReference err; + + public OutputWriter(int edgeId, int bucketNum) { + super(ShuffleConfig.getInstance().getEmitQueueSize()); + this.edgeId = edgeId; + this.bufferSize = ShuffleConfig.getInstance().getEmitBufferSize(); + this.buffers = new ArrayList[bucketNum]; + this.err = new AtomicReference<>(); + for (int i = 0; i < bucketNum; i++) { + this.buffers[i] = new ArrayList<>(this.bufferSize); + } + } + + @Override + public void emit(long windowId, T data, boolean isRetract, int[] targetChannels) { + for (int channel : targetChannels) { + ArrayList buffer = this.buffers[channel]; + buffer.add(data); + if (buffer.size() == this.bufferSize) { + this.checkErr(); + long start = System.currentTimeMillis(); + this.offer(OutputMessage.data(windowId, channel, buffer)); + this.eventMetrics.addShuffleWriteCostMs(System.currentTimeMillis() - start); + this.buffers[channel] = new ArrayList<>(this.bufferSize); + } + } } - public void setCollectors(List collectors) { - this.collectors = collectors; + @Override + public void setResult(long windowId, Shard result) { + this.resultFuture.complete(result); } + + @Override + public Shard finish(long windowId) { + this.checkErr(); + long start = System.currentTimeMillis(); + for (int i = 0; i < this.buffers.length; i++) { + ArrayList buffer = this.buffers[i]; + if (!buffer.isEmpty()) { + this.offer(OutputMessage.data(windowId, i, buffer)); + this.buffers[i] = new ArrayList<>(this.bufferSize); + } + } + + this.offer(OutputMessage.barrier(windowId)); + try { + Shard shard = this.resultFuture.get(); + this.resultFuture = new CompletableFuture<>(); + this.eventMetrics.addShuffleWriteCostMs(System.currentTimeMillis() - start); + return shard; + } catch (InterruptedException | ExecutionException e) { + throw new GeaflowRuntimeException(e); + } + } + + @Override + public void error(Throwable t) { + if (this.err.get() == null) { + this.err.set(t); + } + } + + public void checkErr() { + if (this.err.get() != null) { + throw new GeaflowRuntimeException(this.err.get()); + } + } + + public int getEdgeId() { + return this.edgeId; + } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/AbstractWorkerContext.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/AbstractWorkerContext.java index dee479cef..812d80546 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/AbstractWorkerContext.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/AbstractWorkerContext.java @@ -26,7 +26,6 @@ import com.antgroup.geaflow.processor.Processor; import com.antgroup.geaflow.runtime.core.context.DefaultRuntimeContext; import com.antgroup.geaflow.runtime.core.context.EventContext; -import com.antgroup.geaflow.runtime.core.scheduler.PipelineMaster; import com.antgroup.geaflow.runtime.shuffle.IoDescriptor; import java.util.List; @@ -46,8 +45,7 @@ public abstract class AbstractWorkerContext implements IWorkerContext { protected String driverId; protected MetricGroup metricGroup; protected EventMetrics eventMetrics; - protected PipelineMaster schedulerRpcRef; - protected List collectors; + protected List> collectors; protected long windowId; protected RuntimeContext runtimeContext; @@ -64,14 +62,14 @@ public void init(IEventContext eventContext) { cycleId = context.getCycleId(); pipelineId = context.getPipelineId(); pipelineName = context.getPipelineName(); - schedulerRpcRef = new PipelineMaster(context.getDriverId()); + driverId = context.getDriverId(); ioDescriptor = context.getIoDescriptor(); executionTask = context.getExecutionTask(); processor = executionTask.getProcessor(); taskId = executionTask.getTaskId(); - eventMetrics = new EventMetrics(); windowId = context.getWindowId(); runtimeContext = createRuntimeContext(); + this.initEventMetrics(); } /** @@ -79,7 +77,7 @@ public void init(IEventContext eventContext) { */ private RuntimeContext createRuntimeContext() { return DefaultRuntimeContext.build(config) - .setExecutionTask(executionTask) + .setTaskArgs(this.executionTask.buildTaskArgs()) .setPipelineId(pipelineId) .setPipelineName(pipelineName) .setMetricGroup(metricGroup) @@ -95,8 +93,8 @@ public void setCurrentWindowId(long currentWindowId) { this.currentWindowId = currentWindowId; } - public PipelineMaster getPipelineMaster() { - return schedulerRpcRef; + public String getDriverId() { + return driverId; } public void setTaskId(int taskId) { @@ -123,11 +121,11 @@ public EventMetrics getEventMetrics() { return eventMetrics; } - public List getCollectors() { + public List> getCollectors() { return collectors; } - public void setCollectors(List collectors) { + public void setCollectors(List> collectors) { this.collectors = collectors; } @@ -158,4 +156,16 @@ public long getWindowId() { public RuntimeContext getRuntimeContext() { return runtimeContext; } + + public boolean isIterativeTask() { + return this.executionTask.isIterative(); + } + + public void initEventMetrics() { + this.eventMetrics = new EventMetrics( + this.executionTask.getVertexId(), + this.executionTask.getParallelism(), + this.executionTask.getIndex()); + } + } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/WorkerContext.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/WorkerContext.java index b4c3102ba..59efb8487 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/WorkerContext.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/context/WorkerContext.java @@ -16,13 +16,9 @@ import com.antgroup.geaflow.cluster.task.ITaskContext; import com.antgroup.geaflow.collector.ICollector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class WorkerContext extends AbstractWorkerContext { - private static final Logger LOGGER = LoggerFactory.getLogger(WorkerContext.class); - public WorkerContext(ITaskContext taskContext) { super(taskContext); } @@ -32,7 +28,7 @@ public WorkerContext(ITaskContext taskContext) { */ @Override public void close() { - for (ICollector collector : collectors) { + for (ICollector collector : collectors) { collector.close(); } processor.close(); diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/fetch/FetchListenerImpl.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/fetch/FetchListenerImpl.java deleted file mode 100644 index 6a2b08bc4..000000000 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/fetch/FetchListenerImpl.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2023 AntGroup CO., Ltd. - * - * Licensed 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. - */ - -package com.antgroup.geaflow.runtime.core.worker.fetch; - -import com.antgroup.geaflow.cluster.fetcher.FetchListener; -import com.antgroup.geaflow.cluster.protocol.Message; -import com.antgroup.geaflow.runtime.core.worker.InputReader; -import com.antgroup.geaflow.shuffle.message.PipelineMessage; - -public class FetchListenerImpl implements FetchListener { - - private InputReader inputReader; - - public FetchListenerImpl(InputReader inputReader) { - this.inputReader = inputReader; - } - - /** - * Trigger processor to process message. - */ - @Override - public void onMessage(PipelineMessage message) { - long windowId = message.getWindowId(); - inputReader.add(new Message(windowId, message)); - } - - @Override - public void onCompleted(long windowId, long windowCount) { - inputReader.add(new Message(windowId, windowCount)); - } -} diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/impl/ComputeWorker.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/impl/ComputeWorker.java index 6f956dd0d..df3874c91 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/impl/ComputeWorker.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/main/java/com/antgroup/geaflow/runtime/core/worker/impl/ComputeWorker.java @@ -18,8 +18,6 @@ import com.antgroup.geaflow.cluster.worker.IAffinityWorker; import com.antgroup.geaflow.cluster.worker.IWorkerContext; import com.antgroup.geaflow.collector.ICollector; -import com.antgroup.geaflow.common.metric.EventMetrics; -import com.antgroup.geaflow.common.utils.GcUtil; import com.antgroup.geaflow.model.record.BatchRecord; import com.antgroup.geaflow.runtime.core.context.DefaultRuntimeContext; import com.antgroup.geaflow.runtime.core.worker.AbstractAlignedWorker; @@ -30,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ComputeWorker extends AbstractAlignedWorker implements TransactionTrait, IAffinityWorker { +public class ComputeWorker extends AbstractAlignedWorker implements TransactionTrait, IAffinityWorker { private static final Logger LOGGER = LoggerFactory.getLogger(ComputeWorker.class); private boolean isTransactionProcessor; @@ -44,7 +42,7 @@ public void open(IWorkerContext workerContext) { super.open(workerContext); LOGGER.info("open processor"); context.getProcessor().open( - outputWriter.getCollectors(), + context.getCollectors(), context.getRuntimeContext() ); this.isTransactionProcessor = context.getProcessor() instanceof TransactionTrait; @@ -58,7 +56,7 @@ public void init(long windowId) { } @Override - public R process(BatchRecord batchRecord) { + public R process(BatchRecord batchRecord) { return (R) context.getProcessor().process(batchRecord); } @@ -66,13 +64,7 @@ public R process(BatchRecord batchRecord) { public void finish(long windowId) { LOGGER.info("taskId {} finishes windowId {}, currentBatchId {}", context.getTaskId(), windowId, context.getCurrentWindowId()); - long startFinishTime = System.currentTimeMillis(); context.getProcessor().finish(windowId); - - EventMetrics eventMetrics = context.getEventMetrics(); - eventMetrics.setFinishTime(System.currentTimeMillis() - startFinishTime); - eventMetrics.setExecuteTime(System.currentTimeMillis() - eventMetrics.getStartTs()); - eventMetrics.setGcTime(GcUtil.computeCurrentTotalGcTime() - eventMetrics.getStartGcTs()); finishWindow(context.getCurrentWindowId()); } @@ -98,21 +90,28 @@ public void pop(IWorkerContext workerContext) { context = (AbstractWorkerContext) WorkerContextManager.get(popWorkerContext.getTaskId()); Preconditions.checkArgument(context != null, "not found any context"); - context.setPipelineId(popWorkerContext.getPipelineId()); - context.setPipelineName(popWorkerContext.getPipelineName()); - context.setWindowId(popWorkerContext.getWindowId()); + final long pipelineId = popWorkerContext.getPipelineId(); + final String pipelineName = popWorkerContext.getPipelineName(); + final int cycleId = popWorkerContext.getCycleId(); + final long windowId = popWorkerContext.getWindowId(); + + context.setPipelineId(pipelineId); + context.setPipelineName(pipelineName); + context.setWindowId(windowId); + context.getExecutionTask().buildTaskName(pipelineName, cycleId, windowId); + context.initEventMetrics(); // Update runtime context. DefaultRuntimeContext runtimeContext = (DefaultRuntimeContext) context.getRuntimeContext(); - runtimeContext.setPipelineId(context.getPipelineId()); - runtimeContext.setPipelineName(context.getPipelineName()); - runtimeContext.setWindowId(context.getWindowId()); + runtimeContext.setPipelineId(pipelineId); + runtimeContext.setPipelineName(pipelineName); + runtimeContext.setWindowId(windowId); + runtimeContext.setTaskArgs(context.getExecutionTask().buildTaskArgs()); // Update collectors. for (ICollector collector : context.getCollectors()) { LOGGER.info("setup collector {}", runtimeContext.getTaskArgs()); collector.setUp(runtimeContext); } - getOutputWriter().setCollectors(this.context.getCollectors()); } } diff --git a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/test/java/com/antgroup/geaflow/runtime/core/scheduler/BaseCycleSchedulerTest.java b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/test/java/com/antgroup/geaflow/runtime/core/scheduler/BaseCycleSchedulerTest.java index de19a488c..05da441b2 100644 --- a/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/test/java/com/antgroup/geaflow/runtime/core/scheduler/BaseCycleSchedulerTest.java +++ b/geaflow/geaflow-core/geaflow-runtime/geaflow-runtime-core/src/test/java/com/antgroup/geaflow/runtime/core/scheduler/BaseCycleSchedulerTest.java @@ -121,7 +121,7 @@ public void processInternal(IEvent event) { switch (event.getEventType()) { case LAUNCH_SOURCE: LaunchSourceEvent sourceEvent = (LaunchSourceEvent) event; - response = new DoneEvent(sourceEvent.getCycleId(), sourceEvent.getIterationWindowId(), + response = new DoneEvent<>(sourceEvent.getCycleId(), sourceEvent.getIterationWindowId(), sourceEvent.getWorkerId(), EventType.EXECUTE_COMPUTE); ((IEventListener) scheduler).handleEvent(response); break; @@ -129,7 +129,7 @@ public void processInternal(IEvent event) { case CLEAN_ENV: case STASH_WORKER: AbstractExecutableCommand executableCommand = (AbstractExecutableCommand) event; - response = new DoneEvent(executableCommand.getCycleId(), executableCommand.getIterationWindowId(), + response = new DoneEvent<>(executableCommand.getCycleId(), executableCommand.getIterationWindowId(), executableCommand.getWorkerId(), executableCommand.getEventType()); ((IEventListener) scheduler).handleEvent(response); break; diff --git a/geaflow/geaflow-utils/src/test/java/com/antgroup/geaflow/utils/BlockingListBenchmark.java b/geaflow/geaflow-utils/src/test/java/com/antgroup/geaflow/utils/BlockingListBenchmark.java new file mode 100644 index 000000000..fc4982ad7 --- /dev/null +++ b/geaflow/geaflow-utils/src/test/java/com/antgroup/geaflow/utils/BlockingListBenchmark.java @@ -0,0 +1,70 @@ +/* + * Copyright 2023 AntGroup CO., Ltd. + * + * Licensed 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. + */ + +package com.antgroup.geaflow.utils; + +import java.util.LinkedList; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS) +@Fork(1) +@State(Scope.Benchmark) +public class BlockingListBenchmark { + + private static final String TEST_STR = "test"; + + @Benchmark + public void testQueue() { + LinkedList queue = new LinkedList<>(); + for (int i = 0; i < 10_000_000; i++) { + queue.offer(TEST_STR); + queue.poll(); + } + } + + + @Benchmark + public void testBlockingQueue() throws Exception { + LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); + for (int i = 0; i < 10_000_000; i++) { + queue.offer(TEST_STR); + queue.poll(); + } + } + + public static void main(String[] args) throws RunnerException { + Options opt = new OptionsBuilder() + .include(BlockingListBenchmark.class.getSimpleName()) + .build(); + new Runner(opt).run(); + } + +}