From c162cc44760511f2bf934a1080d6b2dc92d97d4f Mon Sep 17 00:00:00 2001 From: hwarim Date: Thu, 5 Aug 2021 15:06:02 +0900 Subject: [PATCH 01/26] modify input reader --- .../nemo/runtime/common/RuntimeIdManager.java | 9 +++++++ .../datatransfer/BlockInputReader.java | 24 ++++++++++++++++++- .../executor/datatransfer/InputReader.java | 6 +++++ .../datatransfer/PipeInputReader.java | 5 ++++ .../executor/task/ParentTaskDataFetcher.java | 7 +++++- 5 files changed, 49 insertions(+), 2 deletions(-) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 255cca70f5..3943893cbf 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -113,6 +113,15 @@ public static String generateBlockIdWildcard(final String runtimeEdgeId, return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + "*"; } + public static String generateWorkStealingBlockId(final String runtimeEdgeId, + final String producerTaskId, + final String consumerTaskId) { + return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) + + SPLITTER + getAttemptFromTaskId(producerTaskId) + + SPLITTER + getIndexFromTaskId(consumerTaskId) + + SPLITTER + getAttemptFromTaskId(consumerTaskId); + } + /** * Generates the ID for a control message. * diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 7f2a7e3a53..4db65e1ac4 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -90,6 +90,29 @@ public List> read() { } } + public List> read(final boolean enableWorkStealing, + final int maxSplitNum) { + if (!enableWorkStealing) { + return read(); + } + + // change here + final Optional comValueOptional = + runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); + final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); + + switch (comValue) { + case ONE_TO_ONE: + return Collections.singletonList(readOneToOne()); + case BROADCAST: + return readBroadcast(index -> true); + case SHUFFLE: + return readDataInRange(index -> true); + default: + throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); + } + } + @Override public CompletableFuture retry(final int desiredIndex) { final Optional comValueOptional = @@ -189,7 +212,6 @@ private List> readDataInRange(f blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), hashRangeToRead)); } } - return futures; } } diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java index 9cd0195fdf..59bbdc7ade 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java @@ -38,6 +38,12 @@ public interface InputReader { */ List> read(); + /** Reads input data depending on the communication pattern of the srcVertex. + * + * @return the list of iterators. + */ + List> read(boolean enableWorkStealing, int maxSplitNum); + /** * Retry reading input data. * diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java index cab2ed2f43..e00332f739 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java @@ -83,6 +83,11 @@ public List> read() { } } + @Override + public List> read(boolean enableWorkStealing, int maxSplitNum) { + return read(); + } + @Override public CompletableFuture retry(final int index) { throw new UnsupportedOperationException(String.valueOf(index)); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index a8ae4a9306..c78f0ad8b6 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -51,6 +51,10 @@ class ParentTaskDataFetcher extends DataFetcher { private long serBytes = 0; private long encodedBytes = 0; + private final int MAGIC_SPLIT_NUM = 10; + private final boolean ENABLE_WORK_STEALING = true; + + ParentTaskDataFetcher(final IRVertex dataSource, final InputReader inputReader, final OutputCollector outputCollector) { @@ -155,7 +159,8 @@ private void handleIncomingBlock(final int index, } private void fetchDataLazily() { - final List> futures = inputReader.read(); + final List> futures = inputReader + .read(ENABLE_WORK_STEALING, MAGIC_SPLIT_NUM); this.expectedNumOfIterators = futures.size(); for (int i = 0; i < futures.size(); i++) { final int index = i; From 09aa84a32b215be90eff05accbd2440d7d4ada0e Mon Sep 17 00:00:00 2001 From: hwarim Date: Thu, 12 Aug 2021 17:04:10 +0900 Subject: [PATCH 02/26] change task id --- .../nemo/runtime/common/RuntimeIdManager.java | 21 ++++++++++++++-- .../datatransfer/BlockInputReader.java | 25 ++++++------------- .../executor/task/ParentTaskDataFetcher.java | 8 ++++-- .../runtime/executor/task/TaskExecutor.java | 2 +- .../task/ParentTaskDataFetcherTest.java | 2 +- .../nemo/runtime/master/PlanStateManager.java | 2 +- 6 files changed, 36 insertions(+), 24 deletions(-) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 3943893cbf..4d0f76a284 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -71,7 +71,17 @@ public static String generateTaskId(final String stageId, final int index, final if (index < 0 || attempt < 0) { throw new IllegalStateException(index + ", " + attempt); } - return stageId + SPLITTER + index + SPLITTER + attempt; + return stageId + SPLITTER + index + SPLITTER + "*" + SPLITTER + attempt; + } + + public static String generateWorkStealingTaskId(final String stageId, + final int index, + final int partial, + final int attempt) { + if (index < 0 || partial < 0 || attempt < 0) { + throw new IllegalStateException(index + ", " + partial + ", "+ attempt); + } + return stageId + SPLITTER + index + SPLITTER + partial + SPLITTER + attempt; } /** @@ -183,6 +193,13 @@ public static int getIndexFromTaskId(final String taskId) { return Integer.valueOf(split(taskId)[1]); } + public static boolean isWorkStealingTask(final String taskId) { + return !split(taskId)[2].equals("*"); + } + + public static int getPartialFromTaskId(final String taskId) { + return Integer.valueOf(split(taskId)[2]); + } /** * Extracts the attempt from a task ID. * @@ -190,7 +207,7 @@ public static int getIndexFromTaskId(final String taskId) { * @return the attempt. */ public static int getAttemptFromTaskId(final String taskId) { - return Integer.valueOf(split(taskId)[2]); + return Integer.valueOf(split(taskId)[3]); } private static String[] split(final String id) { diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 4db65e1ac4..8a9ef99797 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -91,26 +91,17 @@ public List> read() { } public List> read(final boolean enableWorkStealing, - final int maxSplitNum) { + final int maxSplitNum, + final int index) { + /* First, fetch */ + List> futures = read(); + if (!enableWorkStealing) { - return read(); + return futures; } - // change here - final Optional comValueOptional = - runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); - final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); - - switch (comValue) { - case ONE_TO_ONE: - return Collections.singletonList(readOneToOne()); - case BROADCAST: - return readBroadcast(index -> true); - case SHUFFLE: - return readDataInRange(index -> true); - default: - throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); - } + final int baseNumber = Math.min(futures.size(), maxSplitNum); + return futures.subList(index * (futures.size() / baseNumber), (index+1) * (futures.size() / baseNumber)); } @Override diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index c78f0ad8b6..f531239c43 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -22,6 +22,7 @@ import org.apache.nemo.common.ir.edge.executionproperty.BlockFetchFailureProperty; import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.punctuation.Finishmark; +import org.apache.nemo.runtime.common.RuntimeIdManager; import org.apache.nemo.runtime.executor.data.DataUtil; import org.apache.nemo.runtime.executor.datatransfer.InputReader; import org.slf4j.Logger; @@ -40,6 +41,7 @@ class ParentTaskDataFetcher extends DataFetcher { private static final Logger LOG = LoggerFactory.getLogger(ParentTaskDataFetcher.class); + private final String taskId; private final InputReader inputReader; private final LinkedBlockingQueue iteratorQueue; @@ -57,9 +59,11 @@ class ParentTaskDataFetcher extends DataFetcher { ParentTaskDataFetcher(final IRVertex dataSource, final InputReader inputReader, - final OutputCollector outputCollector) { + final OutputCollector outputCollector, + final String taskId) { super(dataSource, outputCollector); this.inputReader = inputReader; + this.taskId = taskId; this.firstFetch = true; this.currentIteratorIndex = 0; this.iteratorQueue = new LinkedBlockingQueue<>(); @@ -160,7 +164,7 @@ private void handleIncomingBlock(final int index, private void fetchDataLazily() { final List> futures = inputReader - .read(ENABLE_WORK_STEALING, MAGIC_SPLIT_NUM); + .read(ENABLE_WORK_STEALING, MAGIC_SPLIT_NUM, RuntimeIdManager.get); this.expectedNumOfIterators = futures.size(); for (int i = 0; i < futures.size(); i++) { final int index = i; diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java index 2bf574d396..a466c018ab 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java @@ -291,7 +291,7 @@ irVertex, outputCollector, new TransformContextImpl(broadcastManagerWorker), new ParentTaskDataFetcher( parentTaskReader.getSrcIrVertex(), parentTaskReader, - dataFetcherOutputCollector)); + dataFetcherOutputCollector, taskId)); } } }); diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java index ab774e968a..c2b9c9125b 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java @@ -141,7 +141,7 @@ private ParentTaskDataFetcher createFetcher(final InputReader readerForParentTas return new ParentTaskDataFetcher( mock(IRVertex.class), readerForParentTask, // This is the only argument that affects the behavior of ParentTaskDataFetcher - mock(OutputCollector.class)); + mock(OutputCollector.class), ""); } private InputReader generateInputReader(final CompletableFuture completableFuture, diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 53cab57810..f35bc46c06 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -76,7 +76,7 @@ public final class PlanStateManager { private final Map stageIdToState; // list of attempt states sorted by attempt idx - private final Map>> stageIdToTaskIdxToAttemptStates; + private final Map>>> stageIdToTaskIdxToAttemptStates; /** * Used for speculative cloning. (in the unit of milliseconds - ms) From 7abba44120b6418a8d064104e80cd40e168282ff Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 12:27:19 +0900 Subject: [PATCH 03/26] ready for test --- .../nemo/runtime/common/RuntimeIdManager.java | 4 +- .../datatransfer/BlockInputReader.java | 2 +- .../executor/datatransfer/InputReader.java | 4 +- .../datatransfer/PipeInputReader.java | 4 +- .../executor/task/ParentTaskDataFetcher.java | 6 +- .../nemo/runtime/master/PlanStateManager.java | 117 +++++++++++------- 6 files changed, 84 insertions(+), 53 deletions(-) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 4d0f76a284..74e619955e 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -79,7 +79,7 @@ public static String generateWorkStealingTaskId(final String stageId, final int partial, final int attempt) { if (index < 0 || partial < 0 || attempt < 0) { - throw new IllegalStateException(index + ", " + partial + ", "+ attempt); + throw new IllegalStateException(index + ", " + partial + ", " + attempt); } return stageId + SPLITTER + index + SPLITTER + partial + SPLITTER + attempt; } @@ -198,7 +198,7 @@ public static boolean isWorkStealingTask(final String taskId) { } public static int getPartialFromTaskId(final String taskId) { - return Integer.valueOf(split(taskId)[2]); + return split(taskId)[2].equals("*") ? 0 : Integer.valueOf(split(taskId)[2]); } /** * Extracts the attempt from a task ID. diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 8a9ef99797..5bd0a8bbbb 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -101,7 +101,7 @@ public List> read(final boolean } final int baseNumber = Math.min(futures.size(), maxSplitNum); - return futures.subList(index * (futures.size() / baseNumber), (index+1) * (futures.size() / baseNumber)); + return futures.subList(index * (futures.size() / baseNumber), (index + 1) * (futures.size() / baseNumber)); } @Override diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java index 59bbdc7ade..04979714c5 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java @@ -42,7 +42,9 @@ public interface InputReader { * * @return the list of iterators. */ - List> read(boolean enableWorkStealing, int maxSplitNum); + List> read(boolean enableWorkStealing, + int maxSplitNum, + int index); /** * Retry reading input data. diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java index e00332f739..64c40e7f32 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java @@ -84,7 +84,9 @@ public List> read() { } @Override - public List> read(boolean enableWorkStealing, int maxSplitNum) { + public List> read(final boolean enableWorkStealing, + final int maxSplitNum, + final int index) { return read(); } diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index f531239c43..f3e435ee60 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -53,8 +53,8 @@ class ParentTaskDataFetcher extends DataFetcher { private long serBytes = 0; private long encodedBytes = 0; - private final int MAGIC_SPLIT_NUM = 10; - private final boolean ENABLE_WORK_STEALING = true; + private final int magicSplitNum = 10; + private final boolean enableWorkStealing = true; ParentTaskDataFetcher(final IRVertex dataSource, @@ -164,7 +164,7 @@ private void handleIncomingBlock(final int index, private void fetchDataLazily() { final List> futures = inputReader - .read(ENABLE_WORK_STEALING, MAGIC_SPLIT_NUM, RuntimeIdManager.get); + .read(enableWorkStealing, magicSplitNum, RuntimeIdManager.getPartialFromTaskId(taskId)); this.expectedNumOfIterators = futures.size(); for (int i = 0; i < futures.size(); i++) { final int index = i; diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index f35bc46c06..5dc2ff31db 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -192,42 +192,52 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) final List taskAttemptsToSchedule = new ArrayList<>(); final Stage stage = physicalPlan.getStageDAG().getVertexById(stageId); for (final int taskIndex : stage.getTaskIndices()) { - final List attemptStatesForThisTaskIndex = + final List> attemptStatesForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); + for (List attemptStatesForThisPartialTaskIndex : attemptStatesForThisTaskIndex) { - // If one of the attempts is COMPLETE, do not schedule - if (attemptStatesForThisTaskIndex - .stream() - .noneMatch(state -> state.getStateMachine().getCurrentState().equals(TaskState.State.COMPLETE))) { - - // (Step 1) Create new READY attempts, as many as - // # of numOfConcurrentAttempts(including clones) - # of 'not-done' attempts - stageIdToTaskIndexToNumOfClones.putIfAbsent(stageId, new HashMap<>()); - final Optional cloneConf = - stage.getPropertyValue(ClonedSchedulingProperty.class); - final int numOfConcurrentAttempts = cloneConf.isPresent() && cloneConf.get().isUpFrontCloning() - // For now we support up to 1 clone (2 concurrent = 1 original + 1 clone) - ? 2 - // If the property is not set, then we do not clone (= 1 concurrent) - : stageIdToTaskIndexToNumOfClones.get(stageId).getOrDefault(stageId, 1); - final long numOfNotDoneAttempts = attemptStatesForThisTaskIndex.stream().filter(this::isTaskNotDone).count(); - for (int i = 0; i < numOfConcurrentAttempts - numOfNotDoneAttempts; i++) { - attemptStatesForThisTaskIndex.add(new TaskState()); - } + // If one of the attempts is COMPLETE, do not schedule + if (attemptStatesForThisPartialTaskIndex + .stream() + .noneMatch(state -> state.getStateMachine().getCurrentState().equals(TaskState.State.COMPLETE))) { + + // (Step 1) Create new READY attempts, as many as + // # of numOfConcurrentAttempts(including clones) - # of 'not-done' attempts + stageIdToTaskIndexToNumOfClones.putIfAbsent(stageId, new HashMap<>()); + final Optional cloneConf = + stage.getPropertyValue(ClonedSchedulingProperty.class); + final int numOfConcurrentAttempts = cloneConf.isPresent() && cloneConf.get().isUpFrontCloning() + // For now we support up to 1 clone (2 concurrent = 1 original + 1 clone) + ? 2 + // If the property is not set, then we do not clone (= 1 concurrent) + : stageIdToTaskIndexToNumOfClones.get(stageId).getOrDefault(stageId, 1); + final long numOfNotDoneAttempts = attemptStatesForThisPartialTaskIndex + .stream().filter(this::isTaskNotDone).count(); + for (int i = 0; i < numOfConcurrentAttempts - numOfNotDoneAttempts; i++) { + attemptStatesForThisPartialTaskIndex.add(new TaskState()); + } - // (Step 2) Check max attempt - if (attemptStatesForThisTaskIndex.size() > maxScheduleAttempt) { - throw new RuntimeException( - attemptStatesForThisTaskIndex.size() + " exceeds max attempt " + maxScheduleAttempt); - } + // (Step 2) Check max attempt + if (attemptStatesForThisPartialTaskIndex.size() > maxScheduleAttempt) { + throw new RuntimeException( + attemptStatesForThisPartialTaskIndex.size() + " exceeds max attempt " + maxScheduleAttempt); + } - // (Step 3) Return all READY attempts - for (int attempt = 0; attempt < attemptStatesForThisTaskIndex.size(); attempt++) { - if (attemptStatesForThisTaskIndex.get(attempt).getStateMachine().getCurrentState() - .equals(TaskState.State.READY)) { - taskAttemptsToSchedule.add(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt)); + // (Step 3) Return all READY attempts + for (int attempt = 0; attempt < attemptStatesForThisPartialTaskIndex.size(); attempt++) { + if (attemptStatesForThisPartialTaskIndex.get(attempt).getStateMachine().getCurrentState() + .equals(TaskState.State.READY)) { + if (attemptStatesForThisTaskIndex.size() > 1) { + taskAttemptsToSchedule.add(RuntimeIdManager.generateWorkStealingTaskId(stageId, taskIndex, + attemptStatesForThisTaskIndex.indexOf(attemptStatesForThisPartialTaskIndex), attempt)); + } else { + taskAttemptsToSchedule.add(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt)); + } + + } } - } + } + } } @@ -254,13 +264,15 @@ public synchronized Map getExecutingTaskToRunningTimeMs(final Stri final long curTime = System.currentTimeMillis(); final Map result = new HashMap<>(); - final Map> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); + final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); for (final int taskIndex : taskIdToState.keySet()) { - final List attemptStates = taskIdToState.get(taskIndex); - for (int attempt = 0; attempt < attemptStates.size(); attempt++) { - if (TaskState.State.EXECUTING.equals(attemptStates.get(attempt).getStateMachine().getCurrentState())) { - final String taskId = RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt); - result.put(taskId, curTime - taskIdToStartTimeMs.get(taskId)); + final List> listOfAttemptStates = taskIdToState.get(taskIndex); + for (List attemptStates: listOfAttemptStates) { + for (int attempt = 0; attempt < attemptStates.size(); attempt++) { + if (TaskState.State.EXECUTING.equals(attemptStates.get(attempt).getStateMachine().getCurrentState())) { + final String taskId = RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt); + result.put(taskId, curTime - taskIdToStartTimeMs.get(taskId)); + } } } } @@ -325,8 +337,9 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState // Log not-yet-completed tasks for us humans to track progress final String stageId = RuntimeIdManager.getStageIdFromTaskId(taskId); - final Map> taskStatesOfThisStage = stageIdToTaskIdxToAttemptStates.get(stageId); + final Map>> taskStatesOfThisStage = stageIdToTaskIdxToAttemptStates.get(stageId); final long numOfCompletedTaskIndicesInThisStage = taskStatesOfThisStage.values().stream() + .flatMap(Collection::stream) .filter(attempts -> { final List states = attempts .stream() @@ -538,13 +551,25 @@ public synchronized TaskState.State getTaskState(final String taskId) { private Map getTaskAttemptIdsToItsState(final String stageId) { final Map result = new HashMap<>(); - final Map> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); - for (final int taskIndex : taskIdToState.keySet()) { - final List attemptStates = taskIdToState.get(taskIndex); - for (int attempt = 0; attempt < attemptStates.size(); attempt++) { - result.put(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt), - (TaskState.State) attemptStates.get(attempt).getStateMachine().getCurrentState()); + final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); + for (int taskIndex : taskIdToState.keySet()) { + List> partialIdxAttempts = taskIdToState.get(taskIndex); + if (partialIdxAttempts.size() > 1) { + for (int partialIdx = 0; partialIdx < partialIdxAttempts.size(); partialIdx++) { + List attemptStates = partialIdxAttempts.get(partialIdx); + for (int attempt = 0; attempt < attemptStates.size(); attempt++) { + result.put(RuntimeIdManager.generateWorkStealingTaskId(stageId, taskIndex, partialIdx, attempt), + (TaskState.State) attemptStates.get(attempt).getStateMachine().getCurrentState()); + } + } + } else { + List attemptStates = partialIdxAttempts.get(0); + for (int attempt = 0; attempt < attemptStates.size(); attempt++) { + result.put(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt), + (TaskState.State) attemptStates.get(attempt).getStateMachine().getCurrentState()); + } } + } return result; } @@ -553,6 +578,7 @@ private TaskState getTaskStateHelper(final String taskId) { return stageIdToTaskIdxToAttemptStates .get(RuntimeIdManager.getStageIdFromTaskId(taskId)) .get(RuntimeIdManager.getIndexFromTaskId(taskId)) + .get(RuntimeIdManager.getPartialFromTaskId(taskId)) .get(RuntimeIdManager.getAttemptFromTaskId(taskId)); } @@ -566,10 +592,11 @@ private boolean isTaskNotDone(final TaskState taskState) { private List getPeerAttemptsForTheSameTaskIndex(final String taskId) { final String stageId = RuntimeIdManager.getStageIdFromTaskId(taskId); final int taskIndex = RuntimeIdManager.getIndexFromTaskId(taskId); + final int partialIndex = RuntimeIdManager.getPartialFromTaskId(taskId); final int attempt = RuntimeIdManager.getAttemptFromTaskId(taskId); final List otherAttemptsforTheSameTaskIndex = - new ArrayList<>(stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex)); + new ArrayList<>(stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex).get(partialIndex)); otherAttemptsforTheSameTaskIndex.remove(attempt); return otherAttemptsforTheSameTaskIndex.stream() From 6fc86e54806c89b08668fe535b6836cd47883a14 Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 12:38:28 +0900 Subject: [PATCH 04/26] debug --- .../org/apache/nemo/runtime/master/PlanStateManager.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 5dc2ff31db..96ce582a24 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -563,10 +563,11 @@ private Map getTaskAttemptIdsToItsState(final String st } } } else { - List attemptStates = partialIdxAttempts.get(0); - for (int attempt = 0; attempt < attemptStates.size(); attempt++) { - result.put(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt), - (TaskState.State) attemptStates.get(attempt).getStateMachine().getCurrentState()); + for (List attemptStates : partialIdxAttempts) { + for (int attempt = 0; attempt < attemptStates.size(); attempt++) { + result.put(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt), + (TaskState.State) attemptStates.get(attempt).getStateMachine().getCurrentState()); + } } } From f2c579c4333e57b60ecf5d5a84fefd5af37dca34 Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 13:02:57 +0900 Subject: [PATCH 05/26] logs for debugging --- .../java/org/apache/nemo/runtime/master/PlanStateManager.java | 1 + .../org/apache/nemo/runtime/master/scheduler/BatchScheduler.java | 1 + 2 files changed, 2 insertions(+) diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 96ce582a24..4521270d98 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -183,6 +183,7 @@ private void initializeStates() { * @return executable task attempts */ public synchronized List getTaskAttemptsToSchedule(final String stageId) { + LOG.error("GET TASK ATTEMPTS TO SCHEDULE at {}", stageId); if (getStageState(stageId).equals(StageState.State.COMPLETE)) { // This stage is done return new ArrayList<>(0); diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java index 086c9d08bd..f8cb60786a 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java @@ -304,6 +304,7 @@ public void terminate() { * - We make {@link TaskDispatcher} dispatch only the tasks that are READY. */ private void doSchedule() { + LOG.error("DO Schedule"); final Optional> earliest = BatchSchedulerUtils.selectEarliestSchedulableGroup(sortedScheduleGroups, planStateManager); From 58f5b503ca6fe4eaf4069281ba91cb572481d4c1 Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 13:57:06 +0900 Subject: [PATCH 06/26] logs --- .../org/apache/nemo/runtime/master/PlanStateManager.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 4521270d98..591f46c3a3 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -262,7 +262,8 @@ public synchronized Set getAllTaskAttemptsOfStage(final String stageId) * @return a map from an EXECUTING task to its running time so far. */ public synchronized Map getExecutingTaskToRunningTimeMs(final String stageId) { - final long curTime = System.currentTimeMillis(); + LOG.error("get executing task to running time ms"); +; final long curTime = System.currentTimeMillis(); final Map result = new HashMap<>(); final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); @@ -322,6 +323,7 @@ public synchronized boolean setNumOfClones(final String stageId, final int taskI * @param newTaskState the new state of the task. */ public synchronized void onTaskStateChanged(final String taskId, final TaskState.State newTaskState) { + LOG.error("on task state changed"); // Change task state final StateMachine taskState = getTaskStateHelper(taskId).getStateMachine(); LOG.debug("Task State Transition: id {}, from {} to {}", @@ -551,6 +553,7 @@ public synchronized TaskState.State getTaskState(final String taskId) { } private Map getTaskAttemptIdsToItsState(final String stageId) { + LOG.error("get task attempt ids to its state"); final Map result = new HashMap<>(); final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); for (int taskIndex : taskIdToState.keySet()) { @@ -592,6 +595,7 @@ private boolean isTaskNotDone(final TaskState taskState) { } private List getPeerAttemptsForTheSameTaskIndex(final String taskId) { + LOG.error("get peer attempts for the same task index"); final String stageId = RuntimeIdManager.getStageIdFromTaskId(taskId); final int taskIndex = RuntimeIdManager.getIndexFromTaskId(taskId); final int partialIndex = RuntimeIdManager.getPartialFromTaskId(taskId); From 729de29b3f616d0438c1c2a7d8a43d0762a504dd Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 14:28:03 +0900 Subject: [PATCH 07/26] logs --- .../org/apache/nemo/runtime/master/PlanStateManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 591f46c3a3..06c980390a 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -251,6 +251,7 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) * @return all task attempt ids of the stage. */ public synchronized Set getAllTaskAttemptsOfStage(final String stageId) { + LOG.error("get al task attempts of stage"); return getTaskAttemptIdsToItsState(stageId).keySet(); } @@ -263,7 +264,7 @@ public synchronized Set getAllTaskAttemptsOfStage(final String stageId) */ public synchronized Map getExecutingTaskToRunningTimeMs(final String stageId) { LOG.error("get executing task to running time ms"); -; final long curTime = System.currentTimeMillis(); + final long curTime = System.currentTimeMillis(); final Map result = new HashMap<>(); final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); @@ -508,6 +509,7 @@ public PlanState.State waitUntilFinish(final long timeout, final TimeUnit unit) */ @VisibleForTesting public synchronized Map getAllTaskAttemptIdsToItsState() { + LOG.error("get all task attempt ids to its state"); return physicalPlan.getStageDAG().getVertices() .stream() .map(Stage::getId) @@ -680,6 +682,7 @@ public synchronized String toString() { boolean isFirstTask = true; for (final Map.Entry entry : getTaskAttemptIdsToItsState(stage.getId()).entrySet()) { + LOG.error("in tostring"); if (!isFirstTask) { sb.append(", "); } From aa82d76104e77c5b6ee1b9d2c180786265c40f34 Mon Sep 17 00:00:00 2001 From: hwarim Date: Fri, 13 Aug 2021 16:18:56 +0900 Subject: [PATCH 08/26] logs --- ..._wordcount_ws_hurricane_20160731_1628833837515.json | 1 + .../apache/nemo/runtime/master/PlanStateManager.java | 10 +++++++--- .../org/apache/nemo/runtime/master/RuntimeMaster.java | 2 ++ .../nemo/runtime/master/scheduler/TaskDispatcher.java | 1 + 4 files changed, 11 insertions(+), 3 deletions(-) create mode 100644 Metric_wordcount_ws_hurricane_20160731_1628833837515.json diff --git a/Metric_wordcount_ws_hurricane_20160731_1628833837515.json b/Metric_wordcount_ws_hurricane_20160731_1628833837515.json new file mode 100644 index 0000000000..cd39f2df91 --- /dev/null +++ b/Metric_wordcount_ws_hurricane_20160731_1628833837515.json @@ -0,0 +1 @@ +{"JobMetric":{"Plan0":{"id":"Plan0","data":{"id":"Plan0","stateTransitionEvents":[{"timestamp":1628833525477,"prevState":"READY","newState":"EXECUTING"}],"irDagSummary":"rv1_v7_e6_8GB","inputSize":8854645443,"vertexProperties":"100010001:1 100010002:73 100010003:0 100010004:1 100010005:1 100020001:1 100020002:73 100020003:0 100020004:1 100020005:1 100030001:1 100030002:73 100030003:0 100030004:1 100030005:1 100040001:1 100040002:73 100040003:0 100040004:1 100040005:1 100050001:1 100050002:36 100050003:1 100050004:1 100050005:1 100060001:1 100060002:36 100060003:1 100060004:1 100060005:1 100070001:1 100070002:36 100070003:1 100070004:1 100070005:1","edgeProperties":"200010006:1 200010007:1 200010008:1 200010009:1 200010010:0 200010011:1 200010012:1 200010013:1 200010014:0 200010015:1 200010016:1 200010017:1 200020006:1 200020008:1 200020009:1 200020010:0 200020011:1 200020012:1 200020013:1 200020014:0 200020016:1 200020017:1 200030006:1 200030007:2 200030008:1 200030009:1 200030010:0 200030011:1 200030012:1 200030013:1 200030014:0 200030015:2 200030016:1 200030017:1 200040006:1 200040007:2 200040008:1 200040009:3 200040010:2 200040011:2 200040012:1 200040013:1 200040014:1 200040015:2 200040016:1 200040017:0 200050006:1 200050007:2 200050008:1 200050009:1 200050010:0 200050011:1 200050012:1 200050013:1 200050014:0 200050015:2 200050016:1 200050017:1 200060006:1 200060008:1 200060009:1 200060010:0 200060011:1 200060012:1 200060013:1 200060014:0 200060016:1 200060017:1","jobDuration":null,"stage-dag":{"vertices":[{"id":"Stage0","properties":{"scheduleGroup":0,"irDag":{"vertices":[{"id":"vertex1","properties":{"class":"BeamBoundedSourceVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"source":"[]org.apache.beam.sdk.io.Read$Bounded:source=org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:key.class=org.apache.hadoop.io.LongWritable\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:mapreduce.job.inputformat.class=org.apache.hadoop.mapreduce.lib.input.TextInputFormat\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:value.class=org.apache.hadoop.io.Text"}},{"id":"vertex2","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements/Map/ParMultiDo(Anonymous)"}},{"id":"vertex3","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements2/Map/ParMultiDo(Anonymous)"}},{"id":"vertex4","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnPartialTransform@173a7e4d"}}],"edges":[{"src":"vertex3","dst":"vertex4","properties":{"runtimeEdgeId":"edge3","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex2","dst":"vertex3","properties":{"runtimeEdgeId":"edge2","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex1","dst":"vertex2","properties":{"runtimeEdgeId":"edge1","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]},"parallelism":73,"num of task indices":73,"executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"}}},{"id":"Stage1","properties":{"scheduleGroup":1,"irDag":{"vertices":[{"id":"vertex5","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnFinalTransform@6a1b9b9a"}},{"id":"vertex6","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements3/Map/ParMultiDo(Anonymous)"}},{"id":"vertex7","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=ParDo(HDFSWrite)/ParMultiDo(HDFSWrite)"}}],"edges":[{"src":"vertex6","dst":"vertex7","properties":{"runtimeEdgeId":"edge6","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex5","dst":"vertex6","properties":{"runtimeEdgeId":"edge5","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]},"parallelism":36,"num of task indices":36,"executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"}}}],"edges":[{"src":"Stage0","dst":"Stage1","properties":{"runtimeEdgeId":"edge4","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"LOCAL_FILE_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"SHUFFLE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[HASH,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"KEEP","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PULL"},"externalSrcVertexId":"vertex4","externalDstVertexId":"vertex5"}}]},"ir-dag":{"vertices":[{"id":"vertex1","properties":{"class":"BeamBoundedSourceVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"source":"[]org.apache.beam.sdk.io.Read$Bounded:source=org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:key.class=org.apache.hadoop.io.LongWritable\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:mapreduce.job.inputformat.class=org.apache.hadoop.mapreduce.lib.input.TextInputFormat\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:value.class=org.apache.hadoop.io.Text"}},{"id":"vertex2","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements/Map/ParMultiDo(Anonymous)"}},{"id":"vertex3","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements2/Map/ParMultiDo(Anonymous)"}},{"id":"vertex4","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnPartialTransform@173a7e4d"}},{"id":"vertex5","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnFinalTransform@6a1b9b9a"}},{"id":"vertex6","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements3/Map/ParMultiDo(Anonymous)"}},{"id":"vertex7","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=ParDo(HDFSWrite)/ParMultiDo(HDFSWrite)"}}],"edges":[{"src":"vertex4","dst":"vertex5","properties":{"id":"edge4","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"LOCAL_FILE_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"SHUFFLE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[HASH,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"KEEP","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PULL"}}},{"src":"vertex3","dst":"vertex4","properties":{"id":"edge3","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex6","dst":"vertex7","properties":{"id":"edge6","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex5","dst":"vertex6","properties":{"id":"edge5","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex2","dst":"vertex3","properties":{"id":"edge2","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex1","dst":"vertex2","properties":{"id":"edge1","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]}}}}} \ No newline at end of file diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 06c980390a..884249a893 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -141,6 +141,7 @@ public void setMetricStore(final MetricStore metricStore) { */ public synchronized void updatePlan(final PhysicalPlan physicalPlanToUpdate, final int maxScheduleAttemptToSet) { + LOG.error("update plan"); if (!initialized) { // First scheduling. this.initialized = true; @@ -161,6 +162,7 @@ public synchronized void updatePlan(final PhysicalPlan physicalPlanToUpdate, * TODO #182: Consider reshaping in run-time optimization. At now, we only consider plan appending. */ private void initializeStates() { + LOG.error("initialize states"); onPlanStateChanged(PlanState.State.EXECUTING); physicalPlan.getStageDAG().topologicalDo(stage -> { stageIdToState.putIfAbsent(stage.getId(), new StageState()); @@ -168,7 +170,7 @@ private void initializeStates() { // for each task idx of this stage stage.getTaskIndices().forEach(taskIndex -> - stageIdToTaskIdxToAttemptStates.get(stage.getId()).putIfAbsent(taskIndex, new ArrayList<>())); + stageIdToTaskIdxToAttemptStates.get(stage.getId()).putIfAbsent(taskIndex, new ArrayList<>(10))); // task states will be initialized lazily in getTaskAttemptsToSchedule() }); } @@ -193,8 +195,10 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) final List taskAttemptsToSchedule = new ArrayList<>(); final Stage stage = physicalPlan.getStageDAG().getVertexById(stageId); for (final int taskIndex : stage.getTaskIndices()) { + LOG.error("{} task index {}", stageId, taskIndex); final List> attemptStatesForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); + LOG.error("partial index size : {}", attemptStatesForThisTaskIndex.size()); // no tasks in here! for (List attemptStatesForThisPartialTaskIndex : attemptStatesForThisTaskIndex) { // If one of the attempts is COMPLETE, do not schedule @@ -242,7 +246,7 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) } } - + LOG.error("attempts to schedule: {} tasks", taskAttemptsToSchedule.size()); return taskAttemptsToSchedule; } @@ -639,6 +643,7 @@ public synchronized boolean isInitialized() { * @param suffix suffix for file name */ public void storeJSON(final String suffix) { + LOG.error("STORE JSON {}", suffix); if (dagDirectory.equals(EMPTY_DAG_DIRECTORY)) { return; } @@ -682,7 +687,6 @@ public synchronized String toString() { boolean isFirstTask = true; for (final Map.Entry entry : getTaskAttemptIdsToItsState(stage.getId()).entrySet()) { - LOG.error("in tostring"); if (!isFirstTask) { sb.append(", "); } diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java index d3b48f266a..cae3cf47da 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java @@ -217,6 +217,7 @@ private Server startRestMetricServer() { * @param planId the ID of the IR DAG Physical Plan. */ public void recordIRDAGMetrics(final IRDAG irdag, final String planId) { + LOG.error("record metrics of {}", planId); metricStore.getOrCreateMetric(JobMetric.class, planId).setIRDAG(irdag); } @@ -260,6 +261,7 @@ public void flushMetrics() { */ public Pair execute(final PhysicalPlan plan, final int maxScheduleAttempt) { + LOG.error("EXECUTE in runtime master"); final Callable> planExecutionCallable = () -> { this.irVertices.addAll(plan.getIdToIRVertex().values()); try { diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java index e15d8ea13e..559668d54b 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java @@ -183,6 +183,7 @@ void onNewPendingTaskCollectionAvailable() { * Run the dispatcher thread. */ void run() { + LOG.error("Run in task dispatcher"); if (!isTerminated && !isSchedulerRunning) { dispatcherThread.execute(new TaskDispatcherThread()); dispatcherThread.shutdown(); From 9e7c736ec2374cfb24936e8380a3f95f74d32acb Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 16:41:13 +0900 Subject: [PATCH 09/26] backup --- .../EnableWorkStealingExecutionProperty.java | 24 ++++++++++++++ .../annotating/WorkStealingPass.java | 32 +++++++++++++++++++ .../apache/nemo/examples/beam/WordCount.java | 4 +-- .../common/plan/PhysicalPlanGenerator.java | 3 +- .../nemo/runtime/common/plan/Stage.java | 8 ++++- .../datatransfer/DataTransferTest.java | 2 +- .../nemo/runtime/master/PlanStateManager.java | 7 ++++ 7 files changed, 75 insertions(+), 5 deletions(-) create mode 100644 common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java create mode 100644 compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java new file mode 100644 index 0000000000..c8a59e4251 --- /dev/null +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java @@ -0,0 +1,24 @@ +package org.apache.nemo.common.ir.vertex.executionproperty; + +import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; + +public class EnableWorkStealingExecutionProperty extends VertexExecutionProperty { + /** + * Default constructor. + * + * @param value value of the VertexExecutionProperty. + */ + public EnableWorkStealingExecutionProperty(Boolean value) { + super(value); + } + + /** + * Static method exposing the constructor. + * + * @param value value of the new execution property. + * @return the newly created execution property. + */ + public static EnableDynamicTaskSizingProperty of(final Boolean value) { + return new EnableDynamicTaskSizingProperty(value); + } +} diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java new file mode 100644 index 0000000000..0b10bce346 --- /dev/null +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -0,0 +1,32 @@ +package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating; + +import org.apache.nemo.common.ir.IRDAG; +import org.apache.nemo.common.ir.vertex.OperatorVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingExecutionProperty; +import org.apache.nemo.common.ir.vertex.transform.Transform; + +/** + * Optimization pass for tagging parallelism execution property. + */ +@Annotates(EnableWorkStealingExecutionProperty.class) +public class WorkStealingPass extends AnnotatingPass{ + + private final boolean enableWorkStealing; + + public WorkStealingPass(boolean enableWorkStealing) { + super(WorkStealingPass.class); + this.enableWorkStealing = enableWorkStealing; + } + + @Override + public IRDAG apply(IRDAG irdag) { + irdag.topologicalDo(irVertex -> { + if (irVertex instanceof OperatorVertex) { + Transform transform = ((OperatorVertex) irVertex).getTransform(); + + irVertex.setProperty(EnableWorkStealingExecutionProperty.of(true)); + } + }); + return irdag; + } +} diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 367938fd68..748f7ddb00 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -59,10 +59,10 @@ public static void main(final String[] args) { * @return the generated pipeline. */ static Pipeline generateWordCountPipeline(final PipelineOptions options, - final String inputFilePath, final String outputFilePath) { + final String inputFilePath, final String outputFilePath) { final Pipeline p = Pipeline.create(options); final PCollection result = GenericSourceSink.read(p, inputFilePath) - .apply(MapElements.>via(new SimpleFunction>() { + .apply("work stealing", MapElements.>via(new SimpleFunction>() { @Override public KV apply(final String line) { final String[] words = line.split(" +"); diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index 83728e4f03..4b805cdd8a 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -213,7 +213,8 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { taskIndices, stageInternalDAG, stageProperties, - vertexIdToReadables); + vertexIdToReadables, + 1); // ad-hoc for now dagOfStagesBuilder.addVertex(stage); stageIdToStageMap.put(stageId, stage); } diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java index a7f472c0da..ac99b35edd 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java @@ -42,6 +42,7 @@ */ public final class Stage extends Vertex { private final List taskIndices; + private final int subSplitNum; private final DAG> irDag; private final byte[] serializedIRDag; private final List> vertexIdToReadables; @@ -61,13 +62,15 @@ public Stage(final String stageId, final List taskIndices, final DAG> irDag, final ExecutionPropertyMap executionProperties, - final List> vertexIdToReadables) { + final List> vertexIdToReadables, + final int subSplitNum) { // 이거 어떻게 설정해줄 수 있는지 생각!!! -> ws용으로 하겠다는 얘기있으면 10으로 하게... super(stageId); this.taskIndices = taskIndices; this.irDag = irDag; this.serializedIRDag = SerializationUtils.serialize(irDag); this.executionProperties = executionProperties; this.vertexIdToReadables = vertexIdToReadables; + this.subSplitNum = subSplitNum; } /** @@ -155,6 +158,9 @@ public List> getVertexIdToReadables() { return vertexIdToReadables; } + public int getSubSplitNum() { + return subSplitNum; + } @Override public ObjectNode getPropertiesAsJsonNode() { final ObjectNode node = JsonNodeFactory.instance.objectNode(); diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java index 3ce99197e6..65c4ff145c 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java @@ -532,7 +532,7 @@ private Stage setupStages(final String stageId) { IntStream.range(0, PARALLELISM_TEN).boxed().collect(Collectors.toList()), emptyDag, stageExecutionProperty, - Collections.emptyList()); + Collections.emptyList(), 0); } /** diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 884249a893..ad23d8d484 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -185,6 +185,7 @@ private void initializeStates() { * @return executable task attempts */ public synchronized List getTaskAttemptsToSchedule(final String stageId) { + // initialization: 첫번째로 만들어지는케이스를 따로 생각해야 함. LOG.error("GET TASK ATTEMPTS TO SCHEDULE at {}", stageId); if (getStageState(stageId).equals(StageState.State.COMPLETE)) { // This stage is done @@ -199,6 +200,12 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) final List> attemptStatesForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); LOG.error("partial index size : {}", attemptStatesForThisTaskIndex.size()); // no tasks in here! + if (attemptStatesForThisTaskIndex.size() == 0) { + // initialize in here + for (int i = 0; i < stage.getSubSplitNum(); i++) { + attemptStatesForThisTaskIndex.add(new ArrayList<>()); + } + } for (List attemptStatesForThisPartialTaskIndex : attemptStatesForThisTaskIndex) { // If one of the attempts is COMPLETE, do not schedule From eaae64f51e037074d515434cbe2d439ffcc2de90 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 16:55:09 +0900 Subject: [PATCH 10/26] add work stealing policy --- .../annotating/WorkStealingPass.java | 11 +++--- .../optimizer/policy/WorkStealingPolicy.java | 34 +++++++++++++++++++ 2 files changed, 40 insertions(+), 5 deletions(-) create mode 100644 compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index 0b10bce346..1fef37b2b0 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -11,11 +11,9 @@ @Annotates(EnableWorkStealingExecutionProperty.class) public class WorkStealingPass extends AnnotatingPass{ - private final boolean enableWorkStealing; - public WorkStealingPass(boolean enableWorkStealing) { + public WorkStealingPass() { super(WorkStealingPass.class); - this.enableWorkStealing = enableWorkStealing; } @Override @@ -23,8 +21,11 @@ public IRDAG apply(IRDAG irdag) { irdag.topologicalDo(irVertex -> { if (irVertex instanceof OperatorVertex) { Transform transform = ((OperatorVertex) irVertex).getTransform(); - - irVertex.setProperty(EnableWorkStealingExecutionProperty.of(true)); + if (transform.toString().contains("work stealing")) { + irVertex.setProperty(EnableWorkStealingExecutionProperty.of(true)); + } else { + irVertex.setProperty(EnableWorkStealingExecutionProperty.of(false)); + } } }); return irdag; diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java new file mode 100644 index 0000000000..07d3736f81 --- /dev/null +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java @@ -0,0 +1,34 @@ +package org.apache.nemo.compiler.optimizer.policy; + +import com.sun.corba.se.spi.orbutil.threadpool.Work; +import org.apache.nemo.common.ir.IRDAG; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingExecutionProperty; +import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingPass; +import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass; +import org.apache.nemo.compiler.optimizer.pass.runtime.Message; + +public class WorkStealingPolicy implements Policy { + public static final PolicyBuilder BUILDER = + new PolicyBuilder() + .registerCompileTimePass(new DefaultCompositePass()) + .registerCompileTimePass(new WorkStealingPass()); + + private final Policy policy; + + /** + * Default constructor. + */ + public WorkStealingPolicy() { + this.policy = BUILDER.build(); + } + + @Override + public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) { + return this.policy.runCompileTimeOptimization(dag, dagDirectory); + } + + @Override + public IRDAG runRunTimeOptimizations(final IRDAG dag, final Message message) { + return this.policy.runRunTimeOptimizations(dag, message); + } +} From 5c41fc7e3dbed7f49dcd349176d6020ae40e3bdf Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 19:00:07 +0900 Subject: [PATCH 11/26] backup --- ...ount_ws_hurricane_20160731_1628833837515.json | 1 - ...erty.java => EnableWorkStealingProperty.java} | 9 +++++---- .../compiletime/annotating/WorkStealingPass.java | 10 ++++++---- .../optimizer/policy/WorkStealingPolicy.java | 2 -- .../org/apache/nemo/examples/beam/WordCount.java | 5 +++-- .../common/plan/PhysicalPlanGenerator.java | 7 ++++++- .../executor/task/ParentTaskDataFetcher.java | 4 +++- .../nemo/runtime/executor/task/TaskExecutor.java | 16 ++++++++++++++++ .../executor/datatransfer/DataTransferTest.java | 3 ++- .../nemo/runtime/master/PlanStateManager.java | 10 ++++++++-- 10 files changed, 49 insertions(+), 18 deletions(-) delete mode 100644 Metric_wordcount_ws_hurricane_20160731_1628833837515.json rename common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/{EnableWorkStealingExecutionProperty.java => EnableWorkStealingProperty.java} (60%) diff --git a/Metric_wordcount_ws_hurricane_20160731_1628833837515.json b/Metric_wordcount_ws_hurricane_20160731_1628833837515.json deleted file mode 100644 index cd39f2df91..0000000000 --- a/Metric_wordcount_ws_hurricane_20160731_1628833837515.json +++ /dev/null @@ -1 +0,0 @@ -{"JobMetric":{"Plan0":{"id":"Plan0","data":{"id":"Plan0","stateTransitionEvents":[{"timestamp":1628833525477,"prevState":"READY","newState":"EXECUTING"}],"irDagSummary":"rv1_v7_e6_8GB","inputSize":8854645443,"vertexProperties":"100010001:1 100010002:73 100010003:0 100010004:1 100010005:1 100020001:1 100020002:73 100020003:0 100020004:1 100020005:1 100030001:1 100030002:73 100030003:0 100030004:1 100030005:1 100040001:1 100040002:73 100040003:0 100040004:1 100040005:1 100050001:1 100050002:36 100050003:1 100050004:1 100050005:1 100060001:1 100060002:36 100060003:1 100060004:1 100060005:1 100070001:1 100070002:36 100070003:1 100070004:1 100070005:1","edgeProperties":"200010006:1 200010007:1 200010008:1 200010009:1 200010010:0 200010011:1 200010012:1 200010013:1 200010014:0 200010015:1 200010016:1 200010017:1 200020006:1 200020008:1 200020009:1 200020010:0 200020011:1 200020012:1 200020013:1 200020014:0 200020016:1 200020017:1 200030006:1 200030007:2 200030008:1 200030009:1 200030010:0 200030011:1 200030012:1 200030013:1 200030014:0 200030015:2 200030016:1 200030017:1 200040006:1 200040007:2 200040008:1 200040009:3 200040010:2 200040011:2 200040012:1 200040013:1 200040014:1 200040015:2 200040016:1 200040017:0 200050006:1 200050007:2 200050008:1 200050009:1 200050010:0 200050011:1 200050012:1 200050013:1 200050014:0 200050015:2 200050016:1 200050017:1 200060006:1 200060008:1 200060009:1 200060010:0 200060011:1 200060012:1 200060013:1 200060014:0 200060016:1 200060017:1","jobDuration":null,"stage-dag":{"vertices":[{"id":"Stage0","properties":{"scheduleGroup":0,"irDag":{"vertices":[{"id":"vertex1","properties":{"class":"BeamBoundedSourceVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"source":"[]org.apache.beam.sdk.io.Read$Bounded:source=org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:key.class=org.apache.hadoop.io.LongWritable\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:mapreduce.job.inputformat.class=org.apache.hadoop.mapreduce.lib.input.TextInputFormat\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:value.class=org.apache.hadoop.io.Text"}},{"id":"vertex2","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements/Map/ParMultiDo(Anonymous)"}},{"id":"vertex3","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements2/Map/ParMultiDo(Anonymous)"}},{"id":"vertex4","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnPartialTransform@173a7e4d"}}],"edges":[{"src":"vertex3","dst":"vertex4","properties":{"runtimeEdgeId":"edge3","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex2","dst":"vertex3","properties":{"runtimeEdgeId":"edge2","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex1","dst":"vertex2","properties":{"runtimeEdgeId":"edge1","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]},"parallelism":73,"num of task indices":73,"executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"}}},{"id":"Stage1","properties":{"scheduleGroup":1,"irDag":{"vertices":[{"id":"vertex5","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnFinalTransform@6a1b9b9a"}},{"id":"vertex6","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements3/Map/ParMultiDo(Anonymous)"}},{"id":"vertex7","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=ParDo(HDFSWrite)/ParMultiDo(HDFSWrite)"}}],"edges":[{"src":"vertex6","dst":"vertex7","properties":{"runtimeEdgeId":"edge6","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex5","dst":"vertex6","properties":{"runtimeEdgeId":"edge5","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]},"parallelism":36,"num of task indices":36,"executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"}}}],"edges":[{"src":"Stage0","dst":"Stage1","properties":{"runtimeEdgeId":"edge4","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"LOCAL_FILE_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"SHUFFLE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[HASH,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"KEEP","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PULL"},"externalSrcVertexId":"vertex4","externalDstVertexId":"vertex5"}}]},"ir-dag":{"vertices":[{"id":"vertex1","properties":{"class":"BeamBoundedSourceVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"source":"[]org.apache.beam.sdk.io.Read$Bounded:source=org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:key.class=org.apache.hadoop.io.LongWritable\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:mapreduce.job.inputformat.class=org.apache.hadoop.mapreduce.lib.input.TextInputFormat\n[source]org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO$HadoopInputFormatBoundedSource:value.class=org.apache.hadoop.io.Text"}},{"id":"vertex2","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements/Map/ParMultiDo(Anonymous)"}},{"id":"vertex3","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements2/Map/ParMultiDo(Anonymous)"}},{"id":"vertex4","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"73","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"0","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnPartialTransform@173a7e4d"}},{"id":"vertex5","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"CombineFnPartialTransform:org.apache.nemo.compiler.frontend.beam.transform.CombineFnFinalTransform@6a1b9b9a"}},{"id":"vertex6","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=MapElements3/Map/ParMultiDo(Anonymous)"}},{"id":"vertex7","properties":{"class":"OperatorVertex","executionProperties":{"org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty":"None","org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty":"36","org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty":"1","org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty":"true","org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty":"true"},"transform":"DoFnTransform / []org.apache.nemo.compiler.frontend.beam.PipelineTranslator / name=ParDo(HDFSWrite)/ParMultiDo(HDFSWrite)"}}],"edges":[{"src":"vertex4","dst":"vertex5","properties":{"id":"edge4","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"LOCAL_FILE_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"SHUFFLE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[HASH,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"KEEP","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PULL"}}},{"src":"vertex3","dst":"vertex4","properties":{"id":"edge3","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex6","dst":"vertex7","properties":{"id":"edge6","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex5","dst":"vertex6","properties":{"id":"edge5","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.StringUtf8Coder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex2","dst":"vertex3","properties":{"id":"edge2","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}},{"src":"vertex1","dst":"vertex2","properties":{"id":"edge1","executionProperties":{"org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.KeyDecoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty":"MEMORY_STORE","org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty":"ONE_TO_ONE","org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty":"Pair[INTACT,0]","org.apache.nemo.common.ir.edge.executionproperty.KeyExtractorProperty":"org.apache.nemo.compiler.frontend.beam.BeamKeyExtractor@845","org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty":"org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder","org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty":"DISCARD","org.apache.nemo.common.ir.edge.executionproperty.KeyEncoderProperty":"org.apache.beam.sdk.coders.VarLongCoder","org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty":"LZ4","org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty":"PUSH"}}}]}}}}} \ No newline at end of file diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java similarity index 60% rename from common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java rename to common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java index c8a59e4251..a8c925bb14 100644 --- a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingExecutionProperty.java +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java @@ -2,13 +2,14 @@ import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; -public class EnableWorkStealingExecutionProperty extends VertexExecutionProperty { +public class EnableWorkStealingProperty + extends VertexExecutionProperty { /** * Default constructor. * * @param value value of the VertexExecutionProperty. */ - public EnableWorkStealingExecutionProperty(Boolean value) { + public EnableWorkStealingProperty(String value) { super(value); } @@ -18,7 +19,7 @@ public EnableWorkStealingExecutionProperty(Boolean value) { * @param value value of the new execution property. * @return the newly created execution property. */ - public static EnableDynamicTaskSizingProperty of(final Boolean value) { - return new EnableDynamicTaskSizingProperty(value); + public static EnableWorkStealingProperty of(final String value) { + return new EnableWorkStealingProperty(value); } } diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index 1fef37b2b0..0204dee186 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -2,13 +2,13 @@ import org.apache.nemo.common.ir.IRDAG; import org.apache.nemo.common.ir.vertex.OperatorVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingExecutionProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.transform.Transform; /** * Optimization pass for tagging parallelism execution property. */ -@Annotates(EnableWorkStealingExecutionProperty.class) +@Annotates(EnableWorkStealingProperty.class) public class WorkStealingPass extends AnnotatingPass{ @@ -22,9 +22,11 @@ public IRDAG apply(IRDAG irdag) { if (irVertex instanceof OperatorVertex) { Transform transform = ((OperatorVertex) irVertex).getTransform(); if (transform.toString().contains("work stealing")) { - irVertex.setProperty(EnableWorkStealingExecutionProperty.of(true)); + irVertex.setProperty(EnableWorkStealingProperty.of("SPLIT")); + } else if (transform.toString().contains("merge")){ + irVertex.setProperty(EnableWorkStealingProperty.of("MERGE")); } else { - irVertex.setProperty(EnableWorkStealingExecutionProperty.of(false)); + irVertex.setProperty(EnableWorkStealingProperty.of("DEFAULT")); } } }); diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java index 07d3736f81..ca6d3623a2 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java @@ -1,8 +1,6 @@ package org.apache.nemo.compiler.optimizer.policy; -import com.sun.corba.se.spi.orbutil.threadpool.Work; import org.apache.nemo.common.ir.IRDAG; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingExecutionProperty; import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingPass; import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass; import org.apache.nemo.compiler.optimizer.pass.runtime.Message; diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 748f7ddb00..7da3ccd73b 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -62,7 +62,7 @@ static Pipeline generateWordCountPipeline(final PipelineOptions options, final String inputFilePath, final String outputFilePath) { final Pipeline p = Pipeline.create(options); final PCollection result = GenericSourceSink.read(p, inputFilePath) - .apply("work stealing", MapElements.>via(new SimpleFunction>() { + .apply(MapElements.>via(new SimpleFunction>() { @Override public KV apply(final String line) { final String[] words = line.split(" +"); @@ -71,7 +71,8 @@ public KV apply(final String line) { return KV.of(documentId, count); } })) - .apply(Sum.longsPerKey()) + .apply("work stealing", Sum.longsPerKey()) + .apply("merge", Sum.longsPerKey()) .apply(MapElements., String>via(new SimpleFunction, String>() { @Override public String apply(final KV kv) { diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index 4b805cdd8a..fab9da8ec1 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -32,6 +32,7 @@ import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.SourceVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty; import org.apache.nemo.common.ir.vertex.utility.SamplingVertex; @@ -208,13 +209,17 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { if (!stageInternalDAGBuilder.isEmpty()) { final DAG> stageInternalDAG = stageInternalDAGBuilder.buildWithoutSourceSinkCheck(); + // check if this stage is subject of work stealing optimization + boolean isWorkStealingStage = stageInternalDAG.getRootVertices().stream() + .anyMatch(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class) + .orElse("DEFAULT").equals("SPLIT")); final Stage stage = new Stage( stageIdentifier, taskIndices, stageInternalDAG, stageProperties, vertexIdToReadables, - 1); // ad-hoc for now + isWorkStealingStage? 10 : 1); // ad-hoc for now dagOfStagesBuilder.addVertex(stage); stageIdToStageMap.put(stageId, stage); } diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index f3e435ee60..505de4e93e 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -54,7 +54,7 @@ class ParentTaskDataFetcher extends DataFetcher { private long encodedBytes = 0; private final int magicSplitNum = 10; - private final boolean enableWorkStealing = true; + private final boolean enableWorkStealing; ParentTaskDataFetcher(final IRVertex dataSource, @@ -67,6 +67,8 @@ class ParentTaskDataFetcher extends DataFetcher { this.firstFetch = true; this.currentIteratorIndex = 0; this.iteratorQueue = new LinkedBlockingQueue<>(); + + this.enableWorkStealing = RuntimeIdManager.isWorkStealingTask(taskId); } @Override diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java index a466c018ab..c94018540c 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java @@ -30,6 +30,7 @@ import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.SourceVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform; import org.apache.nemo.common.ir.vertex.transform.SignalTransform; import org.apache.nemo.common.ir.vertex.transform.Transform; @@ -82,6 +83,7 @@ public final class TaskExecutor { // Dynamic optimization private String idOfVertexPutOnHold; + private String workStealingStrategy; private final PersistentConnectionToMasterMap persistentConnectionToMasterMap; @@ -120,6 +122,7 @@ public TaskExecutor(final Task task, // Prepare data structures final Pair, List> pair = prepare(task, irVertexDag, intermediateDataIOFactory); + this.workStealingStrategy = getWorkStealingStrategy(irVertexDag); this.dataFetchers = pair.left(); this.sortedHarnesses = pair.right(); @@ -714,4 +717,17 @@ private void finalizeOutputWriters(final VertexHarness vertexHarness) { metricMessageSender.send(TASK_METRIC_ID, taskId, "taskOutputBytes", SerializationUtils.serialize(totalWrittenBytes)); } + + private String getWorkStealingStrategy(DAG> irVertexDag) { + Set strategy = irVertexDag.getVertices().stream() + .map(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class).orElse("DEFAULT")) + .collect(Collectors.toSet()); + if (strategy.contains("SPLIT")) { + return "SPLIT"; + } else if (strategy.contains("MERGE")) { + return "MERGE"; + } else { + return "DEFAULT"; + } + } } diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java index 65c4ff145c..d3cfe82c15 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/datatransfer/DataTransferTest.java @@ -532,7 +532,8 @@ private Stage setupStages(final String stageId) { IntStream.range(0, PARALLELISM_TEN).boxed().collect(Collectors.toList()), emptyDag, stageExecutionProperty, - Collections.emptyList(), 0); + Collections.emptyList(), + 1); } /** diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index ad23d8d484..5c13d9f66d 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -102,6 +102,11 @@ public final class PlanStateManager { private final String dagDirectory; private MetricStore metricStore; + /** + * For dynamic optimization + */ + private final int maxSubTaskSplitNum = 10; + /** * Constructor. */ @@ -170,7 +175,7 @@ private void initializeStates() { // for each task idx of this stage stage.getTaskIndices().forEach(taskIndex -> - stageIdToTaskIdxToAttemptStates.get(stage.getId()).putIfAbsent(taskIndex, new ArrayList<>(10))); + stageIdToTaskIdxToAttemptStates.get(stage.getId()).putIfAbsent(taskIndex, new ArrayList<>(maxSubTaskSplitNum))); // task states will be initialized lazily in getTaskAttemptsToSchedule() }); } @@ -365,6 +370,7 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState }) .count(); if (newTaskState.equals(TaskState.State.COMPLETE)) { + // 여기 나중에 고쳐야 함 LOG.info("{} completed: {} Task(s) out of {} are remaining in this stage", taskId, taskStatesOfThisStage.size() - numOfCompletedTaskIndicesInThisStage, taskStatesOfThisStage.size()); } @@ -389,7 +395,7 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState } break; - // COMPLETE stage + // COMPLETE stage 여기도 고쳐야 함 case COMPLETE: case ON_HOLD: if (numOfCompletedTaskIndicesInThisStage From 8e9fb546e3eff337624c9babe9959e993a3c4ef7 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 21:20:07 +0900 Subject: [PATCH 12/26] add work stealing compile time pass --- .../EnableWorkStealingProperty.java | 31 ++++- .../annotating/WorkStealingPass.java | 116 +++++++++++++++++- .../optimizer/policy/WorkStealingPolicy.java | 24 +++- .../common/plan/PhysicalPlanGenerator.java | 4 +- .../nemo/runtime/master/PlanStateManager.java | 4 +- 5 files changed, 168 insertions(+), 11 deletions(-) diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java index a8c925bb14..1a501dd8a2 100644 --- a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java @@ -1,15 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.nemo.common.ir.vertex.executionproperty; import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; -public class EnableWorkStealingProperty - extends VertexExecutionProperty { +/** + * Marks Work Stealing Strategy of the vertex. + * + * Currently, there are three types: + * SPLIT : vertex which is the subject of work stealing + * MERGE : vertex which merges the effect of work stealing + * DEFAULT : vertex which is not the subject of work stealing + */ +public class EnableWorkStealingProperty extends VertexExecutionProperty { /** * Default constructor. * * @param value value of the VertexExecutionProperty. */ - public EnableWorkStealingProperty(String value) { + public EnableWorkStealingProperty(final String value) { super(value); } diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index 0204dee186..a97acacdd1 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -1,16 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating; +import org.apache.nemo.common.Pair; +import org.apache.nemo.common.dag.Edge; import org.apache.nemo.common.ir.IRDAG; +import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty; +import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.transform.Transform; +import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; +import org.apache.nemo.runtime.common.plan.StagePartitioner; + +import java.util.*; /** * Optimization pass for tagging parallelism execution property. */ @Annotates(EnableWorkStealingProperty.class) +@Requires(CommunicationPatternProperty.class) public class WorkStealingPass extends AnnotatingPass{ + private static final String SPLIT_STRATEGY = "SPLIT"; + private static final String MERGE_STRATEGY = "MERGE"; + private static final String DEFAULT_STRATEGY = "DEFAULT"; + private final StagePartitioner stagePartitioner = new StagePartitioner(); public WorkStealingPass() { super(WorkStealingPass.class); @@ -19,17 +50,94 @@ public WorkStealingPass() { @Override public IRDAG apply(IRDAG irdag) { irdag.topologicalDo(irVertex -> { - if (irVertex instanceof OperatorVertex) { + final boolean notConnectedToO2OEdge = irdag.getIncomingEdgesOf(irVertex).stream() + .map(edge -> edge.getPropertyValue(CommunicationPatternProperty.class).get()) + .noneMatch(property -> property.equals(CommunicationPatternProperty.Value.ONE_TO_ONE)); + if (irVertex instanceof OperatorVertex && notConnectedToO2OEdge) { Transform transform = ((OperatorVertex) irVertex).getTransform(); if (transform.toString().contains("work stealing")) { - irVertex.setProperty(EnableWorkStealingProperty.of("SPLIT")); + irVertex.setProperty(EnableWorkStealingProperty.of(SPLIT_STRATEGY)); } else if (transform.toString().contains("merge")){ - irVertex.setProperty(EnableWorkStealingProperty.of("MERGE")); + irVertex.setProperty(EnableWorkStealingProperty.of(MERGE_STRATEGY)); + } else { + irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + } + } else { + irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + } + }); + return tidyWorkStealingAnnotation(irdag); + } + + + /** + * Tidy annotated dag. + * Cleanup conditions: + * - The number of SPLIT annotations and MERGE annotations should be equal + * - SPLIT and MERGE should not be together in one stage, but needs to be in adjacent stage. + * - For now, nested work stealing optimizations are not provided. If detected, leave only the + * innermost pair. + * + * @param irdag irdag to cleanup. + * @return cleaned irdag. + */ + private IRDAG tidyWorkStealingAnnotation(IRDAG irdag) { + String splitVertexId = null; + + final List> splitMergePairs = new ArrayList<>(); + final Set pairedVertices = new HashSet<>(); + + // Make SPLIT - MERGE vertex pair. + for (IRVertex vertex : irdag.getTopologicalSort()) { + if (vertex.getPropertyValue(EnableWorkStealingProperty.class).get().equals(SPLIT_STRATEGY)) { + if (splitVertexId != null) { + // nested SPLIT vertex detected: delete the prior one. + irdag.getVertexById(splitVertexId).setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + } + splitVertexId = vertex.getId(); + } else if (vertex.getPropertyValue(EnableWorkStealingProperty.class).get().equals(MERGE_STRATEGY)) { + if (splitVertexId != null) { + splitMergePairs.add(Pair.of(splitVertexId, vertex.getId())); + pairedVertices.add(splitVertexId); + pairedVertices.add(vertex.getId()); + splitVertexId = null; } else { - irVertex.setProperty(EnableWorkStealingProperty.of("DEFAULT")); + // no corresponding SPLIT vertex: delete + vertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + } + } + } + + stagePartitioner.addIgnoredPropertyKey(EnableWorkStealingProperty.class); + final Map vertexToStageId = stagePartitioner.apply(irdag); + + for (Pair splitMergePair : splitMergePairs) { + IRVertex splitVertex = irdag.getVertexById(splitMergePair.left()); + IRVertex mergeVertex = irdag.getVertexById(splitMergePair.right()); + + if (vertexToStageId.get(splitVertex) >= vertexToStageId.get(mergeVertex) || + irdag.getIncomingEdgesOf(mergeVertex).stream() + .map(Edge::getSrc) + .map(vertexToStageId::get) + .noneMatch(stageId -> stageId.equals(vertexToStageId.get(splitVertex)))) { + // split vertex is descendent of merge vertex or they are in the same stage, + // or they are not in adjacent stages + splitVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + mergeVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + pairedVertices.remove(splitVertex.getId()); + pairedVertices.remove(mergeVertex.getId()); + } + } + + irdag.topologicalDo(vertex -> { + if (!vertex.getPropertyValue(EnableWorkStealingProperty.class) + .orElse(DEFAULT_STRATEGY).equals(DEFAULT_STRATEGY)) { + if (!pairedVertices.contains(vertex.getId())) { + vertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); } } }); + return irdag; } } diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java index ca6d3623a2..3a189c8542 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.nemo.compiler.optimizer.policy; import org.apache.nemo.common.ir.IRDAG; @@ -5,12 +23,14 @@ import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass; import org.apache.nemo.compiler.optimizer.pass.runtime.Message; -public class WorkStealingPolicy implements Policy { +/** + * Policy for work stealing. + */ +public final class WorkStealingPolicy implements Policy { public static final PolicyBuilder BUILDER = new PolicyBuilder() .registerCompileTimePass(new DefaultCompositePass()) .registerCompileTimePass(new WorkStealingPass()); - private final Policy policy; /** diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index fab9da8ec1..1f2073bc06 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -133,6 +133,8 @@ private void handleDuplicateEdgeGroupProperty(final DAG dagOfS */ public DAG stagePartitionIrDAG(final IRDAG irDAG) { final StagePartitioner stagePartitioner = new StagePartitioner(); + stagePartitioner.addIgnoredPropertyKey(EnableWorkStealingProperty.class); + final DAGBuilder dagOfStagesBuilder = new DAGBuilder<>(); final Set interStageEdges = new HashSet<>(); final Map stageIdToStageMap = new HashMap<>(); @@ -219,7 +221,7 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { stageInternalDAG, stageProperties, vertexIdToReadables, - isWorkStealingStage? 10 : 1); // ad-hoc for now + isWorkStealingStage ? 10 : 1); // ad-hoc for now dagOfStagesBuilder.addVertex(stage); stageIdToStageMap.put(stageId, stage); } diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 5c13d9f66d..fb7993e899 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -371,8 +371,10 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState .count(); if (newTaskState.equals(TaskState.State.COMPLETE)) { // 여기 나중에 고쳐야 함 + final int numOfTasksOfThisStage = taskStatesOfThisStage.values().stream() + .mapToInt(partialTasks -> partialTasks.size()).sum(); LOG.info("{} completed: {} Task(s) out of {} are remaining in this stage", - taskId, taskStatesOfThisStage.size() - numOfCompletedTaskIndicesInThisStage, taskStatesOfThisStage.size()); + taskId, numOfTasksOfThisStage - numOfCompletedTaskIndicesInThisStage, numOfTasksOfThisStage); } // Maintain info for speculative execution From 07ff1f9257fea9d3b3e4ec75d273235fdcb31c25 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 21:56:20 +0900 Subject: [PATCH 13/26] checkstyle and work stealing sub split pass --- .../WorkStealingSubSplitProperty.java | 45 +++++++++++++ .../annotating/WorkStealingPass.java | 21 +++--- .../annotating/WorkStealingSubSplitPass.java | 66 +++++++++++++++++++ .../optimizer/policy/WorkStealingPolicy.java | 4 +- .../common/plan/PhysicalPlanGenerator.java | 9 ++- .../runtime/common/plan/StagePartitioner.java | 10 +++ .../runtime/executor/task/TaskExecutor.java | 2 +- .../nemo/runtime/master/PlanStateManager.java | 2 +- 8 files changed, 142 insertions(+), 17 deletions(-) create mode 100644 common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingSubSplitProperty.java create mode 100644 compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingSubSplitProperty.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingSubSplitProperty.java new file mode 100644 index 0000000000..fe9db872a7 --- /dev/null +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingSubSplitProperty.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.nemo.common.ir.vertex.executionproperty; + +import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; + +/** + * Property to store the sub-split number of work stealing tasks. + */ +public class WorkStealingSubSplitProperty extends VertexExecutionProperty { + /** + * Default constructor. + * + * @param value value of the VertexExecutionProperty. + */ + public WorkStealingSubSplitProperty(final Integer value) { + super(value); + } + + /** + * Static method exposing the constructor. + * + * @param value value of the new execution property. + * @return the newly created execution property. + */ + public static WorkStealingSubSplitProperty of(final Integer value) { + return new WorkStealingSubSplitProperty(value); + } +} diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index a97acacdd1..4211c43c5f 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -32,11 +32,11 @@ import java.util.*; /** - * Optimization pass for tagging parallelism execution property. + * Optimization pass for tagging enable work stealing execution property. */ @Annotates(EnableWorkStealingProperty.class) @Requires(CommunicationPatternProperty.class) -public class WorkStealingPass extends AnnotatingPass{ +public final class WorkStealingPass extends AnnotatingPass { private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; @@ -48,7 +48,7 @@ public WorkStealingPass() { } @Override - public IRDAG apply(IRDAG irdag) { + public IRDAG apply(final IRDAG irdag) { irdag.topologicalDo(irVertex -> { final boolean notConnectedToO2OEdge = irdag.getIncomingEdgesOf(irVertex).stream() .map(edge -> edge.getPropertyValue(CommunicationPatternProperty.class).get()) @@ -57,7 +57,7 @@ public IRDAG apply(IRDAG irdag) { Transform transform = ((OperatorVertex) irVertex).getTransform(); if (transform.toString().contains("work stealing")) { irVertex.setProperty(EnableWorkStealingProperty.of(SPLIT_STRATEGY)); - } else if (transform.toString().contains("merge")){ + } else if (transform.toString().contains("merge")) { irVertex.setProperty(EnableWorkStealingProperty.of(MERGE_STRATEGY)); } else { irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); @@ -81,7 +81,7 @@ public IRDAG apply(IRDAG irdag) { * @param irdag irdag to cleanup. * @return cleaned irdag. */ - private IRDAG tidyWorkStealingAnnotation(IRDAG irdag) { + private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { String splitVertexId = null; final List> splitMergePairs = new ArrayList<>(); @@ -108,18 +108,17 @@ private IRDAG tidyWorkStealingAnnotation(IRDAG irdag) { } } - stagePartitioner.addIgnoredPropertyKey(EnableWorkStealingProperty.class); final Map vertexToStageId = stagePartitioner.apply(irdag); for (Pair splitMergePair : splitMergePairs) { IRVertex splitVertex = irdag.getVertexById(splitMergePair.left()); IRVertex mergeVertex = irdag.getVertexById(splitMergePair.right()); - if (vertexToStageId.get(splitVertex) >= vertexToStageId.get(mergeVertex) || - irdag.getIncomingEdgesOf(mergeVertex).stream() - .map(Edge::getSrc) - .map(vertexToStageId::get) - .noneMatch(stageId -> stageId.equals(vertexToStageId.get(splitVertex)))) { + if (vertexToStageId.get(splitVertex) >= vertexToStageId.get(mergeVertex) + || irdag.getIncomingEdgesOf(mergeVertex).stream() + .map(Edge::getSrc) + .map(vertexToStageId::get) + .noneMatch(stageId -> stageId.equals(vertexToStageId.get(splitVertex)))) { // split vertex is descendent of merge vertex or they are in the same stage, // or they are not in adjacent stages splitVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java new file mode 100644 index 0000000000..7e52c447ac --- /dev/null +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating; + +import org.apache.nemo.common.ir.IRDAG; +import org.apache.nemo.common.ir.edge.IREdge; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; +import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; + +/** + * Optimization pass for tagging work stealing sub-split execution property. + */ +@Annotates(WorkStealingSubSplitProperty.class) +@Requires({EnableWorkStealingProperty.class, ParallelismProperty.class}) +public final class WorkStealingSubSplitPass extends AnnotatingPass { + private static final String SPLIT_STRATEGY = "SPLIT"; + private static final String DEFAULT_STRATEGY = "DEFAULT"; + + private static final int MAX_SUB_SPLIT_NUM = 10; + private static final int DEFAULT_SUB_SPLIT_NUM = 1; + /** + * Default Constructor. + */ + public WorkStealingSubSplitPass() { + super(WorkStealingSubSplitPass.class); + } + + @Override + public IRDAG apply(final IRDAG irdag) { + irdag.topologicalDo(vertex -> { + if (vertex.getPropertyValue(EnableWorkStealingProperty.class) + .orElse(DEFAULT_STRATEGY).equals(SPLIT_STRATEGY)) { + int maxSourceParallelism = irdag.getIncomingEdgesOf(vertex).stream().map(IREdge::getSrc) + .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(DEFAULT_SUB_SPLIT_NUM)) + .max().orElse(DEFAULT_SUB_SPLIT_NUM); + if (maxSourceParallelism > MAX_SUB_SPLIT_NUM) { + vertex.setProperty(WorkStealingSubSplitProperty.of(DEFAULT_SUB_SPLIT_NUM)); + } else { + vertex.setProperty(WorkStealingSubSplitProperty.of(maxSourceParallelism)); + } + } else { + vertex.setProperty(WorkStealingSubSplitProperty.of(DEFAULT_SUB_SPLIT_NUM)); + } + }); + + return irdag; + } +} diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java index 3a189c8542..ba5219b3e8 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java @@ -20,6 +20,7 @@ import org.apache.nemo.common.ir.IRDAG; import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingPass; +import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingSubSplitPass; import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass; import org.apache.nemo.compiler.optimizer.pass.runtime.Message; @@ -30,7 +31,8 @@ public final class WorkStealingPolicy implements Policy { public static final PolicyBuilder BUILDER = new PolicyBuilder() .registerCompileTimePass(new DefaultCompositePass()) - .registerCompileTimePass(new WorkStealingPass()); + .registerCompileTimePass(new WorkStealingPass()) + .registerCompileTimePass(new WorkStealingSubSplitPass()); private final Policy policy; /** diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index 1f2073bc06..b82294bd7b 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -35,6 +35,7 @@ import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.common.ir.vertex.utility.SamplingVertex; import org.apache.nemo.conf.JobConf; import org.apache.nemo.runtime.common.RuntimeIdManager; @@ -133,7 +134,6 @@ private void handleDuplicateEdgeGroupProperty(final DAG dagOfS */ public DAG stagePartitionIrDAG(final IRDAG irDAG) { final StagePartitioner stagePartitioner = new StagePartitioner(); - stagePartitioner.addIgnoredPropertyKey(EnableWorkStealingProperty.class); final DAGBuilder dagOfStagesBuilder = new DAGBuilder<>(); final Set interStageEdges = new HashSet<>(); @@ -212,16 +212,19 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { final DAG> stageInternalDAG = stageInternalDAGBuilder.buildWithoutSourceSinkCheck(); // check if this stage is subject of work stealing optimization - boolean isWorkStealingStage = stageInternalDAG.getRootVertices().stream() + boolean isWorkStealingStage = stageInternalDAG.getVertices().stream() .anyMatch(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class) .orElse("DEFAULT").equals("SPLIT")); + int numSubSplit = stageInternalDAG.getVertices().stream() + .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(1)) + .max().orElse(1); final Stage stage = new Stage( stageIdentifier, taskIndices, stageInternalDAG, stageProperties, vertexIdToReadables, - isWorkStealingStage ? 10 : 1); // ad-hoc for now + isWorkStealingStage ? numSubSplit : 1); dagOfStagesBuilder.addVertex(stage); stageIdToStageMap.put(stageId, stage); } diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java index bc20448220..7c24e01eff 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java @@ -25,6 +25,8 @@ import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty; import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; import org.apache.nemo.common.ir.vertex.IRVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.reef.annotations.audience.DriverSide; import java.util.HashMap; @@ -49,6 +51,14 @@ public final class StagePartitioner implements Function> ignoredPropertyKeys = ConcurrentHashMap.newKeySet(); private final MutableInt nextStageIndex = new MutableInt(0); + /** + * Default Constructor. + */ + public StagePartitioner() { + addIgnoredPropertyKey(EnableWorkStealingProperty.class); + addIgnoredPropertyKey(WorkStealingSubSplitProperty.class); + } + /** * By default, the stage partitioner merges two vertices into one stage if and only if the two vertices have * same set of {@link VertexExecutionProperty}. diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java index c94018540c..a645ce8f6c 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java @@ -718,7 +718,7 @@ private void finalizeOutputWriters(final VertexHarness vertexHarness) { SerializationUtils.serialize(totalWrittenBytes)); } - private String getWorkStealingStrategy(DAG> irVertexDag) { + private String getWorkStealingStrategy(final DAG> irVertexDag) { Set strategy = irVertexDag.getVertices().stream() .map(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class).orElse("DEFAULT")) .collect(Collectors.toSet()); diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index fb7993e899..c5a726e11c 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -103,7 +103,7 @@ public final class PlanStateManager { private MetricStore metricStore; /** - * For dynamic optimization + * For dynamic optimization. */ private final int maxSubTaskSplitNum = 10; From e25355ae5db5c5715e21ad475bed67ca24ba9e31 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sun, 22 Aug 2021 22:07:15 +0900 Subject: [PATCH 14/26] change block output writer --- .../nemo/runtime/common/RuntimeIdManager.java | 19 ++++++++----------- .../datatransfer/BlockOutputWriter.java | 4 ++-- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 74e619955e..fddf26764b 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -102,8 +102,14 @@ public static String generateExecutorId() { */ public static String generateBlockId(final String runtimeEdgeId, final String producerTaskId) { - return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) - + SPLITTER + getAttemptFromTaskId(producerTaskId); + if (isWorkStealingTask(producerTaskId)) { + return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) + + SPLITTER + getPartialFromTaskId(producerTaskId) + + SPLITTER + getAttemptFromTaskId(producerTaskId); + } else { + return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) + + SPLITTER + getAttemptFromTaskId(producerTaskId); + } } /** @@ -123,15 +129,6 @@ public static String generateBlockIdWildcard(final String runtimeEdgeId, return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + "*"; } - public static String generateWorkStealingBlockId(final String runtimeEdgeId, - final String producerTaskId, - final String consumerTaskId) { - return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) - + SPLITTER + getAttemptFromTaskId(producerTaskId) - + SPLITTER + getIndexFromTaskId(consumerTaskId) - + SPLITTER + getAttemptFromTaskId(consumerTaskId); - } - /** * Generates the ID for a control message. * diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockOutputWriter.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockOutputWriter.java index 97cde037c6..2e4add6651 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockOutputWriter.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockOutputWriter.java @@ -72,11 +72,11 @@ public final class BlockOutputWriter implements OutputWriter { this.blockManagerWorker = blockManagerWorker; this.blockStoreValue = runtimeEdge.getPropertyValue(DataStoreProperty.class) .orElseThrow(() -> new RuntimeException("No data store property on the edge")); - blockToWrite = blockManagerWorker.createBlock( + this.blockToWrite = blockManagerWorker.createBlock( RuntimeIdManager.generateBlockId(runtimeEdge.getId(), srcTaskId), blockStoreValue); final Optional duplicateDataProperty = runtimeEdge.getPropertyValue(DuplicateEdgeGroupProperty.class); - nonDummyBlock = !duplicateDataProperty.isPresent() + this.nonDummyBlock = !duplicateDataProperty.isPresent() || duplicateDataProperty.get().getRepresentativeEdgeId().equals(runtimeEdge.getId()) || duplicateDataProperty.get().getGroupSize() <= 1; } From 87f65c16d988c910f82790cc63189879c1aab19e Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 00:21:06 +0900 Subject: [PATCH 15/26] modify block input reader to support SPLIT and MERGE --- .../annotating/WorkStealingPass.java | 19 ++++ .../annotating/WorkStealingSubSplitPass.java | 38 ++++++- .../nemo/runtime/common/RuntimeIdManager.java | 29 +++++- .../datatransfer/BlockInputReader.java | 98 ++++++++++++++----- .../executor/datatransfer/InputReader.java | 2 +- .../datatransfer/PipeInputReader.java | 2 +- .../executor/task/ParentTaskDataFetcher.java | 19 ++-- .../runtime/executor/task/TaskExecutor.java | 13 ++- .../task/ParentTaskDataFetcherTest.java | 5 +- .../runtime/master/BlockManagerMaster.java | 2 +- 10 files changed, 187 insertions(+), 40 deletions(-) diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index 4211c43c5f..0a664a9dcb 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -86,6 +86,7 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { final List> splitMergePairs = new ArrayList<>(); final Set pairedVertices = new HashSet<>(); + final Map> stageIdToStageVertices = new HashMap<>(); // Make SPLIT - MERGE vertex pair. for (IRVertex vertex : irdag.getTopologicalSort()) { @@ -137,6 +138,24 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { } }); + // update execution property of other vertices in same stage. + vertexToStageId.forEach((vertex, stageId) -> { + if (!stageIdToStageVertices.containsKey(stageId)) { + stageIdToStageVertices.put(stageId, new HashSet<>()); + } + stageIdToStageVertices.get(stageId).add(vertex); + }); + + for (String vertexId : pairedVertices) { + IRVertex vertex = irdag.getVertexById(vertexId); + Set stageVertices = stageIdToStageVertices.get(vertexToStageId.get(vertex)); + String strategy = vertex.getPropertyValue(EnableWorkStealingProperty.class) + .orElse(DEFAULT_STRATEGY); + for (IRVertex stageVertex : stageVertices) { + stageVertex.setProperty(EnableWorkStealingProperty.of(strategy)); + } + } + return irdag; } } diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java index 7e52c447ac..35e94792db 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -20,10 +20,17 @@ import org.apache.nemo.common.ir.IRDAG; import org.apache.nemo.common.ir.edge.IREdge; +import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; +import org.apache.nemo.runtime.common.plan.StagePartitioner; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; /** * Optimization pass for tagging work stealing sub-split execution property. @@ -32,10 +39,13 @@ @Requires({EnableWorkStealingProperty.class, ParallelismProperty.class}) public final class WorkStealingSubSplitPass extends AnnotatingPass { private static final String SPLIT_STRATEGY = "SPLIT"; + private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; private static final int MAX_SUB_SPLIT_NUM = 10; private static final int DEFAULT_SUB_SPLIT_NUM = 1; + + private final StagePartitioner stagePartitioner = new StagePartitioner(); /** * Default Constructor. */ @@ -45,7 +55,9 @@ public WorkStealingSubSplitPass() { @Override public IRDAG apply(final IRDAG irdag) { - irdag.topologicalDo(vertex -> { + final Map vertexToSplitNum = new HashMap<>(); + + for (IRVertex vertex : irdag.getTopologicalSort()) { if (vertex.getPropertyValue(EnableWorkStealingProperty.class) .orElse(DEFAULT_STRATEGY).equals(SPLIT_STRATEGY)) { int maxSourceParallelism = irdag.getIncomingEdgesOf(vertex).stream().map(IREdge::getSrc) @@ -55,12 +67,34 @@ public IRDAG apply(final IRDAG irdag) { vertex.setProperty(WorkStealingSubSplitProperty.of(DEFAULT_SUB_SPLIT_NUM)); } else { vertex.setProperty(WorkStealingSubSplitProperty.of(maxSourceParallelism)); + vertexToSplitNum.put(vertex, maxSourceParallelism); } } else { vertex.setProperty(WorkStealingSubSplitProperty.of(DEFAULT_SUB_SPLIT_NUM)); } - }); + } + updateParallelismProperty(irdag, vertexToSplitNum); return irdag; } + + private void updateParallelismProperty(IRDAG irdag, Map vertexToSplitNum) { + final Map vertexToStageId = stagePartitioner.apply(irdag); + + final Map> stageIdToStageVertices = new HashMap<>(); + vertexToStageId.forEach((vertex, stageId) -> { + if (!stageIdToStageVertices.containsKey(stageId)) { + stageIdToStageVertices.put(stageId, new HashSet<>()); + } + stageIdToStageVertices.get(stageId).add(vertex); + }); + + for (IRVertex vertex : vertexToSplitNum.keySet()) { + int numSubSplit = vertexToSplitNum.get(vertex); + for (IRVertex stageVertex : stageIdToStageVertices.get(vertexToStageId.get(vertex))) { + int currentParallelism = stageVertex.getPropertyValue(ParallelismProperty.class).get(); + stageVertex.setProperty(ParallelismProperty.of(currentParallelism * numSubSplit)); + } + } + } } diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index fddf26764b..6465bec5cd 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -125,8 +125,15 @@ public static String generateBlockId(final String runtimeEdgeId, * @return the generated WILDCARD ID */ public static String generateBlockIdWildcard(final String runtimeEdgeId, - final int producerTaskIndex) { - return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + "*"; + final int producerTaskIndex, + final String subSplitIndex) { + if (!subSplitIndex.equals("*")) { + return runtimeEdgeId + SPLITTER + producerTaskIndex + + SPLITTER + subSplitIndex + SPLITTER + "*"; + } else { + return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + "*"; + } + } /** @@ -139,6 +146,9 @@ public static long generateMessageId() { } //////////////////////////////////////////////////////////////// Parse IDs + public static boolean isWorkStealingBlock(final String blockId) { + return split(blockId).length == 4; + } /** * Extracts runtime edge ID from a block ID. @@ -160,6 +170,17 @@ public static int getTaskIndexFromBlockId(final String blockId) { return Integer.valueOf(split(blockId)[1]); } + /** + * + */ + public static String getSubSplitIndexFromBlockId(final String blockId) { + if (isWorkStealingBlock(blockId)) { + return split(blockId)[2]; + } else { + return "*"; + } + } + /** * Extracts wild card from a block ID. * @@ -167,7 +188,9 @@ public static int getTaskIndexFromBlockId(final String blockId) { * @return the wild card. */ public static String getWildCardFromBlockId(final String blockId) { - return generateBlockIdWildcard(getRuntimeEdgeIdFromBlockId(blockId), getTaskIndexFromBlockId(blockId)); + return generateBlockIdWildcard(getRuntimeEdgeIdFromBlockId(blockId), + getTaskIndexFromBlockId(blockId), + getSubSplitIndexFromBlockId(blockId)); } /** diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 5bd0a8bbbb..34dd54e0dd 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -30,6 +30,8 @@ import org.apache.nemo.common.ir.executionproperty.EdgeExecutionProperty; import org.apache.nemo.common.ir.executionproperty.ExecutionPropertyMap; import org.apache.nemo.common.ir.vertex.IRVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.runtime.common.RuntimeIdManager; import org.apache.nemo.runtime.common.plan.RuntimeEdge; import org.apache.nemo.runtime.common.plan.StageEdge; @@ -53,6 +55,10 @@ public final class BlockInputReader implements InputReader { private final String dstTaskId; private final int dstTaskIndex; + private static final String SPLIT_STRATEGY = "SPLIT"; + private static final String MERGE_STRATEGY = "MERGE"; + private static final String DEFAULT_STRATEGY = "DEFAULT"; + /** * Attributes that specify how we should read the input. */ @@ -90,18 +96,52 @@ public List> read() { } } - public List> read(final boolean enableWorkStealing, - final int maxSplitNum, - final int index) { - /* First, fetch */ - List> futures = read(); + @Override + public List> read(final String workStealingState, + final int numSubSplit, + final int subSplitIndex) { + if (workStealingState.equals(MERGE_STRATEGY) && + srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + .equals(SPLIT_STRATEGY)) { + return readSplitBlocks(InputReader.getSourceParallelism(this), + srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); + } else { + List> futures = read(); + + if (!workStealingState.equals(SPLIT_STRATEGY)) { + /* DEFAULT case */ + return futures; + } - if (!enableWorkStealing) { - return futures; + /* SPLIT case*/ + return futures.subList(subSplitIndex * (futures.size() / numSubSplit), + (subSplitIndex + 1) * (futures.size() / numSubSplit)); } - final int baseNumber = Math.min(futures.size(), maxSplitNum); - return futures.subList(index * (futures.size() / baseNumber), (index + 1) * (futures.size() / baseNumber)); + } + + /** + * Read sub-split blocks generated by work stealing SPLIT stage. + * @param srcParallelism src stage parallelism. + * @param numSubSplit number of sub-split blocks per src task index. + * @return List of iterators. + */ + private List> readSplitBlocks(final int srcParallelism, + final int numSubSplit) { + final Optional comValueOptional = + runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); + final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); + + switch (comValue) { + case ONE_TO_ONE: + return Collections.singletonList(readOneToOne()); + case BROADCAST: + return checkSingleElement(readBroadcast(index -> index == desiredIndex)); + case SHUFFLE: + return checkSingleElement(readDataInRange(index -> index == desiredIndex)); + default: + throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); + } } @Override @@ -141,47 +181,54 @@ private CompletableFuture checkSingleElement( } /** - * See {@link RuntimeIdManager#generateBlockIdWildcard(String, int)} for information on block wildcards. + * See {@link RuntimeIdManager#generateBlockIdWildcard(String, int, String)} for information on block wildcards. * * @param producerTaskIndex to use. * @return wildcard block id that corresponds to "ANY" task attempt of the task index. */ - private String generateWildCardBlockId(final int producerTaskIndex) { + private String generateWildCardBlockId(final int producerTaskIndex, + final String subSplitIndex) { final Optional duplicateDataProperty = runtimeEdge.getPropertyValue(DuplicateEdgeGroupProperty.class); if (!duplicateDataProperty.isPresent() || duplicateDataProperty.get().getGroupSize() <= 1) { - return RuntimeIdManager.generateBlockIdWildcard(runtimeEdge.getId(), producerTaskIndex); + return RuntimeIdManager.generateBlockIdWildcard(runtimeEdge.getId(), producerTaskIndex, subSplitIndex); } final String duplicateEdgeId = duplicateDataProperty.get().getRepresentativeEdgeId(); - return RuntimeIdManager.generateBlockIdWildcard(duplicateEdgeId, producerTaskIndex); + return RuntimeIdManager.generateBlockIdWildcard(duplicateEdgeId, producerTaskIndex, subSplitIndex); } private CompletableFuture readOneToOne() { - final String blockIdWildcard = generateWildCardBlockId(dstTaskIndex); + final String blockIdWildcard = generateWildCardBlockId(dstTaskIndex, "1"); return blockManagerWorker.readBlock( blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all()); } - private List> readBroadcast(final Predicate predicate) { + private List> readBroadcast(final Predicate predicate, + final int numSubSplit) { final int numSrcTasks = InputReader.getSourceParallelism(this); final List> futures = new ArrayList<>(); for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) { if (predicate.test(srcTaskIdx)) { - final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx); - futures.add(blockManagerWorker.readBlock( - blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all())); + for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { + final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx, + numSubSplit == 1 ? "*": Integer.toString(subSplitIdx)); + futures.add(blockManagerWorker.readBlock( + blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all())); + } } } - return futures; } /** * Read data in the assigned range of hash value. * + * if @param{numSubSplit} > 1, it indicates that it is reading sub-split blocks from work stealing. + * * @return the list of the completable future of the data. */ - private List> readDataInRange(final Predicate predicate) { + private List> readDataInRange(final Predicate predicate, + final int numSubSplit) { assert (runtimeEdge instanceof StageEdge); final List keyRangeList = ((StageEdge) runtimeEdge).getKeyRanges(); final KeyRange hashRangeToRead = keyRangeList.get(dstTaskIndex); @@ -197,10 +244,13 @@ private List> readDataInRange(f final int numSrcTasks = InputReader.getSourceParallelism(this); final List> futures = new ArrayList<>(); for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) { - if (predicate.test(srcTaskIdx)) { - final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx); - futures.add(blockManagerWorker.readBlock( - blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), hashRangeToRead)); + for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { + if (predicate.test(srcTaskIdx)) { + final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx, + numSubSplit == 1 ? "*": Integer.toString(subSplitIdx)); + futures.add(blockManagerWorker.readBlock( + blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), hashRangeToRead)); + } } } return futures; diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java index 04979714c5..60cc1b9776 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java @@ -42,7 +42,7 @@ public interface InputReader { * * @return the list of iterators. */ - List> read(boolean enableWorkStealing, + List> read(String workStealingState, int maxSplitNum, int index); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java index 64c40e7f32..89061129a3 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java @@ -84,7 +84,7 @@ public List> read() { } @Override - public List> read(final boolean enableWorkStealing, + public List> read(final String workStealingState, final int maxSplitNum, final int index) { return read(); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index 505de4e93e..d6c21cd107 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -53,22 +53,24 @@ class ParentTaskDataFetcher extends DataFetcher { private long serBytes = 0; private long encodedBytes = 0; - private final int magicSplitNum = 10; - private final boolean enableWorkStealing; + private final int subSplitNum; + private final String workStealingState; ParentTaskDataFetcher(final IRVertex dataSource, final InputReader inputReader, final OutputCollector outputCollector, + final String workStealingState, + final int subSplitNum, final String taskId) { super(dataSource, outputCollector); - this.inputReader = inputReader; this.taskId = taskId; + this.inputReader = inputReader; + this.workStealingState = workStealingState; + this.subSplitNum = subSplitNum; this.firstFetch = true; this.currentIteratorIndex = 0; this.iteratorQueue = new LinkedBlockingQueue<>(); - - this.enableWorkStealing = RuntimeIdManager.isWorkStealingTask(taskId); } @Override @@ -166,8 +168,9 @@ private void handleIncomingBlock(final int index, private void fetchDataLazily() { final List> futures = inputReader - .read(enableWorkStealing, magicSplitNum, RuntimeIdManager.getPartialFromTaskId(taskId)); + .read(workStealingState, subSplitNum, RuntimeIdManager.getPartialFromTaskId(taskId)); this.expectedNumOfIterators = futures.size(); + // 여기도 고쳐야 할 듯. index가 기존의 표현이랑 좀 달라지니까. for (int i = 0; i < futures.size(); i++) { final int index = i; final CompletableFuture future = futures.get(i); @@ -206,4 +209,8 @@ private void countBytes(final DataUtil.IteratorWithNumBytes iterator) { public void close() throws Exception { } + + private int translateIndex(final int index) { + + } } diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java index a645ce8f6c..b36b452a2d 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java @@ -31,6 +31,7 @@ import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.SourceVertex; import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform; import org.apache.nemo.common.ir.vertex.transform.SignalTransform; import org.apache.nemo.common.ir.vertex.transform.Transform; @@ -290,11 +291,21 @@ irVertex, outputCollector, new TransformContextImpl(broadcastManagerWorker), parentTaskReader, dataFetcherOutputCollector)); } else { + final String workStealingState = irVertexDag.getVertices().stream() + .map(v -> v.getPropertyValue(EnableWorkStealingProperty.class).orElse("DEFAULT")) + .filter(s -> !s.equals("DEFAULT")) + .findFirst().orElse("DEFAULT"); + final int numSubSplit = irVertexDag.getVertices().stream() + .mapToInt(v -> v.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)) + .max().orElse(1); dataFetcherList.add( new ParentTaskDataFetcher( parentTaskReader.getSrcIrVertex(), parentTaskReader, - dataFetcherOutputCollector, taskId)); + dataFetcherOutputCollector, + workStealingState, + numSubSplit, + taskId)); } } }); diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java index c2b9c9125b..3a35b4f5a9 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java @@ -141,7 +141,10 @@ private ParentTaskDataFetcher createFetcher(final InputReader readerForParentTas return new ParentTaskDataFetcher( mock(IRVertex.class), readerForParentTask, // This is the only argument that affects the behavior of ParentTaskDataFetcher - mock(OutputCollector.class), ""); + mock(OutputCollector.class), + "DEFAULT", + 1, + ""); } private InputReader generateInputReader(final CompletableFuture completableFuture, diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/BlockManagerMaster.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/BlockManagerMaster.java index 410ae7168e..7db30cb30b 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/BlockManagerMaster.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/BlockManagerMaster.java @@ -58,7 +58,7 @@ public final class BlockManagerMaster { private final Map> producerTaskIdToBlockIds; // a task can have multiple out-edges /** - * See {@link RuntimeIdManager#generateBlockIdWildcard(String, int)} for information on block wildcards. + * See {@link RuntimeIdManager#generateBlockIdWildcard(String, String)} for information on block wildcards. */ private final Map> blockIdWildcardToMetadataSet; // a metadata = a task attempt output From c992465dfba28267613a87d50329939b30f21b8d Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 00:45:57 +0900 Subject: [PATCH 16/26] update block input reader: still need to implement retry --- .../executor/datatransfer/BlockInputReader.java | 16 +++++++++------- .../executor/task/ParentTaskDataFetcher.java | 8 +++++++- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 34dd54e0dd..4de7546b0e 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -88,9 +88,9 @@ public List> read() { case ONE_TO_ONE: return Collections.singletonList(readOneToOne()); case BROADCAST: - return readBroadcast(index -> true); + return readBroadcast(index -> true, Optional.empty(), 1); case SHUFFLE: - return readDataInRange(index -> true); + return readDataInRange(index -> true, Optional.empty(), 1); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); } @@ -103,7 +103,7 @@ public List> read(final String if (workStealingState.equals(MERGE_STRATEGY) && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { - return readSplitBlocks(InputReader.getSourceParallelism(this), + return readSplitBlocks(InputReader.getSourceParallelism(this) / numSubSplit, srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); } else { List> futures = read(); @@ -136,9 +136,9 @@ private List> readSplitBlocks(f case ONE_TO_ONE: return Collections.singletonList(readOneToOne()); case BROADCAST: - return checkSingleElement(readBroadcast(index -> index == desiredIndex)); + return readBroadcast(index -> true, Optional.of(srcParallelism), numSubSplit); case SHUFFLE: - return checkSingleElement(readDataInRange(index -> index == desiredIndex)); + return readDataInRange(index -> true, Optional.of(srcParallelism), numSubSplit); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); } @@ -204,8 +204,9 @@ private CompletableFuture readOneToOne() { } private List> readBroadcast(final Predicate predicate, + final Optional numSrcIndex, final int numSubSplit) { - final int numSrcTasks = InputReader.getSourceParallelism(this); + final int numSrcTasks = numSrcIndex.orElse(InputReader.getSourceParallelism(this)); final List> futures = new ArrayList<>(); for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) { if (predicate.test(srcTaskIdx)) { @@ -228,6 +229,7 @@ private List> readBroadcast(fin * @return the list of the completable future of the data. */ private List> readDataInRange(final Predicate predicate, + final Optional numSrcIndex, final int numSubSplit) { assert (runtimeEdge instanceof StageEdge); final List keyRangeList = ((StageEdge) runtimeEdge).getKeyRanges(); @@ -241,7 +243,7 @@ private List> readDataInRange(f - ((HashRange) hashRangeToRead).rangeBeginInclusive(); metricMessageSender.send("TaskMetric", dstTaskId, "taskSizeRatio", SerializationUtils.serialize(partitionerProperty / taskSize)); - final int numSrcTasks = InputReader.getSourceParallelism(this); + final int numSrcTasks = numSrcIndex.orElse(InputReader.getSourceParallelism(this)); final List> futures = new ArrayList<>(); for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) { for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index d6c21cd107..02bd1c288c 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -172,7 +172,7 @@ private void fetchDataLazily() { this.expectedNumOfIterators = futures.size(); // 여기도 고쳐야 할 듯. index가 기존의 표현이랑 좀 달라지니까. for (int i = 0; i < futures.size(); i++) { - final int index = i; + final int index = translateIndex(i); final CompletableFuture future = futures.get(i); future.whenComplete((iterator, exception) -> { handleIncomingBlock(index, future); @@ -211,6 +211,12 @@ public void close() throws Exception { } private int translateIndex(final int index) { + if (workStealingState.equals("SPLIT")) { + } else if (workStealingState.equals("MERGE")) { + + } else { + return index; + } } } From b348f7248ec2ffe3b51193c3f3e1b077c2995166 Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 02:33:39 +0900 Subject: [PATCH 17/26] add retry --- .../datatransfer/BlockInputReader.java | 105 ++++++++++++++---- .../executor/datatransfer/InputReader.java | 11 ++ .../executor/task/ParentTaskDataFetcher.java | 16 +-- 3 files changed, 100 insertions(+), 32 deletions(-) diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 4de7546b0e..d226613151 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -103,7 +103,7 @@ public List> read(final String if (workStealingState.equals(MERGE_STRATEGY) && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { - return readSplitBlocks(InputReader.getSourceParallelism(this) / numSubSplit, + return readSplitBlocks(InputReader.getSourceParallelism(this), srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); } else { List> futures = read(); @@ -114,38 +114,51 @@ public List> read(final String } /* SPLIT case*/ + // 가능한 even 하게 나누는 방법 생각해서 보완 + // 19를 10개로 나누는 건 1*9, 10*1 보다는 2*9, 1*1 이 나음' return futures.subList(subSplitIndex * (futures.size() / numSubSplit), - (subSplitIndex + 1) * (futures.size() / numSubSplit)); + Math.min((subSplitIndex + 1) * (futures.size() / numSubSplit), futures.size())); } } - /** - * Read sub-split blocks generated by work stealing SPLIT stage. - * @param srcParallelism src stage parallelism. - * @param numSubSplit number of sub-split blocks per src task index. - * @return List of iterators. - */ - private List> readSplitBlocks(final int srcParallelism, - final int numSubSplit) { + + @Override + public CompletableFuture retry(final int desiredIndex) { final Optional comValueOptional = runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); switch (comValue) { case ONE_TO_ONE: - return Collections.singletonList(readOneToOne()); + return readOneToOne(); case BROADCAST: - return readBroadcast(index -> true, Optional.of(srcParallelism), numSubSplit); + return checkSingleElement(readBroadcast(index -> index == desiredIndex, Optional.empty(), 1)); case SHUFFLE: - return readDataInRange(index -> true, Optional.of(srcParallelism), numSubSplit); + return checkSingleElement(readDataInRange(index -> index == desiredIndex, Optional.empty(), 1)); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); } } @Override - public CompletableFuture retry(final int desiredIndex) { + public CompletableFuture retry(final String workStealingState, + final int numSubSplit, + final int desiredIndex) { + + final boolean isMergeAfterSplit = workStealingState.equals(MERGE_STRATEGY) && + srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + .equals(SPLIT_STRATEGY); + + if (!isMergeAfterSplit && !workStealingState.equals(SPLIT_STRATEGY)) { + return retry(desiredIndex); + } + + final int srcParallelism = InputReader.getSourceParallelism(this); + final int srcNumSubSplit = srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1); + final int trueIndex = translateIndex(workStealingState, numSubSplit, desiredIndex); + final int subIndex = isMergeAfterSplit ? desiredIndex % srcNumSubSplit : 0; + final Optional comValueOptional = runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); @@ -154,9 +167,13 @@ public CompletableFuture retry(final int desiredI case ONE_TO_ONE: return readOneToOne(); case BROADCAST: - return checkSingleElement(readBroadcast(index -> index == desiredIndex)); + return readBroadcast(index -> index == trueIndex, + isMergeAfterSplit ? Optional.of(srcParallelism / srcNumSubSplit) : Optional.empty(), + isMergeAfterSplit ? srcNumSubSplit : 1).get(subIndex); case SHUFFLE: - return checkSingleElement(readDataInRange(index -> index == desiredIndex)); + return readDataInRange(index -> index == trueIndex, + isMergeAfterSplit ? Optional.of(srcParallelism / srcNumSubSplit) : Optional.empty(), + isMergeAfterSplit ? srcNumSubSplit : 1).get(subIndex); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); } @@ -203,6 +220,12 @@ private CompletableFuture readOneToOne() { blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all()); } + /** + * Read data in full range of hash value. + * if @param{numSubSplit} > 1, it indicates that it is reading sub-split blocks from work stealing. + * + * @return the list of the completable future of the data. + */ private List> readBroadcast(final Predicate predicate, final Optional numSrcIndex, final int numSubSplit) { @@ -223,7 +246,6 @@ private List> readBroadcast(fin /** * Read data in the assigned range of hash value. - * * if @param{numSubSplit} > 1, it indicates that it is reading sub-split blocks from work stealing. * * @return the list of the completable future of the data. @@ -246,10 +268,10 @@ private List> readDataInRange(f final int numSrcTasks = numSrcIndex.orElse(InputReader.getSourceParallelism(this)); final List> futures = new ArrayList<>(); for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) { - for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { - if (predicate.test(srcTaskIdx)) { + if (predicate.test(srcTaskIdx)) { + for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx, - numSubSplit == 1 ? "*": Integer.toString(subSplitIdx)); + numSubSplit == 1 ? "*" : Integer.toString(subSplitIdx)); futures.add(blockManagerWorker.readBlock( blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), hashRangeToRead)); } @@ -257,4 +279,47 @@ private List> readDataInRange(f } return futures; } + + /** + * Read sub-split blocks generated by work stealing SPLIT stage. + * @param srcParallelism src stage parallelism. + * @param srcNumSubSplit number of sub-split blocks per src task index. + * @return List of iterators. + */ + private List> readSplitBlocks(final int srcParallelism, + final int srcNumSubSplit) { + final Optional comValueOptional = + runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); + final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); + + switch (comValue) { + case ONE_TO_ONE: + return Collections.singletonList(readOneToOne()); + case BROADCAST: + return readBroadcast(index -> true, Optional.of(srcParallelism / srcNumSubSplit), srcNumSubSplit); + case SHUFFLE: + return readDataInRange(index -> true, Optional.of(srcParallelism / srcNumSubSplit), srcNumSubSplit); + default: + throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); + } + } + + private int translateIndex(final String workStealingState, + final int numSubSplit, + final int index) { + // 여기서는 source vertex 의 parallelism 이 훨씬 중요함. + if (workStealingState.equals(SPLIT_STRATEGY)) { + // source parallelism, current sub split num, current sub split index, parameter index + int sourceParallelism = InputReader.getSourceParallelism(this); + return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getPartialFromTaskId(dstTaskId) + index; + } else if (workStealingState.equals(MERGE_STRATEGY) && + srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + .equals(SPLIT_STRATEGY)) { + // src parallelism, src original parallelism, source sub split num + int srcNumSubSplit = srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1); + return index / srcNumSubSplit; + } else { + return index; + } + } } diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java index 60cc1b9776..78c804bfc3 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/InputReader.java @@ -54,6 +54,17 @@ List> read(String workStealingS */ CompletableFuture retry(int index); + /** + * Retry reading input data during work stealing. + * @param workStealingState work stealing state (SPLIT, MERGE, DEFAULT) + * @param numSubSplit number of sub-splits in SPLIT state, default by 1 in other states. + * @param index of the failed iterator in the list returned by read(). + * @return the retried iterator. + */ + CompletableFuture retry(String workStealingState, + int numSubSplit, + int index); + IRVertex getSrcIrVertex(); ExecutionPropertyMap getProperties(); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index 02bd1c288c..6b4422ffc1 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -27,6 +27,7 @@ import org.apache.nemo.runtime.executor.datatransfer.InputReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import sun.management.snmp.jvmmib.EnumJvmMemPoolThreshdSupport; import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; @@ -147,7 +148,8 @@ private void handleIncomingBlock(final int index, inputReader.getSrcIrVertex().getId(), index); final int twoSecondsInMs = 2 * 1000; Thread.sleep(twoSecondsInMs); - final CompletableFuture retryFuture = inputReader.retry(index); + final CompletableFuture retryFuture = inputReader.retry( + workStealingState, subSplitNum, index); handleIncomingBlock(index, retryFuture); } else if (fetchFailure.equals(BlockFetchFailureProperty.Value.CANCEL_TASK)) { // Retry the entire task @@ -172,7 +174,7 @@ private void fetchDataLazily() { this.expectedNumOfIterators = futures.size(); // 여기도 고쳐야 할 듯. index가 기존의 표현이랑 좀 달라지니까. for (int i = 0; i < futures.size(); i++) { - final int index = translateIndex(i); + final int index = i; // 여기 더이상 고칠 필요 없는데 원래 뭐였는지 체크 final CompletableFuture future = futures.get(i); future.whenComplete((iterator, exception) -> { handleIncomingBlock(index, future); @@ -209,14 +211,4 @@ private void countBytes(final DataUtil.IteratorWithNumBytes iterator) { public void close() throws Exception { } - - private int translateIndex(final int index) { - if (workStealingState.equals("SPLIT")) { - - } else if (workStealingState.equals("MERGE")) { - - } else { - return index; - } - } } From b3b42d120853db3934075f8b7386bc1a43942497 Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 11:58:10 +0900 Subject: [PATCH 18/26] debug --- .../annotating/WorkStealingSubSplitPass.java | 3 +- .../apache/nemo/examples/beam/WordCount.java | 42 +++++++++++++++++++ .../nemo/examples/beam/WordCountITCase.java | 10 +++++ .../nemo/runtime/common/RuntimeIdManager.java | 6 +++ .../common/plan/PhysicalPlanGenerator.java | 1 - .../datatransfer/BlockInputReader.java | 20 +++++---- .../datatransfer/PipeInputReader.java | 7 ++++ .../executor/task/ParentTaskDataFetcher.java | 4 +- .../nemo/runtime/master/PlanStateManager.java | 15 +++---- 9 files changed, 88 insertions(+), 20 deletions(-) diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java index 35e94792db..cc2eb7c35a 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -78,7 +78,8 @@ public IRDAG apply(final IRDAG irdag) { return irdag; } - private void updateParallelismProperty(IRDAG irdag, Map vertexToSplitNum) { + private void updateParallelismProperty(final IRDAG irdag, + final Map vertexToSplitNum) { final Map vertexToStageId = stagePartitioner.apply(irdag); final Map> stageIdToStageVertices = new HashMap<>(); diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 7da3ccd73b..abb3ce7631 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -34,6 +34,7 @@ public final class WordCount { * Private Constructor. */ private WordCount() { + } /** @@ -45,9 +46,18 @@ public static void main(final String[] args) { final String inputFilePath = args[0]; final String outputFilePath = args[1]; final PipelineOptions options = NemoPipelineOptionsFactory.create(); + + //final boolean enableWorkStealing = true; options.setJobName("WordCount"); final Pipeline p = generateWordCountPipeline(options, inputFilePath, outputFilePath); + /** + if (enableWorkStealing) { + p = generateWordCountPipelineWS(options, inputFilePath, outputFilePath); + } else { + p = generateWordCountPipeline(options, inputFilePath, outputFilePath); + } + */ p.run().waitUntilFinish(); } @@ -82,4 +92,36 @@ public String apply(final KV kv) { GenericSourceSink.write(result, outputFilePath); return p; } + + /** + * Static method to generate the word count Beam pipeline. (work stealing version) + * @param options options for the pipeline. + * @param inputFilePath the input file path. + * @param outputFilePath the output file path. + * @return the generated pipeline. + */ + static Pipeline generateWordCountPipelineWS(final PipelineOptions options, + final String inputFilePath, final String outputFilePath) { + final Pipeline p = Pipeline.create(options); + final PCollection result = GenericSourceSink.read(p, inputFilePath) + .apply(MapElements.>via(new SimpleFunction>() { + @Override + public KV apply(final String line) { + final String[] words = line.split(" +"); + final String documentId = words[0] + "#" + words[1]; + final Long count = Long.parseLong(words[2]); + return KV.of(documentId, count); + } + })) + .apply("work stealing", Sum.longsPerKey()) + .apply("merge", Sum.longsPerKey()) + .apply(MapElements., String>via(new SimpleFunction, String>() { + @Override + public String apply(final KV kv) { + return kv.getKey() + ": " + kv.getValue(); + } + })); + GenericSourceSink.write(result, outputFilePath); + return p; + } } diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java index cfe0571434..a7ebb9d5f9 100644 --- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java +++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java @@ -24,6 +24,7 @@ import org.apache.nemo.common.test.ExampleTestUtil; import org.apache.nemo.compiler.optimizer.policy.ConditionalLargeShufflePolicy; import org.apache.nemo.compiler.optimizer.policy.DynamicTaskSizingPolicy; +import org.apache.nemo.compiler.optimizer.policy.WorkStealingPolicy; import org.apache.nemo.examples.beam.policy.*; import org.junit.After; import org.junit.Before; @@ -73,6 +74,15 @@ public void test() throws Exception { .build()); } + @Test(timeout = ExampleTestArgs.TIMEOUT, expected = Test.None.class) + public void testWorkStealing() throws Exception { + JobLauncher.main(builder + .addResourceJson(executorResourceFileName) + .addJobId(WordCountITCase.class.getSimpleName() + "_workStealing") + .addOptimizationPolicy(WorkStealingPolicy.class.getCanonicalName()) + .build()); + } + @Test(timeout = ExampleTestArgs.TIMEOUT, expected = Test.None.class) public void testLargeShuffle() throws Exception { JobLauncher.main(builder diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 6465bec5cd..556946de1a 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -18,6 +18,9 @@ */ package org.apache.nemo.runtime.common; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -25,6 +28,7 @@ * ID Generator. */ public final class RuntimeIdManager { + private static final Logger LOG = LoggerFactory.getLogger(RuntimeIdManager.class.getName()); private static AtomicInteger physicalPlanIdGenerator = new AtomicInteger(0); private static AtomicInteger executorIdGenerator = new AtomicInteger(0); private static AtomicLong messageIdGenerator = new AtomicLong(1L); @@ -103,6 +107,7 @@ public static String generateExecutorId() { public static String generateBlockId(final String runtimeEdgeId, final String producerTaskId) { if (isWorkStealingTask(producerTaskId)) { + LOG.error("work stealing task detected: {}, {}", producerTaskId, runtimeEdgeId); return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) + SPLITTER + getPartialFromTaskId(producerTaskId) + SPLITTER + getAttemptFromTaskId(producerTaskId); @@ -128,6 +133,7 @@ public static String generateBlockIdWildcard(final String runtimeEdgeId, final int producerTaskIndex, final String subSplitIndex) { if (!subSplitIndex.equals("*")) { + LOG.error("work stealing task detected: {}, {}, {}", runtimeEdgeId, producerTaskIndex, subSplitIndex); return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + subSplitIndex + SPLITTER + "*"; } else { diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index b82294bd7b..78f5f3fd79 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -35,7 +35,6 @@ import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty; -import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.common.ir.vertex.utility.SamplingVertex; import org.apache.nemo.conf.JobConf; import org.apache.nemo.runtime.common.RuntimeIdManager; diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index d226613151..3f103495eb 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -38,6 +38,8 @@ import org.apache.nemo.runtime.executor.MetricMessageSender; import org.apache.nemo.runtime.executor.data.BlockManagerWorker; import org.apache.nemo.runtime.executor.data.DataUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; @@ -50,6 +52,7 @@ * Represents the input data transfer to a task. */ public final class BlockInputReader implements InputReader { + private static final Logger LOG = LoggerFactory.getLogger(BlockInputReader.class.getName()); private final BlockManagerWorker blockManagerWorker; private final MetricMessageSender metricMessageSender; private final String dstTaskId; @@ -100,8 +103,8 @@ public List> read() { public List> read(final String workStealingState, final int numSubSplit, final int subSplitIndex) { - if (workStealingState.equals(MERGE_STRATEGY) && - srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + if (workStealingState.equals(MERGE_STRATEGY) + && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { return readSplitBlocks(InputReader.getSourceParallelism(this), srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); @@ -146,8 +149,8 @@ public CompletableFuture retry(final String workS final int numSubSplit, final int desiredIndex) { - final boolean isMergeAfterSplit = workStealingState.equals(MERGE_STRATEGY) && - srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + final boolean isMergeAfterSplit = workStealingState.equals(MERGE_STRATEGY) + && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY); if (!isMergeAfterSplit && !workStealingState.equals(SPLIT_STRATEGY)) { @@ -205,6 +208,7 @@ private CompletableFuture checkSingleElement( */ private String generateWildCardBlockId(final int producerTaskIndex, final String subSplitIndex) { + LOG.error(""); final Optional duplicateDataProperty = runtimeEdge.getPropertyValue(DuplicateEdgeGroupProperty.class); if (!duplicateDataProperty.isPresent() || duplicateDataProperty.get().getGroupSize() <= 1) { @@ -215,7 +219,7 @@ private String generateWildCardBlockId(final int producerTaskIndex, } private CompletableFuture readOneToOne() { - final String blockIdWildcard = generateWildCardBlockId(dstTaskIndex, "1"); + final String blockIdWildcard = generateWildCardBlockId(dstTaskIndex, "*"); return blockManagerWorker.readBlock( blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all()); } @@ -235,7 +239,7 @@ private List> readBroadcast(fin if (predicate.test(srcTaskIdx)) { for (int subSplitIdx = 0; subSplitIdx < numSubSplit; subSplitIdx++) { final String blockIdWildcard = generateWildCardBlockId(srcTaskIdx, - numSubSplit == 1 ? "*": Integer.toString(subSplitIdx)); + numSubSplit == 1 ? "*" : Integer.toString(subSplitIdx)); futures.add(blockManagerWorker.readBlock( blockIdWildcard, runtimeEdge.getId(), runtimeEdge.getExecutionProperties(), HashRange.all())); } @@ -312,8 +316,8 @@ private int translateIndex(final String workStealingState, // source parallelism, current sub split num, current sub split index, parameter index int sourceParallelism = InputReader.getSourceParallelism(this); return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getPartialFromTaskId(dstTaskId) + index; - } else if (workStealingState.equals(MERGE_STRATEGY) && - srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + } else if (workStealingState.equals(MERGE_STRATEGY) + && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { // src parallelism, src original parallelism, source sub split num int srcNumSubSplit = srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java index 89061129a3..e69a31b6a8 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/PipeInputReader.java @@ -95,6 +95,13 @@ public CompletableFuture retry(final int index) { throw new UnsupportedOperationException(String.valueOf(index)); } + @Override + public CompletableFuture retry(final String workStealingState, + final int numSubSplit, + final int index) { + return retry(index); + } + @Override public ExecutionPropertyMap getProperties() { return runtimeEdge.getExecutionProperties(); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index 6b4422ffc1..feb3f34cc7 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -27,7 +27,6 @@ import org.apache.nemo.runtime.executor.datatransfer.InputReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import sun.management.snmp.jvmmib.EnumJvmMemPoolThreshdSupport; import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; @@ -172,9 +171,8 @@ private void fetchDataLazily() { final List> futures = inputReader .read(workStealingState, subSplitNum, RuntimeIdManager.getPartialFromTaskId(taskId)); this.expectedNumOfIterators = futures.size(); - // 여기도 고쳐야 할 듯. index가 기존의 표현이랑 좀 달라지니까. for (int i = 0; i < futures.size(); i++) { - final int index = i; // 여기 더이상 고칠 필요 없는데 원래 뭐였는지 체크 + final int index = i; final CompletableFuture future = futures.get(i); future.whenComplete((iterator, exception) -> { handleIncomingBlock(index, future); diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index c5a726e11c..5afe25b3ae 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -202,16 +202,16 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) final Stage stage = physicalPlan.getStageDAG().getVertexById(stageId); for (final int taskIndex : stage.getTaskIndices()) { LOG.error("{} task index {}", stageId, taskIndex); - final List> attemptStatesForThisTaskIndex = + final List> attemptStatesPerPartialTaskForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); - LOG.error("partial index size : {}", attemptStatesForThisTaskIndex.size()); // no tasks in here! - if (attemptStatesForThisTaskIndex.size() == 0) { + LOG.error("partial index size : {}", attemptStatesPerPartialTaskForThisTaskIndex.size()); // no tasks in here! + if (attemptStatesPerPartialTaskForThisTaskIndex.size() == 0) { // initialize in here for (int i = 0; i < stage.getSubSplitNum(); i++) { - attemptStatesForThisTaskIndex.add(new ArrayList<>()); + attemptStatesPerPartialTaskForThisTaskIndex.add(new ArrayList<>()); } } - for (List attemptStatesForThisPartialTaskIndex : attemptStatesForThisTaskIndex) { + for (List attemptStatesForThisPartialTaskIndex : attemptStatesPerPartialTaskForThisTaskIndex) { // If one of the attempts is COMPLETE, do not schedule if (attemptStatesForThisPartialTaskIndex @@ -244,9 +244,10 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) for (int attempt = 0; attempt < attemptStatesForThisPartialTaskIndex.size(); attempt++) { if (attemptStatesForThisPartialTaskIndex.get(attempt).getStateMachine().getCurrentState() .equals(TaskState.State.READY)) { - if (attemptStatesForThisTaskIndex.size() > 1) { + if (attemptStatesPerPartialTaskForThisTaskIndex.size() > 1) { + taskAttemptsToSchedule.add(RuntimeIdManager.generateWorkStealingTaskId(stageId, taskIndex, - attemptStatesForThisTaskIndex.indexOf(attemptStatesForThisPartialTaskIndex), attempt)); + attemptStatesPerPartialTaskForThisTaskIndex.indexOf(attemptStatesForThisPartialTaskIndex), attempt)); } else { taskAttemptsToSchedule.add(RuntimeIdManager.generateTaskId(stageId, taskIndex, attempt)); } From c0fe4c91377c7ace2fbbf143a095f6cecbe5bcb7 Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 12:09:41 +0900 Subject: [PATCH 19/26] ready for remote test --- .../apache/nemo/examples/beam/WordCount.java | 40 ------------------- 1 file changed, 40 deletions(-) diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index abb3ce7631..4982097a81 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -47,17 +47,9 @@ public static void main(final String[] args) { final String outputFilePath = args[1]; final PipelineOptions options = NemoPipelineOptionsFactory.create(); - //final boolean enableWorkStealing = true; options.setJobName("WordCount"); final Pipeline p = generateWordCountPipeline(options, inputFilePath, outputFilePath); - /** - if (enableWorkStealing) { - p = generateWordCountPipelineWS(options, inputFilePath, outputFilePath); - } else { - p = generateWordCountPipeline(options, inputFilePath, outputFilePath); - } - */ p.run().waitUntilFinish(); } @@ -92,36 +84,4 @@ public String apply(final KV kv) { GenericSourceSink.write(result, outputFilePath); return p; } - - /** - * Static method to generate the word count Beam pipeline. (work stealing version) - * @param options options for the pipeline. - * @param inputFilePath the input file path. - * @param outputFilePath the output file path. - * @return the generated pipeline. - */ - static Pipeline generateWordCountPipelineWS(final PipelineOptions options, - final String inputFilePath, final String outputFilePath) { - final Pipeline p = Pipeline.create(options); - final PCollection result = GenericSourceSink.read(p, inputFilePath) - .apply(MapElements.>via(new SimpleFunction>() { - @Override - public KV apply(final String line) { - final String[] words = line.split(" +"); - final String documentId = words[0] + "#" + words[1]; - final Long count = Long.parseLong(words[2]); - return KV.of(documentId, count); - } - })) - .apply("work stealing", Sum.longsPerKey()) - .apply("merge", Sum.longsPerKey()) - .apply(MapElements., String>via(new SimpleFunction, String>() { - @Override - public String apply(final KV kv) { - return kv.getKey() + ": " + kv.getValue(); - } - })); - GenericSourceSink.write(result, outputFilePath); - return p; - } } From d09d27edd862f0fc7fd0034db3d46367c63d3203 Mon Sep 17 00:00:00 2001 From: hwarim Date: Mon, 23 Aug 2021 17:09:20 +0900 Subject: [PATCH 20/26] working, but happens with memory leak and OOM --- .../nemo/common/ir/vertex/OperatorVertex.java | 12 +++++++++++ .../frontend/beam/PipelineTranslator.java | 5 +++-- .../annotating/WorkStealingPass.java | 9 +++++++-- .../annotating/WorkStealingSubSplitPass.java | 15 +++++++++----- .../apache/nemo/examples/beam/WordCount.java | 3 ++- .../common/plan/PhysicalPlanGenerator.java | 3 ++- .../nemo/runtime/common/plan/Stage.java | 1 + .../datatransfer/BlockInputReader.java | 20 +++++++++++-------- .../nemo/runtime/master/PlanStateManager.java | 9 ++++++--- .../nemo/runtime/master/RuntimeMaster.java | 2 +- 10 files changed, 56 insertions(+), 23 deletions(-) diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/OperatorVertex.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/OperatorVertex.java index 1b2136d070..be44a8af3d 100644 --- a/common/src/main/java/org/apache/nemo/common/ir/vertex/OperatorVertex.java +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/OperatorVertex.java @@ -27,6 +27,7 @@ */ public class OperatorVertex extends IRVertex { private final Transform transform; + private final String transformFullName; /** * Constructor of OperatorVertex. @@ -36,6 +37,12 @@ public class OperatorVertex extends IRVertex { public OperatorVertex(final Transform t) { super(); this.transform = t; + this.transformFullName = ""; + } + + public OperatorVertex(final Transform t, final String transformFullName) { + this.transform = t; + this.transformFullName = transformFullName; } /** @@ -46,6 +53,7 @@ public OperatorVertex(final Transform t) { private OperatorVertex(final OperatorVertex that) { super(that); this.transform = that.transform; + this.transformFullName = that.transformFullName; } @Override @@ -60,6 +68,10 @@ public final Transform getTransform() { return transform; } + public final String getTransformFullName() { + return transformFullName; + } + @Override public final ObjectNode getPropertiesAsJsonNode() { final ObjectNode node = getIRVertexPropertiesAsJsonNode(); diff --git a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java index cd9d7ad223..b81c98135c 100644 --- a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java +++ b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java @@ -350,6 +350,7 @@ private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslato final PTransform transform) { final Combine.PerKey perKey = (Combine.PerKey) transform; + final String fullName = beamNode.getFullName(); // If there's any side inputs, translate each primitive transforms in this composite transform one by one. if (!perKey.getSideInputs().isEmpty()) { @@ -382,8 +383,8 @@ private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslato // Choose between batch processing and stream processing based on window type and boundedness of data if (isMainInputBounded(beamNode, ctx.getPipeline()) && isGlobalWindow(beamNode, ctx.getPipeline())) { // Batch processing, using CombinePartialTransform and CombineFinalTransform - partialCombine = new OperatorVertex(new CombineFnPartialTransform<>(combineFn)); - finalCombine = new OperatorVertex(new CombineFnFinalTransform<>(combineFn)); + partialCombine = new OperatorVertex(new CombineFnPartialTransform<>(combineFn), fullName); + finalCombine = new OperatorVertex(new CombineFnFinalTransform<>(combineFn), fullName); } else { // Stream data processing, using GBKTransform final AppliedPTransform pTransform = beamNode.toAppliedPTransform(ctx.getPipeline()); diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index 0a664a9dcb..b1abecc01b 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -28,6 +28,8 @@ import org.apache.nemo.common.ir.vertex.transform.Transform; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; import org.apache.nemo.runtime.common.plan.StagePartitioner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; @@ -37,6 +39,7 @@ @Annotates(EnableWorkStealingProperty.class) @Requires(CommunicationPatternProperty.class) public final class WorkStealingPass extends AnnotatingPass { + private static final Logger LOG = LoggerFactory.getLogger(WorkStealingPass.class.getName()); private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; @@ -55,9 +58,11 @@ public IRDAG apply(final IRDAG irdag) { .noneMatch(property -> property.equals(CommunicationPatternProperty.Value.ONE_TO_ONE)); if (irVertex instanceof OperatorVertex && notConnectedToO2OEdge) { Transform transform = ((OperatorVertex) irVertex).getTransform(); - if (transform.toString().contains("work stealing")) { + String transformFullName = ((OperatorVertex) irVertex).getTransformFullName(); + LOG.error("transform full name: {}, {}", irVertex.getId(), transformFullName); + if (transform.toString().contains("work stealing") || transformFullName.contains("work stealing")) { irVertex.setProperty(EnableWorkStealingProperty.of(SPLIT_STRATEGY)); - } else if (transform.toString().contains("merge")) { + } else if (transform.toString().contains("merge") || transformFullName.contains("merge")) { irVertex.setProperty(EnableWorkStealingProperty.of(MERGE_STRATEGY)); } else { irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java index cc2eb7c35a..896f339ff0 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -26,6 +26,8 @@ import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; import org.apache.nemo.runtime.common.plan.StagePartitioner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.HashSet; @@ -38,11 +40,12 @@ @Annotates(WorkStealingSubSplitProperty.class) @Requires({EnableWorkStealingProperty.class, ParallelismProperty.class}) public final class WorkStealingSubSplitPass extends AnnotatingPass { + private static final Logger LOG = LoggerFactory.getLogger(WorkStealingSubSplitPass.class.getName()); private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; - private static final int MAX_SUB_SPLIT_NUM = 10; + private static final int MAX_SUB_SPLIT_NUM = 5; private static final int DEFAULT_SUB_SPLIT_NUM = 1; private final StagePartitioner stagePartitioner = new StagePartitioner(); @@ -63,8 +66,10 @@ public IRDAG apply(final IRDAG irdag) { int maxSourceParallelism = irdag.getIncomingEdgesOf(vertex).stream().map(IREdge::getSrc) .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(DEFAULT_SUB_SPLIT_NUM)) .max().orElse(DEFAULT_SUB_SPLIT_NUM); + LOG.error("source parallelism of {} : {}", vertex.getId(), maxSourceParallelism); if (maxSourceParallelism > MAX_SUB_SPLIT_NUM) { - vertex.setProperty(WorkStealingSubSplitProperty.of(DEFAULT_SUB_SPLIT_NUM)); + vertex.setProperty(WorkStealingSubSplitProperty.of(MAX_SUB_SPLIT_NUM)); + vertexToSplitNum.put(vertex, MAX_SUB_SPLIT_NUM); } else { vertex.setProperty(WorkStealingSubSplitProperty.of(maxSourceParallelism)); vertexToSplitNum.put(vertex, maxSourceParallelism); @@ -74,7 +79,7 @@ public IRDAG apply(final IRDAG irdag) { } } - updateParallelismProperty(irdag, vertexToSplitNum); + // updateParallelismProperty(irdag, vertexToSplitNum); return irdag; } @@ -92,9 +97,9 @@ private void updateParallelismProperty(final IRDAG irdag, for (IRVertex vertex : vertexToSplitNum.keySet()) { int numSubSplit = vertexToSplitNum.get(vertex); + int currentParallelism = vertex.getPropertyValue(ParallelismProperty.class).get(); for (IRVertex stageVertex : stageIdToStageVertices.get(vertexToStageId.get(vertex))) { - int currentParallelism = stageVertex.getPropertyValue(ParallelismProperty.class).get(); - stageVertex.setProperty(ParallelismProperty.of(currentParallelism * numSubSplit)); + irdag.getVertexById(stageVertex.getId()).setProperty(ParallelismProperty.of(currentParallelism * numSubSplit)); } } } diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 4982097a81..1f1327e427 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -75,7 +75,8 @@ public KV apply(final String line) { })) .apply("work stealing", Sum.longsPerKey()) .apply("merge", Sum.longsPerKey()) - .apply(MapElements., String>via(new SimpleFunction, String>() { + .apply("test work stealing", MapElements., String>via( + new SimpleFunction, String>() { @Override public String apply(final KV kv) { return kv.getKey() + ": " + kv.getValue(); diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index 78f5f3fd79..b0e5d56eb7 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -35,6 +35,7 @@ import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.common.ir.vertex.utility.SamplingVertex; import org.apache.nemo.conf.JobConf; import org.apache.nemo.runtime.common.RuntimeIdManager; @@ -215,7 +216,7 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { .anyMatch(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class) .orElse("DEFAULT").equals("SPLIT")); int numSubSplit = stageInternalDAG.getVertices().stream() - .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(1)) + .mapToInt(v -> v.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)) .max().orElse(1); final Stage stage = new Stage( stageIdentifier, diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java index ac99b35edd..df106704e4 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/Stage.java @@ -161,6 +161,7 @@ public List> getVertexIdToReadables() { public int getSubSplitNum() { return subSplitNum; } + @Override public ObjectNode getPropertiesAsJsonNode() { final ObjectNode node = JsonNodeFactory.instance.objectNode(); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 3f103495eb..82c5eb7748 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -113,16 +113,21 @@ public List> read(final String if (!workStealingState.equals(SPLIT_STRATEGY)) { /* DEFAULT case */ + LOG.error("DEFAULT CASE: size {}", futures.size()); return futures; } /* SPLIT case*/ // 가능한 even 하게 나누는 방법 생각해서 보완 // 19를 10개로 나누는 건 1*9, 10*1 보다는 2*9, 1*1 이 나음' - return futures.subList(subSplitIndex * (futures.size() / numSubSplit), - Math.min((subSplitIndex + 1) * (futures.size() / numSubSplit), futures.size())); + final int leftInterval = subSplitIndex * (futures.size() / numSubSplit); + final int rightInterval = numSubSplit == subSplitIndex + 1 + ? futures.size() : (subSplitIndex + 1) * (futures.size() / numSubSplit); + LOG.error("SPLIT CASE: future.size: {}", futures.size()); + LOG.error("SPLIT CASE: numSubSplit, index: {}, {}", numSubSplit, subSplitIndex); + LOG.error("SPLIT CASE: [{}, {})", leftInterval, rightInterval); + return futures.subList(leftInterval, rightInterval); } - } @@ -171,11 +176,11 @@ public CompletableFuture retry(final String workS return readOneToOne(); case BROADCAST: return readBroadcast(index -> index == trueIndex, - isMergeAfterSplit ? Optional.of(srcParallelism / srcNumSubSplit) : Optional.empty(), + isMergeAfterSplit ? Optional.of(srcParallelism) : Optional.empty(), isMergeAfterSplit ? srcNumSubSplit : 1).get(subIndex); case SHUFFLE: return readDataInRange(index -> index == trueIndex, - isMergeAfterSplit ? Optional.of(srcParallelism / srcNumSubSplit) : Optional.empty(), + isMergeAfterSplit ? Optional.of(srcParallelism) : Optional.empty(), isMergeAfterSplit ? srcNumSubSplit : 1).get(subIndex); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); @@ -208,7 +213,6 @@ private CompletableFuture checkSingleElement( */ private String generateWildCardBlockId(final int producerTaskIndex, final String subSplitIndex) { - LOG.error(""); final Optional duplicateDataProperty = runtimeEdge.getPropertyValue(DuplicateEdgeGroupProperty.class); if (!duplicateDataProperty.isPresent() || duplicateDataProperty.get().getGroupSize() <= 1) { @@ -300,9 +304,9 @@ private List> readSplitBlocks(f case ONE_TO_ONE: return Collections.singletonList(readOneToOne()); case BROADCAST: - return readBroadcast(index -> true, Optional.of(srcParallelism / srcNumSubSplit), srcNumSubSplit); + return readBroadcast(index -> true, Optional.of(srcParallelism), srcNumSubSplit); case SHUFFLE: - return readDataInRange(index -> true, Optional.of(srcParallelism / srcNumSubSplit), srcNumSubSplit); + return readDataInRange(index -> true, Optional.of(srcParallelism), srcNumSubSplit); default: throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); } diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 5afe25b3ae..e5aa2f3acf 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -105,7 +105,7 @@ public final class PlanStateManager { /** * For dynamic optimization. */ - private final int maxSubTaskSplitNum = 10; + private final int maxSubTaskSplitNum = 5; /** * Constructor. @@ -204,8 +204,9 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) LOG.error("{} task index {}", stageId, taskIndex); final List> attemptStatesPerPartialTaskForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); - LOG.error("partial index size : {}", attemptStatesPerPartialTaskForThisTaskIndex.size()); // no tasks in here! + LOG.error("partial index size : {}", attemptStatesPerPartialTaskForThisTaskIndex.size()); if (attemptStatesPerPartialTaskForThisTaskIndex.size() == 0) { + LOG.error("in initialization stage: {}", stage.getSubSplitNum()); // initialize in here for (int i = 0; i < stage.getSubSplitNum(); i++) { attemptStatesPerPartialTaskForThisTaskIndex.add(new ArrayList<>()); @@ -401,8 +402,9 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState // COMPLETE stage 여기도 고쳐야 함 case COMPLETE: case ON_HOLD: + Stage currentStage = physicalPlan.getStageDAG().getVertexById(stageId); if (numOfCompletedTaskIndicesInThisStage - == physicalPlan.getStageDAG().getVertexById(stageId).getTaskIndices().size()) { + == (long) currentStage.getTaskIndices().size() * currentStage.getSubSplitNum()) { onStageStateChanged(stageId, StageState.State.COMPLETE); } break; @@ -425,6 +427,7 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState * @param newStageState of the stage. */ private void onStageStateChanged(final String stageId, final StageState.State newStageState) { + LOG.error("{} completed", stageId); // Change stage state final StateMachine stageStateMachine = stageIdToState.get(stageId).getStateMachine(); diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java index cae3cf47da..c3b115f7d5 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java @@ -243,7 +243,7 @@ public void flushMetrics() { // save metric to file metricStore.dumpAllMetricToFile(Paths.get(dagDirectory, - "Metric_" + jobId + "_" + System.currentTimeMillis() + ".json").toString()); + "Metric_" + jobId + ".json").toString()); // save metric to database if (this.dbEnabled) { From cfdf87bbf70b2f60ae523bf704cd08e69d2084ee Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 17:55:33 +0900 Subject: [PATCH 21/26] this may be the solution for OOM --- .../datatransfer/BlockInputReader.java | 58 ++++++++++++++----- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 82c5eb7748..29746a7dda 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -99,6 +99,17 @@ public List> read() { } } + /** + * An extended version of {@link #read()} with work stealing options. + * - DEFAULT STRATEGY: {@link #read()} + * - SPLIT STRATEGY: {@link #readPartial(int, int)} + * - MERGE STRATEGY: {@link #readSplitBlocks(int, int)} + * + * @param workStealingState work stealing strategy. + * @param numSubSplit number to split within a task index. + * @param subSplitIndex index of sub split task. + * @return + */ @Override public List> read(final String workStealingState, final int numSubSplit, @@ -109,24 +120,21 @@ public List> read(final String return readSplitBlocks(InputReader.getSourceParallelism(this), srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); } else { - List> futures = read(); - if (!workStealingState.equals(SPLIT_STRATEGY)) { - /* DEFAULT case */ - LOG.error("DEFAULT CASE: size {}", futures.size()); - return futures; + if (workStealingState.equals(SPLIT_STRATEGY)) { + /* SPLIT case*/ + final int srcParallelism = InputReader.getSourceParallelism(this); + final int leftInterval = subSplitIndex * (srcParallelism / numSubSplit); + final int rightInterval = numSubSplit == subSplitIndex + 1 + ? srcParallelism : (subSplitIndex + 1) * (srcParallelism / numSubSplit); + LOG.error("SPLIT CASE: future.size: {}", srcParallelism); + LOG.error("SPLIT CASE: numSubSplit, index: {}, {}", numSubSplit, subSplitIndex); + LOG.error("SPLIT CASE: [{}, {})", leftInterval, rightInterval); + return readPartial(leftInterval, rightInterval); } - /* SPLIT case*/ - // 가능한 even 하게 나누는 방법 생각해서 보완 - // 19를 10개로 나누는 건 1*9, 10*1 보다는 2*9, 1*1 이 나음' - final int leftInterval = subSplitIndex * (futures.size() / numSubSplit); - final int rightInterval = numSubSplit == subSplitIndex + 1 - ? futures.size() : (subSplitIndex + 1) * (futures.size() / numSubSplit); - LOG.error("SPLIT CASE: future.size: {}", futures.size()); - LOG.error("SPLIT CASE: numSubSplit, index: {}, {}", numSubSplit, subSplitIndex); - LOG.error("SPLIT CASE: [{}, {})", leftInterval, rightInterval); - return futures.subList(leftInterval, rightInterval); + /* DEFAULT case */ + return read(); } } @@ -288,6 +296,26 @@ private List> readDataInRange(f return futures; } + // methods related to work stealing policy + + private List> readPartial(final int startIndex, + final int endIndex) { + final Optional comValueOptional = + runtimeEdge.getPropertyValue(CommunicationPatternProperty.class); + final CommunicationPatternProperty.Value comValue = comValueOptional.orElseThrow(IllegalStateException::new); + + switch (comValue) { + case ONE_TO_ONE: + return Collections.singletonList(readOneToOne()); + case BROADCAST: + return readBroadcast(index -> startIndex <= index && index < endIndex, Optional.empty(), 1); + case SHUFFLE: + return readDataInRange(index -> startIndex <= index && index < endIndex, Optional.empty(), 1); + default: + throw new UnsupportedCommPatternException(new Exception("Communication pattern not supported")); + } + } + /** * Read sub-split blocks generated by work stealing SPLIT stage. * @param srcParallelism src stage parallelism. From 2e097da41fe5a6a2f6bf036d24d2ac85d38f06b2 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 18:29:28 +0900 Subject: [PATCH 22/26] delete unnecessary logs --- .../annotating/WorkStealingPass.java | 4 -- .../annotating/WorkStealingSubSplitPass.java | 30 ------------ .../nemo/runtime/common/RuntimeIdManager.java | 6 --- .../datatransfer/BlockInputReader.java | 48 ++++++++++++++----- .../nemo/runtime/master/RuntimeMaster.java | 2 - .../master/scheduler/BatchScheduler.java | 1 - .../master/scheduler/TaskDispatcher.java | 1 - 7 files changed, 35 insertions(+), 57 deletions(-) diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java index b1abecc01b..70e5795857 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java @@ -28,8 +28,6 @@ import org.apache.nemo.common.ir.vertex.transform.Transform; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; import org.apache.nemo.runtime.common.plan.StagePartitioner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.*; @@ -39,7 +37,6 @@ @Annotates(EnableWorkStealingProperty.class) @Requires(CommunicationPatternProperty.class) public final class WorkStealingPass extends AnnotatingPass { - private static final Logger LOG = LoggerFactory.getLogger(WorkStealingPass.class.getName()); private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; @@ -59,7 +56,6 @@ public IRDAG apply(final IRDAG irdag) { if (irVertex instanceof OperatorVertex && notConnectedToO2OEdge) { Transform transform = ((OperatorVertex) irVertex).getTransform(); String transformFullName = ((OperatorVertex) irVertex).getTransformFullName(); - LOG.error("transform full name: {}, {}", irVertex.getId(), transformFullName); if (transform.toString().contains("work stealing") || transformFullName.contains("work stealing")) { irVertex.setProperty(EnableWorkStealingProperty.of(SPLIT_STRATEGY)); } else if (transform.toString().contains("merge") || transformFullName.contains("merge")) { diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java index 896f339ff0..5080b64cde 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -25,14 +25,9 @@ import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; -import org.apache.nemo.runtime.common.plan.StagePartitioner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; /** * Optimization pass for tagging work stealing sub-split execution property. @@ -40,7 +35,6 @@ @Annotates(WorkStealingSubSplitProperty.class) @Requires({EnableWorkStealingProperty.class, ParallelismProperty.class}) public final class WorkStealingSubSplitPass extends AnnotatingPass { - private static final Logger LOG = LoggerFactory.getLogger(WorkStealingSubSplitPass.class.getName()); private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; @@ -48,7 +42,6 @@ public final class WorkStealingSubSplitPass extends AnnotatingPass { private static final int MAX_SUB_SPLIT_NUM = 5; private static final int DEFAULT_SUB_SPLIT_NUM = 1; - private final StagePartitioner stagePartitioner = new StagePartitioner(); /** * Default Constructor. */ @@ -66,7 +59,6 @@ public IRDAG apply(final IRDAG irdag) { int maxSourceParallelism = irdag.getIncomingEdgesOf(vertex).stream().map(IREdge::getSrc) .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(DEFAULT_SUB_SPLIT_NUM)) .max().orElse(DEFAULT_SUB_SPLIT_NUM); - LOG.error("source parallelism of {} : {}", vertex.getId(), maxSourceParallelism); if (maxSourceParallelism > MAX_SUB_SPLIT_NUM) { vertex.setProperty(WorkStealingSubSplitProperty.of(MAX_SUB_SPLIT_NUM)); vertexToSplitNum.put(vertex, MAX_SUB_SPLIT_NUM); @@ -79,28 +71,6 @@ public IRDAG apply(final IRDAG irdag) { } } - // updateParallelismProperty(irdag, vertexToSplitNum); return irdag; } - - private void updateParallelismProperty(final IRDAG irdag, - final Map vertexToSplitNum) { - final Map vertexToStageId = stagePartitioner.apply(irdag); - - final Map> stageIdToStageVertices = new HashMap<>(); - vertexToStageId.forEach((vertex, stageId) -> { - if (!stageIdToStageVertices.containsKey(stageId)) { - stageIdToStageVertices.put(stageId, new HashSet<>()); - } - stageIdToStageVertices.get(stageId).add(vertex); - }); - - for (IRVertex vertex : vertexToSplitNum.keySet()) { - int numSubSplit = vertexToSplitNum.get(vertex); - int currentParallelism = vertex.getPropertyValue(ParallelismProperty.class).get(); - for (IRVertex stageVertex : stageIdToStageVertices.get(vertexToStageId.get(vertex))) { - irdag.getVertexById(stageVertex.getId()).setProperty(ParallelismProperty.of(currentParallelism * numSubSplit)); - } - } - } } diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 556946de1a..6465bec5cd 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -18,9 +18,6 @@ */ package org.apache.nemo.runtime.common; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -28,7 +25,6 @@ * ID Generator. */ public final class RuntimeIdManager { - private static final Logger LOG = LoggerFactory.getLogger(RuntimeIdManager.class.getName()); private static AtomicInteger physicalPlanIdGenerator = new AtomicInteger(0); private static AtomicInteger executorIdGenerator = new AtomicInteger(0); private static AtomicLong messageIdGenerator = new AtomicLong(1L); @@ -107,7 +103,6 @@ public static String generateExecutorId() { public static String generateBlockId(final String runtimeEdgeId, final String producerTaskId) { if (isWorkStealingTask(producerTaskId)) { - LOG.error("work stealing task detected: {}, {}", producerTaskId, runtimeEdgeId); return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) + SPLITTER + getPartialFromTaskId(producerTaskId) + SPLITTER + getAttemptFromTaskId(producerTaskId); @@ -133,7 +128,6 @@ public static String generateBlockIdWildcard(final String runtimeEdgeId, final int producerTaskIndex, final String subSplitIndex) { if (!subSplitIndex.equals("*")) { - LOG.error("work stealing task detected: {}, {}, {}", runtimeEdgeId, producerTaskIndex, subSplitIndex); return runtimeEdgeId + SPLITTER + producerTaskIndex + SPLITTER + subSplitIndex + SPLITTER + "*"; } else { diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index 29746a7dda..7cab15f7ca 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -38,8 +38,6 @@ import org.apache.nemo.runtime.executor.MetricMessageSender; import org.apache.nemo.runtime.executor.data.BlockManagerWorker; import org.apache.nemo.runtime.executor.data.DataUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; @@ -52,7 +50,6 @@ * Represents the input data transfer to a task. */ public final class BlockInputReader implements InputReader { - private static final Logger LOG = LoggerFactory.getLogger(BlockInputReader.class.getName()); private final BlockManagerWorker blockManagerWorker; private final MetricMessageSender metricMessageSender; private final String dstTaskId; @@ -117,19 +114,17 @@ public List> read(final String if (workStealingState.equals(MERGE_STRATEGY) && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { + /* MERGE case */ return readSplitBlocks(InputReader.getSourceParallelism(this), srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)); } else { if (workStealingState.equals(SPLIT_STRATEGY)) { - /* SPLIT case*/ + /* SPLIT case */ final int srcParallelism = InputReader.getSourceParallelism(this); final int leftInterval = subSplitIndex * (srcParallelism / numSubSplit); final int rightInterval = numSubSplit == subSplitIndex + 1 ? srcParallelism : (subSplitIndex + 1) * (srcParallelism / numSubSplit); - LOG.error("SPLIT CASE: future.size: {}", srcParallelism); - LOG.error("SPLIT CASE: numSubSplit, index: {}, {}", numSubSplit, subSplitIndex); - LOG.error("SPLIT CASE: [{}, {})", leftInterval, rightInterval); return readPartial(leftInterval, rightInterval); } @@ -157,6 +152,14 @@ public CompletableFuture retry(final int desiredI } } + /** + * An extended version of {@link #retry(int)} with work stealing options. + * + * @param workStealingState work stealing strategy. (SPLIT, MERGE, DEFAULT) + * @param numSubSplit number of sub-splits in SPLIT state, default by 1 in other states. + * @param desiredIndex desired index. + * @return iterator to retry. + */ @Override public CompletableFuture retry(final String workStealingState, final int numSubSplit, @@ -238,8 +241,10 @@ private CompletableFuture readOneToOne() { /** * Read data in full range of hash value. - * if @param{numSubSplit} > 1, it indicates that it is reading sub-split blocks from work stealing. * + * @param predicate function of the index. + * @param numSrcIndex not empty only if in MERGE strategy. + * @param numSubSplit > 1 only if in MERGE strategy. * @return the list of the completable future of the data. */ private List> readBroadcast(final Predicate predicate, @@ -262,8 +267,10 @@ private List> readBroadcast(fin /** * Read data in the assigned range of hash value. - * if @param{numSubSplit} > 1, it indicates that it is reading sub-split blocks from work stealing. * + * @param predicate function of the index. + * @param numSrcIndex not empty only if in MERGE strategy. + * @param numSubSplit > 1 only if in MERGE strategy. * @return the list of the completable future of the data. */ private List> readDataInRange(final Predicate predicate, @@ -298,6 +305,13 @@ private List> readDataInRange(f // methods related to work stealing policy + /** + * Read blocks in work stealing SPLIT strategy. + * + * @param startIndex start index (inclusive) to read. + * @param endIndex end index (exclusive) to read. + * @return the list of the completable future of the data. + */ private List> readPartial(final int startIndex, final int endIndex) { final Optional comValueOptional = @@ -317,7 +331,8 @@ private List> readPartial(final } /** - * Read sub-split blocks generated by work stealing SPLIT stage. + * Read blocks in work stealing MERGE strategy. + * * @param srcParallelism src stage parallelism. * @param srcNumSubSplit number of sub-split blocks per src task index. * @return List of iterators. @@ -340,18 +355,25 @@ private List> readSplitBlocks(f } } + /** + * translate index for consistency. + * + * @param workStealingState Work stealing startegy. + * @param numSubSplit number of sub-split tasks of the task with given index. + * @param index task index. + * @return + */ private int translateIndex(final String workStealingState, final int numSubSplit, final int index) { - // 여기서는 source vertex 의 parallelism 이 훨씬 중요함. if (workStealingState.equals(SPLIT_STRATEGY)) { - // source parallelism, current sub split num, current sub split index, parameter index + /* SPLIT strategy */ int sourceParallelism = InputReader.getSourceParallelism(this); return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getPartialFromTaskId(dstTaskId) + index; } else if (workStealingState.equals(MERGE_STRATEGY) && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { - // src parallelism, src original parallelism, source sub split num + /* MERGE strategy*/ int srcNumSubSplit = srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1); return index / srcNumSubSplit; } else { diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java index c3b115f7d5..a5393b3ad9 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/RuntimeMaster.java @@ -217,7 +217,6 @@ private Server startRestMetricServer() { * @param planId the ID of the IR DAG Physical Plan. */ public void recordIRDAGMetrics(final IRDAG irdag, final String planId) { - LOG.error("record metrics of {}", planId); metricStore.getOrCreateMetric(JobMetric.class, planId).setIRDAG(irdag); } @@ -261,7 +260,6 @@ public void flushMetrics() { */ public Pair execute(final PhysicalPlan plan, final int maxScheduleAttempt) { - LOG.error("EXECUTE in runtime master"); final Callable> planExecutionCallable = () -> { this.irVertices.addAll(plan.getIdToIRVertex().values()); try { diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java index f8cb60786a..086c9d08bd 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/BatchScheduler.java @@ -304,7 +304,6 @@ public void terminate() { * - We make {@link TaskDispatcher} dispatch only the tasks that are READY. */ private void doSchedule() { - LOG.error("DO Schedule"); final Optional> earliest = BatchSchedulerUtils.selectEarliestSchedulableGroup(sortedScheduleGroups, planStateManager); diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java index 559668d54b..e15d8ea13e 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/TaskDispatcher.java @@ -183,7 +183,6 @@ void onNewPendingTaskCollectionAvailable() { * Run the dispatcher thread. */ void run() { - LOG.error("Run in task dispatcher"); if (!isTerminated && !isSchedulerRunning) { dispatcherThread.execute(new TaskDispatcherThread()); dispatcherThread.shutdown(); From b00779896da3e942b745c5669bf24dcf81c9b8c1 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 19:18:10 +0900 Subject: [PATCH 23/26] delete logs and organize new passes to a composite pass --- ...ngPass.java => WorkStealingStatePass.java} | 6 +-- .../composite/WorkStealingCompositePass.java | 38 +++++++++++++++++++ .../optimizer/policy/WorkStealingPolicy.java | 6 +-- .../nemo/runtime/master/PlanStateManager.java | 15 -------- 4 files changed, 43 insertions(+), 22 deletions(-) rename compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/{WorkStealingPass.java => WorkStealingStatePass.java} (98%) create mode 100644 compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePass.java diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java similarity index 98% rename from compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java rename to compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java index 70e5795857..76c7890e1a 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java @@ -36,15 +36,15 @@ */ @Annotates(EnableWorkStealingProperty.class) @Requires(CommunicationPatternProperty.class) -public final class WorkStealingPass extends AnnotatingPass { +public final class WorkStealingStatePass extends AnnotatingPass { private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; private static final String DEFAULT_STRATEGY = "DEFAULT"; private final StagePartitioner stagePartitioner = new StagePartitioner(); - public WorkStealingPass() { - super(WorkStealingPass.class); + public WorkStealingStatePass() { + super(WorkStealingStatePass.class); } @Override diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePass.java new file mode 100644 index 0000000000..9c8914ee6f --- /dev/null +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePass.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.nemo.compiler.optimizer.pass.compiletime.composite; + +import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.*; + +import java.util.Arrays; + +/** + * Composite pass for work stealing. + */ +public class WorkStealingCompositePass extends CompositePass { + /** + * Default constructor. + */ + public WorkStealingCompositePass() { + super(Arrays.asList( + new WorkStealingStatePass(), + new WorkStealingSubSplitPass() + )); + } +} diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java index ba5219b3e8..b094606f32 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/WorkStealingPolicy.java @@ -19,9 +19,8 @@ package org.apache.nemo.compiler.optimizer.policy; import org.apache.nemo.common.ir.IRDAG; -import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingPass; -import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.WorkStealingSubSplitPass; import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass; +import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.WorkStealingCompositePass; import org.apache.nemo.compiler.optimizer.pass.runtime.Message; /** @@ -31,8 +30,7 @@ public final class WorkStealingPolicy implements Policy { public static final PolicyBuilder BUILDER = new PolicyBuilder() .registerCompileTimePass(new DefaultCompositePass()) - .registerCompileTimePass(new WorkStealingPass()) - .registerCompileTimePass(new WorkStealingSubSplitPass()); + .registerCompileTimePass(new WorkStealingCompositePass()); private final Policy policy; /** diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index e5aa2f3acf..2b26abee60 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -146,7 +146,6 @@ public void setMetricStore(final MetricStore metricStore) { */ public synchronized void updatePlan(final PhysicalPlan physicalPlanToUpdate, final int maxScheduleAttemptToSet) { - LOG.error("update plan"); if (!initialized) { // First scheduling. this.initialized = true; @@ -167,7 +166,6 @@ public synchronized void updatePlan(final PhysicalPlan physicalPlanToUpdate, * TODO #182: Consider reshaping in run-time optimization. At now, we only consider plan appending. */ private void initializeStates() { - LOG.error("initialize states"); onPlanStateChanged(PlanState.State.EXECUTING); physicalPlan.getStageDAG().topologicalDo(stage -> { stageIdToState.putIfAbsent(stage.getId(), new StageState()); @@ -191,7 +189,6 @@ private void initializeStates() { */ public synchronized List getTaskAttemptsToSchedule(final String stageId) { // initialization: 첫번째로 만들어지는케이스를 따로 생각해야 함. - LOG.error("GET TASK ATTEMPTS TO SCHEDULE at {}", stageId); if (getStageState(stageId).equals(StageState.State.COMPLETE)) { // This stage is done return new ArrayList<>(0); @@ -201,12 +198,9 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) final List taskAttemptsToSchedule = new ArrayList<>(); final Stage stage = physicalPlan.getStageDAG().getVertexById(stageId); for (final int taskIndex : stage.getTaskIndices()) { - LOG.error("{} task index {}", stageId, taskIndex); final List> attemptStatesPerPartialTaskForThisTaskIndex = stageIdToTaskIdxToAttemptStates.get(stageId).get(taskIndex); - LOG.error("partial index size : {}", attemptStatesPerPartialTaskForThisTaskIndex.size()); if (attemptStatesPerPartialTaskForThisTaskIndex.size() == 0) { - LOG.error("in initialization stage: {}", stage.getSubSplitNum()); // initialize in here for (int i = 0; i < stage.getSubSplitNum(); i++) { attemptStatesPerPartialTaskForThisTaskIndex.add(new ArrayList<>()); @@ -260,7 +254,6 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) } } - LOG.error("attempts to schedule: {} tasks", taskAttemptsToSchedule.size()); return taskAttemptsToSchedule; } @@ -269,7 +262,6 @@ public synchronized List getTaskAttemptsToSchedule(final String stageId) * @return all task attempt ids of the stage. */ public synchronized Set getAllTaskAttemptsOfStage(final String stageId) { - LOG.error("get al task attempts of stage"); return getTaskAttemptIdsToItsState(stageId).keySet(); } @@ -281,7 +273,6 @@ public synchronized Set getAllTaskAttemptsOfStage(final String stageId) * @return a map from an EXECUTING task to its running time so far. */ public synchronized Map getExecutingTaskToRunningTimeMs(final String stageId) { - LOG.error("get executing task to running time ms"); final long curTime = System.currentTimeMillis(); final Map result = new HashMap<>(); @@ -342,7 +333,6 @@ public synchronized boolean setNumOfClones(final String stageId, final int taskI * @param newTaskState the new state of the task. */ public synchronized void onTaskStateChanged(final String taskId, final TaskState.State newTaskState) { - LOG.error("on task state changed"); // Change task state final StateMachine taskState = getTaskStateHelper(taskId).getStateMachine(); LOG.debug("Task State Transition: id {}, from {} to {}", @@ -427,7 +417,6 @@ public synchronized void onTaskStateChanged(final String taskId, final TaskState * @param newStageState of the stage. */ private void onStageStateChanged(final String stageId, final StageState.State newStageState) { - LOG.error("{} completed", stageId); // Change stage state final StateMachine stageStateMachine = stageIdToState.get(stageId).getStateMachine(); @@ -532,7 +521,6 @@ public PlanState.State waitUntilFinish(final long timeout, final TimeUnit unit) */ @VisibleForTesting public synchronized Map getAllTaskAttemptIdsToItsState() { - LOG.error("get all task attempt ids to its state"); return physicalPlan.getStageDAG().getVertices() .stream() .map(Stage::getId) @@ -578,7 +566,6 @@ public synchronized TaskState.State getTaskState(final String taskId) { } private Map getTaskAttemptIdsToItsState(final String stageId) { - LOG.error("get task attempt ids to its state"); final Map result = new HashMap<>(); final Map>> taskIdToState = stageIdToTaskIdxToAttemptStates.get(stageId); for (int taskIndex : taskIdToState.keySet()) { @@ -620,7 +607,6 @@ private boolean isTaskNotDone(final TaskState taskState) { } private List getPeerAttemptsForTheSameTaskIndex(final String taskId) { - LOG.error("get peer attempts for the same task index"); final String stageId = RuntimeIdManager.getStageIdFromTaskId(taskId); final int taskIndex = RuntimeIdManager.getIndexFromTaskId(taskId); final int partialIndex = RuntimeIdManager.getPartialFromTaskId(taskId); @@ -662,7 +648,6 @@ public synchronized boolean isInitialized() { * @param suffix suffix for file name */ public void storeJSON(final String suffix) { - LOG.error("STORE JSON {}", suffix); if (dagDirectory.equals(EMPTY_DAG_DIRECTORY)) { return; } From 18ce7353f119d4904c1f5a4b14de96c73d259170 Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 19:40:04 +0900 Subject: [PATCH 24/26] tidy annotations --- ...ty.java => WorkStealingStateProperty.java} | 8 ++-- .../frontend/beam/PipelineTranslator.java | 6 ++- .../annotating/WorkStealingStatePass.java | 34 +++++++------- .../annotating/WorkStealingSubSplitPass.java | 6 +-- .../apache/nemo/examples/beam/WordCount.java | 45 ++++++++++++++++--- .../nemo/examples/beam/WordCountITCase.java | 6 ++- .../nemo/runtime/common/RuntimeIdManager.java | 11 +++-- .../common/plan/PhysicalPlanGenerator.java | 4 +- .../runtime/common/plan/StagePartitioner.java | 4 +- .../datatransfer/BlockInputReader.java | 10 ++--- .../executor/task/ParentTaskDataFetcher.java | 2 +- .../runtime/executor/task/TaskExecutor.java | 6 +-- .../nemo/runtime/master/PlanStateManager.java | 4 +- 13 files changed, 94 insertions(+), 52 deletions(-) rename common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/{EnableWorkStealingProperty.java => WorkStealingStateProperty.java} (85%) diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingStateProperty.java similarity index 85% rename from common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java rename to common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingStateProperty.java index 1a501dd8a2..def6b481ce 100644 --- a/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/EnableWorkStealingProperty.java +++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/executionproperty/WorkStealingStateProperty.java @@ -28,13 +28,13 @@ * MERGE : vertex which merges the effect of work stealing * DEFAULT : vertex which is not the subject of work stealing */ -public class EnableWorkStealingProperty extends VertexExecutionProperty { +public class WorkStealingStateProperty extends VertexExecutionProperty { /** * Default constructor. * * @param value value of the VertexExecutionProperty. */ - public EnableWorkStealingProperty(final String value) { + public WorkStealingStateProperty(final String value) { super(value); } @@ -44,7 +44,7 @@ public EnableWorkStealingProperty(final String value) { * @param value value of the new execution property. * @return the newly created execution property. */ - public static EnableWorkStealingProperty of(final String value) { - return new EnableWorkStealingProperty(value); + public static WorkStealingStateProperty of(final String value) { + return new WorkStealingStateProperty(value); } } diff --git a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java index b81c98135c..f3063a2989 100644 --- a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java +++ b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java @@ -269,7 +269,8 @@ private static void parDoMultiOutputTranslator(final PipelineTranslationContext private static void groupByKeyTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final GroupByKey transform) { - final IRVertex vertex = new OperatorVertex(createGBKTransform(ctx, beamNode)); + final String fullName = beamNode.getFullName(); + final IRVertex vertex = new OperatorVertex(createGBKTransform(ctx, beamNode), fullName); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); @@ -324,7 +325,8 @@ private static void createPCollectionViewTranslator(final PipelineTranslationCon private static void flattenTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final Flatten.PCollections transform) { - final IRVertex vertex = new OperatorVertex(new FlattenTransform()); + final String fullName = beamNode.getFullName(); + final IRVertex vertex = new OperatorVertex(new FlattenTransform(), fullName); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java index 76c7890e1a..15b46e5c77 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingStatePass.java @@ -24,7 +24,7 @@ import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty; import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; import org.apache.nemo.common.ir.vertex.transform.Transform; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; import org.apache.nemo.runtime.common.plan.StagePartitioner; @@ -32,9 +32,9 @@ import java.util.*; /** - * Optimization pass for tagging enable work stealing execution property. + * Optimization pass for annotating {@link WorkStealingStateProperty}. */ -@Annotates(EnableWorkStealingProperty.class) +@Annotates(WorkStealingStateProperty.class) @Requires(CommunicationPatternProperty.class) public final class WorkStealingStatePass extends AnnotatingPass { private static final String SPLIT_STRATEGY = "SPLIT"; @@ -57,14 +57,14 @@ public IRDAG apply(final IRDAG irdag) { Transform transform = ((OperatorVertex) irVertex).getTransform(); String transformFullName = ((OperatorVertex) irVertex).getTransformFullName(); if (transform.toString().contains("work stealing") || transformFullName.contains("work stealing")) { - irVertex.setProperty(EnableWorkStealingProperty.of(SPLIT_STRATEGY)); + irVertex.setProperty(WorkStealingStateProperty.of(SPLIT_STRATEGY)); } else if (transform.toString().contains("merge") || transformFullName.contains("merge")) { - irVertex.setProperty(EnableWorkStealingProperty.of(MERGE_STRATEGY)); + irVertex.setProperty(WorkStealingStateProperty.of(MERGE_STRATEGY)); } else { - irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + irVertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); } } else { - irVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + irVertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); } }); return tidyWorkStealingAnnotation(irdag); @@ -91,13 +91,13 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { // Make SPLIT - MERGE vertex pair. for (IRVertex vertex : irdag.getTopologicalSort()) { - if (vertex.getPropertyValue(EnableWorkStealingProperty.class).get().equals(SPLIT_STRATEGY)) { + if (vertex.getPropertyValue(WorkStealingStateProperty.class).get().equals(SPLIT_STRATEGY)) { if (splitVertexId != null) { // nested SPLIT vertex detected: delete the prior one. - irdag.getVertexById(splitVertexId).setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + irdag.getVertexById(splitVertexId).setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); } splitVertexId = vertex.getId(); - } else if (vertex.getPropertyValue(EnableWorkStealingProperty.class).get().equals(MERGE_STRATEGY)) { + } else if (vertex.getPropertyValue(WorkStealingStateProperty.class).get().equals(MERGE_STRATEGY)) { if (splitVertexId != null) { splitMergePairs.add(Pair.of(splitVertexId, vertex.getId())); pairedVertices.add(splitVertexId); @@ -105,7 +105,7 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { splitVertexId = null; } else { // no corresponding SPLIT vertex: delete - vertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + vertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); } } } @@ -123,18 +123,18 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { .noneMatch(stageId -> stageId.equals(vertexToStageId.get(splitVertex)))) { // split vertex is descendent of merge vertex or they are in the same stage, // or they are not in adjacent stages - splitVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); - mergeVertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + splitVertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); + mergeVertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); pairedVertices.remove(splitVertex.getId()); pairedVertices.remove(mergeVertex.getId()); } } irdag.topologicalDo(vertex -> { - if (!vertex.getPropertyValue(EnableWorkStealingProperty.class) + if (!vertex.getPropertyValue(WorkStealingStateProperty.class) .orElse(DEFAULT_STRATEGY).equals(DEFAULT_STRATEGY)) { if (!pairedVertices.contains(vertex.getId())) { - vertex.setProperty(EnableWorkStealingProperty.of(DEFAULT_STRATEGY)); + vertex.setProperty(WorkStealingStateProperty.of(DEFAULT_STRATEGY)); } } }); @@ -150,10 +150,10 @@ private IRDAG tidyWorkStealingAnnotation(final IRDAG irdag) { for (String vertexId : pairedVertices) { IRVertex vertex = irdag.getVertexById(vertexId); Set stageVertices = stageIdToStageVertices.get(vertexToStageId.get(vertex)); - String strategy = vertex.getPropertyValue(EnableWorkStealingProperty.class) + String strategy = vertex.getPropertyValue(WorkStealingStateProperty.class) .orElse(DEFAULT_STRATEGY); for (IRVertex stageVertex : stageVertices) { - stageVertex.setProperty(EnableWorkStealingProperty.of(strategy)); + stageVertex.setProperty(WorkStealingStateProperty.of(strategy)); } } diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java index 5080b64cde..d0d072aab2 100644 --- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java +++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/WorkStealingSubSplitPass.java @@ -21,7 +21,7 @@ import org.apache.nemo.common.ir.IRDAG; import org.apache.nemo.common.ir.edge.IREdge; import org.apache.nemo.common.ir.vertex.IRVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires; @@ -33,7 +33,7 @@ * Optimization pass for tagging work stealing sub-split execution property. */ @Annotates(WorkStealingSubSplitProperty.class) -@Requires({EnableWorkStealingProperty.class, ParallelismProperty.class}) +@Requires({WorkStealingStateProperty.class, ParallelismProperty.class}) public final class WorkStealingSubSplitPass extends AnnotatingPass { private static final String SPLIT_STRATEGY = "SPLIT"; private static final String MERGE_STRATEGY = "MERGE"; @@ -54,7 +54,7 @@ public IRDAG apply(final IRDAG irdag) { final Map vertexToSplitNum = new HashMap<>(); for (IRVertex vertex : irdag.getTopologicalSort()) { - if (vertex.getPropertyValue(EnableWorkStealingProperty.class) + if (vertex.getPropertyValue(WorkStealingStateProperty.class) .orElse(DEFAULT_STRATEGY).equals(SPLIT_STRATEGY)) { int maxSourceParallelism = irdag.getIncomingEdgesOf(vertex).stream().map(IREdge::getSrc) .mapToInt(v -> v.getPropertyValue(ParallelismProperty.class).orElse(DEFAULT_SUB_SPLIT_NUM)) diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 1f1327e427..0d2216d3d0 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -34,7 +34,6 @@ public final class WordCount { * Private Constructor. */ private WordCount() { - } /** @@ -45,11 +44,15 @@ private WordCount() { public static void main(final String[] args) { final String inputFilePath = args[0]; final String outputFilePath = args[1]; + final boolean enableWorkStealing = Boolean.parseBoolean(args[2]); final PipelineOptions options = NemoPipelineOptionsFactory.create(); options.setJobName("WordCount"); - final Pipeline p = generateWordCountPipeline(options, inputFilePath, outputFilePath); + final Pipeline p = enableWorkStealing + ? generateWordCountPipelineForWorkStealing(options, inputFilePath, outputFilePath) + : generateWordCountPipeline(options, inputFilePath, outputFilePath); + p.run().waitUntilFinish(); } @@ -73,9 +76,8 @@ public KV apply(final String line) { return KV.of(documentId, count); } })) - .apply("work stealing", Sum.longsPerKey()) - .apply("merge", Sum.longsPerKey()) - .apply("test work stealing", MapElements., String>via( + .apply(Sum.longsPerKey()) + .apply(MapElements., String>via( new SimpleFunction, String>() { @Override public String apply(final KV kv) { @@ -85,4 +87,37 @@ public String apply(final KV kv) { GenericSourceSink.write(result, outputFilePath); return p; } + + /** + * Static method to generate the word count Beam pipeline with work stealing optimization. + * @param options options for the pipeline. + * @param inputFilePath the input file path. + * @param outputFilePath the output file path. + * @return the generated pipeline. + */ + static Pipeline generateWordCountPipelineForWorkStealing(final PipelineOptions options, + final String inputFilePath, final String outputFilePath) { + final Pipeline p = Pipeline.create(options); + final PCollection result = GenericSourceSink.read(p, inputFilePath) + .apply(MapElements.>via(new SimpleFunction>() { + @Override + public KV apply(final String line) { + final String[] words = line.split(" +"); + final String documentId = words[0] + "#" + words[1]; + final Long count = Long.parseLong(words[2]); + return KV.of(documentId, count); + } + })) + .apply("work stealing", Sum.longsPerKey()) + .apply("merge", Sum.longsPerKey()) + .apply("test work stealing", MapElements., String>via( + new SimpleFunction, String>() { + @Override + public String apply(final KV kv) { + return kv.getKey() + ": " + kv.getValue(); + } + })); + GenericSourceSink.write(result, outputFilePath); + return p; + } } diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java index a7ebb9d5f9..a609dd1469 100644 --- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java +++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/WordCountITCase.java @@ -53,7 +53,7 @@ public final class WordCountITCase { public void setUp() throws Exception { builder = new ArgBuilder() .addUserMain(WordCount.class.getCanonicalName()) - .addUserArgs(inputFilePath, outputFilePath); + .addUserArgs(inputFilePath, outputFilePath, "false"); } @After @@ -76,7 +76,9 @@ public void test() throws Exception { @Test(timeout = ExampleTestArgs.TIMEOUT, expected = Test.None.class) public void testWorkStealing() throws Exception { - JobLauncher.main(builder + JobLauncher.main(new ArgBuilder() + .addUserMain(WordCount.class.getCanonicalName()) + .addUserArgs(inputFilePath, outputFilePath, "true") .addResourceJson(executorResourceFileName) .addJobId(WordCountITCase.class.getSimpleName() + "_workStealing") .addOptimizationPolicy(WorkStealingPolicy.class.getCanonicalName()) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java index 6465bec5cd..6d69b992fc 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/RuntimeIdManager.java @@ -104,7 +104,7 @@ public static String generateBlockId(final String runtimeEdgeId, final String producerTaskId) { if (isWorkStealingTask(producerTaskId)) { return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) - + SPLITTER + getPartialFromTaskId(producerTaskId) + + SPLITTER + getSubSplitIndexFromTaskId(producerTaskId) + SPLITTER + getAttemptFromTaskId(producerTaskId); } else { return runtimeEdgeId + SPLITTER + getIndexFromTaskId(producerTaskId) @@ -171,9 +171,12 @@ public static int getTaskIndexFromBlockId(final String blockId) { } /** + * Extracts task index from a block ID. * + * @param blockId the block ID to extract. + * @return the task index. */ - public static String getSubSplitIndexFromBlockId(final String blockId) { + public static String getTaskSubSplitIndexFromBlockId(final String blockId) { if (isWorkStealingBlock(blockId)) { return split(blockId)[2]; } else { @@ -190,7 +193,7 @@ public static String getSubSplitIndexFromBlockId(final String blockId) { public static String getWildCardFromBlockId(final String blockId) { return generateBlockIdWildcard(getRuntimeEdgeIdFromBlockId(blockId), getTaskIndexFromBlockId(blockId), - getSubSplitIndexFromBlockId(blockId)); + getTaskSubSplitIndexFromBlockId(blockId)); } /** @@ -217,7 +220,7 @@ public static boolean isWorkStealingTask(final String taskId) { return !split(taskId)[2].equals("*"); } - public static int getPartialFromTaskId(final String taskId) { + public static int getSubSplitIndexFromTaskId(final String taskId) { return split(taskId)[2].equals("*") ? 0 : Integer.valueOf(split(taskId)[2]); } /** diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java index b0e5d56eb7..c63134e6f3 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGenerator.java @@ -32,7 +32,7 @@ import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.SourceVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty; import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; @@ -213,7 +213,7 @@ public DAG stagePartitionIrDAG(final IRDAG irDAG) { = stageInternalDAGBuilder.buildWithoutSourceSinkCheck(); // check if this stage is subject of work stealing optimization boolean isWorkStealingStage = stageInternalDAG.getVertices().stream() - .anyMatch(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class) + .anyMatch(vertex -> vertex.getPropertyValue(WorkStealingStateProperty.class) .orElse("DEFAULT").equals("SPLIT")); int numSubSplit = stageInternalDAG.getVertices().stream() .mapToInt(v -> v.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1)) diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java index 7c24e01eff..ec4db68219 100644 --- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java +++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/StagePartitioner.java @@ -25,7 +25,7 @@ import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty; import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty; import org.apache.nemo.common.ir.vertex.IRVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.reef.annotations.audience.DriverSide; @@ -55,7 +55,7 @@ public final class StagePartitioner implements Function> read(final String final int numSubSplit, final int subSplitIndex) { if (workStealingState.equals(MERGE_STRATEGY) - && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + && srcVertex.getPropertyValue(WorkStealingStateProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { /* MERGE case */ return readSplitBlocks(InputReader.getSourceParallelism(this), @@ -166,7 +166,7 @@ public CompletableFuture retry(final String workS final int desiredIndex) { final boolean isMergeAfterSplit = workStealingState.equals(MERGE_STRATEGY) - && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + && srcVertex.getPropertyValue(WorkStealingStateProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY); if (!isMergeAfterSplit && !workStealingState.equals(SPLIT_STRATEGY)) { @@ -369,9 +369,9 @@ private int translateIndex(final String workStealingState, if (workStealingState.equals(SPLIT_STRATEGY)) { /* SPLIT strategy */ int sourceParallelism = InputReader.getSourceParallelism(this); - return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getPartialFromTaskId(dstTaskId) + index; + return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getSubSplitIndexFromTaskId(dstTaskId) + index; } else if (workStealingState.equals(MERGE_STRATEGY) - && srcVertex.getPropertyValue(EnableWorkStealingProperty.class).orElse(DEFAULT_STRATEGY) + && srcVertex.getPropertyValue(WorkStealingStateProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { /* MERGE strategy*/ int srcNumSubSplit = srcVertex.getPropertyValue(WorkStealingSubSplitProperty.class).orElse(1); diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java index feb3f34cc7..990b57b68f 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcher.java @@ -169,7 +169,7 @@ private void handleIncomingBlock(final int index, private void fetchDataLazily() { final List> futures = inputReader - .read(workStealingState, subSplitNum, RuntimeIdManager.getPartialFromTaskId(taskId)); + .read(workStealingState, subSplitNum, RuntimeIdManager.getSubSplitIndexFromTaskId(taskId)); this.expectedNumOfIterators = futures.size(); for (int i = 0; i < futures.size(); i++) { final int index = i; diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java index b36b452a2d..22bede2ed2 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/task/TaskExecutor.java @@ -30,7 +30,7 @@ import org.apache.nemo.common.ir.vertex.IRVertex; import org.apache.nemo.common.ir.vertex.OperatorVertex; import org.apache.nemo.common.ir.vertex.SourceVertex; -import org.apache.nemo.common.ir.vertex.executionproperty.EnableWorkStealingProperty; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingSubSplitProperty; import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform; import org.apache.nemo.common.ir.vertex.transform.SignalTransform; @@ -292,7 +292,7 @@ irVertex, outputCollector, new TransformContextImpl(broadcastManagerWorker), dataFetcherOutputCollector)); } else { final String workStealingState = irVertexDag.getVertices().stream() - .map(v -> v.getPropertyValue(EnableWorkStealingProperty.class).orElse("DEFAULT")) + .map(v -> v.getPropertyValue(WorkStealingStateProperty.class).orElse("DEFAULT")) .filter(s -> !s.equals("DEFAULT")) .findFirst().orElse("DEFAULT"); final int numSubSplit = irVertexDag.getVertices().stream() @@ -731,7 +731,7 @@ private void finalizeOutputWriters(final VertexHarness vertexHarness) { private String getWorkStealingStrategy(final DAG> irVertexDag) { Set strategy = irVertexDag.getVertices().stream() - .map(vertex -> vertex.getPropertyValue(EnableWorkStealingProperty.class).orElse("DEFAULT")) + .map(vertex -> vertex.getPropertyValue(WorkStealingStateProperty.class).orElse("DEFAULT")) .collect(Collectors.toSet()); if (strategy.contains("SPLIT")) { return "SPLIT"; diff --git a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java index 096c53287e..23d19f52a7 100644 --- a/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java +++ b/runtime/master/src/main/java/org/apache/nemo/runtime/master/PlanStateManager.java @@ -595,7 +595,7 @@ private TaskState getTaskStateHelper(final String taskId) { return stageIdToTaskIdxToAttemptStates .get(RuntimeIdManager.getStageIdFromTaskId(taskId)) .get(RuntimeIdManager.getIndexFromTaskId(taskId)) - .get(RuntimeIdManager.getPartialFromTaskId(taskId)) + .get(RuntimeIdManager.getSubSplitIndexFromTaskId(taskId)) .get(RuntimeIdManager.getAttemptFromTaskId(taskId)); } @@ -609,7 +609,7 @@ private boolean isTaskNotDone(final TaskState taskState) { private List getPeerAttemptsForTheSameTaskIndex(final String taskId) { final String stageId = RuntimeIdManager.getStageIdFromTaskId(taskId); final int taskIndex = RuntimeIdManager.getIndexFromTaskId(taskId); - final int partialIndex = RuntimeIdManager.getPartialFromTaskId(taskId); + final int partialIndex = RuntimeIdManager.getSubSplitIndexFromTaskId(taskId); final int attempt = RuntimeIdManager.getAttemptFromTaskId(taskId); final List otherAttemptsforTheSameTaskIndex = From 9667326f36273096681e9e754a09922644c55f9c Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 19:56:40 +0900 Subject: [PATCH 25/26] checkstyles --- .../nemo/runtime/executor/datatransfer/BlockInputReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java index ce59e6d328..280d6aeceb 100644 --- a/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java +++ b/runtime/executor/src/main/java/org/apache/nemo/runtime/executor/datatransfer/BlockInputReader.java @@ -368,8 +368,8 @@ private int translateIndex(final String workStealingState, final int index) { if (workStealingState.equals(SPLIT_STRATEGY)) { /* SPLIT strategy */ - int sourceParallelism = InputReader.getSourceParallelism(this); - return Math.round(sourceParallelism / numSubSplit) * RuntimeIdManager.getSubSplitIndexFromTaskId(dstTaskId) + index; + int srcParallelism = InputReader.getSourceParallelism(this); + return Math.round(srcParallelism / numSubSplit) * RuntimeIdManager.getSubSplitIndexFromTaskId(dstTaskId) + index; } else if (workStealingState.equals(MERGE_STRATEGY) && srcVertex.getPropertyValue(WorkStealingStateProperty.class).orElse(DEFAULT_STRATEGY) .equals(SPLIT_STRATEGY)) { From 463a201bd5b6eba3b4551748e73b38037564d4dc Mon Sep 17 00:00:00 2001 From: hwarim Date: Sat, 28 Aug 2021 22:38:40 +0900 Subject: [PATCH 26/26] add tests --- .../nemo/compiler/CompilerTestUtil.java | 12 ++++ .../WorkStealingCompositePassTest.java | 66 +++++++++++++++++++ .../apache/nemo/examples/beam/WordCount.java | 2 +- .../task/ParentTaskDataFetcherTest.java | 12 ++-- .../executor/task/TaskExecutorTest.java | 1 + 5 files changed, 86 insertions(+), 7 deletions(-) create mode 100644 compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePassTest.java diff --git a/compiler/test/src/main/java/org/apache/nemo/compiler/CompilerTestUtil.java b/compiler/test/src/main/java/org/apache/nemo/compiler/CompilerTestUtil.java index 3f5001a0b7..4f8dafdc2b 100644 --- a/compiler/test/src/main/java/org/apache/nemo/compiler/CompilerTestUtil.java +++ b/compiler/test/src/main/java/org/apache/nemo/compiler/CompilerTestUtil.java @@ -104,6 +104,18 @@ public static IRDAG compileWordCountDAG() throws Exception { return compileDAG(mrArgBuilder.build()); } + public static IRDAG compileWordCountWorkStealingDAG() throws Exception { + final String input = ROOT_DIR + "/examples/resources/inputs/test_input_wordcount"; + final String output = ROOT_DIR + "/examples/resources/inputs/test_output"; + final String main = "org.apache.nemo.examples.beam.WordCount"; + + final ArgBuilder mrArgBuilder = new ArgBuilder() + .addJobId("WordCount") + .addUserMain(main) + .addUserArgs(input, output, "true"); + return compileDAG(mrArgBuilder.build()); + } + public static IRDAG compileALSDAG() throws Exception { final String input = ROOT_DIR + "/examples/resources/inputs/test_input_als"; final String numFeatures = "10"; diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePassTest.java new file mode 100644 index 0000000000..03d7a0980e --- /dev/null +++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/WorkStealingCompositePassTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.nemo.compiler.optimizer.pass.compiletime.composite; + +import org.apache.nemo.client.JobLauncher; +import org.apache.nemo.common.ir.IRDAG; +import org.apache.nemo.common.ir.vertex.IRVertex; +import org.apache.nemo.common.ir.vertex.executionproperty.WorkStealingStateProperty; +import org.apache.nemo.compiler.CompilerTestUtil; +import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static junit.framework.TestCase.assertEquals; + +/** + * Test {@link WorkStealingCompositePass} with MR workload. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(JobLauncher.class) +public class WorkStealingCompositePassTest { + private IRDAG mrDAG; + + @Before + public void setUp() throws Exception { + } + + @Test + public void testWorkStealingPass() throws Exception { + mrDAG = CompilerTestUtil.compileWordCountWorkStealingDAG(); + + final IRDAG processedDAG = new WorkStealingCompositePass().apply(new DefaultParallelismPass().apply(mrDAG)); + + int numSplitSVertex = 0; + int numMergeVertex = 0; + + for (IRVertex vertex : processedDAG.getTopologicalSort()) { + if (vertex.getPropertyValue(WorkStealingStateProperty.class).equals("SPLIT")) { + numSplitSVertex++; + } else if (vertex.getPropertyValue(WorkStealingStateProperty.class).equals("MERGE")) { + numMergeVertex++; + } + } + + assertEquals(numSplitSVertex, numMergeVertex); + } +} diff --git a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java index 0d2216d3d0..b1cd4ed462 100644 --- a/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java +++ b/examples/beam/src/main/java/org/apache/nemo/examples/beam/WordCount.java @@ -44,7 +44,7 @@ private WordCount() { public static void main(final String[] args) { final String inputFilePath = args[0]; final String outputFilePath = args[1]; - final boolean enableWorkStealing = Boolean.parseBoolean(args[2]); + final boolean enableWorkStealing = args.length > 2 && Boolean.parseBoolean(args[2]); final PipelineOptions options = NemoPipelineOptionsFactory.create(); options.setJobName("WordCount"); diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java index 3a35b4f5a9..832c676fdf 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java @@ -43,8 +43,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -59,7 +58,6 @@ public final class ParentTaskDataFetcherTest { public void testEmpty() throws Exception { final List empty = new ArrayList<>(0); // empty data final InputReader inputReader = generateInputReader(generateCompletableFuture(empty.iterator())); - // Fetcher final ParentTaskDataFetcher fetcher = createFetcher(inputReader); assertEquals(Finishmark.getInstance(), fetcher.fetchDataElement()); @@ -70,7 +68,6 @@ public void testNull() throws Exception { final List oneNull = new ArrayList<>(1); // empty data oneNull.add(null); final InputReader inputReader = generateInputReader(generateCompletableFuture(oneNull.iterator())); - // Fetcher final ParentTaskDataFetcher fetcher = createFetcher(inputReader); @@ -119,7 +116,9 @@ public void testErrorWhenFutureWithRetry() throws Exception { when(inputReader.retry(anyInt())) .thenReturn(generateCompletableFuture( empty.iterator())); // success upon retry - + when(inputReader.retry(anyString(), anyInt(), anyInt())) + .thenReturn(generateCompletableFuture( + empty.iterator())); // success upon retry // Fetcher should work on retry final ParentTaskDataFetcher fetcher = createFetcher(inputReader); assertEquals(Finishmark.getInstance(), fetcher.fetchDataElement()); @@ -144,7 +143,7 @@ private ParentTaskDataFetcher createFetcher(final InputReader readerForParentTas mock(OutputCollector.class), "DEFAULT", 1, - ""); + "DUMMY-0-*-0"); } private InputReader generateInputReader(final CompletableFuture completableFuture, @@ -152,6 +151,7 @@ private InputReader generateInputReader(final CompletableFuture completableFutur final InputReader inputReader = mock(InputReader.class, Mockito.CALLS_REAL_METHODS); when(inputReader.getSrcIrVertex()).thenReturn(mock(IRVertex.class)); when(inputReader.read()).thenReturn(Arrays.asList(completableFuture)); + when(inputReader.read(anyString(), anyInt(), anyInt())).thenReturn(Arrays.asList(completableFuture)); final ExecutionPropertyMap propertyMap = new ExecutionPropertyMap<>(""); for (final EdgeExecutionProperty p : properties) { propertyMap.put(p); diff --git a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/TaskExecutorTest.java b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/TaskExecutorTest.java index 3e33fec1f4..6c6eeb79dd 100644 --- a/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/TaskExecutorTest.java +++ b/runtime/executor/src/test/java/org/apache/nemo/runtime/executor/task/TaskExecutorTest.java @@ -589,6 +589,7 @@ public InputReader answer(final InvocationOnMock invocationOnMock) throws Throwa srcVertex.setProperty(ParallelismProperty.of(SOURCE_PARALLELISM)); when(inputReader.getSrcIrVertex()).thenReturn(srcVertex); when(inputReader.read()).thenReturn(inputFutures); + when(inputReader.read(anyString(), anyInt(), anyInt())).thenReturn(inputFutures); when(inputReader.getProperties()).thenReturn(new ExecutionPropertyMap<>("")); return inputReader; }