You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nemo.apache.org by sa...@apache.org on 2019/02/01 01:32:23 UTC

[incubator-nemo] branch master updated: [NEMO-328] Refactor IRDAG (#190)

This is an automated email from the ASF dual-hosted git repository.

sanha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-nemo.git


The following commit(s) were added to refs/heads/master by this push:
     new a49b050  [NEMO-328] Refactor IRDAG (#190)
a49b050 is described below

commit a49b05076d85c746d85d77710b8987ff4e4906ce
Author: John Yang <jo...@gmail.com>
AuthorDate: Fri Feb 1 10:32:18 2019 +0900

    [NEMO-328] Refactor IRDAG (#190)
    
    JIRA: [NEMO-328: Refactor IRDAG](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-328)
    
    **Major changes:**
    - Modifies the Pass interface to use IRDAG, rather than directly using DAG
    - Modifies reshaping passes (e.g., LargeShuffle, Skew) to use the IRDAG reshaping methods
    
    **Minor changes to note:**
    - Merges multiple annotating LargeShufflePasses into a single LargeShuffleAnnotatingPass
    
    **Tests for the changes:**
    - Existing tests pass
    
    **Other comments:**
    - N/A
    
    Closes #190
---
 .../java/org/apache/nemo/client/JobLauncher.java   |  10 +-
 .../nemo/common/coder/PairDecoderFactory.java      |  11 +
 .../nemo/common/coder/PairEncoderFactory.java      |  11 +
 .../main/java/org/apache/nemo/common/dag/DAG.java  | 161 ++-------
 .../org/apache/nemo/common/dag/DAGBuilder.java     |  23 +-
 .../org/apache/nemo/common/dag/DAGInterface.java   | 216 ++++++++++++
 .../main/java/org/apache/nemo/common/ir/IRDAG.java | 384 +++++++++++++++++++++
 .../apache/nemo/common/ir/vertex/LoopVertex.java   |   5 +
 .../nemo/common/ir/vertex/OperatorVertex.java      |   8 +-
 .../ir/vertex/system/MessageAggregatorVertex.java  |  27 +-
 .../ir/vertex/system/MessageBarrierVertex.java     |  47 +++
 .../nemo/common/ir/vertex/system/StreamVertex.java |  23 +-
 ...nsform.java => MessageAggregatorTransform.java} |  21 +-
 ...Transform.java => MessageBarrierTransform.java} |  47 ++-
 .../{RelayTransform.java => StreamTransform.java}  |   8 +-
 .../java/org/apache/nemo/common/pass/Pass.java     |  12 +-
 .../apache/nemo/common/test/EmptyComponents.java   |  10 +-
 .../org/apache/nemo/common/ir/LoopVertexTest.java  |   9 +-
 .../org/apache/nemo/compiler/backend/Backend.java  |   6 +-
 .../nemo/compiler/backend/nemo/NemoBackend.java    |   5 +-
 .../compiler/backend/nemo/NemoBackendTest.java     |   7 +-
 .../compiler/frontend/beam/PipelineVisitor.java    |   8 +-
 .../frontend/spark/core/SparkFrontendUtils.java    |   3 +-
 .../compiler/frontend/spark/core/rdd/RDD.scala     |   3 +-
 .../nemo/compiler/optimizer/NemoOptimizer.java     |  91 ++---
 .../apache/nemo/compiler/optimizer/Optimizer.java  |   6 +-
 .../MapReduceDisaggregationOptimization.java       |   9 +-
 .../pass/compiletime/CompileTimePass.java          |   6 +-
 .../AggressiveSpeculativeCloningPass.java          |   6 +-
 .../compiletime/annotating/CompressionPass.java    |   7 +-
 .../compiletime/annotating/DecompressionPass.java  |   7 +-
 .../annotating/DefaultDataPersistencePass.java     |   8 +-
 .../annotating/DefaultDataStorePass.java           |   6 +-
 .../annotating/DefaultEdgeDecoderPass.java         |   6 +-
 .../annotating/DefaultEdgeEncoderPass.java         |   6 +-
 .../compiletime/annotating/DefaultMetricPass.java  |   6 +-
 .../annotating/DefaultParallelismPass.java         |  10 +-
 .../annotating/DefaultScheduleGroupPass.java       |   8 +-
 .../DisaggregationEdgeDataStorePass.java           |   5 +-
 .../annotating/DuplicateEdgeGroupSizePass.java     |   7 +-
 .../annotating/LargeShuffleAnnotatingPass.java     |  96 ++++++
 .../annotating/LargeShuffleCompressionPass.java    |  62 ----
 .../annotating/LargeShuffleDataFlowPass.java       |  59 ----
 .../LargeShuffleDataPersistencePass.java           |  54 ---
 .../annotating/LargeShuffleDataStorePass.java      |  63 ----
 .../annotating/LargeShuffleDecoderPass.java        |  59 ----
 .../annotating/LargeShuffleDecompressionPass.java  |  63 ----
 .../annotating/LargeShuffleEncoderPass.java        |  61 ----
 .../annotating/LargeShufflePartitionerPass.java    |  63 ----
 .../annotating/LargeShuffleResourceSlotPass.java   |  60 ----
 .../annotating/PipeTransferForAllEdgesPass.java    |   6 +-
 .../annotating/ResourceLocalityPass.java           |   6 +-
 .../compiletime/annotating/ResourceSitePass.java   |  18 +-
 .../compiletime/annotating/ResourceSlotPass.java   |   6 +-
 .../annotating/ShuffleEdgePushPass.java            |   5 +-
 .../annotating/SkewPartitionerPass.java            |   6 +-
 .../annotating/SkewResourceSkewedDataPass.java     |   6 +-
 .../annotating/TransientResourceDataFlowPass.java  |   5 +-
 .../annotating/TransientResourceDataStorePass.java |   5 +-
 .../annotating/TransientResourcePriorityPass.java  |   5 +-
 .../compiletime/annotating/UpfrontCloningPass.java |   6 +-
 .../pass/compiletime/composite/CompositePass.java  |  14 +-
 .../composite/LargeShuffleCompositePass.java       |  14 +-
 .../compiletime/composite/SkewCompositePass.java   |   3 +-
 .../CommonSubexpressionEliminationPass.java        | 103 +++---
 .../reshaping/LargeShuffleRelayReshapingPass.java  |  85 -----
 .../LargeShuffleReshapingPass.java}                |  41 +--
 .../compiletime/reshaping/LoopExtractionPass.java  |  19 +-
 .../compiletime/reshaping/LoopOptimizations.java   | 199 ++++++-----
 .../compiletime/reshaping/LoopUnrollingPass.java   |   8 +-
 .../compiletime/reshaping/SkewReshapingPass.java   | 265 ++++----------
 .../compiler/optimizer/policy/BasicPullPolicy.java |   8 +-
 .../compiler/optimizer/policy/BasicPushPolicy.java |   8 +-
 .../policy/ConditionalLargeShufflePolicy.java      |   8 +-
 .../compiler/optimizer/policy/DataSkewPolicy.java  |  10 +-
 .../compiler/optimizer/policy/DefaultPolicy.java   |   6 +-
 .../policy/DefaultPolicyWithSeparatePass.java      |   8 +-
 .../optimizer/policy/DisaggregationPolicy.java     |   8 +-
 .../optimizer/policy/LargeShufflePolicy.java       |   8 +-
 .../nemo/compiler/optimizer/policy/Policy.java     |   6 +-
 .../compiler/optimizer/policy/PolicyBuilder.java   |  15 +-
 .../nemo/compiler/optimizer/policy/PolicyImpl.java |  54 +--
 .../compiler/optimizer/policy/StreamingPolicy.java |   6 +-
 .../optimizer/policy/TransientResourcePolicy.java  |   6 +-
 .../compiler/optimizer/policy/PolicyImplTest.java  |   8 +-
 .../org/apache/nemo/compiler/CompilerTestUtil.java |  16 +-
 .../nemo/compiler/optimizer/policy/TestPolicy.java |   6 +-
 .../compiler/backend/nemo/DAGConverterTest.java    |   5 +-
 .../frontend/beam/BeamFrontendALSTest.java         |   3 +-
 .../frontend/beam/BeamFrontendMLRTest.java         |   3 +-
 .../annotating/DefaultEdgeCoderPassTest.java       |   7 +-
 .../annotating/DefaultParallelismPassTest.java     |   7 +-
 .../annotating/DefaultScheduleGroupPassTest.java   |  32 +-
 .../composite/DisaggregationPassTest.java          |   8 +-
 .../composite/LargeShuffleCompositePassTest.java   |   5 +-
 .../composite/SkewCompositePassTest.java           |  31 +-
 .../TransientResourceCompositePassTest.java        |   5 +-
 .../CommonSubexpressionEliminationPassTest.java    |  17 +-
 .../reshaping/LoopExtractionPassTest.java          |   5 +-
 .../compiletime/reshaping/LoopFusionPassTest.java  |  70 ++--
 .../LoopInvariantCodeMotionALSInefficientTest.java |   7 +-
 .../reshaping/LoopInvariantCodeMotionPassTest.java |  13 +-
 .../reshaping/LoopUnrollingPassTest.java           |   5 +-
 ...iveSpeculativeCloningPolicyParallelismFive.java |   6 +-
 .../beam/policy/DataSkewPolicyParallelismFive.java |   3 +-
 .../beam/policy/DefaultPolicyParallelismFive.java  |   3 +-
 .../DisaggregationPolicyParallelismFive.java       |   3 +-
 .../policy/LargeShufflePolicyParallelismFive.java  |   3 +-
 .../policy/StreamingPolicyParallelismFive.java     |   3 +-
 .../TransientResourcePolicyParallelismFive.java    |   3 +-
 .../TransientResourcePolicyParallelismTen.java     |   3 +-
 .../UpfrontSchedulingPolicyParallelismFive.java    |   3 +-
 .../DedicatedKeyPerElementPartitioner.java         |   2 +-
 .../runtime/common/plan/PhysicalPlanGenerator.java |   7 +-
 .../nemo/runtime/common/plan/StagePartitioner.java |   8 +-
 .../common/plan/PhysicalPlanGeneratorTest.java     |   5 +-
 .../runtime/common/plan/StagePartitionerTest.java  |  15 +-
 .../apache/nemo/driver/UserApplicationRunner.java  |   8 +-
 .../nemo/runtime/executor/task/TaskExecutor.java   |   4 +-
 .../runtime/executor/task/TaskExecutorTest.java    |  32 +-
 .../runtime/master/scheduler/BatchScheduler.java   |   2 +-
 .../runtime/common/plan/TestPlanGenerator.java     |  13 +-
 122 files changed, 1572 insertions(+), 1682 deletions(-)

diff --git a/client/src/main/java/org/apache/nemo/client/JobLauncher.java b/client/src/main/java/org/apache/nemo/client/JobLauncher.java
index e849aca..113e916 100644
--- a/client/src/main/java/org/apache/nemo/client/JobLauncher.java
+++ b/client/src/main/java/org/apache/nemo/client/JobLauncher.java
@@ -20,7 +20,7 @@ package org.apache.nemo.client;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.conf.JobConf;
 import org.apache.nemo.driver.NemoDriver;
 import org.apache.nemo.runtime.common.comm.ControlMessage;
@@ -206,7 +206,7 @@ public final class JobLauncher {
    * @param dag the application DAG.
    */
   // When modifying the signature of this method, see CompilerTestUtil#compileDAG and make corresponding changes
-  public static void launchDAG(final DAG dag) {
+  public static void launchDAG(final IRDAG dag) {
     launchDAG(dag, Collections.emptyMap(), "");
   }
 
@@ -214,7 +214,7 @@ public final class JobLauncher {
    * @param dag the application DAG.
    * @param jobId job ID.
    */
-  public static void launchDAG(final DAG dag, final String jobId) {
+  public static void launchDAG(final IRDAG dag, final String jobId) {
     launchDAG(dag, Collections.emptyMap(), jobId);
   }
 
@@ -223,7 +223,9 @@ public final class JobLauncher {
    * @param broadcastVariables broadcast variables (can be empty).
    * @param jobId job ID.
    */
-  public static void launchDAG(final DAG dag, final Map<Serializable, Object> broadcastVariables, final String jobId) {
+  public static void launchDAG(final IRDAG dag,
+                               final Map<Serializable, Object> broadcastVariables,
+                               final String jobId) {
     // launch driver if it hasn't been already
     if (driverReadyLatch == null) {
       try {
diff --git a/common/src/main/java/org/apache/nemo/common/coder/PairDecoderFactory.java b/common/src/main/java/org/apache/nemo/common/coder/PairDecoderFactory.java
index 519a2e2..790690d 100644
--- a/common/src/main/java/org/apache/nemo/common/coder/PairDecoderFactory.java
+++ b/common/src/main/java/org/apache/nemo/common/coder/PairDecoderFactory.java
@@ -63,6 +63,17 @@ public final class PairDecoderFactory<A, B> implements DecoderFactory<Pair<A, B>
     return new PairDecoder<>(inputStream, leftDecoderFactory, rightDecoderFactory);
   }
 
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder();
+    sb.append("Pair(");
+    sb.append(leftDecoderFactory.toString());
+    sb.append(", ");
+    sb.append(rightDecoderFactory.toString());
+    sb.append(")");
+    return sb.toString();
+  }
+
   /**
    * PairDecoder.
    * @param <T1> type for the left coder.
diff --git a/common/src/main/java/org/apache/nemo/common/coder/PairEncoderFactory.java b/common/src/main/java/org/apache/nemo/common/coder/PairEncoderFactory.java
index 12edfcc..030c336 100644
--- a/common/src/main/java/org/apache/nemo/common/coder/PairEncoderFactory.java
+++ b/common/src/main/java/org/apache/nemo/common/coder/PairEncoderFactory.java
@@ -62,6 +62,17 @@ public final class PairEncoderFactory<A, B> implements EncoderFactory<Pair<A, B>
     return new PairEncoder<>(outputStream, leftEncoderFactory, rightEncoderFactory);
   }
 
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder();
+    sb.append("Pair(");
+    sb.append(leftEncoderFactory.toString());
+    sb.append(", ");
+    sb.append(rightEncoderFactory.toString());
+    sb.append(")");
+    return sb.toString();
+  }
+
   /**
    * PairEncoder.
    * @param <T1> type for the left coder.
diff --git a/common/src/main/java/org/apache/nemo/common/dag/DAG.java b/common/src/main/java/org/apache/nemo/common/dag/DAG.java
index b0b34d4..e6a55b7 100644
--- a/common/src/main/java/org/apache/nemo/common/dag/DAG.java
+++ b/common/src/main/java/org/apache/nemo/common/dag/DAG.java
@@ -30,7 +30,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.io.Serializable;
 import java.util.*;
 import java.util.function.Consumer;
 import java.util.function.Predicate;
@@ -41,7 +40,7 @@ import java.util.stream.Collectors;
  * @param <V> the vertex type
  * @param <E> the edge type
  */
-public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializable {
+public final class DAG<V extends Vertex, E extends Edge<V>> implements DAGInterface<V, E> {
   private static final Logger LOG = LoggerFactory.getLogger(DAG.class.getName());
 
   private final List<V> vertices;
@@ -88,12 +87,7 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     loopStackDepthMap.forEach(((v, integer) -> this.loopStackDepthMap.put(v.getId(), integer)));
   }
 
-  /**
-   * Retrieves the vertex given its ID.
-   * @param id of the vertex to retrieve
-   * @return the vertex
-   * @throws IllegalVertexOperationException when the requested vertex does not exist.
-   */
+  @Override
   public V getVertexById(final String id) {
     for (final V vertex : vertices) {
       if (vertex.getId().equals(id)) {
@@ -103,88 +97,47 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     throw new IllegalVertexOperationException("There is no vertex of id: " + id);
   }
 
-  /**
-   * Retrieves the vertices of this DAG.
-   * @return the set of vertices.
-   * Note that the result is never null, ensured by {@link DAGBuilder}.
-   */
+  @Override
   public List<V> getVertices() {
     return vertices;
   }
 
-  /**
-   * Retrieves the root vertices of this DAG.
-   * @return the set of root vertices.
-   */
+  @Override
   public List<V> getRootVertices() {
     return rootVertices;
   }
 
-  /**
-   * Retrieves the incoming edges of the given vertex.
-   * @param v the subject vertex.
-   * @return the set of incoming edges to the vertex.
-   * Note that the result is never null, ensured by {@link DAGBuilder}.
-   */
+  @Override
   public List<E> getIncomingEdgesOf(final V v) {
     return getIncomingEdgesOf(v.getId());
   }
 
-  /**
-   * Retrieves the incoming edges of the given vertex.
-   * @param vertexId the ID of the subject vertex.
-   * @return the set of incoming edges to the vertex.
-   * Note that the result is never null, ensured by {@link DAGBuilder}.
-   */
+  @Override
   public List<E> getIncomingEdgesOf(final String vertexId) {
     return incomingEdges.get(vertexId);
   }
 
-  /**
-   * Retrieves the outgoing edges of the given vertex.
-   * @param v the subject vertex.
-   * @return the set of outgoing edges to the vertex.
-   * Note that the result is never null, ensured by {@link DAGBuilder}.
-   */
+  @Override
   public List<E> getOutgoingEdgesOf(final V v) {
     return getOutgoingEdgesOf(v.getId());
   }
 
-  /**
-   * Retrieves the outgoing edges of the given vertex.
-   * @param vertexId the ID of the subject vertex.
-   * @return the set of outgoing edges to the vertex.
-   * Note that the result is never null, ensured by {@link DAGBuilder}.
-   */
+  @Override
   public List<E> getOutgoingEdgesOf(final String vertexId) {
     return outgoingEdges.get(vertexId);
   }
 
-  /**
-   * Retrieves the parent vertices of the given vertex.
-   * @param vertexId the ID of the subject vertex.
-   * @return the list of parent vertices.
-   */
+  @Override
   public List<V> getParents(final String vertexId) {
     return incomingEdges.get(vertexId).stream().map(Edge::getSrc).collect(Collectors.toList());
   }
 
-  /**
-   * Retrieves the children vertices of the given vertex.
-   * @param vertexId the ID of the subject vertex.
-   * @return the list of children vertices.
-   */
+  @Override
   public List<V> getChildren(final String vertexId) {
     return outgoingEdges.get(vertexId).stream().map(Edge::getDst).collect(Collectors.toList());
   }
 
-  /**
-   * Retrieves the edge between two vertices.
-   * @param srcVertexId the ID of the source vertex.
-   * @param dstVertexId the ID of the destination vertex.
-   * @return the edge if exists.
-   * @throws IllegalEdgeOperationException otherwise.
-   */
+  @Override
   public E getEdgeBetween(final String srcVertexId, final String dstVertexId) throws IllegalEdgeOperationException {
     for (E e : incomingEdges.get(dstVertexId)) {
       if (e.getSrc().getId().equals(srcVertexId)) {
@@ -195,30 +148,14 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
         new Throwable("There exists no edge from " + srcVertexId + " to " + dstVertexId));
   }
 
-  /**
-   * Indicates the traversal order of this DAG.
-   */
-  private enum TraversalOrder {
-    PreOrder,
-    PostOrder
-  }
-
-  /**
-   * Gets the DAG's vertices in topologically sorted order.
-   * This function brings consistent results.
-   * @return the sorted list of vertices in topological order.
-   */
+  @Override
   public List<V> getTopologicalSort() {
     final List<V> sortedList = new ArrayList<>(vertices.size());
     topologicalDo(sortedList::add);
     return sortedList;
   }
 
-  /**
-   * Retrieves the ancestors of a vertex.
-   * @param vertexId to find the ancestors for.
-   * @return the list of ancestors.
-   */
+  @Override
   public List<V> getAncestors(final String vertexId) {
     final List<V> ancestors = new ArrayList<>();
     addAncestors(ancestors, vertexId);
@@ -237,11 +174,7 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     });
   }
 
-  /**
-   * Retrieves the descendants of a vertex.
-   * @param vertexId to find the descendants for.
-   * @return the list of descendants.
-   */
+  @Override
   public List<V> getDescendants(final String vertexId) {
     final List<V> descendants = new ArrayList<>();
     final Set<V> visited = new HashSet<>();
@@ -251,21 +184,13 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     return descendants;
   }
 
-  /**
-   * Filters the vertices according to the given condition.
-   * @param condition that must be satisfied to be included in the filtered list.
-   * @return the list of vertices that meet the condition.
-   */
+  @Override
   public List<V> filterVertices(final Predicate<V> condition) {
     final List<V> filteredVertices = vertices.stream().filter(condition).collect(Collectors.toList());
     return filteredVertices;
   }
 
-  /**
-   * Applies the function to each node in the DAG in a topological order.
-   * This function brings consistent results.
-   * @param function to apply.
-   */
+  @Override
   public void topologicalDo(final Consumer<V> function) {
     final Stack<V> stack = new Stack<>();
     dfsTraverse(stack::push, TraversalOrder.PostOrder);
@@ -274,26 +199,16 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     }
   }
 
-  /**
-   * Traverses the DAG by DFS, applying the given function.
-   * @param function to apply.
-   * @param traversalOrder which the DFS should be conducted.
-   */
-  private void dfsTraverse(final Consumer<V> function, final TraversalOrder traversalOrder) {
+  @Override
+  public void dfsTraverse(final Consumer<V> function, final TraversalOrder traversalOrder) {
     final Set<V> visited = new HashSet<>();
     getVertices().stream().filter(vertex -> incomingEdges.get(vertex.getId()).isEmpty()) // root Operators
         .filter(vertex -> !visited.contains(vertex))
         .forEachOrdered(vertex -> dfsDo(vertex, function, traversalOrder, visited));
   }
 
-  /**
-   * A recursive helper function for {@link #dfsTraverse(Consumer, TraversalOrder)}.
-   * @param vertex the root vertex of the remaining DAG.
-   * @param vertexConsumer the function to apply.
-   * @param traversalOrder which the DFS should be conducted.
-   * @param visited the set of nodes visited.
-   */
-  private void dfsDo(final V vertex,
+  @Override
+  public void dfsDo(final V vertex,
                      final Consumer<V> vertexConsumer,
                      final TraversalOrder traversalOrder,
                      final Set<V> visited) {
@@ -312,12 +227,7 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     }
   }
 
-  /**
-   * Function checks whether there is a path between two vertices.
-   * @param v1 First vertex to check.
-   * @param v2 Second vertex to check.
-   * @return Whether or not there is a path between two vertices.
-   */
+  @Override
   public Boolean pathExistsBetween(final V v1, final V v2) {
     final Set<V> reachableFromV1 = new HashSet<>();
     final Set<V> reachableFromV2 = new HashSet<>();
@@ -328,36 +238,22 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
     return reachableFromV1.contains(v2) || reachableFromV2.contains(v1);
   }
 
-  /**
-   * Checks whether the given vertex is assigned with a wrapping LoopVertex.
-   * @param v Vertex to check.
-   * @return whether or not it is wrapped by a LoopVertex
-   */
+  @Override
   public Boolean isCompositeVertex(final V v) {
     return this.assignedLoopVertexMap.containsKey(v.getId());
   }
 
-  /**
-   * Retrieves the wrapping LoopVertex of the vertex.
-   * @param v Vertex to check.
-   * @return The wrapping LoopVertex.
-   */
+  @Override
   public LoopVertex getAssignedLoopVertexOf(final V v) {
     return this.assignedLoopVertexMap.get(v.getId());
   }
 
-  /**
-   * Retrieves the stack depth of the given vertex.
-   * @param v Vertex to check.
-   * @return The depth of the stack of LoopVertices for the vertex.
-   */
+  @Override
   public Integer getLoopStackDepthOf(final V v) {
     return this.loopStackDepthMap.get(v.getId());
   }
 
-  /**
-   * @return {@link com.fasterxml.jackson.databind.JsonNode} for this DAG.
-   */
+  @Override
   public ObjectNode asJsonNode() {
     final ObjectMapper mapper = new ObjectMapper();
     final ObjectNode node = mapper.createObjectNode();
@@ -393,12 +289,7 @@ public final class DAG<V extends Vertex, E extends Edge<V>> implements Serializa
 
   public static final String EMPTY_DAG_DIRECTORY = "";
 
-  /**
-   * Stores JSON representation of this DAG into a file.
-   * @param directory the directory which JSON representation is saved to
-   * @param name name of this DAG
-   * @param description description of this DAG
-   */
+  @Override
   public void storeJSON(final String directory, final String name, final String description) {
     if (directory == null || directory.equals(EMPTY_DAG_DIRECTORY)) {
       return;
diff --git a/common/src/main/java/org/apache/nemo/common/dag/DAGBuilder.java b/common/src/main/java/org/apache/nemo/common/dag/DAGBuilder.java
index 30826a8..5d7a311 100644
--- a/common/src/main/java/org/apache/nemo/common/dag/DAGBuilder.java
+++ b/common/src/main/java/org/apache/nemo/common/dag/DAGBuilder.java
@@ -24,7 +24,7 @@ import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
 import org.apache.nemo.common.ir.vertex.*;
 import org.apache.nemo.common.exception.IllegalVertexOperationException;
-import org.apache.nemo.common.ir.vertex.transform.AggregateMetricTransform;
+import org.apache.nemo.common.ir.vertex.system.MessageAggregatorVertex;
 
 import java.io.Serializable;
 import java.util.*;
@@ -210,7 +210,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
     stack.push(vertex);
     // When we encounter a vertex that we've already gone through, then there is a cycle.
     if (outgoingEdges.get(vertex).stream().map(Edge::getDst).anyMatch(stack::contains)) {
-      throw new CompileTimeOptimizationException("DAG contains a cycle");
+      throw getException("DAG contains a cycle", vertex.toString());
     } else {
       outgoingEdges.get(vertex).stream().map(Edge::getDst)
           .filter(v -> !visited.contains(v))
@@ -232,7 +232,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
           .filter(v -> !(v instanceof SourceVertex))
           .map(V::getId)
           .collect(Collectors.toList()).toString();
-      throw new CompileTimeOptimizationException("DAG source check failed while building DAG. " + problematicVertices);
+      throw getException("DAG source check failed while building DAG", problematicVertices);
     }
   }
 
@@ -250,7 +250,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
       final String problematicVertices = verticesToObserve.get().filter(v ->
           !(v instanceof OperatorVertex || v instanceof LoopVertex))
           .map(V::getId).collect(Collectors.toList()).toString();
-      throw new CompileTimeOptimizationException("DAG sink check failed while building DAG: " + problematicVertices);
+      throw getException("DAG sink check failed while building DAG", problematicVertices);
     }
   }
 
@@ -262,7 +262,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
     vertices.forEach(v -> incomingEdges.get(v).stream().filter(e -> e instanceof IREdge).map(e -> (IREdge) e)
         .filter(e -> e.getPropertyValue(MetricCollectionProperty.class).isPresent())
         .filter(e -> !(e.getDst() instanceof OperatorVertex
-          && ((OperatorVertex) e.getDst()).getTransform() instanceof AggregateMetricTransform))
+          && e.getDst() instanceof MessageAggregatorVertex))
         .filter(e -> DataFlowProperty.Value.Push.equals(e.getPropertyValue(DataFlowProperty.class).get()))
         .forEach(e -> {
           throw new CompileTimeOptimizationException("DAG execution property check: "
@@ -330,4 +330,17 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
     integrityCheck(true, true, true, true);
     return new DAG<>(vertices, incomingEdges, outgoingEdges, assignedLoopVertexMap, loopStackDepthMap);
   }
+
+  /**
+   * Generates a user-friendly exception message.
+   * @param reason of the exception.
+   * @param problematicObjects that caused the exception.
+   * @return exception object.
+   */
+  private CompileTimeOptimizationException getException(final String reason, final String problematicObjects) {
+    final DAG erroredDAG = new DAG<>(vertices, incomingEdges, outgoingEdges, assignedLoopVertexMap, loopStackDepthMap);
+    erroredDAG.storeJSON("debug", "errored_ir", "Errored IR");
+    return new CompileTimeOptimizationException(reason + " /// Problematic objects are: "
+      + problematicObjects + " /// see the debug directory for the errored_ir");
+  }
 }
diff --git a/common/src/main/java/org/apache/nemo/common/dag/DAGInterface.java b/common/src/main/java/org/apache/nemo/common/dag/DAGInterface.java
new file mode 100644
index 0000000..40cb72c
--- /dev/null
+++ b/common/src/main/java/org/apache/nemo/common/dag/DAGInterface.java
@@ -0,0 +1,216 @@
+/*
+ * 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.dag;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.nemo.common.exception.IllegalEdgeOperationException;
+import org.apache.nemo.common.ir.vertex.LoopVertex;
+
+import java.io.Serializable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+/**
+ * Methods for querying a directed-acyclic graph (DAG).
+ * Implementations of the methods must not modify the topology of the DAG.
+ *
+ * @param <V> the vertex type
+ * @param <E> the edge type
+ */
+public interface DAGInterface<V extends Vertex, E extends Edge<V>> extends Serializable {
+  /**
+   * Retrieves the vertex given its ID.
+   * @param id of the vertex to retrieve
+   * @return the vertex
+   */
+  V getVertexById(final String id);
+
+  /**
+   * Retrieves the vertices of this DAG.
+   * @return the set of vertices.
+   * Note that the result is never null, ensured by {@link DAGBuilder}.
+   */
+  List<V> getVertices();
+
+  /**
+   * Retrieves the root vertices of this DAG.
+   * @return the set of root vertices.
+   */
+  List<V> getRootVertices();
+
+  /**
+   * Retrieves the incoming edges of the given vertex.
+   * @param v the subject vertex.
+   * @return the set of incoming edges to the vertex.
+   * Note that the result is never null, ensured by {@link DAGBuilder}.
+   */
+  List<E> getIncomingEdgesOf(final V v);
+
+  /**
+   * Retrieves the incoming edges of the given vertex.
+   * @param vertexId the ID of the subject vertex.
+   * @return the set of incoming edges to the vertex.
+   * Note that the result is never null, ensured by {@link DAGBuilder}.
+   */
+  List<E> getIncomingEdgesOf(final String vertexId);
+
+  /**
+   * Retrieves the outgoing edges of the given vertex.
+   * @param v the subject vertex.
+   * @return the set of outgoing edges to the vertex.
+   * Note that the result is never null, ensured by {@link DAGBuilder}.
+   */
+  List<E> getOutgoingEdgesOf(final V v);
+
+  /**
+   * Retrieves the outgoing edges of the given vertex.
+   * @param vertexId the ID of the subject vertex.
+   * @return the set of outgoing edges to the vertex.
+   * Note that the result is never null, ensured by {@link DAGBuilder}.
+   */
+  List<E> getOutgoingEdgesOf(final String vertexId);
+
+  /**
+   * Retrieves the parent vertices of the given vertex.
+   * @param vertexId the ID of the subject vertex.
+   * @return the list of parent vertices.
+   */
+  List<V> getParents(final String vertexId);
+
+  /**
+   * Retrieves the children vertices of the given vertex.
+   * @param vertexId the ID of the subject vertex.
+   * @return the list of children vertices.
+   */
+  List<V> getChildren(final String vertexId);
+
+  /**
+   * Retrieves the edge between two vertices.
+   * @param srcVertexId the ID of the source vertex.
+   * @param dstVertexId the ID of the destination vertex.
+   * @return the edge if exists.
+   * @throws IllegalEdgeOperationException otherwise.
+   */
+  E getEdgeBetween(final String srcVertexId, final String dstVertexId) throws IllegalEdgeOperationException;
+
+  /**
+   * Gets the DAG's vertices in topologically sorted order.
+   * This function brings consistent results.
+   * @return the sorted list of vertices in topological order.
+   */
+  List<V> getTopologicalSort();
+
+  /**
+   * Retrieves the ancestors of a vertex.
+   * @param vertexId to find the ancestors for.
+   * @return the list of ancestors.
+   */
+  List<V> getAncestors(final String vertexId);
+
+  /**
+   * Retrieves the descendants of a vertex.
+   * @param vertexId to find the descendants for.
+   * @return the list of descendants.
+   */
+  List<V> getDescendants(final String vertexId);
+
+  /**
+   * Filters the vertices according to the given condition.
+   * @param condition that must be satisfied to be included in the filtered list.
+   * @return the list of vertices that meet the condition.
+   */
+  List<V> filterVertices(final Predicate<V> condition);
+
+  /**
+   * Applies the function to each node in the DAG in a topological order.
+   * This function brings consistent results.
+   * @param function to apply.
+   */
+  void topologicalDo(final Consumer<V> function);
+
+  /**
+   * Indicates the traversal order of this DAG.
+   */
+  enum TraversalOrder {
+    PreOrder,
+    PostOrder
+  }
+
+  /**
+   * Traverses the DAG by DFS, applying the given function.
+   * @param function to apply.
+   * @param traversalOrder which the DFS should be conducted.
+   */
+  void dfsTraverse(final Consumer<V> function, final TraversalOrder traversalOrder);
+
+  /**
+   * A recursive helper function for {@link #dfsTraverse(Consumer, TraversalOrder)}.
+   * @param vertex the root vertex of the remaining DAG.
+   * @param vertexConsumer the function to apply.
+   * @param traversalOrder which the DFS should be conducted.
+   * @param visited the set of nodes visited.
+   */
+  void dfsDo(final V vertex,
+             final Consumer<V> vertexConsumer,
+             final TraversalOrder traversalOrder,
+             final Set<V> visited);
+
+  /**
+   * Function checks whether there is a path between two vertices.
+   * @param v1 First vertex to check.
+   * @param v2 Second vertex to check.
+   * @return Whether or not there is a path between two vertices.
+   */
+  Boolean pathExistsBetween(final V v1, final V v2);
+
+  /**
+   * Checks whether the given vertex is assigned with a wrapping LoopVertex.
+   * @param v Vertex to check.
+   * @return whether or not it is wrapped by a LoopVertex
+   */
+  Boolean isCompositeVertex(final V v);
+
+  /**
+   * Retrieves the stack depth of the given vertex.
+   * @param v Vertex to check.
+   * @return The depth of the stack of LoopVertices for the vertex.
+   */
+  Integer getLoopStackDepthOf(final V v);
+
+  /**
+   * Retrieves the wrapping LoopVertex of the vertex.
+   * @param v Vertex to check.
+   * @return The wrapping LoopVertex.
+   */
+  LoopVertex getAssignedLoopVertexOf(final V v);
+
+  /**
+   * @return {@link com.fasterxml.jackson.databind.JsonNode} for this DAG.
+   */
+  ObjectNode asJsonNode();
+
+  /**
+   * Stores JSON representation of this DAG into a file.
+   * @param directory the directory which JSON representation is saved to
+   * @param name name of this DAG
+   * @param description description of this DAG
+   */
+  void storeJSON(final String directory, final String name, final String description);
+}
diff --git a/common/src/main/java/org/apache/nemo/common/ir/IRDAG.java b/common/src/main/java/org/apache/nemo/common/ir/IRDAG.java
new file mode 100644
index 0000000..e2e9480
--- /dev/null
+++ b/common/src/main/java/org/apache/nemo/common/ir/IRDAG.java
@@ -0,0 +1,384 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.nemo.common.KeyExtractor;
+import org.apache.nemo.common.Pair;
+import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.dag.DAGInterface;
+import org.apache.nemo.common.exception.IllegalEdgeOperationException;
+import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.edge.executionproperty.*;
+import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.vertex.LoopVertex;
+import org.apache.nemo.common.ir.vertex.system.MessageAggregatorVertex;
+import org.apache.nemo.common.ir.vertex.system.MessageBarrierVertex;
+import org.apache.nemo.common.ir.vertex.system.StreamVertex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * An IRDAG object captures a high-level data processing application (e.g., Spark/Beam application).
+ * - IRVertex: A data-parallel operation. (e.g., map)
+ * - IREdge: A data dependency between two operations. (e.g., shuffle)
+ *
+ * Largely two types of IRDAG optimization(modification) methods are provided.
+ * All of these methods preserve application semantics.
+ * - Annotation: setProperty(), getPropertyValue() on each IRVertex/IREdge
+ * - Reshaping: insert(), delete() on the IRDAG
+ */
+public final class IRDAG implements DAGInterface<IRVertex, IREdge> {
+  private static final Logger LOG = LoggerFactory.getLogger(IRDAG.class.getName());
+
+  private final AtomicInteger metricCollectionId;
+
+  private DAG<IRVertex, IREdge> dagSnapshot; // the DAG that was saved most recently.
+  private DAG<IRVertex, IREdge> modifiedDAG; // the DAG that is being updated.
+
+  /**
+   * @param originalUserApplicationDAG the initial DAG.
+   */
+  public IRDAG(final DAG<IRVertex, IREdge> originalUserApplicationDAG) {
+    this.modifiedDAG = originalUserApplicationDAG;
+    this.dagSnapshot = originalUserApplicationDAG;
+    this.metricCollectionId = new AtomicInteger(0);
+  }
+
+  //////////////////////////////////////////////////
+
+  /**
+   * Used internally by Nemo to advance the DAG snapshot after applying each pass.
+   * @param checker that compares the dagSnapshot and the modifiedDAG
+   *                to determine if the snapshot can be set the current modifiedDAG.
+   * @return true if the checker passes, false otherwise.
+   */
+  public boolean advanceDAGSnapshot(final BiFunction<IRDAG, IRDAG, Boolean> checker) {
+    final boolean canAdvance = checker.apply(new IRDAG(dagSnapshot), new IRDAG(modifiedDAG));
+    if (canAdvance) {
+      dagSnapshot = modifiedDAG;
+    }
+    return canAdvance;
+  }
+
+  ////////////////////////////////////////////////// Methods for reshaping the DAG topology.
+
+  /**
+   * Inserts a new vertex that streams data.
+   *
+   * Before: src - edgeToStreamize - dst
+   * After: src - edgeToStreamizeWithNewDestination - streamVertex - oneToOneEdge - dst
+   * (replaces the "Before" relationships)
+   *
+   * @param streamVertex to insert.
+   * @param edgeToStreamize to modify.
+   */
+  public void insert(final StreamVertex streamVertex, final IREdge edgeToStreamize) {
+    // Create a completely new DAG with the vertex inserted.
+    final DAGBuilder builder = new DAGBuilder();
+
+    // Insert the vertex.
+    builder.addVertex(streamVertex);
+
+    // Build the new DAG to reflect the new topology.
+    modifiedDAG.topologicalDo(v -> {
+      builder.addVertex(v); // None of the existing vertices are deleted.
+
+      for (final IREdge edge : modifiedDAG.getIncomingEdgesOf(v)) {
+        if (edge.equals(edgeToStreamize)) {
+          // MATCH!
+
+          // Edge to the streamVertex
+          final IREdge edgeToStreamizeWithNewDestination = new IREdge(
+            edgeToStreamize.getPropertyValue(CommunicationPatternProperty.class).get(),
+            edgeToStreamize.getSrc(),
+            streamVertex);
+          edgeToStreamize.copyExecutionPropertiesTo(edgeToStreamizeWithNewDestination);
+
+          // Edge from the streamVertex.
+          final IREdge oneToOneEdge = new IREdge(CommunicationPatternProperty.Value.OneToOne, streamVertex, v);
+          oneToOneEdge.setProperty(EncoderProperty.of(edgeToStreamize.getPropertyValue(EncoderProperty.class).get()));
+          oneToOneEdge.setProperty(DecoderProperty.of(edgeToStreamize.getPropertyValue(DecoderProperty.class).get()));
+
+          // Track the new edges.
+          builder.connectVertices(edgeToStreamizeWithNewDestination);
+          builder.connectVertices(oneToOneEdge);
+        } else {
+          // NO MATCH, so simply connect vertices as before.
+          builder.connectVertices(edge);
+        }
+      }
+    });
+
+    modifiedDAG = builder.build(); // update the DAG.
+  }
+
+  /**
+   * Inserts a new vertex that analyzes intermediate data, and triggers a dynamic optimization.
+   *
+   * For each edge in edgesToGetStatisticsOf...
+   *
+   * Before: src - edge - dst
+   * After: src - oneToOneEdge(a clone of edge) - messageBarrierVertex -
+   *        shuffleEdge - messageAggregatorVertex - broadcastEdge - dst
+   * (the "Before" relationships are unmodified)
+   *
+   * @param messageBarrierVertex to insert.
+   * @param messageAggregatorVertex to insert.
+   * @param mbvOutputEncoder to use.
+   * @param mbvOutputDecoder to use.
+   * @param edgesToGetStatisticsOf to examine.
+   */
+  public void insert(final MessageBarrierVertex messageBarrierVertex,
+                     final MessageAggregatorVertex messageAggregatorVertex,
+                     final EncoderProperty mbvOutputEncoder,
+                     final DecoderProperty mbvOutputDecoder,
+                     final Set<IREdge> edgesToGetStatisticsOf) {
+    if (edgesToGetStatisticsOf.stream().map(edge -> edge.getDst().getId()).collect(Collectors.toSet()).size() != 1) {
+      throw new IllegalArgumentException("Not destined to the same vertex: " + edgesToGetStatisticsOf.toString());
+    }
+    final IRVertex dst = edgesToGetStatisticsOf.iterator().next().getDst();
+
+    // Create a completely new DAG with the vertex inserted.
+    final DAGBuilder builder = new DAGBuilder();
+
+    // Current metric collection id.
+    final int currentMetricCollectionId = metricCollectionId.incrementAndGet();
+
+    // First, add all the vertices.
+    modifiedDAG.topologicalDo(v -> builder.addVertex(v));
+
+    // Add a control dependency (no output) from the messageAggregatorVertex to the destination.
+    builder.addVertex(messageAggregatorVertex);
+    final IREdge noDataEdge = new IREdge(CommunicationPatternProperty.Value.BroadCast, messageAggregatorVertex, dst);
+    builder.connectVertices(noDataEdge);
+
+    // Add the edges and the messageBarrierVertex.
+    modifiedDAG.topologicalDo(v -> {
+      for (final IREdge edge : modifiedDAG.getIncomingEdgesOf(v)) {
+        if (edgesToGetStatisticsOf.contains(edge)) {
+          // MATCH!
+          final MessageBarrierVertex mbv = new MessageBarrierVertex<>(messageBarrierVertex.getMessageFunction());
+          builder.addVertex(mbv);
+
+          // Clone the edgeToGetStatisticsOf
+          final IREdge clone = new IREdge(CommunicationPatternProperty.Value.OneToOne, edge.getSrc(), mbv);
+          clone.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
+          clone.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
+          edge.getPropertyValue(AdditionalOutputTagProperty.class).ifPresent(tag -> {
+            clone.setProperty(AdditionalOutputTagProperty.of(tag));
+          });
+          builder.connectVertices(clone);
+
+          // messageBarrierVertex to the messageAggregatorVertex
+          final IREdge edgeToABV = edgeBetweenMessageVertices(mbv,
+            messageAggregatorVertex, mbvOutputEncoder, mbvOutputDecoder, currentMetricCollectionId);
+          builder.connectVertices(edgeToABV);
+
+          // The original edge
+          // We then insert the vertex with MessageBarrierTransform and vertex with MessageAggregatorTransform
+          // between the vertex and incoming vertices.
+          final IREdge edgeToOriginalDst =
+            new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(), edge.getSrc(), v);
+          edge.copyExecutionPropertiesTo(edgeToOriginalDst);
+          edgeToOriginalDst.setPropertyPermanently(MetricCollectionProperty.of(currentMetricCollectionId));
+          builder.connectVertices(edgeToOriginalDst);
+        } else {
+          // NO MATCH, so simply connect vertices as before.
+          builder.connectVertices(edge);
+        }
+      }
+    });
+
+    modifiedDAG = builder.build(); // update the DAG.
+  }
+
+  /**
+   * Reshape unsafely, without guarantees on preserving application semantics.
+   * TODO #330: Refactor Unsafe Reshaping Passes
+   * @param unsafeReshapingFunction takes as input the underlying DAG, and outputs a reshaped DAG.
+   */
+  public void reshapeUnsafely(final Function<DAG<IRVertex, IREdge>, DAG<IRVertex, IREdge>> unsafeReshapingFunction) {
+    modifiedDAG = unsafeReshapingFunction.apply(modifiedDAG);
+  }
+
+  ////////////////////////////////////////////////// Private helper methods.
+
+  /**
+   * @param mbv src.
+   * @param mav dst.
+   * @param encoder src-dst encoder.
+   * @param decoder src-dst decoder.
+   * @param currentMetricCollectionId of the edge.
+   * @return the edge.
+   */
+  private IREdge edgeBetweenMessageVertices(final MessageBarrierVertex mbv,
+                                            final MessageAggregatorVertex mav,
+                                            final EncoderProperty encoder,
+                                            final DecoderProperty decoder,
+                                            final int currentMetricCollectionId) {
+    final IREdge newEdge = new IREdge(CommunicationPatternProperty.Value.Shuffle, mbv, mav);
+    newEdge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+    newEdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Keep));
+    newEdge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
+    newEdge.setPropertyPermanently(MetricCollectionProperty.of(currentMetricCollectionId));
+    final KeyExtractor pairKeyExtractor = (element) -> {
+      if (element instanceof Pair) {
+        return ((Pair) element).left();
+      } else {
+        throw new IllegalStateException(element.toString());
+      }
+    };
+    newEdge.setProperty(KeyExtractorProperty.of(pairKeyExtractor));
+    newEdge.setPropertyPermanently(encoder);
+    newEdge.setPropertyPermanently(decoder);
+    return newEdge;
+  }
+
+  ////////////////////////////////////////////////// DAGInterface methods - forward calls to the underlying DAG.
+
+  @Override
+  public void topologicalDo(final Consumer<IRVertex> function) {
+    modifiedDAG.topologicalDo(function);
+  }
+
+  @Override
+  public void dfsTraverse(final Consumer<IRVertex> function, final TraversalOrder traversalOrder) {
+    modifiedDAG.dfsTraverse(function, traversalOrder);
+  }
+
+  @Override
+  public void dfsDo(final IRVertex vertex,
+                    final Consumer<IRVertex> vertexConsumer,
+                    final TraversalOrder traversalOrder,
+                    final Set<IRVertex> visited) {
+    modifiedDAG.dfsDo(vertex, vertexConsumer, traversalOrder, visited);
+  }
+
+  @Override
+  public Boolean pathExistsBetween(final IRVertex v1, final IRVertex v2) {
+    return modifiedDAG.pathExistsBetween(v1, v2);
+  }
+
+  @Override
+  public Boolean isCompositeVertex(final IRVertex irVertex) {
+    return modifiedDAG.isCompositeVertex(irVertex);
+  }
+
+  @Override
+  public Integer getLoopStackDepthOf(final IRVertex irVertex) {
+    return modifiedDAG.getLoopStackDepthOf(irVertex);
+  }
+
+  @Override
+  public LoopVertex getAssignedLoopVertexOf(final IRVertex irVertex) {
+    return modifiedDAG.getAssignedLoopVertexOf(irVertex);
+  }
+
+  @Override
+  public ObjectNode asJsonNode() {
+    return modifiedDAG.asJsonNode();
+  }
+
+  @Override
+  public void storeJSON(final String directory, final String name, final String description) {
+    modifiedDAG.storeJSON(directory, name, description);
+  }
+
+  @Override
+  public IRVertex getVertexById(final String id) {
+    return modifiedDAG.getVertexById(id);
+  }
+
+  @Override
+  public List<IRVertex> getVertices() {
+    return modifiedDAG.getVertices();
+  }
+
+  @Override
+  public List<IRVertex> getRootVertices() {
+    return modifiedDAG.getRootVertices();
+  }
+
+  @Override
+  public List<IREdge> getIncomingEdgesOf(final IRVertex v) {
+    return modifiedDAG.getIncomingEdgesOf(v);
+  }
+
+  @Override
+  public List<IREdge> getIncomingEdgesOf(final String vertexId) {
+    return modifiedDAG.getIncomingEdgesOf(vertexId);
+  }
+
+  @Override
+  public List<IREdge> getOutgoingEdgesOf(final IRVertex v) {
+    return modifiedDAG.getOutgoingEdgesOf(v);
+  }
+
+  @Override
+  public List<IREdge> getOutgoingEdgesOf(final String vertexId) {
+    return modifiedDAG.getOutgoingEdgesOf(vertexId);
+  }
+
+  @Override
+  public List<IRVertex> getParents(final String vertexId) {
+    return modifiedDAG.getParents(vertexId);
+  }
+
+  @Override
+  public List<IRVertex> getChildren(final String vertexId) {
+    return modifiedDAG.getChildren(vertexId);
+  }
+
+  @Override
+  public IREdge getEdgeBetween(final String srcVertexId,
+                               final String dstVertexId) throws IllegalEdgeOperationException {
+    return modifiedDAG.getEdgeBetween(srcVertexId, dstVertexId);
+  }
+
+  @Override
+  public List<IRVertex> getTopologicalSort() {
+    return modifiedDAG.getTopologicalSort();
+  }
+
+  @Override
+  public List<IRVertex> getAncestors(final String vertexId) {
+    return modifiedDAG.getAncestors(vertexId);
+  }
+
+  @Override
+  public List<IRVertex> getDescendants(final String vertexId) {
+    return modifiedDAG.getDescendants(vertexId);
+  }
+
+  @Override
+  public List<IRVertex> filterVertices(final Predicate<IRVertex> condition) {
+    return modifiedDAG.filterVertices(condition);
+  }
+}
diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/LoopVertex.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/LoopVertex.java
index dd12406..f279783 100644
--- a/common/src/main/java/org/apache/nemo/common/ir/vertex/LoopVertex.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/LoopVertex.java
@@ -310,6 +310,11 @@ public final class LoopVertex extends IRVertex {
     this.maxNumberOfIterations--;
   }
 
+  /**
+   * Check termination condition.
+   * @param that another vertex.
+   * @return true if equals.
+   */
   public boolean terminationConditionEquals(final LoopVertex that) {
     if (this.maxNumberOfIterations.equals(that.getMaxNumberOfIterations()) && Util
         .checkEqualityOfIntPredicates(this.terminationCondition, that.getTerminationCondition(),
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 dc0e11e..7e2b953 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
@@ -25,7 +25,7 @@ import org.apache.nemo.common.ir.vertex.transform.Transform;
  * IRVertex that transforms input data.
  * It is to be constructed in the compiler frontend with language-specific data transform logic.
  */
-public final class OperatorVertex extends IRVertex {
+public class OperatorVertex extends IRVertex {
   private final Transform transform;
 
   /**
@@ -47,19 +47,19 @@ public final class OperatorVertex extends IRVertex {
   }
 
   @Override
-  public OperatorVertex getClone() {
+  public final OperatorVertex getClone() {
     return new OperatorVertex(this);
   }
 
   /**
    * @return the transform in the OperatorVertex.
    */
-  public Transform getTransform() {
+  public final Transform getTransform() {
     return transform;
   }
 
   @Override
-  public ObjectNode getPropertiesAsJsonNode() {
+  public final ObjectNode getPropertiesAsJsonNode() {
     final ObjectNode node = getIRVertexPropertiesAsJsonNode();
     node.put("transform", transform.toString());
     return node;
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageAggregatorVertex.java
similarity index 54%
copy from compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java
copy to common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageAggregatorVertex.java
index e7a4101..574fa01 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageAggregatorVertex.java
@@ -16,21 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.nemo.compiler.optimizer;
+package org.apache.nemo.common.ir.vertex.system;
 
-import org.apache.nemo.common.KeyExtractor;
 import org.apache.nemo.common.Pair;
+import org.apache.nemo.common.ir.vertex.OperatorVertex;
+import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform;
+
+import java.util.function.BiFunction;
 
 /**
- * Extracts the key from a pair element.
+ * Aggregates upstream messages.
+ * @param <K> of the input pair.
+ * @param <V> of the input pair.
+ * @param <O> of the output aggregated message.
  */
-public final class PairKeyExtractor implements KeyExtractor {
-  @Override
-  public Object extractKey(final Object element) {
-    if (element instanceof Pair) {
-      return ((Pair) element).left();
-    } else {
-      throw new IllegalStateException(element.toString());
-    }
+public class MessageAggregatorVertex<K, V, O> extends OperatorVertex {
+  /**
+   * @param initialState to use.
+   * @param userFunction for aggregating the messages.
+   */
+  public MessageAggregatorVertex(final O initialState, final BiFunction<Pair<K, V>, O, O> userFunction) {
+    super(new MessageAggregatorTransform<>(initialState, userFunction));
   }
 }
diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageBarrierVertex.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageBarrierVertex.java
new file mode 100644
index 0000000..98fddbd
--- /dev/null
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/MessageBarrierVertex.java
@@ -0,0 +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.common.ir.vertex.system;
+
+import org.apache.nemo.common.ir.vertex.OperatorVertex;
+import org.apache.nemo.common.ir.vertex.transform.MessageBarrierTransform;
+
+import java.util.Map;
+import java.util.function.BiFunction;
+
+/**
+ * Generates messages.
+ * @param <I> input type
+ * @param <K> of the output pair.
+ * @param <V> of the output pair.
+ */
+public final class MessageBarrierVertex<I, K, V> extends OperatorVertex {
+  private final BiFunction<I, Map<K, V>, Map<K, V>> messageFunction;
+
+  /**
+   * @param messageFunction for producing a message.
+   */
+  public MessageBarrierVertex(final BiFunction<I, Map<K, V>, Map<K, V>> messageFunction) {
+    super(new MessageBarrierTransform<>(messageFunction));
+    this.messageFunction = messageFunction;
+  }
+
+  public BiFunction<I, Map<K, V>, Map<K, V>> getMessageFunction() {
+    return messageFunction;
+  }
+}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/StreamVertex.java
similarity index 64%
rename from compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java
rename to common/src/main/java/org/apache/nemo/common/ir/vertex/system/StreamVertex.java
index e7a4101..4f4a9ca 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/PairKeyExtractor.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/system/StreamVertex.java
@@ -16,21 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.nemo.compiler.optimizer;
+package org.apache.nemo.common.ir.vertex.system;
 
-import org.apache.nemo.common.KeyExtractor;
-import org.apache.nemo.common.Pair;
+import org.apache.nemo.common.ir.vertex.OperatorVertex;
+import org.apache.nemo.common.ir.vertex.transform.StreamTransform;
 
 /**
- * Extracts the key from a pair element.
+ * IRVertex that transforms input data.
+ * It is to be constructed in the compiler frontend with language-specific data transform logic.
  */
-public final class PairKeyExtractor implements KeyExtractor {
-  @Override
-  public Object extractKey(final Object element) {
-    if (element instanceof Pair) {
-      return ((Pair) element).left();
-    } else {
-      throw new IllegalStateException(element.toString());
-    }
+public final class StreamVertex extends OperatorVertex {
+  /**
+   * Constructor.
+   */
+  public StreamVertex() {
+    super(new StreamTransform());
   }
 }
diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/AggregateMetricTransform.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageAggregatorTransform.java
similarity index 72%
rename from common/src/main/java/org/apache/nemo/common/ir/vertex/transform/AggregateMetricTransform.java
rename to common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageAggregatorTransform.java
index e912125..d4dd77b 100644
--- a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/AggregateMetricTransform.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageAggregatorTransform.java
@@ -18,6 +18,7 @@
  */
 package org.apache.nemo.common.ir.vertex.transform;
 
+import org.apache.nemo.common.Pair;
 import org.apache.nemo.common.ir.OutputCollector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -25,25 +26,25 @@ import org.slf4j.LoggerFactory;
 import java.util.function.BiFunction;
 
 /**
- * A {@link Transform} that aggregates stage-level statistics sent to the master side optimizer
- * for dynamic optimization.
+ * A {@link Transform} that aggregates statistics generated by the {@link MessageBarrierTransform}.
  *
- * @param <I> input type.
+ * @param <K> input key type.
+ * @param <V> input value type.
  * @param <O> output type.
  */
-public final class AggregateMetricTransform<I, O> extends NoWatermarkEmitTransform<I, O> {
-  private static final Logger LOG = LoggerFactory.getLogger(AggregateMetricTransform.class.getName());
+public final class MessageAggregatorTransform<K, V, O> extends NoWatermarkEmitTransform<Pair<K, V>, O> {
+  private static final Logger LOG = LoggerFactory.getLogger(MessageAggregatorTransform.class.getName());
   private OutputCollector<O> outputCollector;
   private O aggregatedDynOptData;
-  private final BiFunction<Object, O, O> dynOptDataAggregator;
+  private final BiFunction<Pair<K, V>, O, O> dynOptDataAggregator;
 
   /**
    * Default constructor.
    * @param aggregatedDynOptData per-stage aggregated dynamic optimization data.
    * @param dynOptDataAggregator aggregator to use.
    */
-  public AggregateMetricTransform(final O aggregatedDynOptData,
-                                  final BiFunction<Object, O, O> dynOptDataAggregator) {
+  public MessageAggregatorTransform(final O aggregatedDynOptData,
+                                    final BiFunction<Pair<K, V>, O, O> dynOptDataAggregator) {
     this.aggregatedDynOptData = aggregatedDynOptData;
     this.dynOptDataAggregator = dynOptDataAggregator;
   }
@@ -54,7 +55,7 @@ public final class AggregateMetricTransform<I, O> extends NoWatermarkEmitTransfo
   }
 
   @Override
-  public void onData(final I element) {
+  public void onData(final Pair<K, V> element) {
     aggregatedDynOptData = dynOptDataAggregator.apply(element, aggregatedDynOptData);
   }
 
@@ -66,7 +67,7 @@ public final class AggregateMetricTransform<I, O> extends NoWatermarkEmitTransfo
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder();
-    sb.append(AggregateMetricTransform.class);
+    sb.append(MessageAggregatorTransform.class);
     sb.append(":");
     sb.append(super.toString());
     return sb.toString();
diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MetricCollectTransform.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageBarrierTransform.java
similarity index 54%
rename from common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MetricCollectTransform.java
rename to common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageBarrierTransform.java
index 39dc6e0..4c9a007 100644
--- a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MetricCollectTransform.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/MessageBarrierTransform.java
@@ -18,60 +18,59 @@
  */
 package org.apache.nemo.common.ir.vertex.transform;
 
+import org.apache.nemo.common.Pair;
 import org.apache.nemo.common.ir.OutputCollector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.function.BiFunction;
 
 /**
  * A {@link Transform} that collects task-level statistics used for dynamic optimization.
- * The collected statistics is sent to vertex with {@link AggregateMetricTransform} as a tagged output
- * when this transform is closed.
- *
  * @param <I> input type.
- * @param <O> output type.
+ * @param <K> output key type.
+ * @param <V> output value type.
  */
-public final class MetricCollectTransform<I, O> extends NoWatermarkEmitTransform<I, O> {
-  private static final Logger LOG = LoggerFactory.getLogger(MetricCollectTransform.class.getName());
-  private OutputCollector<O> outputCollector;
-  private O dynOptData;
-  private final BiFunction<Object, O, O> dynOptDataCollector;
-  private final BiFunction<O, OutputCollector, O> closer;
+public final class MessageBarrierTransform<I, K, V> extends NoWatermarkEmitTransform<I, Pair<K, V>> {
+  private static final Logger LOG = LoggerFactory.getLogger(MessageBarrierTransform.class.getName());
+  private final BiFunction<I, Map<K, V>, Map<K, V>> userFunction;
+
+  private OutputCollector<Pair<K, V>> outputCollector;
+  private Map<K, V> holder;
 
   /**
-   * MetricCollectTransform constructor.
-   * @param dynOptData per-task dynamic optimization data.
-   * @param dynOptDataCollector that collects the data.
-   * @param closer callback function to be invoked when closing the transform.
+   * MessageBarrierTransform constructor.
+   * @param userFunction that analyzes the data.
    */
-  public MetricCollectTransform(final O dynOptData,
-                                final BiFunction<Object, O, O> dynOptDataCollector,
-                                final BiFunction<O, OutputCollector, O> closer) {
-    this.dynOptData = dynOptData;
-    this.dynOptDataCollector = dynOptDataCollector;
-    this.closer = closer;
+  public MessageBarrierTransform(final BiFunction<I, Map<K, V>, Map<K, V>> userFunction) {
+    this.userFunction = userFunction;
   }
 
   @Override
-  public void prepare(final Context context, final OutputCollector<O> oc) {
+  public void prepare(final Context context, final OutputCollector<Pair<K, V>> oc) {
     this.outputCollector = oc;
+    this.holder = new HashMap<>();
   }
 
   @Override
   public void onData(final I element) {
-    dynOptData = dynOptDataCollector.apply(element, dynOptData);
+    holder = userFunction.apply(element, holder);
   }
 
   @Override
   public void close() {
-    closer.apply(dynOptData, outputCollector);
+    holder.forEach((k, v) -> {
+      final Pair<K, V> pairData = Pair.of(k, v);
+      outputCollector.emit(pairData);
+    });
   }
 
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder();
-    sb.append(MetricCollectTransform.class);
+    sb.append(MessageBarrierTransform.class);
     sb.append(":");
     sb.append(super.toString());
     return sb.toString();
diff --git a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/RelayTransform.java b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/StreamTransform.java
similarity index 89%
rename from common/src/main/java/org/apache/nemo/common/ir/vertex/transform/RelayTransform.java
rename to common/src/main/java/org/apache/nemo/common/ir/vertex/transform/StreamTransform.java
index cd713d3..97a7013 100644
--- a/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/RelayTransform.java
+++ b/common/src/main/java/org/apache/nemo/common/ir/vertex/transform/StreamTransform.java
@@ -28,14 +28,14 @@ import org.slf4j.LoggerFactory;
  * This transform can be used for merging input data into the {@link OutputCollector}.
  * @param <T> input/output type.
  */
-public final class RelayTransform<T> implements Transform<T, T> {
+public final class StreamTransform<T> implements Transform<T, T> {
   private OutputCollector<T> outputCollector;
-  private static final Logger LOG = LoggerFactory.getLogger(RelayTransform.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(StreamTransform.class.getName());
 
   /**
    * Default constructor.
    */
-  public RelayTransform() {
+  public StreamTransform() {
     // Do nothing.
   }
 
@@ -62,7 +62,7 @@ public final class RelayTransform<T> implements Transform<T, T> {
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder();
-    sb.append(RelayTransform.class);
+    sb.append(StreamTransform.class);
     sb.append(":");
     sb.append(super.toString());
     return sb.toString();
diff --git a/common/src/main/java/org/apache/nemo/common/pass/Pass.java b/common/src/main/java/org/apache/nemo/common/pass/Pass.java
index a2297e1..7a540c9 100644
--- a/common/src/main/java/org/apache/nemo/common/pass/Pass.java
+++ b/common/src/main/java/org/apache/nemo/common/pass/Pass.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.common.pass;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 
 import java.io.Serializable;
 import java.util.function.Predicate;
@@ -29,7 +27,7 @@ import java.util.function.Predicate;
  * Abstract class for optimization passes. All passes basically extends this class.
  */
 public abstract class Pass implements Serializable {
-  private Predicate<DAG<IRVertex, IREdge>> condition;
+  private Predicate<IRDAG> condition;
 
   /**
    * Default constructor.
@@ -42,7 +40,7 @@ public abstract class Pass implements Serializable {
    * Constructor.
    * @param condition condition under which to run the pass.
    */
-  private Pass(final Predicate<DAG<IRVertex, IREdge>> condition) {
+  private Pass(final Predicate<IRDAG> condition) {
     this.condition = condition;
   }
 
@@ -50,7 +48,7 @@ public abstract class Pass implements Serializable {
    * Getter for the condition under which to apply the pass.
    * @return the condition under which to apply the pass.
    */
-  public final Predicate<DAG<IRVertex, IREdge>> getCondition() {
+  public final Predicate<IRDAG> getCondition() {
     return this.condition;
   }
 
@@ -59,7 +57,7 @@ public abstract class Pass implements Serializable {
    * @param newCondition the new condition to add to the existing condition.
    * @return the condition with the new condition added.
    */
-  public final Pass addCondition(final Predicate<DAG<IRVertex, IREdge>> newCondition) {
+  public final Pass addCondition(final Predicate<IRDAG> newCondition) {
     this.condition = this.condition.and(newCondition);
     return this;
   }
diff --git a/common/src/main/java/org/apache/nemo/common/test/EmptyComponents.java b/common/src/main/java/org/apache/nemo/common/test/EmptyComponents.java
index 2a0fe36..4e005e1 100644
--- a/common/src/main/java/org/apache/nemo/common/test/EmptyComponents.java
+++ b/common/src/main/java/org/apache/nemo/common/test/EmptyComponents.java
@@ -21,8 +21,8 @@ package org.apache.nemo.common.test;
 import org.apache.nemo.common.KeyExtractor;
 import org.apache.nemo.common.coder.DecoderFactory;
 import org.apache.nemo.common.coder.EncoderFactory;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.OutputCollector;
 import org.apache.nemo.common.ir.Readable;
 import org.apache.nemo.common.ir.edge.IREdge;
@@ -54,7 +54,7 @@ public final class EmptyComponents {
    * Builds dummy IR DAG for testing.
    * @return the dummy IR DAG.
    */
-  public static DAG<IRVertex, IREdge> buildEmptyDAG() {
+  public static IRDAG buildEmptyDAG() {
     DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
     final IRVertex s = new EmptyComponents.EmptySourceVertex<>("s");
     final IRVertex t1 = new OperatorVertex(new EmptyComponents.EmptyTransform("t1"));
@@ -73,7 +73,7 @@ public final class EmptyComponents {
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, t2, t3));
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, t3, t4));
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, t2, t5));
-    return dagBuilder.build();
+    return new IRDAG(dagBuilder.build());
   }
 
   /**
@@ -82,7 +82,7 @@ public final class EmptyComponents {
    * and KeyExtractorProperty by default.
    * @return the dummy IR DAG.
    */
-  public static DAG<IRVertex, IREdge> buildEmptyDAGForSkew() {
+  public static IRDAG buildEmptyDAGForSkew() {
     DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
     final IRVertex s = new EmptyComponents.EmptySourceVertex<>("s");
     final IRVertex t1 = new OperatorVertex(new EmptyComponents.EmptyTransform("t1"));
@@ -116,7 +116,7 @@ public final class EmptyComponents {
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, t2, t3));
     dagBuilder.connectVertices(shuffleEdgeBetweenT3AndT4);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, t2, t5));
-    return dagBuilder.build();
+    return new IRDAG(dagBuilder.build());
   }
 
   /**
diff --git a/common/src/test/java/org/apache/nemo/common/ir/LoopVertexTest.java b/common/src/test/java/org/apache/nemo/common/ir/LoopVertexTest.java
index e56154e..da500b1 100644
--- a/common/src/test/java/org/apache/nemo/common/ir/LoopVertexTest.java
+++ b/common/src/test/java/org/apache/nemo/common/ir/LoopVertexTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.nemo.common.ir;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
@@ -41,8 +40,8 @@ import static org.junit.Assert.assertEquals;
  */
 public class LoopVertexTest {
   private final LoopVertex loopVertex = new LoopVertex("fakeTransform");
-  private DAG<IRVertex, IREdge> originalDAG;
-  private DAG<IRVertex, IREdge> newDAG;
+  private IRDAG originalDAG;
+  private IRDAG newDAG;
 
   private final IRVertex source = new EmptyComponents.EmptySourceVertex<>("Source");
   private final IRVertex map1 = new OperatorVertex(new EmptyComponents.EmptyTransform("MapElements"));
@@ -62,12 +61,12 @@ public class LoopVertexTest {
     loopVertex.addDagIncomingEdge(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1));
     loopVertex.addIterativeIncomingEdge(new IREdge(CommunicationPatternProperty.Value.OneToOne, map2, map1));
 
-    originalDAG = builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
+    originalDAG = new IRDAG(builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
-        .build();
+        .build());
   }
 
   @Test
diff --git a/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/Backend.java b/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/Backend.java
index 7e343db..bc3906f 100644
--- a/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/Backend.java
+++ b/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/Backend.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.backend;
 
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.backend.nemo.NemoBackend;
 import org.apache.reef.tang.annotations.DefaultImplementation;
 
@@ -37,5 +35,5 @@ public interface Backend<Plan> {
    * @return the execution plan generated.
    * @throws Exception Exception on the way.
    */
-  Plan compile(DAG<IRVertex, IREdge> dag) throws Exception;
+  Plan compile(IRDAG dag) throws Exception;
 }
diff --git a/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/nemo/NemoBackend.java b/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/nemo/NemoBackend.java
index 3b68482..e0722ee 100644
--- a/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/nemo/NemoBackend.java
+++ b/compiler/backend/src/main/java/org/apache/nemo/compiler/backend/nemo/NemoBackend.java
@@ -19,9 +19,8 @@
 package org.apache.nemo.compiler.backend.nemo;
 
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.backend.Backend;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.runtime.common.RuntimeIdManager;
 import org.apache.nemo.runtime.common.plan.PhysicalPlan;
 import org.apache.nemo.runtime.common.plan.PhysicalPlanGenerator;
@@ -52,7 +51,7 @@ public final class NemoBackend implements Backend<PhysicalPlan> {
    * @param irDAG the IR DAG to compile.
    * @return the execution plan to be submitted to Runtime.
    */
-  public PhysicalPlan compile(final DAG<IRVertex, IREdge> irDAG) {
+  public PhysicalPlan compile(final IRDAG irDAG) {
 
     final DAG<Stage, StageEdge> stageDAG = physicalPlanGenerator.apply(irDAG);
     return new PhysicalPlan(RuntimeIdManager.generatePhysicalPlanId(), stageDAG);
diff --git a/compiler/backend/src/test/java/org/apache/nemo/compiler/backend/nemo/NemoBackendTest.java b/compiler/backend/src/test/java/org/apache/nemo/compiler/backend/nemo/NemoBackendTest.java
index 96a7aef..335ef1a 100644
--- a/compiler/backend/src/test/java/org/apache/nemo/compiler/backend/nemo/NemoBackendTest.java
+++ b/compiler/backend/src/test/java/org/apache/nemo/compiler/backend/nemo/NemoBackendTest.java
@@ -19,6 +19,7 @@
 package org.apache.nemo.compiler.backend.nemo;
 
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
@@ -48,16 +49,16 @@ public final class NemoBackendTest<I, O> {
   private NemoBackend nemoBackend;
 
   private final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
-  private DAG<IRVertex, IREdge> dag;
+  private IRDAG dag;
 
   @Before
   public void setUp() throws Exception {
-    this.dag = builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
+    this.dag = new IRDAG(builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
-        .build();
+        .build());
 
     this.dag = new TransientResourcePolicy().runCompileTimeOptimization(dag, EMPTY_DAG_DIRECTORY);
 
diff --git a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineVisitor.java b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineVisitor.java
index efd77a0..6c02088 100644
--- a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineVisitor.java
+++ b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineVisitor.java
@@ -20,9 +20,7 @@ package org.apache.nemo.compiler.frontend.beam;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 
 /**
  * Uses the translator and the context to build a Nemo IR DAG.
@@ -63,7 +61,7 @@ public final class PipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
   /**
    * @return the converted pipeline.
    */
-  DAG<IRVertex, IREdge> getConvertedPipeline() {
-    return context.getBuilder().build();
+  IRDAG getConvertedPipeline() {
+    return new IRDAG(context.getBuilder().build());
   }
 }
diff --git a/compiler/frontend/spark/src/main/java/org/apache/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java b/compiler/frontend/spark/src/main/java/org/apache/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
index 0a5c0e5..680a29b 100644
--- a/compiler/frontend/spark/src/main/java/org/apache/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
+++ b/compiler/frontend/spark/src/main/java/org/apache/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
@@ -21,6 +21,7 @@ package org.apache.nemo.compiler.frontend.spark.core;
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
@@ -105,7 +106,7 @@ public final class SparkFrontendUtils {
     builder.connectVertices(newEdge);
 
     // launch DAG
-    JobLauncher.launchDAG(builder.build(), SparkBroadcastVariables.getAll(), "");
+    JobLauncher.launchDAG(new IRDAG(builder.build()), SparkBroadcastVariables.getAll(), "");
 
     return (List<T>) JobLauncher.getCollectedData();
   }
diff --git a/compiler/frontend/spark/src/main/scala/org/apache/nemo/compiler/frontend/spark/core/rdd/RDD.scala b/compiler/frontend/spark/src/main/scala/org/apache/nemo/compiler/frontend/spark/core/rdd/RDD.scala
index 234e867..64aeb02 100644
--- a/compiler/frontend/spark/src/main/scala/org/apache/nemo/compiler/frontend/spark/core/rdd/RDD.scala
+++ b/compiler/frontend/spark/src/main/scala/org/apache/nemo/compiler/frontend/spark/core/rdd/RDD.scala
@@ -35,6 +35,7 @@ import org.apache.nemo.compiler.frontend.spark.core.SparkFrontendUtils
 import org.apache.nemo.compiler.frontend.spark.transform._
 import org.apache.hadoop.io.WritableFactory
 import org.apache.hadoop.io.compress.CompressionCodec
+import org.apache.nemo.common.ir.IRDAG
 import org.apache.spark.api.java.function.{FlatMapFunction, Function, Function2}
 import org.apache.spark.partial.{BoundedDouble, PartialResult}
 import org.apache.spark.rdd.{AsyncRDDActions, DoubleRDDFunctions, OrderedRDDFunctions, PartitionCoalescer, SequenceFileRDDFunctions}
@@ -230,7 +231,7 @@ final class RDD[T: ClassTag] protected[rdd] (
     newEdge.setProperty(keyExtractorProperty)
 
     builder.connectVertices(newEdge)
-    JobLauncher.launchDAG(builder.build, SparkBroadcastVariables.getAll, "")
+    JobLauncher.launchDAG(new IRDAG(builder.build), SparkBroadcastVariables.getAll, "")
   }
 
   /////////////// CACHING ///////////////
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/NemoOptimizer.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/NemoOptimizer.java
index 3ebc1de..3c4cf82 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/NemoOptimizer.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/NemoOptimizer.java
@@ -18,21 +18,21 @@
  */
 package org.apache.nemo.compiler.optimizer;
 
-import org.apache.nemo.common.dag.DAG;
+import net.jcip.annotations.NotThreadSafe;
 import org.apache.nemo.common.dag.DAGBuilder;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
 import org.apache.nemo.common.exception.CompileTimeOptimizationException;
 import org.apache.nemo.common.exception.DynamicOptimizationException;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.CachedSourceVertex;
 import org.apache.nemo.common.ir.edge.executionproperty.CacheIDProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import org.apache.nemo.common.ir.vertex.CachedSourceVertex;
 import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.executionproperty.IgnoreSchedulingTempDataReceiverProperty;
 import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
 import org.apache.nemo.compiler.optimizer.policy.Policy;
 import org.apache.nemo.conf.JobConf;
-import net.jcip.annotations.NotThreadSafe;
 import org.apache.reef.tang.Injector;
 import org.apache.reef.tang.annotations.Parameter;
 
@@ -78,20 +78,20 @@ public final class NemoOptimizer implements Optimizer {
    * @return optimized DAG, reshaped or tagged with execution properties.
    */
   @Override
-  public DAG<IRVertex, IREdge> optimizeDag(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG optimizeDag(final IRDAG dag) {
     final String irDagId = "ir-" + irDagCount++ + "-";
     dag.storeJSON(dagDirectory, irDagId, "IR before optimization");
 
-    final DAG<IRVertex, IREdge> optimizedDAG;
+    final IRDAG optimizedDAG;
     final Policy optimizationPolicy;
     final Map<UUID, IREdge> cacheIdToEdge = new HashMap<>();
 
     try {
       // Handle caching first.
-      final DAG<IRVertex, IREdge> cacheFilteredDag = handleCaching(dag, cacheIdToEdge);
+      final IRDAG cacheFilteredDag = handleCaching(dag, cacheIdToEdge);
       if (!cacheIdToEdge.isEmpty()) {
         cacheFilteredDag.storeJSON(dagDirectory, irDagId + "FilterCache",
-            "IR after cache filtering");
+          "IR after cache filtering");
       }
 
       // Conduct compile-time optimization.
@@ -102,8 +102,9 @@ public final class NemoOptimizer implements Optimizer {
       }
 
       optimizedDAG = optimizationPolicy.runCompileTimeOptimization(cacheFilteredDag, dagDirectory);
-      optimizedDAG.storeJSON(dagDirectory, irDagId + optimizationPolicy.getClass().getSimpleName(),
-          "IR optimized for " + optimizationPolicy.getClass().getSimpleName());
+      optimizedDAG
+        .storeJSON(dagDirectory, irDagId + optimizationPolicy.getClass().getSimpleName(),
+        "IR optimized for " + optimizationPolicy.getClass().getSimpleName());
     } catch (final Exception e) {
       throw new CompileTimeOptimizationException(e);
     }
@@ -121,8 +122,9 @@ public final class NemoOptimizer implements Optimizer {
     cacheIdToEdge.forEach((cacheId, edge) -> {
       if (!cacheIdToParallelism.containsKey(cacheId)) {
         cacheIdToParallelism.put(
-            cacheId, optimizedDAG.getVertexById(edge.getDst().getId()).getPropertyValue(ParallelismProperty.class)
-                .orElseThrow(() -> new RuntimeException("No parallelism on an IR vertex.")));
+          cacheId, optimizedDAG
+            .getVertexById(edge.getDst().getId()).getPropertyValue(ParallelismProperty.class)
+            .orElseThrow(() -> new RuntimeException("No parallelism on an IR vertex.")));
       }
     });
 
@@ -142,26 +144,25 @@ public final class NemoOptimizer implements Optimizer {
    * @param cacheIdToEdge the map from cache ID to edge to update.
    * @return the cropped dag regarding to caching.
    */
-  private DAG<IRVertex, IREdge> handleCaching(final DAG<IRVertex, IREdge> dag,
-                                              final Map<UUID, IREdge> cacheIdToEdge) {
+  private IRDAG handleCaching(final IRDAG dag, final Map<UUID, IREdge> cacheIdToEdge) {
     dag.topologicalDo(irVertex ->
-        dag.getIncomingEdgesOf(irVertex).forEach(
-            edge -> edge.getPropertyValue(CacheIDProperty.class).
-                ifPresent(cacheId -> cacheIdToEdge.put(cacheId, edge))
-        ));
+      dag.getIncomingEdgesOf(irVertex).forEach(
+        edge -> edge.getPropertyValue(CacheIDProperty.class).
+          ifPresent(cacheId -> cacheIdToEdge.put(cacheId, edge))
+      ));
 
     if (cacheIdToEdge.isEmpty()) {
       return dag;
     } else {
       final DAGBuilder<IRVertex, IREdge> filteredDagBuilder = new DAGBuilder<>();
       final List<IRVertex> sinkVertices = dag.getVertices().stream()
-          .filter(irVertex -> dag.getOutgoingEdgesOf(irVertex).isEmpty())
-          .collect(Collectors.toList());
+        .filter(irVertex -> dag.getOutgoingEdgesOf(irVertex).isEmpty())
+        .collect(Collectors.toList());
       sinkVertices.forEach(filteredDagBuilder::addVertex); // Sink vertex cannot be cached already.
 
       sinkVertices.forEach(sinkVtx -> addNonCachedVerticesAndEdges(dag, sinkVtx, filteredDagBuilder));
 
-      return filteredDagBuilder.buildWithoutSourceCheck();
+      return new IRDAG(filteredDagBuilder.buildWithoutSourceCheck());
     }
   }
 
@@ -172,45 +173,45 @@ public final class NemoOptimizer implements Optimizer {
    * @param irVertex the ir vertex to consider to add.
    * @param builder  the filtered dag builder.
    */
-  private void addNonCachedVerticesAndEdges(final DAG<IRVertex, IREdge> dag,
+  private void addNonCachedVerticesAndEdges(final IRDAG dag,
                                             final IRVertex irVertex,
                                             final DAGBuilder<IRVertex, IREdge> builder) {
     if (irVertex.getPropertyValue(IgnoreSchedulingTempDataReceiverProperty.class).orElse(false)
-        && dag.getIncomingEdgesOf(irVertex).stream()
-        .filter(irEdge -> irEdge.getPropertyValue(CacheIDProperty.class).isPresent())
-        .anyMatch(irEdge -> cacheIdToParallelism
-            .containsKey(irEdge.getPropertyValue(CacheIDProperty.class).get()))) {
+      && dag.getIncomingEdgesOf(irVertex).stream()
+      .filter(irEdge -> irEdge.getPropertyValue(CacheIDProperty.class).isPresent())
+      .anyMatch(irEdge -> cacheIdToParallelism
+        .containsKey(irEdge.getPropertyValue(CacheIDProperty.class).get()))) {
       builder.removeVertex(irVertex); // Ignore ghost vertex which was cached once.
       return;
     }
 
     dag.getIncomingEdgesOf(irVertex).stream()
-        .forEach(edge -> {
-      final Optional<UUID> cacheId = dag.getOutgoingEdgesOf(edge.getSrc()).stream()
+      .forEach(edge -> {
+        final Optional<UUID> cacheId = dag.getOutgoingEdgesOf(edge.getSrc()).stream()
           .filter(edgeToFilter -> edgeToFilter.getPropertyValue(CacheIDProperty.class).isPresent())
           .map(edgeToMap -> edgeToMap.getPropertyValue(CacheIDProperty.class).get())
           .findFirst();
-      if (cacheId.isPresent() && cacheIdToParallelism.get(cacheId.get()) != null) { // Cached already.
-        // Replace the vertex emitting cached edge with a cached source vertex.
-        final IRVertex cachedDataRelayVertex = new CachedSourceVertex(cacheIdToParallelism.get(cacheId.get()));
-        cachedDataRelayVertex.setPropertyPermanently(ParallelismProperty.of(cacheIdToParallelism.get(cacheId.get())));
+        if (cacheId.isPresent() && cacheIdToParallelism.get(cacheId.get()) != null) { // Cached already.
+          // Replace the vertex emitting cached edge with a cached source vertex.
+          final IRVertex cachedDataRelayVertex = new CachedSourceVertex(cacheIdToParallelism.get(cacheId.get()));
+          cachedDataRelayVertex.setPropertyPermanently(ParallelismProperty.of(cacheIdToParallelism.get(cacheId.get())));
 
-        builder.addVertex(cachedDataRelayVertex);
-        final IREdge newEdge = new IREdge(
+          builder.addVertex(cachedDataRelayVertex);
+          final IREdge newEdge = new IREdge(
             edge.getPropertyValue(CommunicationPatternProperty.class)
-                .orElseThrow(() -> new RuntimeException("No communication pattern on an ir edge")),
+              .orElseThrow(() -> new RuntimeException("No communication pattern on an ir edge")),
             cachedDataRelayVertex,
             irVertex);
-        edge.copyExecutionPropertiesTo(newEdge);
-        newEdge.setProperty(CacheIDProperty.of(cacheId.get()));
-        builder.connectVertices(newEdge);
-        // Stop the recursion for this vertex.
-      } else {
-        final IRVertex srcVtx = edge.getSrc();
-        builder.addVertex(srcVtx);
-        builder.connectVertices(edge);
-        addNonCachedVerticesAndEdges(dag, srcVtx, builder);
-      }
-    });
+          edge.copyExecutionPropertiesTo(newEdge);
+          newEdge.setProperty(CacheIDProperty.of(cacheId.get()));
+          builder.connectVertices(newEdge);
+          // Stop the recursion for this vertex.
+        } else {
+          final IRVertex srcVtx = edge.getSrc();
+          builder.addVertex(srcVtx);
+          builder.connectVertices(edge);
+          addNonCachedVerticesAndEdges(dag, srcVtx, builder);
+        }
+      });
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/Optimizer.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/Optimizer.java
index 9e5159b..3c89e87 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/Optimizer.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/Optimizer.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.reef.tang.annotations.DefaultImplementation;
 
 /**
@@ -36,5 +34,5 @@ public interface Optimizer {
    * @param dag the input DAG to optimize.
    * @return optimized DAG, reshaped or tagged with execution properties.
    */
-  DAG<IRVertex, IREdge> optimizeDag(DAG<IRVertex, IREdge> dag);
+  IRDAG optimizeDag(IRDAG dag);
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
index eeb855d..ec533ad 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
@@ -18,15 +18,14 @@
  */
 package org.apache.nemo.compiler.optimizer.examples;
 
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
+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.test.EmptyComponents;
 import org.apache.nemo.compiler.optimizer.policy.DisaggregationPolicy;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,12 +65,12 @@ public final class MapReduceDisaggregationOptimization {
     final IREdge edge2 = new IREdge(CommunicationPatternProperty.Value.Shuffle, map, reduce);
     builder.connectVertices(edge2);
 
-    final DAG<IRVertex, IREdge> dag = builder.build();
+    final IRDAG dag = new IRDAG(builder.build());
     LOG.info("Before Optimization");
     LOG.info(dag.toString());
 
     // Optimize
-    final DAG optimizedDAG = new DisaggregationPolicy().runCompileTimeOptimization(dag, EMPTY_DAG_DIRECTORY);
+    final IRDAG optimizedDAG = new DisaggregationPolicy().runCompileTimeOptimization(dag, EMPTY_DAG_DIRECTORY);
 
     // After
     LOG.info("After Optimization");
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/CompileTimePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/CompileTimePass.java
index 27d9da8..5f5dbb7 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/CompileTimePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/CompileTimePass.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime;
 
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.executionproperty.ExecutionProperty;
 import org.apache.nemo.common.pass.Pass;
 
@@ -31,7 +29,7 @@ import java.util.function.Function;
  * Abstract class for compile-time optimization passes that processes the DAG.
  * It is a function that takes an original DAG to produce a processed DAG, after an optimization.
  */
-public abstract class CompileTimePass extends Pass implements Function<DAG<IRVertex, IREdge>, DAG<IRVertex, IREdge>> {
+public abstract class CompileTimePass extends Pass implements Function<IRDAG, IRDAG> {
   /**
    * Getter for prerequisite execution properties.
    * @return set of prerequisite execution properties.
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/AggressiveSpeculativeCloningPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/AggressiveSpeculativeCloningPass.java
index 89651d9..818a803 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/AggressiveSpeculativeCloningPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/AggressiveSpeculativeCloningPass.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ClonedSchedulingProperty;
 
 /**
@@ -37,7 +35,7 @@ public final class AggressiveSpeculativeCloningPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     // Speculative execution policy.
     final double fractionToWaitFor = 0.00000001; // Aggressive
     final double medianTimeMultiplier = 1.00000001; // Aggressive
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/CompressionPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/CompressionPass.java
index 8ac538a..e3154d3 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/CompressionPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/CompressionPass.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 
 /**
@@ -48,11 +46,10 @@ public final class CompressionPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(vertex -> dag.getIncomingEdgesOf(vertex).stream()
         .filter(edge -> !edge.getPropertyValue(CompressionProperty.class).isPresent())
         .forEach(edge -> edge.setProperty(CompressionProperty.of(compression))));
-
     return dag;
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DecompressionPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DecompressionPass.java
index 05121dd..b4c1e08 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DecompressionPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DecompressionPass.java
@@ -18,11 +18,9 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 
@@ -42,14 +40,13 @@ public final class DecompressionPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(vertex -> dag.getIncomingEdgesOf(vertex).stream()
         // Find edges which have a compression property but not decompression property.
         .filter(edge -> edge.getPropertyValue(CompressionProperty.class).isPresent()
             && !edge.getPropertyValue(DecompressionProperty.class).isPresent())
         .forEach(edge -> edge.setProperty(DecompressionProperty.of(
             edge.getPropertyValue(CompressionProperty.class).get()))));
-
     return dag;
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java
index 95240f3..1a947f4 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java
@@ -18,11 +18,9 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 /**
@@ -40,7 +38,7 @@ public final class DefaultDataPersistencePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(irVertex ->
         dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
           if (!irEdge.getPropertyValue(DataPersistenceProperty.class).isPresent()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java
index 7e0c966..f8ac7e6 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 /**
  * Edge data store pass to process inter-stage memory store edges.
@@ -36,7 +34,7 @@ public final class DefaultDataStorePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> {
       dag.getIncomingEdgesOf(vertex).stream()
           .filter(edge -> !edge.getPropertyValue(DataStoreProperty.class).isPresent())
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeDecoderPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeDecoderPass.java
index f1113c9..e513065 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeDecoderPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeDecoderPass.java
@@ -19,10 +19,8 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
 import org.apache.nemo.common.coder.DecoderFactory;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 /**
  * Pass for initiating IREdge Decoder ExecutionProperty with default dummy coder.
@@ -41,7 +39,7 @@ public final class DefaultEdgeDecoderPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(irVertex ->
         dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
           if (!irEdge.getPropertyValue(DecoderProperty.class).isPresent()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeEncoderPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeEncoderPass.java
index 21a4134..790391c 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeEncoderPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeEncoderPass.java
@@ -19,10 +19,8 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
 import org.apache.nemo.common.coder.EncoderFactory;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 /**
  * Pass for initiating IREdge Encoder ExecutionProperty with default dummy coder.
@@ -41,7 +39,7 @@ public final class DefaultEdgeEncoderPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(irVertex ->
         dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
           if (!irEdge.getPropertyValue(EncoderProperty.class).isPresent()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultMetricPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultMetricPass.java
index fe3baa8..5b6b89c 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultMetricPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultMetricPass.java
@@ -21,11 +21,9 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 import org.apache.nemo.common.DataSkewMetricFactory;
 import org.apache.nemo.common.HashRange;
 import org.apache.nemo.common.KeyRange;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataSkewMetricProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
 
 import java.util.HashMap;
@@ -44,7 +42,7 @@ public final class DefaultMetricPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(dst ->
       dag.getIncomingEdgesOf(dst).forEach(edge -> {
         if (CommunicationPatternProperty.Value.Shuffle
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
index 4fedc6a..73fc7a3 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
@@ -18,12 +18,11 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.SourceVertex;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -60,7 +59,7 @@ public final class DefaultParallelismPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     // Propagate forward source parallelism
     dag.topologicalDo(vertex -> {
       try {
@@ -103,8 +102,7 @@ public final class DefaultParallelismPass extends AnnotatingPass {
         throw new RuntimeException(e);
       }
     });
-    final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>(dag);
-    return builder.build();
+    return dag;
   }
 
   /**
@@ -114,7 +112,7 @@ public final class DefaultParallelismPass extends AnnotatingPass {
    * @param parallelism the parallelism of the most recently updated descendant.
    * @return the max value of parallelism among those observed.
    */
-  static Integer recursivelySynchronizeO2OParallelism(final DAG<IRVertex, IREdge> dag, final IRVertex vertex,
+  static Integer recursivelySynchronizeO2OParallelism(final IRDAG dag, final IRVertex vertex,
                                                       final Integer parallelism) {
     final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
     final Integer ancestorParallelism = inEdges.stream()
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
index a36612f..f000fce 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
@@ -18,17 +18,17 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
+import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.nemo.common.dag.DAGBuilder;
 import org.apache.nemo.common.dag.Edge;
 import org.apache.nemo.common.dag.Vertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-import org.apache.commons.lang3.mutable.MutableInt;
 
 import java.util.*;
 
@@ -83,7 +83,7 @@ public final class DefaultScheduleGroupPass extends AnnotatingPass {
 
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     final Map<IRVertex, ScheduleGroup> irVertexToScheduleGroupMap = new HashMap<>();
     final Set<ScheduleGroup> scheduleGroups = new HashSet<>();
     dag.topologicalDo(irVertex -> {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
index 7eb1043..b6b9fac 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
@@ -18,10 +18,9 @@
  */
 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.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 import java.util.List;
@@ -41,7 +40,7 @@ public final class DisaggregationEdgeDataStorePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> { // Initialize the DataStore of the DAG with GlusterFileStore.
       final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
       inEdges.forEach(edge ->
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DuplicateEdgeGroupSizePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DuplicateEdgeGroupSizePass.java
index 6119658..0e49943 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DuplicateEdgeGroupSizePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DuplicateEdgeGroupSizePass.java
@@ -18,11 +18,9 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupPropertyValue;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 import java.util.HashMap;
 import java.util.Optional;
@@ -41,7 +39,7 @@ public final class DuplicateEdgeGroupSizePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     final HashMap<String, Integer> groupIdToGroupSize = new HashMap<>();
     dag.topologicalDo(vertex -> dag.getIncomingEdgesOf(vertex)
         .forEach(e -> {
@@ -65,7 +63,6 @@ public final class DuplicateEdgeGroupSizePass extends AnnotatingPass {
             }
           }
         }));
-
     return dag;
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleAnnotatingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleAnnotatingPass.java
new file mode 100644
index 0000000..124b1a6
--- /dev/null
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleAnnotatingPass.java
@@ -0,0 +1,96 @@
+/*
+ * 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.coder.BytesDecoderFactory;
+import org.apache.nemo.common.coder.BytesEncoderFactory;
+import org.apache.nemo.common.ir.IRDAG;
+import org.apache.nemo.common.ir.edge.executionproperty.*;
+import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import org.apache.nemo.common.ir.vertex.system.StreamVertex;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
+
+/**
+ * This pass assumes that a StreamVertex was previously inserted to receive each shuffle edge.
+ *
+ * src - shuffle-edge - streamvertex - one-to-one-edge - dst
+ *
+ * (1) shuffle-edge
+ * Encode/compress into byte[], and have the receiver read data as the same byte[], rather than decompressing/decoding.
+ * Perform a push-based in-memory shuffle with discarding on.
+ *
+ * (2) streamvertex
+ * Ignore resource slots, such that all tasks fetch the in-memory input data blocks as soon as they become available.
+ *
+ * (3) one-to-one-edge
+ * Do not encode/compress the byte[]
+ * Perform a pull-based and on-disk data transfer with the DedicatedKeyPerElementPartitioner.
+ */
+@Annotates({CompressionProperty.class, DataFlowProperty.class, CompressionProperty.class,
+  DataPersistenceProperty.class, DataStoreProperty.class, DecoderProperty.class, DecompressionProperty.class,
+  EncoderProperty.class, PartitionerProperty.class, ResourceSlotProperty.class})
+@Requires(CommunicationPatternProperty.class)
+public final class LargeShuffleAnnotatingPass extends AnnotatingPass {
+  /**
+   * Default constructor.
+   */
+  public LargeShuffleAnnotatingPass() {
+    super(LargeShuffleAnnotatingPass.class);
+  }
+
+  @Override
+  public IRDAG apply(final IRDAG dag) {
+    dag.topologicalDo(irVertex ->
+      dag.getIncomingEdgesOf(irVertex).forEach(edge -> {
+        if (edge.getDst().getClass().equals(StreamVertex.class)) {
+          // CASE #1: To a stream vertex
+
+          // Coder and Compression
+          edge.setPropertyPermanently(DecoderProperty.of(BytesDecoderFactory.of()));
+          edge.setPropertyPermanently(CompressionProperty.of(CompressionProperty.Value.LZ4));
+          edge.setPropertyPermanently(DecompressionProperty.of(CompressionProperty.Value.None));
+
+          // Data transfers
+          edge.setPropertyPermanently(DataFlowProperty.of(DataFlowProperty.Value.Push));
+          edge.setPropertyPermanently(DataPersistenceProperty.of(DataPersistenceProperty.Value.Discard));
+          edge.setPropertyPermanently(DataStoreProperty.of(DataStoreProperty.Value.SerializedMemoryStore));
+
+          // Resource slots
+          edge.getDst().setPropertyPermanently(ResourceSlotProperty.of(false));
+        } else if (edge.getSrc().getClass().equals(StreamVertex.class)) {
+          // CASE #2: From a stream vertex
+
+          // Coder and Compression
+          edge.setPropertyPermanently(EncoderProperty.of(BytesEncoderFactory.of()));
+          edge.setPropertyPermanently(CompressionProperty.of(CompressionProperty.Value.None));
+          edge.setPropertyPermanently(DecompressionProperty.of(CompressionProperty.Value.LZ4));
+
+          // Data transfers
+          edge.setPropertyPermanently(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+          edge.setPropertyPermanently(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+          edge.setPropertyPermanently(
+            PartitionerProperty.of(PartitionerProperty.Value.DedicatedKeyPerElementPartitioner));
+        } else {
+          // CASE #3: Unrelated to any stream vertices
+          edge.setPropertyPermanently(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+        }
+      }));
+    return dag;
+  }
+}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleCompressionPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleCompressionPass.java
deleted file mode 100644
index 0a3969e..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleCompressionPass.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to support Sailfish-like shuffle by tagging edges.
- * This pass modifies the encoder property toward {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform}
- * to write data as byte arrays.
- */
-@Annotates(CompressionProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleCompressionPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleCompressionPass() {
-    super(LargeShuffleCompressionPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          edge.setPropertyPermanently(CompressionProperty.of(CompressionProperty.Value.LZ4));
-
-          dag.getOutgoingEdgesOf(edge.getDst())
-              .forEach(edgeFromRelay ->
-                  edgeFromRelay.setPropertyPermanently(CompressionProperty.of(CompressionProperty.Value.None)));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java
deleted file mode 100644
index 3818a5a..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass handles the DataFlowModel ExecutionProperty.
- */
-@Annotates(DataFlowProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleDataFlowPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleDataFlowPass() {
-    super(LargeShuffleDataFlowPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          edge.setPropertyPermanently(DataFlowProperty.of(DataFlowProperty.Value.Push)); // Push to the merger vertex.
-        } else {
-          edge.setPropertyPermanently(DataFlowProperty.of(DataFlowProperty.Value.Pull));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java
deleted file mode 100644
index f72ea26..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DataPersistenceProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass handles the data persistence ExecutionProperty.
- */
-@Annotates(DataPersistenceProperty.class)
-@Requires(DataFlowProperty.class)
-public final class LargeShuffleDataPersistencePass extends AnnotatingPass {
-
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleDataPersistencePass() {
-    super(LargeShuffleDataPersistencePass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.topologicalDo(irVertex ->
-        dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
-          final DataFlowProperty.Value dataFlowModel = irEdge.getPropertyValue(DataFlowProperty.class).get();
-          if (DataFlowProperty.Value.Push.equals(dataFlowModel)) {
-            irEdge.setPropertyPermanently(DataPersistenceProperty.of(DataPersistenceProperty.Value.Discard));
-          }
-        }));
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java
deleted file mode 100644
index 76c2038..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass handles the DataStore ExecutionProperty.
- */
-@Annotates(DataStoreProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleDataStorePass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleDataStorePass() {
-    super(LargeShuffleDataStorePass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      // Find the merger vertex inserted by reshaping pass.
-      if (dag.getIncomingEdgesOf(vertex).stream().anyMatch(irEdge ->
-              CommunicationPatternProperty.Value.Shuffle
-          .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
-        dag.getIncomingEdgesOf(vertex).forEach(edgeToMerger -> {
-          if (CommunicationPatternProperty.Value.Shuffle
-          .equals(edgeToMerger.getPropertyValue(CommunicationPatternProperty.class).get())) {
-            // Pass data through memory to the merger vertex.
-            edgeToMerger.setPropertyPermanently(DataStoreProperty.of(DataStoreProperty.Value.SerializedMemoryStore));
-          }
-        });
-        dag.getOutgoingEdgesOf(vertex).forEach(edgeFromMerger ->
-            // Merge the input data and write it immediately to the remote disk.
-            edgeFromMerger.setPropertyPermanently(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore)));
-      }
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java
deleted file mode 100644
index c69c921..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.coder.BytesDecoderFactory;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass modifies the decoder property toward {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform}
- * to read data as byte arrays.
- */
-@Annotates(DecoderProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleDecoderPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleDecoderPass() {
-    super(LargeShuffleDecoderPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          edge.setPropertyPermanently(DecoderProperty.of(BytesDecoderFactory.of()));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecompressionPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecompressionPass.java
deleted file mode 100644
index bd775f3..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecompressionPass.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.CompressionProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DecompressionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to support Sailfish-like shuffle by tagging edges.
- * This pass modifies the decoder property toward {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform}
- * to read data as byte arrays.
- */
-@Annotates(DecompressionProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleDecompressionPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleDecompressionPass() {
-    super(LargeShuffleDecompressionPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          edge.setPropertyPermanently(DecompressionProperty.of(CompressionProperty.Value.None));
-
-          dag.getOutgoingEdgesOf(edge.getDst())
-              .forEach(edgeFromRelay ->
-                  edgeFromRelay.setPropertyPermanently(DecompressionProperty.of(CompressionProperty.Value.LZ4)));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java
deleted file mode 100644
index 5defb59..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.coder.BytesEncoderFactory;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass modifies the encoder property toward {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform}
- * to write data as byte arrays.
- */
-@Annotates(EncoderProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleEncoderPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleEncoderPass() {
-    super(LargeShuffleEncoderPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          dag.getOutgoingEdgesOf(edge.getDst())
-              .forEach(edgeFromRelay ->
-                  edgeFromRelay.setPropertyPermanently(EncoderProperty.of(BytesEncoderFactory.of())));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShufflePartitionerPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShufflePartitionerPass.java
deleted file mode 100644
index d5958f1..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShufflePartitionerPass.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-import java.util.List;
-
-/**
- * A pass to support Sailfish-like shuffle by tagging edges.
- * This pass modifies the partitioner property from {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform}
- * to write an element as a partition.
- * This enables that every byte[] element, which was a partition for the reduce task, becomes one partition again
- * and flushed to disk write after it is relayed.
- */
-@Annotates(PartitionerProperty.class)
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShufflePartitionerPass extends AnnotatingPass {
-  /**
-   * Default constructor.
-   */
-  public LargeShufflePartitionerPass() {
-    super(LargeShufflePartitionerPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      inEdges.forEach(edge -> {
-        if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-            .equals(CommunicationPatternProperty.Value.Shuffle)) {
-          dag.getOutgoingEdgesOf(edge.getDst())
-              .forEach(edgeFromRelay ->
-                  edgeFromRelay.setPropertyPermanently(PartitionerProperty.of(
-                      PartitionerProperty.Value.DedicatedKeyPerElementPartitioner)));
-        }
-      });
-    });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java
deleted file mode 100644
index eb5525d..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-/**
- * Sets {@link ResourceSlotProperty}.
- */
-@Annotates(ResourceSlotProperty.class)
-@Requires(DataFlowProperty.class)
-public final class LargeShuffleResourceSlotPass extends AnnotatingPass {
-
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleResourceSlotPass() {
-    super(LargeShuffleResourceSlotPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    // On every vertex that receive push edge, if ResourceSlotProperty is not set, put it as false.
-    // For other vertices, if ResourceSlotProperty is not set, put it as true.
-    dag.getVertices().stream()
-        .filter(v -> !v.getPropertyValue(ResourceSlotProperty.class).isPresent())
-        .forEach(v -> {
-          if (dag.getIncomingEdgesOf(v).stream().anyMatch(
-              e -> e.getPropertyValue(DataFlowProperty.class)
-                  .orElseThrow(() -> new RuntimeException(String.format("DataFlowProperty for %s must be set",
-                      e.getId()))).equals(DataFlowProperty.Value.Push))) {
-            v.setPropertyPermanently(ResourceSlotProperty.of(false));
-          } else {
-            v.setPropertyPermanently(ResourceSlotProperty.of(true));
-          }
-        });
-    return dag;
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/PipeTransferForAllEdgesPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/PipeTransferForAllEdgesPass.java
index 25999dc..37ad2cb 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/PipeTransferForAllEdgesPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/PipeTransferForAllEdgesPass.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 
 /**
  * Annotate 'Pipe' on all edges.
@@ -36,7 +34,7 @@ public final class PipeTransferForAllEdgesPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> {
       dag.getIncomingEdgesOf(vertex).stream()
           .forEach(edge -> edge.setPropertyPermanently(
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java
index 141adbc..8294beb 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty;
 
 /**
@@ -37,7 +35,7 @@ public final class ResourceLocalityPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     // On every vertex, if ResourceLocalityProperty is not set, put it as true.
     dag.getVertices().stream()
         .filter(v -> !v.getPropertyValue(ResourceLocalityProperty.class).isPresent())
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java
index c822039..1820a7c 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java
@@ -20,18 +20,18 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
 import com.fasterxml.jackson.core.TreeNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSiteProperty;
-import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 import org.apache.commons.math3.optim.BaseOptimizer;
 import org.apache.commons.math3.optim.PointValuePair;
 import org.apache.commons.math3.optim.linear.*;
 import org.apache.commons.math3.optim.nonlinear.scalar.GoalType;
 import org.apache.commons.math3.util.Incrementor;
+import org.apache.nemo.common.ir.IRDAG;
+import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
+import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSiteProperty;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,7 +74,7 @@ public final class ResourceSitePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     if (bandwidthSpecificationString.isEmpty()) {
       dag.topologicalDo(irVertex -> irVertex.setProperty(ResourceSiteProperty.of(EMPTY_MAP)));
     } else {
@@ -110,7 +110,7 @@ public final class ResourceSitePass extends AnnotatingPass {
    * @param bandwidthSpecification bandwidth specification.
    */
   private static void assignNodeShares(
-      final DAG<IRVertex, IREdge> dag,
+      final IRDAG dag,
       final BandwidthSpecification bandwidthSpecification) {
     dag.topologicalDo(irVertex -> {
       final Collection<IREdge> inEdges = dag.getIncomingEdgesOf(irVertex);
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java
index dc9cbe9..abc4c77 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java
@@ -18,9 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
 
 /**
@@ -37,7 +35,7 @@ public final class ResourceSlotPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     // On every vertex, if ResourceSlotProperty is not set, put it as true.
     dag.getVertices().stream()
         .filter(v -> !v.getPropertyValue(ResourceSlotProperty.class).isPresent())
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
index 08855b2..007ce0e 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
@@ -18,11 +18,10 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 import java.util.List;
@@ -42,7 +41,7 @@ public final class ShuffleEdgePushPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> {
       final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
       if (!inEdges.isEmpty()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java
index 45401e0..5d71ffc 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -39,7 +37,7 @@ public final class SkewPartitionerPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices()
       .forEach(v -> dag.getOutgoingEdgesOf(v).stream()
         .filter(edge -> edge.getPropertyValue(MetricCollectionProperty.class).isPresent())
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java
index 425c939..2da624d 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java
@@ -18,8 +18,7 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSkewedDataProperty;
@@ -43,7 +42,7 @@ public final class SkewResourceSkewedDataPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices()
       .forEach(v -> dag.getOutgoingEdgesOf(v).stream()
         .filter(edge -> edge.getPropertyValue(MetricCollectionProperty.class).isPresent())
@@ -55,7 +54,6 @@ public final class SkewResourceSkewedDataPass extends AnnotatingPass {
           });
         })
       );
-
     return dag;
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java
index 99f16b5..5f1daca 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java
@@ -18,10 +18,9 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -43,7 +42,7 @@ public final class TransientResourceDataFlowPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> {
       final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
       if (!inEdges.isEmpty()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java
index baafd83..325aa84 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java
@@ -18,11 +18,10 @@
  */
 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.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -42,7 +41,7 @@ public final class TransientResourceDataStorePass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().forEach(vertex -> {
       final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
       if (!inEdges.isEmpty()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java
index 52e28c0..5988d61 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java
@@ -18,10 +18,9 @@
  */
 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.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -41,7 +40,7 @@ public final class TransientResourcePriorityPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.topologicalDo(vertex -> {
       final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
       if (inEdges.isEmpty()) {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/UpfrontCloningPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/UpfrontCloningPass.java
index 105ddd1..17492d4 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/UpfrontCloningPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/UpfrontCloningPass.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+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.executionproperty.ClonedSchedulingProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
@@ -39,7 +37,7 @@ public final class UpfrontCloningPass extends AnnotatingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG dag) {
     dag.getVertices().stream()
         .filter(vertex -> dag.getIncomingEdgesOf(vertex.getId())
           .stream()
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/CompositePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/CompositePass.java
index eb478f5..19c2a64 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/CompositePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/CompositePass.java
@@ -18,14 +18,15 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.executionproperty.ExecutionProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AnnotatingPass;
 
-import java.util.*;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
 /**
  * A compile-time pass composed of multiple compile-time passes, which each modifies an IR DAG.
@@ -58,7 +59,7 @@ public abstract class CompositePass extends CompileTimePass {
   }
 
   @Override
-  public final DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> irVertexIREdgeDAG) {
+  public final IRDAG apply(final IRDAG irVertexIREdgeDAG) {
     return recursivelyApply(irVertexIREdgeDAG, getPassList().iterator());
   }
 
@@ -68,8 +69,7 @@ public abstract class CompositePass extends CompileTimePass {
    * @param passIterator pass iterator.
    * @return dag.
    */
-  private DAG<IRVertex, IREdge> recursivelyApply(final DAG<IRVertex, IREdge> dag,
-                                                 final Iterator<CompileTimePass> passIterator) {
+  private IRDAG recursivelyApply(final IRDAG dag, final Iterator<CompileTimePass> passIterator) {
     if (passIterator.hasNext()) {
       return recursivelyApply(passIterator.next().apply(dag), passIterator);
     } else {
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java
index f69b2b8..2752c04 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java
@@ -19,7 +19,7 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.*;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping.LargeShuffleRelayReshapingPass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping.LargeShuffleReshapingPass;
 
 import java.util.Arrays;
 
@@ -33,16 +33,8 @@ public final class LargeShuffleCompositePass extends CompositePass {
    */
   public LargeShuffleCompositePass() {
     super(Arrays.asList(
-        new LargeShuffleRelayReshapingPass(),
-        new LargeShuffleDataFlowPass(),
-        new LargeShuffleDataStorePass(),
-        new LargeShuffleDecoderPass(),
-        new LargeShuffleEncoderPass(),
-        new LargeShufflePartitionerPass(),
-        new LargeShuffleCompressionPass(),
-        new LargeShuffleDecompressionPass(),
-        new LargeShuffleDataPersistencePass(),
-        new LargeShuffleResourceSlotPass()
+        new LargeShuffleReshapingPass(),
+        new LargeShuffleAnnotatingPass()
     ));
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java
index 2ca955c..2e3f108 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java
@@ -18,7 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
-import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.*;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.SkewPartitionerPass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.SkewResourceSkewedDataPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping.SkewReshapingPass;
 
 import java.util.Arrays;
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
index e7c1282..9e066e1 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
@@ -20,6 +20,9 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.common.coder.DecoderFactory;
 import org.apache.nemo.common.coder.EncoderFactory;
+import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
@@ -27,8 +30,6 @@ import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.OperatorVertex;
 import org.apache.nemo.common.ir.vertex.transform.Transform;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.dag.DAGBuilder;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 import java.util.*;
@@ -50,68 +51,69 @@ public final class CommonSubexpressionEliminationPass extends ReshapingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+  public IRDAG apply(final IRDAG inputDAG) {
     // find and collect vertices with equivalent transforms
     final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
     final Map<Transform, List<OperatorVertex>> operatorVerticesToBeMerged = new HashMap<>();
     final Map<OperatorVertex, Set<IREdge>> inEdges = new HashMap<>();
     final Map<OperatorVertex, Set<IREdge>> outEdges = new HashMap<>();
 
-    dag.topologicalDo(irVertex -> {
-      if (irVertex instanceof OperatorVertex) {
-        final OperatorVertex operatorVertex = (OperatorVertex) irVertex;
-        operatorVerticesToBeMerged.putIfAbsent(operatorVertex.getTransform(), new ArrayList<>());
-        operatorVerticesToBeMerged.get(operatorVertex.getTransform()).add(operatorVertex);
+    inputDAG.reshapeUnsafely(dag -> {
+      dag.topologicalDo(irVertex -> {
+        if (irVertex instanceof OperatorVertex) {
+          final OperatorVertex operatorVertex = (OperatorVertex) irVertex;
+          operatorVerticesToBeMerged.putIfAbsent(operatorVertex.getTransform(), new ArrayList<>());
+          operatorVerticesToBeMerged.get(operatorVertex.getTransform()).add(operatorVertex);
 
-        dag.getIncomingEdgesOf(operatorVertex).forEach(irEdge -> {
-          inEdges.putIfAbsent(operatorVertex, new HashSet<>());
-          inEdges.get(operatorVertex).add(irEdge);
-          if (irEdge.getSrc() instanceof OperatorVertex) {
-            final OperatorVertex source = (OperatorVertex) irEdge.getSrc();
-            outEdges.putIfAbsent(source, new HashSet<>());
-            outEdges.get(source).add(irEdge);
-          }
-        });
-      } else {
-        builder.addVertex(irVertex, dag);
-        dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
-          if (irEdge.getSrc() instanceof OperatorVertex) {
-            final OperatorVertex source = (OperatorVertex) irEdge.getSrc();
-            outEdges.putIfAbsent(source, new HashSet<>());
-            outEdges.get(source).add(irEdge);
-          } else {
-            builder.connectVertices(irEdge);
-          }
-        });
-      }
-    });
+          dag.getIncomingEdgesOf(operatorVertex).forEach(irEdge -> {
+            inEdges.putIfAbsent(operatorVertex, new HashSet<>());
+            inEdges.get(operatorVertex).add(irEdge);
+            if (irEdge.getSrc() instanceof OperatorVertex) {
+              final OperatorVertex source = (OperatorVertex) irEdge.getSrc();
+              outEdges.putIfAbsent(source, new HashSet<>());
+              outEdges.get(source).add(irEdge);
+            }
+          });
+        } else {
+          builder.addVertex(irVertex, dag);
+          dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
+            if (irEdge.getSrc() instanceof OperatorVertex) {
+              final OperatorVertex source = (OperatorVertex) irEdge.getSrc();
+              outEdges.putIfAbsent(source, new HashSet<>());
+              outEdges.get(source).add(irEdge);
+            } else {
+              builder.connectVertices(irEdge);
+            }
+          });
+        }
+      });
 
-    // merge them if they are not dependent on each other, and add IRVertices to the builder.
-    operatorVerticesToBeMerged.forEach(((transform, operatorVertices) -> {
-      final Map<Set<IRVertex>, List<OperatorVertex>> verticesToBeMergedWithIdenticalSources = new HashMap<>();
+      // merge them if they are not dependent on each other, and add IRVertices to the builder.
+      operatorVerticesToBeMerged.forEach(((transform, operatorVertices) -> {
+        final Map<Set<IRVertex>, List<OperatorVertex>> verticesToBeMergedWithIdenticalSources = new HashMap<>();
 
-      operatorVertices.forEach(operatorVertex -> {
-        // compare if incoming vertices are identical.
-        final Set<IRVertex> incomingVertices = dag.getIncomingEdgesOf(operatorVertex).stream().map(IREdge::getSrc)
+        operatorVertices.forEach(operatorVertex -> {
+          // compare if incoming vertices are identical.
+          final Set<IRVertex> incomingVertices = dag.getIncomingEdgesOf(operatorVertex).stream().map(IREdge::getSrc)
             .collect(Collectors.toSet());
-        if (verticesToBeMergedWithIdenticalSources.keySet().stream()
+          if (verticesToBeMergedWithIdenticalSources.keySet().stream()
             .anyMatch(lst -> lst.containsAll(incomingVertices) && incomingVertices.containsAll(lst))) {
-          final Set<IRVertex> foundKey = verticesToBeMergedWithIdenticalSources.keySet().stream()
+            final Set<IRVertex> foundKey = verticesToBeMergedWithIdenticalSources.keySet().stream()
               .filter(vs -> vs.containsAll(incomingVertices) && incomingVertices.containsAll(vs))
               .findFirst().get();
-          verticesToBeMergedWithIdenticalSources.get(foundKey).add(operatorVertex);
-        } else {
-          verticesToBeMergedWithIdenticalSources.putIfAbsent(incomingVertices, new ArrayList<>());
-          verticesToBeMergedWithIdenticalSources.get(incomingVertices).add(operatorVertex);
-        }
-      });
+            verticesToBeMergedWithIdenticalSources.get(foundKey).add(operatorVertex);
+          } else {
+            verticesToBeMergedWithIdenticalSources.putIfAbsent(incomingVertices, new ArrayList<>());
+            verticesToBeMergedWithIdenticalSources.get(incomingVertices).add(operatorVertex);
+          }
+        });
 
-      verticesToBeMergedWithIdenticalSources.values().forEach(ovs ->
+        verticesToBeMergedWithIdenticalSources.values().forEach(ovs ->
           mergeAndAddToBuilder(ovs, builder, dag, inEdges, outEdges));
-    }));
+      }));
 
-    // process IREdges
-    operatorVerticesToBeMerged.values().forEach(operatorVertices ->
+      // process IREdges
+      operatorVerticesToBeMerged.values().forEach(operatorVertices ->
         operatorVertices.forEach(operatorVertex -> {
           inEdges.getOrDefault(operatorVertex, new HashSet<>()).forEach(e -> {
             if (builder.contains(operatorVertex) && builder.contains(e.getSrc())) {
@@ -125,7 +127,10 @@ public final class CommonSubexpressionEliminationPass extends ReshapingPass {
           });
         }));
 
-    return builder.build();
+      return builder.build();
+    });
+
+    return inputDAG;
   }
 
   /**
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java
deleted file mode 100644
index c34615e..0000000
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.reshaping;
-
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.dag.DAGBuilder;
-import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.OperatorVertex;
-import org.apache.nemo.common.ir.vertex.transform.RelayTransform;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
-
-/**
- * Pass to modify the DAG for a job to batch the disk seek.
- * It adds a {@link OperatorVertex} with {@link RelayTransform} before the vertices
- * receiving shuffle edges,
- * to merge the shuffled data in memory and write to the disk at once.
- */
-@Requires(CommunicationPatternProperty.class)
-public final class LargeShuffleRelayReshapingPass extends ReshapingPass {
-
-  /**
-   * Default constructor.
-   */
-  public LargeShuffleRelayReshapingPass() {
-    super(LargeShuffleRelayReshapingPass.class);
-  }
-
-  @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
-    dag.topologicalDo(v -> {
-      builder.addVertex(v);
-      // We care about OperatorVertices that have any incoming edge that
-      // has Shuffle as data communication pattern.
-      if (v instanceof OperatorVertex && dag.getIncomingEdgesOf(v).stream().anyMatch(irEdge ->
-        CommunicationPatternProperty.Value.Shuffle
-          .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
-        dag.getIncomingEdgesOf(v).forEach(edge -> {
-          if (CommunicationPatternProperty.Value.Shuffle
-            .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
-            // Insert a merger vertex having transform that write received data immediately
-            // before the vertex receiving shuffled data.
-            final OperatorVertex iFileMergerVertex = new OperatorVertex(new RelayTransform());
-
-            builder.addVertex(iFileMergerVertex);
-            final IREdge newEdgeToMerger =
-              new IREdge(CommunicationPatternProperty.Value.Shuffle, edge.getSrc(), iFileMergerVertex);
-            edge.copyExecutionPropertiesTo(newEdgeToMerger);
-            final IREdge newEdgeFromMerger = new IREdge(CommunicationPatternProperty.Value.OneToOne,
-              iFileMergerVertex, v);
-            newEdgeFromMerger.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
-            newEdgeFromMerger.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
-            builder.connectVertices(newEdgeToMerger);
-            builder.connectVertices(newEdgeFromMerger);
-          } else {
-            builder.connectVertices(edge);
-          }
-        });
-      } else { // Others are simply added to the builder.
-        dag.getIncomingEdgesOf(v).forEach(builder::connectVertices);
-      }
-    });
-    return builder.build();
-  }
-}
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleReshapingPass.java
similarity index 50%
copy from compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
copy to compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleReshapingPass.java
index 08855b2..b865bb3 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleReshapingPass.java
@@ -16,43 +16,36 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
+package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.vertex.system.StreamVertex;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
-import java.util.List;
-
 /**
- * A pass for tagging shuffle edges different from the default ones.
- * It sets DataFlowModel ExecutionProperty as "push".
+ * Inserts the StreamVertex for each shuffle edge.
  */
-@Annotates(DataFlowProperty.class)
 @Requires(CommunicationPatternProperty.class)
-public final class ShuffleEdgePushPass extends AnnotatingPass {
+public final class LargeShuffleReshapingPass extends ReshapingPass {
+
   /**
    * Default constructor.
    */
-  public ShuffleEdgePushPass() {
-    super(ShuffleEdgePushPass.class);
+  public LargeShuffleReshapingPass() {
+    super(LargeShuffleReshapingPass.class);
   }
 
+
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    dag.getVertices().forEach(vertex -> {
-      final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
-      if (!inEdges.isEmpty()) {
-        inEdges.forEach(edge -> {
-          if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
-              .equals(CommunicationPatternProperty.Value.Shuffle)) {
-            edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
-          }
-        });
-      }
+  public IRDAG apply(final IRDAG dag) {
+    dag.topologicalDo(vertex -> {
+      dag.getIncomingEdgesOf(vertex).forEach(edge -> {
+        if (CommunicationPatternProperty.Value.Shuffle
+          .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
+          dag.insert(new StreamVertex(), edge);
+        }
+      });
     });
     return dag;
   }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
index 1a8f281..ce10847 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
@@ -18,17 +18,21 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
+import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.dag.DAGBuilder;
 import org.apache.nemo.common.ir.vertex.LoopVertex;
 import org.apache.nemo.common.ir.vertex.OperatorVertex;
 import org.apache.nemo.common.ir.vertex.SourceVertex;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
-import java.util.*;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.OptionalInt;
 
 /**
  * Pass for extracting and  grouping each loops together using the LoopVertex.
@@ -46,9 +50,12 @@ public final class LoopExtractionPass extends ReshapingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    final Integer maxStackDepth = this.findMaxLoopVertexStackDepth(dag);
-    return groupLoops(dag, maxStackDepth);
+  public IRDAG apply(final IRDAG inputDAG) {
+    inputDAG.reshapeUnsafely(dag -> {
+      final Integer maxStackDepth = this.findMaxLoopVertexStackDepth(dag);
+      return groupLoops(dag, maxStackDepth);
+    });
+    return inputDAG;
   }
 
   /**
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
index f3d09ad..56013c0 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
@@ -18,14 +18,15 @@
  */
 package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
+import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.common.ir.vertex.LoopVertex;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.dag.DAGBuilder;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 
 import java.util.*;
@@ -112,93 +113,98 @@ public final class LoopOptimizations {
     }
 
     @Override
-    public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-      final List<LoopVertex> loopVertices = new ArrayList<>();
-      final Map<LoopVertex, List<IREdge>> inEdges = new HashMap<>();
-      final Map<LoopVertex, List<IREdge>> outEdges = new HashMap<>();
-      final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
+    public IRDAG apply(final IRDAG inputDAG) {
+      inputDAG.reshapeUnsafely(dag -> {
+        final List<LoopVertex> loopVertices = new ArrayList<>();
+        final Map<LoopVertex, List<IREdge>> inEdges = new HashMap<>();
+        final Map<LoopVertex, List<IREdge>> outEdges = new HashMap<>();
+        final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
 
-      collectLoopVertices(dag, loopVertices, inEdges, outEdges, builder);
+        collectLoopVertices(dag, loopVertices, inEdges, outEdges, builder);
 
-      // Collect and group those with same termination condition.
-      final Set<Set<LoopVertex>> setOfLoopsToBeFused = new HashSet<>();
-      loopVertices.forEach(loopVertex -> {
-        final IntPredicate terminationCondition = loopVertex.getTerminationCondition();
-        final Integer numberOfIterations = loopVertex.getMaxNumberOfIterations();
-        // We want loopVertices that are not dependent on each other or the list that is potentially going to be merged.
-        final List<LoopVertex> independentLoops = loopVertices.stream().filter(loop ->
+        // Collect and group those with same termination condition.
+        final Set<Set<LoopVertex>> setOfLoopsToBeFused = new HashSet<>();
+        loopVertices.forEach(loopVertex -> {
+          final IntPredicate terminationCondition = loopVertex.getTerminationCondition();
+          final Integer numberOfIterations = loopVertex.getMaxNumberOfIterations();
+          // We want loopVertices that are not dependent on each other
+          // or the list that is potentially going to be merged.
+          final List<LoopVertex> independentLoops = loopVertices.stream().filter(loop ->
             setOfLoopsToBeFused.stream().anyMatch(list -> list.contains(loop))
-                ? setOfLoopsToBeFused.stream().filter(list -> list.contains(loop))
-                .findFirst()
-                .map(list -> list.stream().noneMatch(loopV -> dag.pathExistsBetween(loopV, loopVertex)))
-                .orElse(false)
-                : !dag.pathExistsBetween(loop, loopVertex)).collect(Collectors.toList());
+              ? setOfLoopsToBeFused.stream().filter(list -> list.contains(loop))
+              .findFirst()
+              .map(list -> list.stream().noneMatch(loopV -> dag.pathExistsBetween(loopV, loopVertex)))
+              .orElse(false)
+              : !dag.pathExistsBetween(loop, loopVertex)).collect(Collectors.toList());
 
-        // Find loops to be fused together.
-        final Set<LoopVertex> loopsToBeFused = new HashSet<>();
-        loopsToBeFused.add(loopVertex);
-        independentLoops.forEach(independentLoop -> {
-          // add them to the list if those independent loops have equal termination conditions.
-          if (loopVertex.terminationConditionEquals(independentLoop)) {
-            loopsToBeFused.add(independentLoop);
-          }
-        });
+          // Find loops to be fused together.
+          final Set<LoopVertex> loopsToBeFused = new HashSet<>();
+          loopsToBeFused.add(loopVertex);
+          independentLoops.forEach(independentLoop -> {
+            // add them to the list if those independent loops have equal termination conditions.
+            if (loopVertex.terminationConditionEquals(independentLoop)) {
+              loopsToBeFused.add(independentLoop);
+            }
+          });
 
-        // add this information to the setOfLoopsToBeFused set.
-        final Optional<Set<LoopVertex>> listToAddVerticesTo = setOfLoopsToBeFused.stream()
+          // add this information to the setOfLoopsToBeFused set.
+          final Optional<Set<LoopVertex>> listToAddVerticesTo = setOfLoopsToBeFused.stream()
             .filter(list -> list.stream().anyMatch(loopsToBeFused::contains)).findFirst();
-        if (listToAddVerticesTo.isPresent()) {
-          listToAddVerticesTo.get().addAll(loopsToBeFused);
-        } else {
-          setOfLoopsToBeFused.add(loopsToBeFused);
-        }
-      });
+          if (listToAddVerticesTo.isPresent()) {
+            listToAddVerticesTo.get().addAll(loopsToBeFused);
+          } else {
+            setOfLoopsToBeFused.add(loopsToBeFused);
+          }
+        });
 
-      // merge and add to builder.
-      setOfLoopsToBeFused.forEach(loops -> {
-        if (loops.size() > 1) {
-          final LoopVertex newLoopVertex = mergeLoopVertices(loops);
-          builder.addVertex(newLoopVertex, dag);
-          loops.forEach(loopVertex -> {
-            // inEdges.
-            inEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
-              if (builder.contains(irEdge.getSrc())) {
-                final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
+        // merge and add to builder.
+        setOfLoopsToBeFused.forEach(loops -> {
+          if (loops.size() > 1) {
+            final LoopVertex newLoopVertex = mergeLoopVertices(loops);
+            builder.addVertex(newLoopVertex, dag);
+            loops.forEach(loopVertex -> {
+              // inEdges.
+              inEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
+                if (builder.contains(irEdge.getSrc())) {
+                  final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
                     .get(), irEdge.getSrc(), newLoopVertex);
-                irEdge.copyExecutionPropertiesTo(newIREdge);
-                builder.connectVertices(newIREdge);
-              }
-            });
-            // outEdges.
-            outEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
-              if (builder.contains(irEdge.getDst())) {
-                final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
+                  irEdge.copyExecutionPropertiesTo(newIREdge);
+                  builder.connectVertices(newIREdge);
+                }
+              });
+              // outEdges.
+              outEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
+                if (builder.contains(irEdge.getDst())) {
+                  final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
                     .get(), newLoopVertex, irEdge.getDst());
-                irEdge.copyExecutionPropertiesTo(newIREdge);
-                builder.connectVertices(newIREdge);
-              }
-            });
-          });
-        } else {
-          loops.forEach(loopVertex -> {
-            builder.addVertex(loopVertex);
-            // inEdges.
-            inEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(edge -> {
-              if (builder.contains(edge.getSrc())) {
-                builder.connectVertices(edge);
-              }
+                  irEdge.copyExecutionPropertiesTo(newIREdge);
+                  builder.connectVertices(newIREdge);
+                }
+              });
             });
-            // outEdges.
-            outEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(edge -> {
-              if (builder.contains(edge.getDst())) {
-                builder.connectVertices(edge);
-              }
+          } else {
+            loops.forEach(loopVertex -> {
+              builder.addVertex(loopVertex);
+              // inEdges.
+              inEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(edge -> {
+                if (builder.contains(edge.getSrc())) {
+                  builder.connectVertices(edge);
+                }
+              });
+              // outEdges.
+              outEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(edge -> {
+                if (builder.contains(edge.getDst())) {
+                  builder.connectVertices(edge);
+                }
+              });
             });
-          });
-        }
+          }
+        });
+
+        return builder.build();
       });
 
-      return builder.build();
+      return inputDAG;
     }
 
     /**
@@ -240,7 +246,14 @@ public final class LoopOptimizations {
     }
 
     @Override
-    public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
+    public IRDAG apply(final IRDAG inputDAG) {
+      inputDAG.reshapeUnsafely(dag -> {
+        return recursivelyOptimize(dag);
+      });
+      return inputDAG;
+    }
+
+    DAG<IRVertex, IREdge> recursivelyOptimize(final DAG<IRVertex, IREdge> dag) {
       final List<LoopVertex> loopVertices = new ArrayList<>();
       final Map<LoopVertex, List<IREdge>> inEdges = new HashMap<>();
       final Map<LoopVertex, List<IREdge>> outEdges = new HashMap<>();
@@ -251,11 +264,11 @@ public final class LoopOptimizations {
       // Refactor those with same data scan / operation, without dependencies in the loop.
       loopVertices.forEach(loopVertex -> {
         final List<Map.Entry<IRVertex, Set<IREdge>>> candidates = loopVertex.getNonIterativeIncomingEdges().entrySet()
-            .stream().filter(entry ->
-                loopVertex.getDAG().getIncomingEdgesOf(entry.getKey()).size() == 0 // no internal inEdges
-                    // no external inEdges
-                    && loopVertex.getIterativeIncomingEdges().getOrDefault(entry.getKey(), new HashSet<>()).size() == 0)
-            .collect(Collectors.toList());
+          .stream().filter(entry ->
+            loopVertex.getDAG().getIncomingEdgesOf(entry.getKey()).size() == 0 // no internal inEdges
+              // no external inEdges
+              && loopVertex.getIterativeIncomingEdges().getOrDefault(entry.getKey(), new HashSet<>()).size() == 0)
+          .collect(Collectors.toList());
         candidates.forEach(candidate -> {
           // add refactored vertex to builder.
           builder.addVertex(candidate.getKey());
@@ -268,16 +281,16 @@ public final class LoopOptimizations {
           final List<IREdge> edgesToRemove = new ArrayList<>();
           final List<IREdge> edgesToAdd = new ArrayList<>();
           inEdges.getOrDefault(loopVertex, new ArrayList<>()).stream().filter(e ->
-              // filter edges that have their sources as the refactored vertices.
-              candidate.getValue().stream().map(IREdge::getSrc).anyMatch(edgeSrc -> edgeSrc.equals(e.getSrc())))
-              .forEach(edge -> {
-                edgesToRemove.add(edge);
-                final IREdge newEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
-                    candidate.getKey(), edge.getDst());
-                newEdge.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
-                newEdge.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
-                edgesToAdd.add(newEdge);
-              });
+            // filter edges that have their sources as the refactored vertices.
+            candidate.getValue().stream().map(IREdge::getSrc).anyMatch(edgeSrc -> edgeSrc.equals(e.getSrc())))
+            .forEach(edge -> {
+              edgesToRemove.add(edge);
+              final IREdge newEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+                candidate.getKey(), edge.getDst());
+              newEdge.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
+              newEdge.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
+              edgesToAdd.add(newEdge);
+            });
           final List<IREdge> listToModify = inEdges.getOrDefault(loopVertex, new ArrayList<>());
           listToModify.removeAll(edgesToRemove);
           listToModify.addAll(edgesToAdd);
@@ -299,7 +312,7 @@ public final class LoopOptimizations {
       if (dag.getVertices().size() == newDag.getVertices().size()) {
         return newDag;
       } else {
-        return apply(newDag);
+        return recursivelyOptimize(newDag);
       }
     }
   }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPass.java
index ba6ef70..3d8b119 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPass.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+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.LoopVertex;
@@ -38,8 +39,11 @@ public final class LoopUnrollingPass extends ReshapingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    return recursivelyUnroll(dag);
+  public IRDAG apply(final IRDAG inputDAG) {
+    inputDAG.reshapeUnsafely(dag -> {
+      return recursivelyUnroll(dag);
+    });
+    return inputDAG;
   }
 
   /**
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java
index 57a1647..9157e3a 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java
@@ -21,43 +21,36 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 import org.apache.nemo.common.KeyExtractor;
 import org.apache.nemo.common.Pair;
 import org.apache.nemo.common.coder.*;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.dag.DAGBuilder;
-import org.apache.nemo.common.ir.OutputCollector;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.*;
+import org.apache.nemo.common.ir.vertex.system.MessageAggregatorVertex;
+import org.apache.nemo.common.ir.vertex.system.MessageBarrierVertex;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.OperatorVertex;
-import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
-import org.apache.nemo.common.ir.vertex.transform.AggregateMetricTransform;
-import org.apache.nemo.common.ir.vertex.transform.MetricCollectTransform;
-import org.apache.nemo.compiler.optimizer.PairKeyExtractor;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.Requires;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.Annotates;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.*;
 import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /**
  * Pass to reshape the IR DAG for skew handling.
- *
- * This pass inserts vertices to perform two-step dynamic optimization for skew handling.
- * 1) Task-level statistic collection is done via vertex with {@link MetricCollectTransform}
- * 2) Stage-level statistic aggregation is done via vertex with {@link AggregateMetricTransform}
- * inserted before shuffle edges.
+ * We insert a {@link MessageBarrierVertex} for each shuffle edge,
+ * and aggregate messages for multiple same-destination shuffle edges.
  * */
 @Annotates(MetricCollectionProperty.class)
 @Requires(CommunicationPatternProperty.class)
 public final class SkewReshapingPass extends ReshapingPass {
   private static final Logger LOG = LoggerFactory.getLogger(SkewReshapingPass.class.getName());
-  private static final String ADDITIONAL_OUTPUT_TAG = "DynOptData";
+  private static final String MAIN_OUTPUT_TAG = "MAIN_OUTPUT_TAG";
+
   /**
    * Default constructor.
    */
@@ -66,186 +59,66 @@ public final class SkewReshapingPass extends ReshapingPass {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
-    int mcCount = 0;
-    // destination vertex ID to metric aggregation vertex - ID pair map
-    final Map<String, Pair<OperatorVertex, Integer>> dstVtxIdToABV = new HashMap<>();
-    final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
-
-    for (final IRVertex v : dag.getTopologicalSort()) {
-      // We care about OperatorVertices that have shuffle incoming edges with main output.
-      // TODO #210: Data-aware dynamic optimization at run-time
-      if (v instanceof OperatorVertex && dag.getIncomingEdgesOf(v).stream().anyMatch(irEdge ->
-          CommunicationPatternProperty.Value.Shuffle
-          .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))
-        && dag.getIncomingEdgesOf(v).stream().noneMatch(irEdge ->
-      irEdge.getPropertyValue(AdditionalOutputTagProperty.class).isPresent())) {
-
-        for (final IREdge edge : dag.getIncomingEdgesOf(v)) {
-          if (CommunicationPatternProperty.Value.Shuffle
-            .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
-            final String dstId = edge.getDst().getId();
-
-            // Get or generate a metric collection vertex.
-            final int metricCollectionId;
-            final OperatorVertex abv;
-            if (!dstVtxIdToABV.containsKey(dstId)) {
-              // There is no metric aggregation vertex for this destination vertex.
-              metricCollectionId = mcCount++;
-              abv = generateMetricAggregationVertex();
-              builder.addVertex(abv);
-
-              abv.setPropertyPermanently(ResourceSlotProperty.of(false));
-              dstVtxIdToABV.put(dstId, Pair.of(abv, metricCollectionId));
-            } else {
-              // There is a metric aggregation vertex for this destination vertex already.
-              final Pair<OperatorVertex, Integer> aggrPair = dstVtxIdToABV.get(dstId);
-              metricCollectionId = aggrPair.right();
-              abv = aggrPair.left();
-            }
-
-            final OperatorVertex mcv = generateMetricCollectVertex(edge);
-            builder.addVertex(v);
-            builder.addVertex(mcv);
-
-            // We then insert the vertex with MetricCollectTransform and vertex with AggregateMetricTransform
-            // between the vertex and incoming vertices.
-            final IREdge edgeToMCV = generateEdgeToMCV(edge, mcv);
-            final IREdge edgeToABV = generateEdgeToABV(edge, mcv, abv);
-            final IREdge edgeToOriginalDstV =
-              new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(), edge.getSrc(), v);
-            edge.copyExecutionPropertiesTo(edgeToOriginalDstV);
-            edgeToOriginalDstV.setPropertyPermanently(MetricCollectionProperty.of(metricCollectionId));
-            edgeToABV.setPropertyPermanently(MetricCollectionProperty.of(metricCollectionId));
-
-            builder.connectVertices(edgeToMCV);
-            builder.connectVertices(edgeToABV);
-            builder.connectVertices(edgeToOriginalDstV);
+  public IRDAG apply(final IRDAG dag) {
+    // TODO #210: Data-aware dynamic optimization at run-time
+    dag.topologicalDo(v -> {
 
-            // Add an control dependency (no output)
-            final IREdge emptyEdge =
-              new IREdge(CommunicationPatternProperty.Value.BroadCast, abv, v);
-            builder.connectVertices(emptyEdge);
-          } else {
-            builder.connectVertices(edge);
-          }
-        }
-      } else { // Others are simply added to the builder, unless it comes from an updated vertex
-        builder.addVertex(v);
-        dag.getIncomingEdgesOf(v).forEach(builder::connectVertices);
-      }
-    }
-    final DAG<IRVertex, IREdge> newDAG = builder.build();
-    return newDAG;
-  }
-
-  /**
-   * @return the generated vertex.
-   */
-  private OperatorVertex generateMetricAggregationVertex() {
-    // Define a custom data aggregator for skew handling.
-    // Here, the aggregator gathers key frequency data used in shuffle data repartitioning.
-    final BiFunction<Object, Map<Object, Long>, Map<Object, Long>> dynOptDataAggregator =
-      (BiFunction<Object, Map<Object, Long>, Map<Object, Long>> & Serializable)
-      (element, aggregatedDynOptData) -> {
-        final Object key = ((Pair<Object, Long>) element).left();
-        final Long count = ((Pair<Object, Long>) element).right();
-
-        final Map<Object, Long> aggregatedDynOptDataMap = (Map<Object, Long>) aggregatedDynOptData;
-        if (aggregatedDynOptDataMap.containsKey(key)) {
-          aggregatedDynOptDataMap.compute(key, (existingKey, accumulatedCount) -> accumulatedCount + count);
-        } else {
-          aggregatedDynOptDataMap.put(key, count);
-        }
-        return aggregatedDynOptData;
+      // Incoming shuffle edges grouped by the AdditionalOutputTagProperty.
+      final Function<IREdge, String> groupingFunction = irEdge -> {
+        return irEdge.getPropertyValue(AdditionalOutputTagProperty.class).orElse(MAIN_OUTPUT_TAG);
       };
-    final AggregateMetricTransform abt =
-      new AggregateMetricTransform<Pair<Object, Long>, Map<Object, Long>>(new HashMap<>(), dynOptDataAggregator);
-    return new OperatorVertex(abt);
-  }
-
-  /**
-   * @param edge to collect the metric.
-   * @return the generated vertex.
-   */
-  private OperatorVertex generateMetricCollectVertex(final IREdge edge) {
-    final KeyExtractor keyExtractor = edge.getPropertyValue(KeyExtractorProperty.class).get();
-    // Define a custom data collector for skew handling.
-    // Here, the collector gathers key frequency data used in shuffle data repartitioning.
-    final BiFunction<Object, Map<Object, Object>, Map<Object, Object>> dynOptDataCollector =
-      (BiFunction<Object, Map<Object, Object>, Map<Object, Object>> & Serializable)
-        (element, dynOptData) -> {
-          Object key = keyExtractor.extractKey(element);
-          if (dynOptData.containsKey(key)) {
-            dynOptData.compute(key, (existingKey, existingCount) -> (long) existingCount + 1L);
-          } else {
-            dynOptData.put(key, 1L);
-          }
-          return dynOptData;
-        };
-
-    // Define a custom transform closer for skew handling.
-    // Here, we emit key to frequency data map type data when closing transform.
-    final BiFunction<Map<Object, Object>, OutputCollector, Map<Object, Object>> closer =
-      (BiFunction<Map<Object, Object>, OutputCollector, Map<Object, Object>> & Serializable)
-        (dynOptData, outputCollector)-> {
-          dynOptData.forEach((k, v) -> {
-            final Pair<Object, Object> pairData = Pair.of(k, v);
-            outputCollector.emit(ADDITIONAL_OUTPUT_TAG, pairData);
-          });
-          return dynOptData;
-        };
-
-    final MetricCollectTransform mct
-      = new MetricCollectTransform(new HashMap<>(), dynOptDataCollector, closer);
-    return new OperatorVertex(mct);
-  }
-
-  /**
-   * @param edge the original shuffle edge.
-   * @param mcv the vertex with MetricCollectTransform.
-   * @return the generated edge to {@code mcv}.
-   */
-  private IREdge generateEdgeToMCV(final IREdge edge, final OperatorVertex mcv) {
-    final IREdge newEdge =
-      new IREdge(CommunicationPatternProperty.Value.OneToOne, edge.getSrc(), mcv);
-    newEdge.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
-    newEdge.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
-    return newEdge;
-  }
-
-  /**
-   * @param edge the original shuffle edge.
-   * @param mcv the vertex with MetricCollectTransform.
-   * @param abv the vertex with AggregateMetricTransform.
-   * @return the generated egde from {@code mcv} to {@code abv}.
-   */
-  private IREdge generateEdgeToABV(final IREdge edge,
-                                   final OperatorVertex mcv,
-                                   final OperatorVertex abv) {
-    final IREdge newEdge = new IREdge(CommunicationPatternProperty.Value.Shuffle, mcv, abv);
-    newEdge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
-    newEdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Keep));
-    newEdge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
-    newEdge.setProperty(KeyExtractorProperty.of(new PairKeyExtractor()));
-    newEdge.setProperty(AdditionalOutputTagProperty.of(ADDITIONAL_OUTPUT_TAG));
-
-    // Dynamic optimization handles statistics on key-value data by default.
-    // We need to get coders for encoding/decoding the keys to send data to
-    // vertex with AggregateMetricTransform.
-    if (edge.getPropertyValue(KeyEncoderProperty.class).isPresent()
-      && edge.getPropertyValue(KeyDecoderProperty.class).isPresent()) {
-      final EncoderFactory keyEncoderFactory = edge.getPropertyValue(KeyEncoderProperty.class).get();
-      final DecoderFactory keyDecoderFactory = edge.getPropertyValue(KeyDecoderProperty.class).get();
-      newEdge.setPropertyPermanently(
-        EncoderProperty.of(PairEncoderFactory.of(keyEncoderFactory, LongEncoderFactory.of())));
-      newEdge.setPropertyPermanently(
-        DecoderProperty.of(PairDecoderFactory.of(keyDecoderFactory, LongDecoderFactory.of())));
-    } else {
-      // If not specified, follow encoder/decoder of the given shuffle edge.
-      throw new RuntimeException("Skew optimization request for none key - value format data!");
-    }
-
-    return newEdge;
+      final Map<String, Set<IREdge>> shuffleEdgesGroupedByTag = dag.getIncomingEdgesOf(v).stream()
+        .filter(e -> CommunicationPatternProperty.Value.Shuffle
+          .equals(e.getPropertyValue(CommunicationPatternProperty.class).get()))
+        .collect(Collectors.groupingBy(groupingFunction, Collectors.toSet()));
+
+      // For each shuffle edge group...
+      for (final Set<IREdge> shuffleEdgeGroup : shuffleEdgesGroupedByTag.values()) {
+        final IREdge representativeEdge = shuffleEdgeGroup.iterator().next();
+
+        // Get the key extractor
+        final KeyExtractor keyExtractor = representativeEdge.getPropertyValue(KeyExtractorProperty.class).get();
+
+        // For collecting the data
+        final BiFunction<Object, Map<Object, Long>, Map<Object, Long>> dynOptDataCollector =
+          (BiFunction<Object, Map<Object, Long>, Map<Object, Long>> & Serializable)
+            (element, dynOptData) -> {
+              Object key = keyExtractor.extractKey(element);
+              if (dynOptData.containsKey(key)) {
+                dynOptData.compute(key, (existingKey, existingCount) -> (long) existingCount + 1L);
+              } else {
+                dynOptData.put(key, 1L);
+              }
+              return dynOptData;
+            };
+
+        // For aggregating the collected data
+        final BiFunction<Pair<Object, Long>, Map<Object, Long>, Map<Object, Long>> dynOptDataAggregator =
+          (BiFunction<Pair<Object, Long>, Map<Object, Long>, Map<Object, Long>> & Serializable)
+            (element, aggregatedDynOptData) -> {
+              final Object key = element.left();
+              final Long count = element.right();
+              if (aggregatedDynOptData.containsKey(key)) {
+                aggregatedDynOptData.compute(key, (existingKey, accumulatedCount) -> accumulatedCount + count);
+              } else {
+                aggregatedDynOptData.put(key, count);
+              }
+              return aggregatedDynOptData;
+            };
+
+        // Coders to use
+        final EncoderProperty encoderProperty = EncoderProperty.of(PairEncoderFactory.
+          of(representativeEdge.getPropertyValue(KeyEncoderProperty.class).get(), LongEncoderFactory.of()));
+        final DecoderProperty decoderProperty = DecoderProperty.of(PairDecoderFactory
+          .of(representativeEdge.getPropertyValue(KeyDecoderProperty.class).get(), LongDecoderFactory.of()));
+
+        // Insert the vertices
+        final MessageBarrierVertex mbv = new MessageBarrierVertex<>(dynOptDataCollector);
+        final MessageAggregatorVertex mav = new MessageAggregatorVertex(new HashMap(), dynOptDataAggregator);
+        dag.insert(mbv, mav, encoderProperty, decoderProperty, shuffleEdgeGroup);
+      }
+    });
+    return dag;
   }
 }
+
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPullPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPullPolicy.java
index bac43c9..9f49c26 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPullPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPullPolicy.java
@@ -18,12 +18,10 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AggressiveSpeculativeCloningPass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
 import org.apache.reef.tang.Injector;
 
 /**
@@ -45,7 +43,7 @@ public final class BasicPullPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPushPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPushPolicy.java
index 9daa907..c843176 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPushPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/BasicPushPolicy.java
@@ -18,13 +18,11 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AggressiveSpeculativeCloningPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.ShuffleEdgePushPass;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AggressiveSpeculativeCloningPass;
 import org.apache.reef.tang.Injector;
 
 /**
@@ -47,7 +45,7 @@ public final class BasicPushPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/ConditionalLargeShufflePolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/ConditionalLargeShufflePolicy.java
index 53b3b7a..5179a26 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/ConditionalLargeShufflePolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/ConditionalLargeShufflePolicy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LargeShuffleCompositePass;
@@ -51,14 +49,14 @@ public final class ConditionalLargeShufflePolicy implements Policy {
    * @param dag dag to observe.
    * @return the maximum parallelism, or 1 by default.
    */
-  private static int getMaxParallelism(final DAG<IRVertex, IREdge> dag) {
+  private static int getMaxParallelism(final IRDAG dag) {
     return dag.getVertices().stream()
         .mapToInt(vertex -> vertex.getPropertyValue(ParallelismProperty.class).orElse(1))
         .max().orElse(1);
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DataSkewPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DataSkewPolicy.java
index abb0bcf..cbd584e 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DataSkewPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DataSkewPolicy.java
@@ -18,13 +18,11 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.SkewCompositePass;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.SkewCompositePass;
 import org.apache.nemo.runtime.common.optimizer.pass.runtime.DataSkewRuntimePass;
 import org.apache.reef.tang.Injector;
 
@@ -48,7 +46,7 @@ public final class DataSkewPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicy.java
index 6ac8af5..4bfbf35 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
 import org.apache.reef.tang.Injector;
 
@@ -42,7 +40,7 @@ public final class DefaultPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
index 448229c..fb8e01e 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
@@ -18,12 +18,10 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultDataStorePass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.CompositePass;
 import org.apache.reef.tang.Injector;
@@ -50,7 +48,7 @@ public final class DefaultPolicyWithSeparatePass implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DisaggregationPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
index c7cc866..38da050 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
@@ -18,11 +18,9 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.*;
+import org.apache.nemo.common.ir.IRDAG;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DisaggregationEdgeDataStorePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
 import org.apache.reef.tang.Injector;
@@ -46,7 +44,7 @@ public final class DisaggregationPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/LargeShufflePolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/LargeShufflePolicy.java
index 6a88994..cb309d8 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/LargeShufflePolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/LargeShufflePolicy.java
@@ -18,13 +18,11 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
-import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LargeShuffleCompositePass;
+import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
 import org.apache.reef.tang.Injector;
 
 /**
@@ -46,7 +44,7 @@ public final class LargeShufflePolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/Policy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/Policy.java
index 8d1a438..8a9bebc 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/Policy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/Policy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.reef.tang.Injector;
 
 import java.io.Serializable;
@@ -39,7 +37,7 @@ public interface Policy extends Serializable {
    * @param dagDirectory directory to save the DAG information.
    * @return optimized DAG, reshaped or tagged with execution properties.
    */
-  DAG<IRVertex, IREdge> runCompileTimeOptimization(DAG<IRVertex, IREdge> dag, String dagDirectory);
+  IRDAG runCompileTimeOptimization(IRDAG dag, String dagDirectory);
 
   /**
    * Register runtime optimizations to the event handler.
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyBuilder.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyBuilder.java
index 770cf73..a011ba1 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyBuilder.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyBuilder.java
@@ -18,24 +18,25 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.exception.CompileTimeOptimizationException;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.PartitionerProperty;
 import org.apache.nemo.common.ir.executionproperty.ExecutionProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
 import org.apache.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
+import org.apache.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AnnotatingPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.CompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping.ReshapingPass;
 import org.apache.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.function.Predicate;
 
 /**
@@ -103,7 +104,7 @@ public final class PolicyBuilder {
    * @return the PolicyBuilder which registers the compileTimePass.
    */
   public PolicyBuilder registerCompileTimePass(final CompileTimePass compileTimePass,
-                                               final Predicate<DAG<IRVertex, IREdge>> condition) {
+                                               final Predicate<IRDAG> condition) {
     compileTimePass.addCondition(condition);
     return this.registerCompileTimePass(compileTimePass);
   }
@@ -130,7 +131,7 @@ public final class PolicyBuilder {
    */
   public PolicyBuilder registerRuntimePass(final RuntimePass<?> runtimePass,
                                            final CompileTimePass runtimePassRegisterer,
-                                           final Predicate<DAG<IRVertex, IREdge>> condition) {
+                                           final Predicate<IRDAG> condition) {
     runtimePass.addCondition(condition);
     return this.registerRuntimePass(runtimePass, runtimePassRegisterer);
   }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyImpl.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyImpl.java
index 102dccb..c72ffa5 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyImpl.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/PolicyImpl.java
@@ -18,10 +18,10 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
 import org.apache.nemo.common.eventhandler.RuntimeEventHandler;
 import org.apache.nemo.common.exception.CompileTimeOptimizationException;
+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.compiler.optimizer.pass.compiletime.CompileTimePass;
@@ -54,7 +54,7 @@ public final class PolicyImpl implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     LOG.info("Launch Compile-time optimizations");
     return process(dag, compileTimePasses.iterator(), dagDirectory);
   }
@@ -67,27 +67,33 @@ public final class PolicyImpl implements Policy {
    * @return the processed DAG.
    * @throws Exception Exceptions on the way.
    */
-  private static DAG<IRVertex, IREdge> process(final DAG<IRVertex, IREdge> dag,
-                                               final Iterator<CompileTimePass> passes,
-                                               final String dagDirectory) {
+  private static IRDAG process(final IRDAG dag,
+                               final Iterator<CompileTimePass> passes,
+                               final String dagDirectory) {
     if (passes.hasNext()) {
       final CompileTimePass passToApply = passes.next();
-      final DAG<IRVertex, IREdge> processedDAG;
+      final IRDAG processedDAG;
 
       if (passToApply.getCondition().test(dag)) {
         LOG.info("Apply {} to the DAG", passToApply.getClass().getSimpleName());
         // Apply the pass to the DAG.
         processedDAG = passToApply.apply(dag);
-        // Ensure AnnotatingPass and ReshapingPass functions as intended.
-        if ((passToApply instanceof AnnotatingPass && !checkAnnotatingPass(dag, processedDAG))
-            || (passToApply instanceof ReshapingPass && !checkReshapingPass(dag, processedDAG))) {
+
+        final boolean advanced = processedDAG.advanceDAGSnapshot((beforePass, afterPass) -> {
+          // Ensure AnnotatingPass and ReshapingPass functions as intended.
+          return !((passToApply instanceof AnnotatingPass && !checkAnnotatingPass(beforePass, afterPass))
+            || (passToApply instanceof ReshapingPass && !checkReshapingPass(beforePass, afterPass)));
+        });
+
+        if (!advanced) {
           throw new CompileTimeOptimizationException(passToApply.getClass().getSimpleName()
-              + " is implemented in a way that doesn't follow its original intention of annotating or reshaping. "
-              + "Modify it or use a general CompileTimePass");
+            + " is implemented in a way that doesn't follow its original intention of annotating or reshaping. "
+            + "Modify it or use a general CompileTimePass");
         }
+
         // Save the processed JSON DAG.
         processedDAG.storeJSON(dagDirectory, "ir-after-" + passToApply.getClass().getSimpleName(),
-            "DAG after optimization");
+          "DAG after optimization");
       } else {
         LOG.info("Condition unmet for applying {} to the DAG", passToApply.getClass().getSimpleName());
         processedDAG = dag;
@@ -106,7 +112,7 @@ public final class PolicyImpl implements Policy {
    * @param after DAG after modification.
    * @return true if there is no problem, false if there is a problem.
    */
-  private static Boolean checkAnnotatingPass(final DAG<IRVertex, IREdge> before, final DAG<IRVertex, IREdge> after) {
+  private static Boolean checkAnnotatingPass(final IRDAG before, final IRDAG after) {
     final Iterator<IRVertex> beforeVertices = before.getTopologicalSort().iterator();
     final Iterator<IRVertex> afterVertices = after.getTopologicalSort().iterator();
     while (beforeVertices.hasNext() && afterVertices.hasNext()) {
@@ -134,7 +140,7 @@ public final class PolicyImpl implements Policy {
       }
       // number of edges should match.
       if (beforeVertexIncomingEdges.hasNext() || afterVertexIncomingEdges.hasNext()
-          || beforeVertexOutgoingEdges.hasNext() || afterVertexOutgoingEdges.hasNext()) {
+        || beforeVertexOutgoingEdges.hasNext() || afterVertexOutgoingEdges.hasNext()) {
         return false;
       }
     }
@@ -149,7 +155,7 @@ public final class PolicyImpl implements Policy {
    * @param after DAG after modification.
    * @return true if there is no problem, false if there is a problem.
    */
-  private static Boolean checkReshapingPass(final DAG<IRVertex, IREdge> before, final DAG<IRVertex, IREdge> after) {
+  private static Boolean checkReshapingPass(final IRDAG before, final IRDAG after) {
     final List<IRVertex> previousVertices = before.getVertices();
     for (final IRVertex irVertex : after.getVertices()) {
       final Integer indexOfVertex = previousVertices.indexOf(irVertex);
@@ -176,14 +182,14 @@ public final class PolicyImpl implements Policy {
   public void registerRunTimeOptimizations(final Injector injector, final PubSubEventHandlerWrapper pubSubWrapper) {
     LOG.info("Register run-time optimizations to the PubSubHandler");
     runtimePasses.forEach(runtimePass ->
-        runtimePass.getEventHandlerClasses().forEach(runtimeEventHandlerClass -> {
-          try {
-            final RuntimeEventHandler runtimeEventHandler = injector.getInstance(runtimeEventHandlerClass);
-            pubSubWrapper.getPubSubEventHandler()
-                .subscribe(runtimeEventHandler.getEventClass(), runtimeEventHandler);
-          } catch (final Exception e) {
-            throw new RuntimeException(e);
-          }
-        }));
+      runtimePass.getEventHandlerClasses().forEach(runtimeEventHandlerClass -> {
+        try {
+          final RuntimeEventHandler runtimeEventHandler = injector.getInstance(runtimeEventHandlerClass);
+          pubSubWrapper.getPubSubEventHandler()
+            .subscribe(runtimeEventHandler.getEventClass(), runtimeEventHandler);
+        } catch (final Exception e) {
+          throw new RuntimeException(e);
+        }
+      }));
   }
 }
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/StreamingPolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/StreamingPolicy.java
index 9bdf2d4..f99d363 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/StreamingPolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/StreamingPolicy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.PipeTransferForAllEdgesPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
 import org.apache.reef.tang.Injector;
@@ -43,7 +41,7 @@ public final class StreamingPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/TransientResourcePolicy.java b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/TransientResourcePolicy.java
index efe2995..eac876f 100644
--- a/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/TransientResourcePolicy.java
+++ b/compiler/optimizer/src/main/java/org/apache/nemo/compiler/optimizer/policy/TransientResourcePolicy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.composite.TransientResourceCompositePass;
@@ -46,7 +44,7 @@ public final class TransientResourcePolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/optimizer/src/test/java/org/apache/nemo/compiler/optimizer/policy/PolicyImplTest.java b/compiler/optimizer/src/test/java/org/apache/nemo/compiler/optimizer/policy/PolicyImplTest.java
index 8e34ff7..48bc896 100644
--- a/compiler/optimizer/src/test/java/org/apache/nemo/compiler/optimizer/policy/PolicyImplTest.java
+++ b/compiler/optimizer/src/test/java/org/apache/nemo/compiler/optimizer/policy/PolicyImplTest.java
@@ -20,9 +20,7 @@ package org.apache.nemo.compiler.optimizer.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.exception.CompileTimeOptimizationException;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
-import org.apache.nemo.common.ir.vertex.OperatorVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.test.EmptyComponents;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
 import org.apache.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
@@ -35,8 +33,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 public final class PolicyImplTest {
-  private DAG dag;
-  private DAG dagForSkew;
+  private IRDAG dag;
+  private IRDAG dagForSkew;
 
   @Before
   public void setUp() {
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 d948138..1ed114a 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
@@ -18,12 +18,10 @@
  */
 package org.apache.nemo.compiler;
 
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.test.ArgBuilder;
 import org.apache.nemo.conf.JobConf;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.client.JobLauncher;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.Injector;
 import org.apache.reef.tang.Tang;
@@ -68,7 +66,7 @@ public final class CompilerTestUtil {
     return folder.getAbsolutePath();
   }
 
-  private static DAG<IRVertex, IREdge> compileDAG(final String[] args) throws Exception {
+  private static IRDAG compileDAG(final String[] args) throws Exception {
     final String userMainClassName;
     final String[] userMainMethodArgs;
 
@@ -85,7 +83,7 @@ public final class CompilerTestUtil {
     final Class userMainClass = Class.forName(userMainClassName);
     final Method userMainMethod = userMainClass.getMethod("main", String[].class);
 
-    final ArgumentCaptor<DAG> captor = ArgumentCaptor.forClass(DAG.class);
+    final ArgumentCaptor<IRDAG> captor = ArgumentCaptor.forClass(IRDAG.class);
     final ArgumentCaptor<String> stringArg = ArgumentCaptor.forClass(String.class);
     PowerMockito.mockStatic(JobLauncher.class);
     PowerMockito.doNothing().when(JobLauncher.class, "launchDAG", captor.capture(), stringArg.capture());
@@ -93,7 +91,7 @@ public final class CompilerTestUtil {
     return captor.getValue();
   }
 
-  public static DAG<IRVertex, IREdge> compileWordCountDAG() throws Exception {
+  public static IRDAG compileWordCountDAG() 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";
@@ -105,7 +103,7 @@ public final class CompilerTestUtil {
     return compileDAG(mrArgBuilder.build());
   }
 
-  public static DAG<IRVertex, IREdge> compileALSDAG() throws Exception {
+  public static IRDAG compileALSDAG() throws Exception {
     final String input = ROOT_DIR + "/examples/resources/inputs/test_input_als";
     final String numFeatures = "10";
     final String numIteration = "3";
@@ -118,7 +116,7 @@ public final class CompilerTestUtil {
     return compileDAG(alsArgBuilder.build());
   }
 
-  public static DAG<IRVertex, IREdge> compileALSInefficientDAG() throws Exception {
+  public static IRDAG compileALSInefficientDAG() throws Exception {
     final String input = ROOT_DIR + "/examples/resources/inputs/test_input_als";
     final String numFeatures = "10";
     final String numIteration = "3";
@@ -131,7 +129,7 @@ public final class CompilerTestUtil {
     return compileDAG(alsArgBuilder.build());
   }
 
-  public static DAG<IRVertex, IREdge> compileMLRDAG() throws Exception {
+  public static IRDAG compileMLRDAG() throws Exception {
     final String input = ROOT_DIR + "/examples/resources/inputs/test_input_mlr";
     final String numFeatures = "100";
     final String numClasses = "5";
diff --git a/compiler/test/src/main/java/org/apache/nemo/compiler/optimizer/policy/TestPolicy.java b/compiler/test/src/main/java/org/apache/nemo/compiler/optimizer/policy/TestPolicy.java
index 39d5a70..ca95fb7 100644
--- a/compiler/test/src/main/java/org/apache/nemo/compiler/optimizer/policy/TestPolicy.java
+++ b/compiler/test/src/main/java/org/apache/nemo/compiler/optimizer/policy/TestPolicy.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.compiler.optimizer.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.*;
 import org.apache.reef.tang.Injector;
@@ -51,7 +49,7 @@ public final class TestPolicy implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/backend/nemo/DAGConverterTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/backend/nemo/DAGConverterTest.java
index 3d43b1c..d9b0162 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/backend/nemo/DAGConverterTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/backend/nemo/DAGConverterTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.backend.nemo;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
@@ -75,8 +76,8 @@ public final class DAGConverterTest {
     final IREdge e = new IREdge(CommunicationPatternProperty.Value.Shuffle, v1, v2);
     irDAGBuilder.connectVertices(e);
 
-    final DAG<IRVertex, IREdge> irDAG = new TestPolicy().runCompileTimeOptimization(
-        irDAGBuilder.buildWithoutSourceSinkCheck(), DAG.EMPTY_DAG_DIRECTORY);
+    final IRDAG irDAG = new TestPolicy().runCompileTimeOptimization(
+        new IRDAG(irDAGBuilder.buildWithoutSourceSinkCheck()), DAG.EMPTY_DAG_DIRECTORY);
     final DAG<Stage, StageEdge> DAGOfStages = physicalPlanGenerator.stagePartitionIrDAG(irDAG);
     final DAG<Stage, StageEdge> physicalDAG = physicalPlanGenerator.apply(irDAG);
 
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendALSTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendALSTest.java
index 278c83c..5be2bcb 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendALSTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendALSTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.frontend.beam;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+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.compiler.CompilerTestUtil;
@@ -38,7 +39,7 @@ import static org.junit.Assert.assertEquals;
 public final class BeamFrontendALSTest {
   @Test
   public void testALSDAG() throws Exception {
-    final DAG<IRVertex, IREdge> producedDAG = CompilerTestUtil.compileALSDAG();
+    final IRDAG producedDAG = CompilerTestUtil.compileALSDAG();
 
     assertEquals(producedDAG.getTopologicalSort(), producedDAG.getTopologicalSort());
     assertEquals(44, producedDAG.getVertices().size());
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendMLRTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendMLRTest.java
index c6f100d..231d9ea 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendMLRTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/BeamFrontendMLRTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.frontend.beam;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+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.compiler.CompilerTestUtil;
@@ -38,7 +39,7 @@ import static org.junit.Assert.assertEquals;
 public class BeamFrontendMLRTest {
   @Test
   public void testMLRDAG() throws Exception {
-    final DAG<IRVertex, IREdge> producedDAG = CompilerTestUtil.compileMLRDAG();
+    final IRDAG producedDAG = CompilerTestUtil.compileMLRDAG();
 
     assertEquals(producedDAG.getTopologicalSort(), producedDAG.getTopologicalSort());
     assertEquals(39, producedDAG.getVertices().size());
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeCoderPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeCoderPassTest.java
index e88b536..88343af 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeCoderPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeCoderPassTest.java
@@ -22,6 +22,7 @@ import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.coder.DecoderFactory;
 import org.apache.nemo.common.coder.EncoderFactory;
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.EncoderProperty;
@@ -42,7 +43,7 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class DefaultEdgeCoderPassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -63,7 +64,7 @@ public class DefaultEdgeCoderPassTest {
     final IREdge irEdge = compiledDAG.getOutgoingEdgesOf(compiledDAG.getTopologicalSort().get(0)).get(0);
     final EncoderFactory compiledEncoderFactory = irEdge.getPropertyValue(EncoderProperty.class).get();
     final DecoderFactory compiledDecoderFactory = irEdge.getPropertyValue(DecoderProperty.class).get();
-    DAG<IRVertex, IREdge> processedDAG = new DefaultEdgeEncoderPass().apply(compiledDAG);
+    IRDAG processedDAG = new DefaultEdgeEncoderPass().apply(compiledDAG);
     processedDAG = new DefaultEdgeDecoderPass().apply(processedDAG);
 
     // Get the first coder from the processed DAG
@@ -80,7 +81,7 @@ public class DefaultEdgeCoderPassTest {
     final IREdge irEdge = compiledDAG.getOutgoingEdgesOf(compiledDAG.getTopologicalSort().get(0)).get(0);
     irEdge.getExecutionProperties().remove(EncoderProperty.class);
     irEdge.getExecutionProperties().remove(DecoderProperty.class);
-    DAG<IRVertex, IREdge> processedDAG = new DefaultEdgeEncoderPass().apply(compiledDAG);
+    IRDAG processedDAG = new DefaultEdgeEncoderPass().apply(compiledDAG);
     processedDAG = new DefaultEdgeDecoderPass().apply(processedDAG);
 
     // Check whether the pass set the empty coder to our default encoder & decoder.
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPassTest.java
index 1d824ca..1453c98 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPassTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.annotating;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+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.SourceVertex;
@@ -40,7 +41,7 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class DefaultParallelismPassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -55,7 +56,7 @@ public class DefaultParallelismPassTest {
 
   @Test
   public void testParallelismOne() {
-    final DAG<IRVertex, IREdge> processedDAG = new DefaultParallelismPass().apply(compiledDAG);
+    final IRDAG processedDAG = new DefaultParallelismPass().apply(compiledDAG);
 
     processedDAG.getTopologicalSort().forEach(irVertex ->
         assertEquals(1, irVertex.getPropertyValue(ParallelismProperty.class).get().longValue()));
@@ -64,7 +65,7 @@ public class DefaultParallelismPassTest {
   @Test
   public void testParallelismTen() {
     final int desiredSourceParallelism = 10;
-    final DAG<IRVertex, IREdge> processedDAG = new DefaultParallelismPass(desiredSourceParallelism, 2).apply(compiledDAG);
+    final IRDAG processedDAG = new DefaultParallelismPass(desiredSourceParallelism, 2).apply(compiledDAG);
 
     processedDAG.getTopologicalSort().stream()
         .filter(irVertex -> irVertex instanceof SourceVertex)
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
index 3f7050d..b8d2d38 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
@@ -22,6 +22,7 @@ import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.Pair;
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
@@ -59,13 +60,14 @@ public final class DefaultScheduleGroupPassTest {
    */
   @Test
   public void testTopologicalOrdering() throws Exception {
-    final DAG<IRVertex, IREdge> compiledDAG = CompilerTestUtil.compileALSDAG();
-    final DAG<IRVertex, IREdge> processedDAG = new TestPolicy().runCompileTimeOptimization(compiledDAG,
+    final IRDAG compiledDAG = CompilerTestUtil.compileALSDAG();
+    final IRDAG processedDAG = new TestPolicy().runCompileTimeOptimization(compiledDAG,
         DAG.EMPTY_DAG_DIRECTORY);
 
     for (final IRVertex irVertex : processedDAG.getTopologicalSort()) {
       final Integer currentScheduleGroup = irVertex.getPropertyValue(ScheduleGroupProperty.class).get();
-      final Integer largestScheduleGroupOfParent = processedDAG.getParents(irVertex.getId()).stream()
+      final Integer largestScheduleGroupOfParent =
+        processedDAG.getParents(irVertex.getId()).stream()
           .mapToInt(v -> v.getPropertyValue(ScheduleGroupProperty.class).get())
           .max().orElse(0);
       assertTrue(currentScheduleGroup >= largestScheduleGroupOfParent);
@@ -83,7 +85,7 @@ public final class DefaultScheduleGroupPassTest {
    * @param dataFlowModel {@link DataFlowProperty.Value} for the edges
    * @return a {@link Pair} of {@link DAG} and {@link List} of {@link IRVertex}
    */
-  private static Pair<DAG<IRVertex, IREdge>, List<IRVertex>> generateBranchDAG(
+  private static Pair<IRDAG, List<IRVertex>> generateBranchDAG(
       final CommunicationPatternProperty.Value communicationPattern,
       final DataFlowProperty.Value dataFlowModel) {
     final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
@@ -108,7 +110,7 @@ public final class DefaultScheduleGroupPassTest {
       edge.setProperty(DataFlowProperty.of(dataFlowModel));
       dagBuilder.connectVertices(edge);
     }
-    return Pair.of(dagBuilder.buildWithoutSourceSinkCheck(), vertices);
+    return Pair.of(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()), vertices);
   }
 
   /**
@@ -122,7 +124,7 @@ public final class DefaultScheduleGroupPassTest {
    * @param dataFlowModel {@link DataFlowProperty.Value} for the edges
    * @return a {@link Pair} of {@link DAG} and {@link List} of {@link IRVertex}
    */
-  private static Pair<DAG<IRVertex, IREdge>, List<IRVertex>> generateJoinDAG(
+  private static Pair<IRDAG, List<IRVertex>> generateJoinDAG(
       final CommunicationPatternProperty.Value communicationPattern,
       final DataFlowProperty.Value dataFlowModel) {
     final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
@@ -148,7 +150,7 @@ public final class DefaultScheduleGroupPassTest {
       edge.setProperty(DataFlowProperty.of(dataFlowModel));
       dagBuilder.connectVertices(edge);
     }
-    return Pair.of(dagBuilder.buildWithoutSourceSinkCheck(), vertices);
+    return Pair.of(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()), vertices);
   }
 
   /**
@@ -188,7 +190,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testBranch() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateBranchDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
     pass.apply(dag.left());
     dag.right().forEach(v -> assertScheduleGroup(0, v));
@@ -200,7 +202,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testBranchWhenMultipleInEdgeNotAllowed() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, false, false);
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateBranchDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
     pass.apply(dag.left());
     dag.right().subList(0, 4).forEach(v -> assertScheduleGroup(0, v));
@@ -213,7 +215,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testBranchWithPush() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, false, false);
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateBranchDAG(CommunicationPatternProperty.Value.Shuffle, DataFlowProperty.Value.Push);
     pass.apply(dag.left());
     dag.right().forEach(v -> assertScheduleGroup(0, v));
@@ -225,7 +227,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testBranchWithBroadcast() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, true, true);
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateBranchDAG(CommunicationPatternProperty.Value.BroadCast, DataFlowProperty.Value.Pull);
     assertDifferentScheduleGroup(pass.apply(dag.left()).getVertices());
   }
@@ -236,7 +238,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testBranchWithShuffle() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(true, false, true);
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateBranchDAG(CommunicationPatternProperty.Value.Shuffle, DataFlowProperty.Value.Pull);
     assertDifferentScheduleGroup(pass.apply(dag.left()).getVertices());
   }
@@ -247,7 +249,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testJoin() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
     pass.apply(dag.left());
     final int idxForFirstScheduleGroup = getScheduleGroup(dag.right().get(0));
@@ -263,7 +265,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testJoinWithPush() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Push);
     pass.apply(dag.left());
     dag.right().forEach(v -> assertScheduleGroup(0, v));
@@ -275,7 +277,7 @@ public final class DefaultScheduleGroupPassTest {
   @Test
   public void testJoinWithSinglePush() {
     final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
-    final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
+    final Pair<IRDAG, List<IRVertex>> dag
         = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Push);
     dag.left().getOutgoingEdgesOf(dag.right().get(1)).iterator().next()
         .setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
index 8831a61..139bfd4 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
@@ -19,10 +19,8 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
 import org.apache.nemo.client.JobLauncher;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import org.apache.nemo.common.ir.vertex.IRVertex;
 import org.apache.nemo.compiler.CompilerTestUtil;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.DisaggregationEdgeDataStorePass;
@@ -41,7 +39,7 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class DisaggregationPassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -50,7 +48,7 @@ public class DisaggregationPassTest {
 
   @Test
   public void testDisaggregation() throws Exception {
-    final DAG<IRVertex, IREdge> processedDAG =
+    final IRDAG processedDAG =
         new DisaggregationEdgeDataStorePass().apply(
             new DefaultDataStorePass().apply(
                   new DefaultParallelismPass().apply(compiledDAG)));
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java
index c27a676..640874c 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java
@@ -22,6 +22,7 @@ import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.coder.BytesDecoderFactory;
 import org.apache.nemo.common.coder.BytesEncoderFactory;
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.*;
 import org.apache.nemo.common.ir.vertex.IRVertex;
@@ -40,7 +41,7 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LargeShuffleCompositePassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -49,7 +50,7 @@ public class LargeShuffleCompositePassTest {
 
   @Test
   public void testLargeShuffle() {
-    final DAG<IRVertex, IREdge> processedDAG = new LargeShuffleCompositePass().apply(compiledDAG);
+    final IRDAG processedDAG = new LargeShuffleCompositePass().apply(compiledDAG);
 
     processedDAG.getTopologicalSort().forEach(irVertex -> {
       if (processedDAG.getIncomingEdgesOf(irVertex).stream().anyMatch(irEdge ->
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java
index 46937e9..6a87aba 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java
@@ -19,15 +19,13 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
 import org.apache.nemo.client.JobLauncher;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.edge.executionproperty.AdditionalOutputTagProperty;
+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.executionproperty.ExecutionProperty;
 import org.apache.nemo.common.ir.vertex.OperatorVertex;
-import org.apache.nemo.common.ir.vertex.transform.MetricCollectTransform;
-import org.apache.nemo.common.ir.vertex.transform.AggregateMetricTransform;
+import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform;
+import org.apache.nemo.common.ir.vertex.transform.MessageBarrierTransform;
 import org.apache.nemo.compiler.CompilerTestUtil;
 import org.apache.nemo.common.ir.vertex.executionproperty.ResourceSkewedDataProperty;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AnnotatingPass;
@@ -50,7 +48,7 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class SkewCompositePassTest {
-  private DAG<IRVertex, IREdge> mrDAG;
+  private IRDAG mrDAG;
   private static final long NUM_OF_PASSES_IN_DATA_SKEW_PASS = 3;
 
   @Before
@@ -78,33 +76,32 @@ public class SkewCompositePassTest {
 
   /**
    * Test for {@link SkewCompositePass} with MR workload.
-   * It should have inserted vertex with {@link MetricCollectTransform}
-   * and vertex with {@link AggregateMetricTransform}
-   * before each shuffle edge with no additional output tags.
+   * It should have inserted vertex with {@link MessageBarrierTransform}
+   * and vertex with {@link MessageAggregatorTransform} for each shuffle edge.
    * @throws Exception exception on the way.
    */
   @Test
   public void testDataSkewPass() throws Exception {
     mrDAG = CompilerTestUtil.compileWordCountDAG();
     final Integer originalVerticesNum = mrDAG.getVertices().size();
-    final Long numOfShuffleEdgesWithOutAdditionalOutputTag =
+
+    final Long numOfShuffleEdges =
       mrDAG.getVertices().stream().filter(irVertex ->
         mrDAG.getIncomingEdgesOf(irVertex).stream().anyMatch(irEdge ->
           CommunicationPatternProperty.Value.Shuffle
-            .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get())
-            && !irEdge.getPropertyValue(AdditionalOutputTagProperty.class).isPresent()))
+            .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get())))
       .count();
-    final DAG<IRVertex, IREdge> processedDAG = new SkewCompositePass().apply(mrDAG);
-    assertEquals(originalVerticesNum + numOfShuffleEdgesWithOutAdditionalOutputTag * 2,
-      processedDAG.getVertices().size());
+
+    final IRDAG processedDAG = new SkewCompositePass().apply(mrDAG);
+    assertEquals(originalVerticesNum + numOfShuffleEdges * 2, processedDAG.getVertices().size());
 
     processedDAG.filterVertices(v -> v instanceof OperatorVertex
-      && ((OperatorVertex) v).getTransform() instanceof MetricCollectTransform)
+      && ((OperatorVertex) v).getTransform() instanceof MessageBarrierTransform)
       .forEach(metricV -> {
           final List<IRVertex> reducerV = processedDAG.getChildren(metricV.getId());
           reducerV.forEach(rV -> {
             if (rV instanceof OperatorVertex &&
-              !(((OperatorVertex) rV).getTransform() instanceof AggregateMetricTransform)) {
+              !(((OperatorVertex) rV).getTransform() instanceof MessageAggregatorTransform)) {
               assertTrue(rV.getPropertyValue(ResourceSkewedDataProperty.class).get());
             }
           });
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java
index 426976d..1f86296 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.composite;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataStoreProperty;
@@ -42,7 +43,7 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class TransientResourceCompositePassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -51,7 +52,7 @@ public class TransientResourceCompositePassTest {
 
   @Test
   public void testTransientResourcePass() throws Exception {
-    final DAG<IRVertex, IREdge> processedDAG = new TransientResourceCompositePass().apply(compiledDAG);
+    final IRDAG processedDAG = new TransientResourceCompositePass().apply(compiledDAG);
 
     final IRVertex vertexX = processedDAG.getTopologicalSort().get(0);
     assertEquals(ResourcePriorityProperty.TRANSIENT, vertexX.getPropertyValue(ResourcePriorityProperty.class).get());
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
index 07c1a71..83f7787 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
@@ -51,25 +52,25 @@ public class CommonSubexpressionEliminationPassTest {
   private final IRVertex combine2 = new OperatorVertex(new EmptyComponents.EmptyTransform("Combine2"));
   private final IRVertex map22 = new OperatorVertex(new EmptyComponents.EmptyTransform("Map2"));
 
-  private DAG<IRVertex, IREdge> dagNotToOptimize;
-  private DAG<IRVertex, IREdge> dagToOptimize;
+  private IRDAG dagNotToOptimize;
+  private IRDAG dagToOptimize;
 
   @Before
   public void setUp() {
     final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
-    dagNotToOptimize = dagBuilder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine)
+    dagNotToOptimize = new IRDAG(dagBuilder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine)
         .addVertex(map2)
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
-        .build();
-    dagToOptimize = dagBuilder.addVertex(map1clone).addVertex(groupByKey2).addVertex(combine2).addVertex(map22)
+        .build());
+    dagToOptimize = new IRDAG(dagBuilder.addVertex(map1clone).addVertex(groupByKey2).addVertex(combine2).addVertex(map22)
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1clone))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1clone, groupByKey2))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey2, combine2))
         .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine2, map22))
-        .build();
+        .build());
   }
 
   @Test
@@ -77,10 +78,10 @@ public class CommonSubexpressionEliminationPassTest {
     final long originalVerticesNum = dagNotToOptimize.getVertices().size();
     final long optimizedVerticesNum = dagToOptimize.getVertices().size();
 
-    final DAG<IRVertex, IREdge> processedDAG = new CommonSubexpressionEliminationPass().apply(dagToOptimize);
+    final IRDAG processedDAG = new CommonSubexpressionEliminationPass().apply(dagToOptimize);
     assertEquals(optimizedVerticesNum - 1, processedDAG.getVertices().size());
 
-    final DAG<IRVertex, IREdge> notProcessedDAG = new CommonSubexpressionEliminationPass().apply(dagNotToOptimize);
+    final IRDAG notProcessedDAG = new CommonSubexpressionEliminationPass().apply(dagNotToOptimize);
     assertEquals(originalVerticesNum, notProcessedDAG.getVertices().size());
   }
 }
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
index 4af6214..d6b0768 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+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.compiler.CompilerTestUtil;
@@ -37,7 +38,7 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LoopExtractionPassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -46,7 +47,7 @@ public class LoopExtractionPassTest {
 
   @Test
   public void testLoopGrouping() {
-    final DAG<IRVertex, IREdge> processedDAG = new LoopExtractionPass().apply(compiledDAG);
+    final IRDAG processedDAG = new LoopExtractionPass().apply(compiledDAG);
 
     assertEquals(9, processedDAG.getTopologicalSort().size());
   }
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
index ee02bdd..a188ff3 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
@@ -19,8 +19,8 @@
 package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.client.JobLauncher;
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
@@ -45,13 +45,13 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LoopFusionPassTest {
-  private DAG<IRVertex, IREdge> originalALSDAG;
-  private DAG<IRVertex, IREdge> groupedDAG;
-  private DAG<IRVertex, IREdge> dagToBeFused;
-  private DAG<IRVertex, IREdge> dagNotToBeFused;
-  private DAG<IRVertex, IREdge> dagToBePartiallyFused;
-  private DAG<IRVertex, IREdge> dagToBePartiallyFused2;
-  private DAG<IRVertex, IREdge> dagToBePartiallyFused3;
+  private IRDAG originalALSDAG;
+  private IRDAG groupedDAG;
+  private IRDAG dagToBeFused;
+  private IRDAG dagNotToBeFused;
+  private IRDAG dagToBePartiallyFused;
+  private IRDAG dagToBePartiallyFused2;
+  private IRDAG dagToBePartiallyFused3;
 
   @Before
   public void setUp() throws Exception {
@@ -60,16 +60,20 @@ public class LoopFusionPassTest {
 
     originalALSDAG = CompilerTestUtil.compileALSDAG();
     groupedDAG = new LoopExtractionPass().apply(originalALSDAG);
+    groupedDAG.reshapeUnsafely(unsafeDAG -> {
+      unsafeDAG.topologicalDo(v -> {
+        dagToBeFusedBuilder.addVertex(v, unsafeDAG);
+        unsafeDAG.getIncomingEdgesOf(v).forEach(dagToBeFusedBuilder::connectVertices);
 
-    groupedDAG.topologicalDo(v -> {
-      dagToBeFusedBuilder.addVertex(v, groupedDAG);
-      groupedDAG.getIncomingEdgesOf(v).forEach(dagToBeFusedBuilder::connectVertices);
+        dagNotToBeFusedBuilder.addVertex(v, unsafeDAG);
+        unsafeDAG.getIncomingEdgesOf(v).forEach(dagNotToBeFusedBuilder::connectVertices);
+      });
 
-      dagNotToBeFusedBuilder.addVertex(v, groupedDAG);
-      groupedDAG.getIncomingEdgesOf(v).forEach(dagNotToBeFusedBuilder::connectVertices);
+      return unsafeDAG;
     });
+
     final Optional<LoopVertex> loopInDAG = groupedDAG.getTopologicalSort().stream()
-        .filter(irVertex -> irVertex instanceof LoopVertex).map(irVertex -> (LoopVertex) irVertex).findFirst();
+      .filter(irVertex -> irVertex instanceof LoopVertex).map(irVertex -> (LoopVertex) irVertex).findFirst();
     assertTrue(loopInDAG.isPresent());
 
     final IRVertex vertexFollowedByLoop = groupedDAG.getIncomingEdgesOf(loopInDAG.get()).get(0).getSrc();
@@ -77,27 +81,27 @@ public class LoopFusionPassTest {
     // We're going to put this additional loop to the DAG, to test out the LoopFusion.
     final LoopVertex newLoop = loopInDAG.get().getClone();
     addLoopVertexToBuilder(dagToBeFusedBuilder, vertexFollowedByLoop, newLoop);
-    dagToBeFused = dagToBeFusedBuilder.build();
+    dagToBeFused = new IRDAG(dagToBeFusedBuilder.build());
 
     // additional Loop with different condition.
     final LoopVertex newLoopWithDiffCondition = loopInDAG.get().getClone();
     newLoopWithDiffCondition.setTerminationCondition((i) -> i < 100);
     addLoopVertexToBuilder(dagNotToBeFusedBuilder, vertexFollowedByLoop, newLoopWithDiffCondition);
-    dagNotToBeFused = dagNotToBeFusedBuilder.build();
+    dagNotToBeFused = new IRDAG(dagNotToBeFusedBuilder.build());
 
     // partially fused: two and one.
     addLoopVertexToBuilder(dagToBeFusedBuilder, newLoop, newLoopWithDiffCondition);
-    dagToBePartiallyFused = dagToBeFusedBuilder.build();
+    dagToBePartiallyFused = new IRDAG(dagToBeFusedBuilder.build());
 
     // partially fused2: two and two.
     final LoopVertex newLoopWithDiffDiffCondition = newLoopWithDiffCondition.getClone();
     newLoopWithDiffDiffCondition.setTerminationCondition((i) -> i < 100);
     addLoopVertexToBuilder(dagToBeFusedBuilder, loopInDAG.get(), newLoopWithDiffDiffCondition);
-    dagToBePartiallyFused2 = dagToBeFusedBuilder.build();
+    dagToBePartiallyFused2 = new IRDAG(dagToBeFusedBuilder.build());
 
     // partially fused3: due to dependency - two and two and one.
     addLoopVertexToBuilder(dagToBeFusedBuilder, newLoopWithDiffCondition, newLoopWithDiffCondition.getClone());
-    dagToBePartiallyFused3 = dagToBeFusedBuilder.build();
+    dagToBePartiallyFused3 = new IRDAG(dagToBeFusedBuilder.build());
   }
 
   /**
@@ -115,14 +119,14 @@ public class LoopFusionPassTest {
     builder.addVertex(loopVertexToFollow);
     loopVertexToFollow.getIterativeIncomingEdges().values().forEach(irEdges -> irEdges.forEach(irEdge -> {
       final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
-          vertexToBeFollowed, loopVertexToFollow);
+        vertexToBeFollowed, loopVertexToFollow);
       newIREdge.setProperty(EncoderProperty.of(irEdge.getPropertyValue(EncoderProperty.class).get()));
       newIREdge.setProperty(DecoderProperty.of(irEdge.getPropertyValue(DecoderProperty.class).get()));
       builder.connectVertices(newIREdge);
     }));
     loopVertexToFollow.getNonIterativeIncomingEdges().values().forEach(irEdges -> irEdges.forEach(irEdge -> {
       final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
-          irEdge.getSrc(), loopVertexToFollow);
+        irEdge.getSrc(), loopVertexToFollow);
       newIREdge.setProperty(EncoderProperty.of(irEdge.getPropertyValue(EncoderProperty.class).get()));
       newIREdge.setProperty(DecoderProperty.of(irEdge.getPropertyValue(DecoderProperty.class).get()));
       builder.connectVertices(newIREdge);
@@ -132,38 +136,38 @@ public class LoopFusionPassTest {
   @Test
   public void testLoopFusionPass() throws Exception {
     final long numberOfGroupedVertices = groupedDAG.getVertices().size();
-    final DAG<IRVertex, IREdge> processedDAG = LoopOptimizations.getLoopFusionPass().apply(dagToBeFused);
+    final IRDAG processedDAG = LoopOptimizations.getLoopFusionPass().apply(dagToBeFused);
     assertEquals(numberOfGroupedVertices, processedDAG.getVertices().size());
 
     // no loop
     final long numberOfOriginalVertices = originalALSDAG.getVertices().size();
-    final DAG<IRVertex, IREdge> processedNoLoopDAG =
-        LoopOptimizations.getLoopFusionPass().apply(originalALSDAG);
+    final IRDAG processedNoLoopDAG =
+      LoopOptimizations.getLoopFusionPass().apply(originalALSDAG);
     assertEquals(numberOfOriginalVertices, processedNoLoopDAG.getVertices().size());
 
     // one loop
-    final DAG<IRVertex, IREdge> processedOneLoopDAG = LoopOptimizations.getLoopFusionPass().apply(groupedDAG);
+    final IRDAG processedOneLoopDAG = LoopOptimizations.getLoopFusionPass().apply(groupedDAG);
     assertEquals(numberOfGroupedVertices, processedOneLoopDAG.getVertices().size());
 
     // not to be fused loops
     final long numberOfNotToBeFusedVertices = dagNotToBeFused.getVertices().size();
-    final DAG<IRVertex, IREdge> processedNotToBeFusedDAG =
-        LoopOptimizations.getLoopFusionPass().apply(dagNotToBeFused);
+    final IRDAG processedNotToBeFusedDAG =
+      LoopOptimizations.getLoopFusionPass().apply(dagNotToBeFused);
     assertEquals(numberOfNotToBeFusedVertices, processedNotToBeFusedDAG.getVertices().size());
 
     // to be partially fused loops: two and one
-    final DAG<IRVertex, IREdge> processedToBePartiallyFusedDAG =
-        LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused);
+    final IRDAG processedToBePartiallyFusedDAG =
+      LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused);
     assertEquals(numberOfNotToBeFusedVertices, processedToBePartiallyFusedDAG.getVertices().size());
 
     // to be partially fused loops: two and two
-    final DAG<IRVertex, IREdge> processedToBePartiallyFusedDAG2 =
-        LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused2);
+    final IRDAG processedToBePartiallyFusedDAG2 =
+      LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused2);
     assertEquals(numberOfNotToBeFusedVertices, processedToBePartiallyFusedDAG2.getVertices().size());
 
     // to be partially fused, due to dependency: two and two and one
-    final DAG<IRVertex, IREdge> processedToBePartiallyFusedDAG3 =
-        LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused3);
+    final IRDAG processedToBePartiallyFusedDAG3 =
+      LoopOptimizations.getLoopFusionPass().apply(dagToBePartiallyFused3);
     assertEquals(numberOfNotToBeFusedVertices + 1, processedToBePartiallyFusedDAG3.getVertices().size());
   }
 }
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionALSInefficientTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionALSInefficientTest.java
index d1a66b8..8c953fc 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionALSInefficientTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionALSInefficientTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
+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.compiler.CompilerTestUtil;
@@ -37,8 +38,8 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LoopInvariantCodeMotionALSInefficientTest {
-  private DAG<IRVertex, IREdge> inefficientALSDAG;
-  private DAG<IRVertex, IREdge> groupedDAG;
+  private IRDAG inefficientALSDAG;
+  private IRDAG groupedDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -50,7 +51,7 @@ public class LoopInvariantCodeMotionALSInefficientTest {
   public void testForInefficientALSDAG() throws Exception {
     final long expectedNumOfVertices = groupedDAG.getVertices().size() + 3;
 
-    final DAG<IRVertex, IREdge> processedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
+    final IRDAG processedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
         .apply(groupedDAG);
     assertEquals(expectedNumOfVertices, processedDAG.getVertices().size());
   }
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
index e2f2c0a..db2e7c5 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
@@ -21,6 +21,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DecoderProperty;
@@ -48,9 +49,9 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LoopInvariantCodeMotionPassTest {
-  private DAG<IRVertex, IREdge> originalALSDAG;
-  private DAG<IRVertex, IREdge> groupedDAG;
-  private DAG<IRVertex, IREdge> dagToBeRefactored;
+  private IRDAG originalALSDAG;
+  private IRDAG groupedDAG;
+  private IRDAG dagToBeRefactored;
 
   @Before
   public void setUp() throws Exception {
@@ -102,18 +103,18 @@ public class LoopInvariantCodeMotionPassTest {
       }
     });
 
-    dagToBeRefactored = builder.build();
+    dagToBeRefactored = new IRDAG(builder.build());
   }
 
   @Test
   public void testLoopInvariantCodeMotionPass() throws Exception {
     final long numberOfGroupedVertices = groupedDAG.getVertices().size();
 
-    final DAG<IRVertex, IREdge> processedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
+    final IRDAG processedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
         .apply(dagToBeRefactored);
     assertEquals(numberOfGroupedVertices, processedDAG.getVertices().size());
 
-    final DAG<IRVertex, IREdge> notProcessedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
+    final IRDAG notProcessedDAG = LoopOptimizations.getLoopInvariantCodeMotionPass()
         .apply(groupedDAG);
     assertEquals(numberOfGroupedVertices, notProcessedDAG.getVertices().size());
   }
diff --git a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPassTest.java b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPassTest.java
index bbce319..5c93ce5 100644
--- a/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPassTest.java
+++ b/compiler/test/src/test/java/org/apache/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopUnrollingPassTest.java
@@ -21,6 +21,7 @@ package org.apache.nemo.compiler.optimizer.pass.compiletime.reshaping;
 import org.apache.nemo.client.JobLauncher;
 import org.apache.nemo.common.Pair;
 import org.apache.nemo.common.dag.DAG;
+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.compiler.CompilerTestUtil;
@@ -42,7 +43,7 @@ import static org.junit.Assert.assertEquals;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public class LoopUnrollingPassTest {
-  private DAG<IRVertex, IREdge> compiledDAG;
+  private IRDAG compiledDAG;
 
   @Before
   public void setUp() throws Exception {
@@ -51,7 +52,7 @@ public class LoopUnrollingPassTest {
 
   @Test
   public void testLoopUnrollingPass() throws Exception {
-    final DAG<IRVertex, IREdge> processedDAG =
+    final IRDAG processedDAG =
         new LoopUnrollingPass().apply(new LoopExtractionPass().apply(compiledDAG));
 
     assertEquals(compiledDAG.getTopologicalSort().size(), processedDAG.getTopologicalSort().size());
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/AggressiveSpeculativeCloningPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/AggressiveSpeculativeCloningPolicyParallelismFive.java
index d991808..0e18dfc 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/AggressiveSpeculativeCloningPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/AggressiveSpeculativeCloningPolicyParallelismFive.java
@@ -18,10 +18,8 @@
  */
 package org.apache.nemo.examples.beam.policy;
 
-import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
 import org.apache.nemo.compiler.optimizer.pass.compiletime.annotating.AggressiveSpeculativeCloningPass;
 import org.apache.nemo.compiler.optimizer.policy.DefaultPolicy;
@@ -44,7 +42,7 @@ public final class AggressiveSpeculativeCloningPolicyParallelismFive implements
         DefaultPolicy.BUILDER.getRuntimePasses());
   }
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
   @Override
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DataSkewPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DataSkewPolicyParallelismFive.java
index 86eca52..08317e7 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DataSkewPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DataSkewPolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.DataSkewPolicy;
@@ -40,7 +41,7 @@ public final class DataSkewPolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DefaultPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DefaultPolicyParallelismFive.java
index 7f35676..06178e7 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DefaultPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DefaultPolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.DefaultPolicy;
@@ -40,7 +41,7 @@ public final class DefaultPolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DisaggregationPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DisaggregationPolicyParallelismFive.java
index e8f306f..1823f10 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DisaggregationPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/DisaggregationPolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.DisaggregationPolicy;
@@ -41,7 +42,7 @@ public final class DisaggregationPolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java
index a0ea6da..fbef26c 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.LargeShufflePolicy;
@@ -40,7 +41,7 @@ public final class LargeShufflePolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/StreamingPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/StreamingPolicyParallelismFive.java
index ee59ea3..116a679 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/StreamingPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/StreamingPolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.pass.compiletime.annotating.PipeTransferForAllEdgesPass;
@@ -46,7 +47,7 @@ public final class StreamingPolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java
index 350e25d..8970214 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.PolicyImpl;
@@ -41,7 +42,7 @@ public final class TransientResourcePolicyParallelismFive implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java
index ee73269..1cb9571 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.policy.PolicyImpl;
@@ -41,7 +42,7 @@ public final class TransientResourcePolicyParallelismTen implements Policy {
   }
 
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
 
diff --git a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/UpfrontSchedulingPolicyParallelismFive.java b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/UpfrontSchedulingPolicyParallelismFive.java
index e326a6d..acf6e37 100644
--- a/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/UpfrontSchedulingPolicyParallelismFive.java
+++ b/examples/beam/src/test/java/org/apache/nemo/examples/beam/policy/UpfrontSchedulingPolicyParallelismFive.java
@@ -20,6 +20,7 @@ package org.apache.nemo.examples.beam.policy;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.eventhandler.PubSubEventHandlerWrapper;
+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.compiler.optimizer.pass.compiletime.CompileTimePass;
@@ -43,7 +44,7 @@ public final class UpfrontSchedulingPolicyParallelismFive implements Policy {
         DefaultPolicy.BUILDER.getRuntimePasses());
   }
   @Override
-  public DAG<IRVertex, IREdge> runCompileTimeOptimization(final DAG<IRVertex, IREdge> dag, final String dagDirectory) {
+  public IRDAG runCompileTimeOptimization(final IRDAG dag, final String dagDirectory) {
     return this.policy.runCompileTimeOptimization(dag, dagDirectory);
   }
   @Override
diff --git a/runtime/common/src/main/java/org/apache/nemo/runtime/common/partitioner/DedicatedKeyPerElementPartitioner.java b/runtime/common/src/main/java/org/apache/nemo/runtime/common/partitioner/DedicatedKeyPerElementPartitioner.java
index e9504c2..63fa8f9 100644
--- a/runtime/common/src/main/java/org/apache/nemo/runtime/common/partitioner/DedicatedKeyPerElementPartitioner.java
+++ b/runtime/common/src/main/java/org/apache/nemo/runtime/common/partitioner/DedicatedKeyPerElementPartitioner.java
@@ -22,7 +22,7 @@ package org.apache.nemo.runtime.common.partitioner;
  * An implementation of {@link Partitioner} which assigns a dedicated key per an output data from a task.
  * WARNING: Because this partitioner assigns a dedicated key per element, it should be used under specific circumstances
  * that the number of output element is not that many. For example, every output element of
- * {@link org.apache.nemo.common.ir.vertex.transform.RelayTransform} inserted by large shuffle optimization is always
+ * StreamTransform inserted by large shuffle optimization is always
  * a partition. In this case, assigning a key for each element can be useful.
  */
 @DedicatedKeyPerElement
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 39f4801..d22ed21 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
@@ -18,6 +18,7 @@
  */
 package org.apache.nemo.runtime.common.plan;
 
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.Readable;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupProperty;
@@ -46,7 +47,7 @@ import java.util.function.Function;
 /**
  * A function that converts an IR DAG to physical DAG.
  */
-public final class PhysicalPlanGenerator implements Function<DAG<IRVertex, IREdge>, DAG<Stage, StageEdge>> {
+public final class PhysicalPlanGenerator implements Function<IRDAG, DAG<Stage, StageEdge>> {
   private final String dagDirectory;
   private final StagePartitioner stagePartitioner;
   private static final Logger LOG = LoggerFactory.getLogger(PhysicalPlanGenerator.class.getName());
@@ -71,7 +72,7 @@ public final class PhysicalPlanGenerator implements Function<DAG<IRVertex, IREdg
    * @return {@link PhysicalPlan} to execute.
    */
   @Override
-  public DAG<Stage, StageEdge> apply(final DAG<IRVertex, IREdge> irDAG) {
+  public DAG<Stage, StageEdge> apply(final IRDAG irDAG) {
     // first, stage-partition the IR DAG.
     final DAG<Stage, StageEdge> dagOfStages = stagePartitionIrDAG(irDAG);
 
@@ -131,7 +132,7 @@ public final class PhysicalPlanGenerator implements Function<DAG<IRVertex, IREdg
    * @param irDAG stage-partitioned IR DAG.
    * @return the DAG composed of stages and stage edges.
    */
-  public DAG<Stage, StageEdge> stagePartitionIrDAG(final DAG<IRVertex, IREdge> irDAG) {
+  public DAG<Stage, StageEdge> stagePartitionIrDAG(final IRDAG irDAG) {
     final DAGBuilder<Stage, StageEdge> dagOfStagesBuilder = new DAGBuilder<>();
     final Set<IREdge> interStageEdges = new HashSet<>();
     final Map<Integer, Stage> stageIdToStageMap = new HashMap<>();
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 6007405..a4d1ca5 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
@@ -18,7 +18,7 @@
  */
 package org.apache.nemo.runtime.common.plan;
 
-import org.apache.nemo.common.dag.DAG;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty;
@@ -44,7 +44,7 @@ import java.util.stream.Collectors;
  */
 @DriverSide
 @ThreadSafe
-public final class StagePartitioner implements Function<DAG<IRVertex, IREdge>, Map<IRVertex, Integer>> {
+public final class StagePartitioner implements Function<IRDAG, Map<IRVertex, Integer>> {
   private final Set<Class<? extends VertexExecutionProperty>> ignoredPropertyKeys = ConcurrentHashMap.newKeySet();
   private final MutableInt nextStageIndex = new MutableInt(0);
 
@@ -68,7 +68,7 @@ public final class StagePartitioner implements Function<DAG<IRVertex, IREdge>, M
    * @return a map between IR vertex and the corresponding stage id
    */
   @Override
-  public Map<IRVertex, Integer> apply(final DAG<IRVertex, IREdge> irDAG) {
+  public Map<IRVertex, Integer> apply(final IRDAG irDAG) {
     final Map<IRVertex, Integer> vertexToStageIdMap = new HashMap<>();
     irDAG.topologicalDo(irVertex -> {
       // Base case: for root vertices
@@ -102,7 +102,7 @@ public final class StagePartitioner implements Function<DAG<IRVertex, IREdge>, M
    * @param dag IR DAG which contains {@code edge}
    * @return {@code true} if and only if the source and the destination vertex of the edge can be merged into one stage.
    */
-  private boolean testMergeability(final IREdge edge, final DAG<IRVertex, IREdge> dag) {
+  private boolean testMergeability(final IREdge edge, final IRDAG dag) {
     // If the destination vertex has multiple inEdges, return false
     if (dag.getIncomingEdgesOf(edge.getDst()).size() > 1) {
       return false;
diff --git a/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java b/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
index cc30b30..7e6dcbe 100644
--- a/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
+++ b/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
@@ -20,6 +20,7 @@ package org.apache.nemo.runtime.common.plan;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
@@ -52,12 +53,12 @@ public final class PhysicalPlanGeneratorTest {
 
     final IRVertex v0 = newIRVertex(0, 5);
     final IRVertex v1 = newIRVertex(0, 3);
-    final DAG<IRVertex, IREdge> irDAG = new DAGBuilder<IRVertex, IREdge>()
+    final IRDAG irDAG = new IRDAG(new DAGBuilder<IRVertex, IREdge>()
         .addVertex(v0)
         .addVertex(v1)
         .connectVertices(newIREdge(v0, v1, CommunicationPatternProperty.Value.OneToOne,
             DataFlowProperty.Value.Pull))
-        .buildWithoutSourceSinkCheck();
+        .buildWithoutSourceSinkCheck());
 
     final DAG<Stage, StageEdge> stageDAG = physicalPlanGenerator.apply(irDAG);
     final Iterator<Stage> stages = stageDAG.getVertices().iterator();
diff --git a/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/StagePartitionerTest.java b/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/StagePartitionerTest.java
index 3c0cadb..2b98f86 100644
--- a/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/StagePartitionerTest.java
+++ b/runtime/common/src/test/java/org/apache/nemo/runtime/common/plan/StagePartitionerTest.java
@@ -19,6 +19,7 @@
 package org.apache.nemo.runtime.common.plan;
 
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.executionproperty.VertexExecutionProperty;
@@ -78,7 +79,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertEquals(0, (int) partitioning.get(v0));
     assertEquals(0, (int) partitioning.get(v1));
   }
@@ -94,7 +95,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertNotEquals(partitioning.get(v0), partitioning.get(v1));
   }
 
@@ -109,7 +110,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertNotEquals(partitioning.get(v0), partitioning.get(v1));
   }
 
@@ -124,7 +125,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertNotEquals(partitioning.get(v0), partitioning.get(v1));
   }
 
@@ -140,7 +141,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertNotEquals(partitioning.get(v0), partitioning.get(v1));
   }
 
@@ -156,7 +157,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v0);
     dagBuilder.addVertex(v1);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertEquals(0, (int) partitioning.get(v0));
     assertEquals(0, (int) partitioning.get(v1));
   }
@@ -175,7 +176,7 @@ public final class StagePartitionerTest {
     dagBuilder.addVertex(v2);
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v2));
     dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v1, v2));
-    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
+    final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(new IRDAG(dagBuilder.buildWithoutSourceSinkCheck()));
     assertNotEquals(partitioning.get(v0), partitioning.get(v1));
     assertNotEquals(partitioning.get(v1), partitioning.get(v2));
     assertNotEquals(partitioning.get(v2), partitioning.get(v0));
diff --git a/runtime/driver/src/main/java/org/apache/nemo/driver/UserApplicationRunner.java b/runtime/driver/src/main/java/org/apache/nemo/driver/UserApplicationRunner.java
index 3f7a839..c9f7f9f 100644
--- a/runtime/driver/src/main/java/org/apache/nemo/driver/UserApplicationRunner.java
+++ b/runtime/driver/src/main/java/org/apache/nemo/driver/UserApplicationRunner.java
@@ -19,9 +19,7 @@
 package org.apache.nemo.driver;
 
 import org.apache.nemo.common.Pair;
-import org.apache.nemo.common.dag.DAG;
-import org.apache.nemo.common.ir.edge.IREdge;
-import org.apache.nemo.common.ir.vertex.IRVertex;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.compiler.backend.Backend;
 import org.apache.nemo.compiler.optimizer.Optimizer;
 import org.apache.nemo.conf.JobConf;
@@ -71,8 +69,8 @@ public final class UserApplicationRunner {
     try {
       LOG.info("##### Nemo Compiler Start #####");
 
-      final DAG<IRVertex, IREdge> dag = SerializationUtils.deserialize(Base64.getDecoder().decode(dagString));
-      final DAG<IRVertex, IREdge> optimizedDAG = optimizer.optimizeDag(dag);
+      final IRDAG dag = SerializationUtils.deserialize(Base64.getDecoder().decode(dagString));
+      final IRDAG optimizedDAG = optimizer.optimizeDag(dag);
       final PhysicalPlan physicalPlan = backend.compile(optimizedDAG);
 
       LOG.info("##### Nemo Compiler Finish #####");
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 af31979..50d80c2 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
@@ -26,7 +26,7 @@ import org.apache.nemo.common.ir.OutputCollector;
 import org.apache.nemo.common.ir.Readable;
 import org.apache.nemo.common.ir.edge.executionproperty.AdditionalOutputTagProperty;
 import org.apache.nemo.common.ir.vertex.*;
-import org.apache.nemo.common.ir.vertex.transform.AggregateMetricTransform;
+import org.apache.nemo.common.ir.vertex.transform.MessageAggregatorTransform;
 import org.apache.nemo.common.ir.vertex.transform.Transform;
 import org.apache.nemo.common.punctuation.Watermark;
 import org.apache.nemo.runtime.executor.datatransfer.MultiInputWatermarkManager;
@@ -226,7 +226,7 @@ public final class TaskExecutor {
       final OutputCollector outputCollector;
 
       if (irVertex instanceof OperatorVertex
-        && ((OperatorVertex) irVertex).getTransform() instanceof AggregateMetricTransform) {
+        && ((OperatorVertex) irVertex).getTransform() instanceof MessageAggregatorTransform) {
         outputCollector = new DynOptDataOutputCollector(
           irVertex, persistentConnectionToMasterMap, this);
       } else {
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 919143b..838271c 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
@@ -203,7 +203,7 @@ public final class TaskExecutorTest {
     vertexIdToReadable.put(sourceIRVertex.getId(), readable);
     final List<Watermark> emittedWatermarks = new LinkedList<>();
 
-    final Transform transform = new RelayTransformNoWatermarkEmit(emittedWatermarks);
+    final Transform transform = new StreamTransformNoWatermarkEmit(emittedWatermarks);
     final OperatorVertex operatorVertex = new OperatorVertex(transform);
 
     final DAG<IRVertex, RuntimeEdge<IRVertex>> taskDag =
@@ -240,7 +240,7 @@ public final class TaskExecutorTest {
    */
   @Test(timeout=5000)
   public void testParentTaskDataFetching() throws Exception {
-    final IRVertex vertex = new OperatorVertex(new RelayTransform());
+    final IRVertex vertex = new OperatorVertex(new StreamTransform());
 
     final DAG<IRVertex, RuntimeEdge<IRVertex>> taskDag = new DAGBuilder<IRVertex, RuntimeEdge<IRVertex>>()
         .addVertex(vertex)
@@ -290,9 +290,9 @@ public final class TaskExecutorTest {
   @Test()
   public void testMultipleIncomingEdges() throws Exception {
     final List<Watermark> emittedWatermarks = new ArrayList<>();
-    final IRVertex operatorIRVertex1 = new OperatorVertex(new RelayTransform());
-    final IRVertex operatorIRVertex2 = new OperatorVertex(new RelayTransformNoWatermarkEmit(emittedWatermarks));
-    final IRVertex operatorIRVertex3 = new OperatorVertex(new RelayTransform());
+    final IRVertex operatorIRVertex1 = new OperatorVertex(new StreamTransform());
+    final IRVertex operatorIRVertex2 = new OperatorVertex(new StreamTransformNoWatermarkEmit(emittedWatermarks));
+    final IRVertex operatorIRVertex3 = new OperatorVertex(new StreamTransform());
 
     final IRVertex sourceIRVertex1 = new TestUnboundedSourceVertex();
     final IRVertex sourceIRVertex2 = new TestUnboundedSourceVertex();
@@ -376,8 +376,8 @@ public final class TaskExecutorTest {
    */
   @Test(timeout=5000)
   public void testTwoOperators() throws Exception {
-    final IRVertex operatorIRVertex1 = new OperatorVertex(new RelayTransform());
-    final IRVertex operatorIRVertex2 = new OperatorVertex(new RelayTransform());
+    final IRVertex operatorIRVertex1 = new OperatorVertex(new StreamTransform());
+    final IRVertex operatorIRVertex2 = new OperatorVertex(new StreamTransform());
 
     final String edgeId = "edge";
     final DAG<IRVertex, RuntimeEdge<IRVertex>> taskDag = new DAGBuilder<IRVertex, RuntimeEdge<IRVertex>>()
@@ -409,7 +409,7 @@ public final class TaskExecutorTest {
     final Transform singleListTransform = new CreateSingleListTransform();
 
     final long broadcastId = 0;
-    final IRVertex operatorIRVertex1 = new OperatorVertex(new RelayTransform());
+    final IRVertex operatorIRVertex1 = new OperatorVertex(new StreamTransform());
     final IRVertex operatorIRVertex2 = new OperatorVertex(new BroadcastVariablePairingTransform(broadcastId));
 
     final String edgeId = "edge";
@@ -460,9 +460,9 @@ public final class TaskExecutorTest {
 
     final IRVertex routerVertex = new OperatorVertex(
       new RoutingTransform(Arrays.asList(additionalTag1, additionalTag2)));
-    final IRVertex mainVertex= new OperatorVertex(new RelayTransform());
-    final IRVertex bonusVertex1 = new OperatorVertex(new RelayTransform());
-    final IRVertex bonusVertex2 = new OperatorVertex(new RelayTransform());
+    final IRVertex mainVertex= new OperatorVertex(new StreamTransform());
+    final IRVertex bonusVertex1 = new OperatorVertex(new StreamTransform());
+    final IRVertex bonusVertex2 = new OperatorVertex(new StreamTransform());
 
     final RuntimeEdge<IRVertex> edge1 = createEdge(routerVertex, mainVertex, "edge-1");
     final RuntimeEdge<IRVertex> edge2 = createEdge(routerVertex, bonusVertex1, "edge-2");
@@ -523,7 +523,7 @@ public final class TaskExecutorTest {
     return new StageEdge("SEdge" + RUNTIME_EDGE_ID.getAndIncrement(),
         ExecutionPropertyMap.of(mock(IREdge.class), CommunicationPatternProperty.Value.OneToOne),
         irVertex,
-        new OperatorVertex(new RelayTransform()),
+        new OperatorVertex(new StreamTransform()),
         mock(Stage.class),
         mock(Stage.class));
   }
@@ -533,7 +533,7 @@ public final class TaskExecutorTest {
       ExecutionPropertyMap.of(mock(IREdge.class), CommunicationPatternProperty.Value.OneToOne);
     return new StageEdge("runtime outgoing edge id",
       executionPropertyMap,
-      new OperatorVertex(new RelayTransform()),
+      new OperatorVertex(new StreamTransform()),
       irVertex,
       mock(Stage.class),
       mock(Stage.class));
@@ -591,11 +591,11 @@ public final class TaskExecutorTest {
    * because OutputWriter currently does not support watermarks (TODO #245)
    * @param <T> type
    */
-  private class RelayTransformNoWatermarkEmit<T> implements Transform<T, T> {
+  private class StreamTransformNoWatermarkEmit<T> implements Transform<T, T> {
     private OutputCollector<T> outputCollector;
     private final List<Watermark> emittedWatermarks;
 
-    RelayTransformNoWatermarkEmit(final List<Watermark> emittedWatermarks) {
+    StreamTransformNoWatermarkEmit(final List<Watermark> emittedWatermarks) {
       this.emittedWatermarks = emittedWatermarks;
     }
 
@@ -714,7 +714,7 @@ public final class TaskExecutorTest {
    * Simple identity function for testing.
    * @param <T> input/output type.
    */
-  private class RelayTransform<T> implements Transform<T, T> {
+  private class StreamTransform<T> implements Transform<T, T> {
     private OutputCollector<T> outputCollector;
 
     @Override
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 5605658..08db4b1 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
@@ -441,7 +441,7 @@ public final class BatchScheduler implements Scheduler {
       .findFirst()
       .orElseThrow(() -> new RuntimeException());
 
-    // Stage put on hold, i.e. stage with vertex containing AggregateMetricTransform
+    // Stage put on hold, i.e. stage with vertex containing MessageAggregatorTransform
     // should have a parent stage whose outgoing edges contain the target edge of dynamic optimization.
     final List<StageEdge> edgesToStagePutOnHold = stageDag.getIncomingEdgesOf(stagePutOnHold);
     if (edgesToStagePutOnHold.isEmpty()) {
diff --git a/runtime/test/src/main/java/org/apache/nemo/runtime/common/plan/TestPlanGenerator.java b/runtime/test/src/main/java/org/apache/nemo/runtime/common/plan/TestPlanGenerator.java
index 8cc5f1a..849f32e 100644
--- a/runtime/test/src/main/java/org/apache/nemo/runtime/common/plan/TestPlanGenerator.java
+++ b/runtime/test/src/main/java/org/apache/nemo/runtime/common/plan/TestPlanGenerator.java
@@ -20,6 +20,7 @@ package org.apache.nemo.runtime.common.plan;
 
 import org.apache.nemo.common.dag.DAG;
 import org.apache.nemo.common.dag.DAGBuilder;
+import org.apache.nemo.common.ir.IRDAG;
 import org.apache.nemo.common.ir.edge.IREdge;
 import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
 import org.apache.nemo.common.ir.vertex.IRVertex;
@@ -94,9 +95,9 @@ public final class TestPlanGenerator {
    * @return convert an IR into a physical plan using the given policy.
    * @throws Exception exception.
    */
-  private static PhysicalPlan convertIRToPhysical(final DAG<IRVertex, IREdge> irDAG,
+  private static PhysicalPlan convertIRToPhysical(final IRDAG irDAG,
                                                   final Policy policy) throws Exception {
-    final DAG<IRVertex, IREdge> optimized = policy.runCompileTimeOptimization(irDAG, EMPTY_DAG_DIRECTORY);
+    final IRDAG optimized = policy.runCompileTimeOptimization(irDAG, EMPTY_DAG_DIRECTORY);
     final DAG<Stage, StageEdge> physicalDAG = PLAN_GENERATOR.apply(optimized);
     return new PhysicalPlan("TestPlan", physicalDAG);
   }
@@ -104,7 +105,7 @@ public final class TestPlanGenerator {
   /**
    * @return a dag that joins two vertices.
    */
-  private static DAG<IRVertex, IREdge> getTwoVerticesJoinedDAG() {
+  private static IRDAG getTwoVerticesJoinedDAG() {
     final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
 
     final Transform t = new EmptyComponents.EmptyTransform("empty");
@@ -145,14 +146,14 @@ public final class TestPlanGenerator {
     final IREdge e4 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v4, v5);
     dagBuilder.connectVertices(e4);
 
-    return dagBuilder.buildWithoutSourceSinkCheck();
+    return new IRDAG(dagBuilder.buildWithoutSourceSinkCheck());
   }
 
   /**
    * @param sameContainerType whether all three vertices are of the same container type
    * @return a dag with 3 sequential vertices.
    */
-  private static DAG<IRVertex, IREdge> getThreeSequentialVerticesDAG(final boolean sameContainerType) {
+  private static IRDAG getThreeSequentialVerticesDAG(final boolean sameContainerType) {
     final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
 
     final Transform t = new EmptyComponents.EmptyTransform("empty");
@@ -185,6 +186,6 @@ public final class TestPlanGenerator {
     final IREdge e2 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v2, v3);
     dagBuilder.connectVertices(e2);
 
-    return dagBuilder.buildWithoutSourceSinkCheck();
+    return new IRDAG(dagBuilder.buildWithoutSourceSinkCheck());
   }
 }