You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nemo.apache.org by jo...@apache.org on 2018/07/13 10:55:30 UTC

[incubator-nemo] branch master updated: [NEMO-161] Make all injectable constructors private (#77)

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

johnyangk 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 9c83146  [NEMO-161] Make all injectable constructors private (#77)
9c83146 is described below

commit 9c83146b737dc95fc643903aaa62fdcd6288086a
Author: Jangho Seo <ja...@jangho.io>
AuthorDate: Fri Jul 13 19:55:28 2018 +0900

    [NEMO-161] Make all injectable constructors private (#77)
    
    JIRA: [NEMO-161: Make all injectable constructors private](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-161)
    
    **Major changes:**
    - Made all injectable constructors private (e.g. BatchSingleScheduler, SchedulerRunner, ExecutorRegistry, scheduling constraints, ...)
    - Added LocalMessageDispatcher#getInjector, LocalMessageDispatcher#forkInjector, and LocalMessageEnvironment#forkInjector for ease of writing unit tests
    
    **Minor changes to note:**
    - Made NemoBackend() injectable constructor
    - Added MessageEnvironment#getId method
    
    **Tests for the changes:**
    - Existing tests, modified for private constructors, should continue to work.
    
    **Other comments:**
    - N/A
    
    resolves [NEMO-161](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-161)
---
 .../edu/snu/nemo/client/ClientEndpointTest.java    | 20 ------
 .../edu/snu/nemo/common/test/EmptyComponents.java  |  6 +-
 .../nemo/compiler/backend/nemo/NemoBackend.java    |  5 +-
 .../runtime/common/message/MessageEnvironment.java |  5 ++
 .../message/PersistentConnectionToMasterMap.java   |  2 +-
 .../message/grpc/GrpcMessageEnvironment.java       |  7 ++
 .../message/local/LocalMessageDispatcher.java      | 28 +++++++-
 .../message/local/LocalMessageEnvironment.java     | 32 ++++++++-
 .../common/message/ncs/NcsMessageEnvironment.java  |  1 +
 .../common/message/local/LocalMessageTest.java     | 25 ++++---
 .../edu/snu/nemo/driver/UserApplicationRunner.java |  3 +-
 .../edu/snu/nemo/runtime/executor/Executor.java    | 12 ++--
 .../runtime/executor/data/SerializerManager.java   |  2 +-
 .../nemo/runtime/executor/data/BlockStoreTest.java | 15 ++---
 .../executor/datatransfer/DataTransferTest.java    | 77 ++++++++--------------
 .../edu/snu/nemo/runtime/master/RuntimeMaster.java | 19 +++---
 .../master/scheduler/BatchSingleJobScheduler.java  | 12 ++--
 .../ContainerTypeAwareSchedulingConstraint.java    |  4 +-
 .../runtime/master/scheduler/ExecutorRegistry.java |  2 +-
 .../MinOccupancyFirstSchedulingPolicy.java         |  4 +-
 .../scheduler/PendingTaskCollectionPointer.java    |  2 +-
 .../runtime/master/scheduler/SchedulerRunner.java  | 10 ++-
 .../SourceLocationAwareSchedulingConstraint.java   |  4 +-
 .../runtime/master/BlockManagerMasterTest.java     |  7 +-
 .../nemo/runtime/master/JobStateManagerTest.java   | 13 +---
 .../scheduler/BatchSingleJobSchedulerTest.java     | 23 ++-----
 ...ContainerTypeAwareSchedulingConstraintTest.java |  7 +-
 .../FreeSlotSchedulingConstraintTest.java          |  1 +
 .../MinOccupancyFirstSchedulingPolicyTest.java     |  7 +-
 .../PendingTaskCollectionPointerTest.java          |  7 +-
 ...ourceLocationAwareSchedulingConstraintTest.java | 12 ++--
 .../runtime/master/scheduler/TaskRetryTest.java    | 15 ++---
 .../common/plan/PhysicalPlanGeneratorTest.java     |  4 +-
 .../runtime/common/plan/StagePartitionerTest.java  |  3 +-
 .../snu/nemo/runtime/master/MetricFlushTest.java   | 32 +++------
 .../compiler/backend/nemo/NemoBackendTest.java     |  5 +-
 .../annotating/DefaultScheduleGroupPassTest.java   |  3 +-
 .../CommonSubexpressionEliminationPassTest.java    |  2 +-
 .../reshaping/LoopExtractionPassTest.java          |  2 +-
 39 files changed, 216 insertions(+), 224 deletions(-)

diff --git a/client/src/test/java/edu/snu/nemo/client/ClientEndpointTest.java b/client/src/test/java/edu/snu/nemo/client/ClientEndpointTest.java
index a3ce968..19f41f5 100644
--- a/client/src/test/java/edu/snu/nemo/client/ClientEndpointTest.java
+++ b/client/src/test/java/edu/snu/nemo/client/ClientEndpointTest.java
@@ -15,27 +15,12 @@
  */
 package edu.snu.nemo.client;
 
-import edu.snu.nemo.common.dag.DAG;
-import edu.snu.nemo.common.dag.DAGBuilder;
-import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.test.EmptyComponents;
-import edu.snu.nemo.conf.JobConf;
-import edu.snu.nemo.runtime.common.message.MessageEnvironment;
-import edu.snu.nemo.runtime.common.message.local.LocalMessageDispatcher;
-import edu.snu.nemo.runtime.common.message.local.LocalMessageEnvironment;
 import edu.snu.nemo.runtime.common.plan.PhysicalPlan;
-import edu.snu.nemo.runtime.common.plan.PhysicalPlanGenerator;
-import edu.snu.nemo.runtime.common.plan.Stage;
-import edu.snu.nemo.runtime.common.plan.StageEdge;
 import edu.snu.nemo.runtime.common.state.JobState;
 import edu.snu.nemo.runtime.common.state.TaskState;
 import edu.snu.nemo.runtime.master.MetricMessageHandler;
-import edu.snu.nemo.runtime.master.BlockManagerMaster;
 import edu.snu.nemo.runtime.master.JobStateManager;
 import edu.snu.nemo.runtime.plangenerator.TestPlanGenerator;
-import org.apache.reef.tang.Injector;
-import org.apache.reef.tang.Tang;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -73,11 +58,6 @@ public class ClientEndpointTest {
     // Create a JobStateManager of a dag and create a DriverEndpoint with it.
     final PhysicalPlan physicalPlan =
         TestPlanGenerator.generatePhysicalPlan(TestPlanGenerator.PlanType.TwoVerticesJoined, false);
-    final LocalMessageDispatcher messageDispatcher = new LocalMessageDispatcher();
-    final LocalMessageEnvironment messageEnvironment =
-        new LocalMessageEnvironment(MessageEnvironment.MASTER_COMMUNICATION_ID, messageDispatcher);
-    final Injector injector = Tang.Factory.getTang().newInjector();
-    injector.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
     final JobStateManager jobStateManager =
         new JobStateManager(physicalPlan, metricMessageHandler, MAX_SCHEDULE_ATTEMPT);
 
diff --git a/common/src/main/java/edu/snu/nemo/common/test/EmptyComponents.java b/common/src/main/java/edu/snu/nemo/common/test/EmptyComponents.java
index 5f59bf0..5af5688 100644
--- a/common/src/main/java/edu/snu/nemo/common/test/EmptyComponents.java
+++ b/common/src/main/java/edu/snu/nemo/common/test/EmptyComponents.java
@@ -26,7 +26,11 @@ import java.util.List;
 /**
  * Empty components to mock transform and source, for tests and examples.
  */
-public class EmptyComponents {
+public final class EmptyComponents {
+  public static final Transform EMPTY_TRANSFORM = new EmptyTransform("");
+
+  private EmptyComponents() {
+  }
 
   /**
    * An empty transform.
diff --git a/compiler/backend/src/main/java/edu/snu/nemo/compiler/backend/nemo/NemoBackend.java b/compiler/backend/src/main/java/edu/snu/nemo/compiler/backend/nemo/NemoBackend.java
index 981ee72..465fb77 100644
--- a/compiler/backend/src/main/java/edu/snu/nemo/compiler/backend/nemo/NemoBackend.java
+++ b/compiler/backend/src/main/java/edu/snu/nemo/compiler/backend/nemo/NemoBackend.java
@@ -26,6 +26,8 @@ import edu.snu.nemo.runtime.common.plan.Stage;
 import edu.snu.nemo.runtime.common.plan.StageEdge;
 import org.apache.reef.tang.Tang;
 
+import javax.inject.Inject;
+
 /**
  * Backend component for Nemo Runtime.
  */
@@ -33,7 +35,8 @@ public final class NemoBackend implements Backend<PhysicalPlan> {
   /**
    * Constructor.
    */
-  public NemoBackend() {
+  @Inject
+  private NemoBackend() {
   }
 
   /**
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/MessageEnvironment.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/MessageEnvironment.java
index b7d5dc1..c4c5b7b 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/MessageEnvironment.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/MessageEnvironment.java
@@ -66,4 +66,9 @@ public interface MessageEnvironment {
    * @throws Exception while closing
    */
   void close() throws Exception;
+
+  /**
+   * @return identifier for this node
+   */
+  String getId();
 }
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/PersistentConnectionToMasterMap.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/PersistentConnectionToMasterMap.java
index 8a2b8bc..6d362f5 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/PersistentConnectionToMasterMap.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/PersistentConnectionToMasterMap.java
@@ -31,7 +31,7 @@ public final class PersistentConnectionToMasterMap {
   private final MessageEnvironment messageEnvironment;
 
   @Inject
-  public PersistentConnectionToMasterMap(final MessageEnvironment messageEnvironment) {
+  private PersistentConnectionToMasterMap(final MessageEnvironment messageEnvironment) {
     this.messageEnvironment = messageEnvironment;
     messageSenders = new HashMap<>();
     try {
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/grpc/GrpcMessageEnvironment.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/grpc/GrpcMessageEnvironment.java
index edb1399..8829d2b 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/grpc/GrpcMessageEnvironment.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/grpc/GrpcMessageEnvironment.java
@@ -54,6 +54,7 @@ public final class GrpcMessageEnvironment implements MessageEnvironment {
   private final NameResolver nameResolver;
   private final IdentifierFactory idFactory;
   private final GrpcMessageServer grpcServer;
+  private final String localSenderId;
 
   @Inject
   private GrpcMessageEnvironment(
@@ -64,6 +65,7 @@ public final class GrpcMessageEnvironment implements MessageEnvironment {
     this.nameResolver = nameResolver;
     this.idFactory = idFactory;
     this.grpcServer = new GrpcMessageServer(localAddressProvider, nameResolver, idFactory, localSenderId);
+    this.localSenderId = localSenderId;
 
     try {
       this.grpcServer.start();
@@ -108,4 +110,9 @@ public final class GrpcMessageEnvironment implements MessageEnvironment {
   public void close() throws Exception {
     grpcServer.close();
   }
+
+  @Override
+  public String getId() {
+    return localSenderId;
+  }
 }
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageDispatcher.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageDispatcher.java
index 941f744..2db27bb 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageDispatcher.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageDispatcher.java
@@ -17,7 +17,11 @@ package edu.snu.nemo.runtime.common.message.local;
 
 import edu.snu.nemo.runtime.common.message.MessageListener;
 import edu.snu.nemo.runtime.common.message.MessageSender;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 
+import javax.inject.Inject;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
@@ -27,10 +31,12 @@ import java.util.concurrent.ConcurrentMap;
  * Dispatch messages on a single machine.
  */
 public final class LocalMessageDispatcher {
+  private static final Tang TANG = Tang.Factory.getTang();
 
   private final ConcurrentMap<String, ConcurrentMap<String, MessageListener>> nodeIdToMessageListenersMap;
 
-  public LocalMessageDispatcher() {
+  @Inject
+  private LocalMessageDispatcher() {
     this.nodeIdToMessageListenersMap = new ConcurrentHashMap<>();
   }
 
@@ -94,4 +100,24 @@ public final class LocalMessageDispatcher {
       super(message);
     }
   }
+
+  /**
+   * @return an {@link Injector} which has {@link LocalMessageDispatcher} for testing.
+   * @throws InjectionException when failed to inject {@link LocalMessageDispatcher}
+   */
+  public static Injector getInjector() throws InjectionException {
+    return forkInjector(TANG.newInjector());
+  }
+
+  /**
+   * @param baseInjector base {@link Injector} to extend upon
+   * @return an {@link Injector} which has {@link LocalMessageDispatcher} for testing.
+   * @throws InjectionException when failed to inject {@link LocalMessageDispatcher}
+   */
+  public static Injector forkInjector(final Injector baseInjector) throws InjectionException {
+    final Injector injector = baseInjector
+        .forkInjector(LocalMessageEnvironment.LOCAL_MESSAGE_ENVIRONMENT_CONFIGURATION);
+    injector.getInstance(LocalMessageDispatcher.class);
+    return injector;
+  }
 }
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageEnvironment.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageEnvironment.java
index df3a973..3a138d6 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageEnvironment.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/local/LocalMessageEnvironment.java
@@ -17,8 +17,15 @@ package edu.snu.nemo.runtime.common.message.local;
 
 import edu.snu.nemo.runtime.common.message.MessageEnvironment;
 import edu.snu.nemo.runtime.common.message.MessageListener;
+import edu.snu.nemo.runtime.common.message.MessageParameters;
 import edu.snu.nemo.runtime.common.message.MessageSender;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.exceptions.InjectionException;
 
+import javax.inject.Inject;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Future;
 
@@ -27,12 +34,16 @@ import java.util.concurrent.Future;
  * Used for unit tests.
  */
 public final class LocalMessageEnvironment implements MessageEnvironment {
+  private static final Tang TANG = Tang.Factory.getTang();
+  public static final Configuration LOCAL_MESSAGE_ENVIRONMENT_CONFIGURATION = TANG.newConfigurationBuilder()
+      .bindImplementation(MessageEnvironment.class, LocalMessageEnvironment.class).build();
 
   private final String currentNodeId;
   private final LocalMessageDispatcher dispatcher;
 
-  public LocalMessageEnvironment(final String currentNodeId,
-                                 final LocalMessageDispatcher dispatcher) {
+  @Inject
+  private LocalMessageEnvironment(@Parameter(MessageParameters.SenderId.class) final String currentNodeId,
+                                  final LocalMessageDispatcher dispatcher) {
     this.currentNodeId = currentNodeId;
     this.dispatcher = dispatcher;
   }
@@ -55,6 +66,7 @@ public final class LocalMessageEnvironment implements MessageEnvironment {
         currentNodeId, targetId, messageTypeId, dispatcher));
   }
 
+  @Override
   public String getId() {
     return currentNodeId;
   }
@@ -63,4 +75,20 @@ public final class LocalMessageEnvironment implements MessageEnvironment {
   public void close() {
     // No-ops.
   }
+
+  /**
+   * Extends {@code baseInjector} to have {@link LocalMessageEnvironment} instance for the given {@code senderId}.
+   *
+   * @param baseInjector provided by {@link LocalMessageDispatcher#getInjector()}
+   *                     or {@link LocalMessageDispatcher#forkInjector(Injector)}
+   * @param senderId  the identifier for the sender
+   * @return an {@link Injector} which has {@link LocalMessageDispatcher} instance for {@link MessageEnvironment}
+   * @throws InjectionException when fails to inject {@link MessageEnvironment}
+   */
+  public static Injector forkInjector(final Injector baseInjector, final String senderId) throws InjectionException {
+    final Injector injector = baseInjector.forkInjector(TANG.newConfigurationBuilder()
+        .bindNamedParameter(MessageParameters.SenderId.class, senderId).build());
+    injector.getInstance(MessageEnvironment.class);
+    return injector;
+  }
 }
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/ncs/NcsMessageEnvironment.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/ncs/NcsMessageEnvironment.java
index 676cb15..a5062fe 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/ncs/NcsMessageEnvironment.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/message/ncs/NcsMessageEnvironment.java
@@ -105,6 +105,7 @@ public final class NcsMessageEnvironment implements MessageEnvironment {
     }
   }
 
+  @Override
   public String getId() {
     return senderId;
   }
diff --git a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/message/local/LocalMessageTest.java b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/message/local/LocalMessageTest.java
index a39916b..08d6300 100644
--- a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/message/local/LocalMessageTest.java
+++ b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/message/local/LocalMessageTest.java
@@ -15,10 +15,9 @@
  */
 package edu.snu.nemo.runtime.common.message.local;
 
-import edu.snu.nemo.runtime.common.message.MessageContext;
-import edu.snu.nemo.runtime.common.message.MessageEnvironment;
-import edu.snu.nemo.runtime.common.message.MessageListener;
-import edu.snu.nemo.runtime.common.message.MessageSender;
+import edu.snu.nemo.runtime.common.message.*;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -30,7 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  * Tests local messaging components.
  */
 public class LocalMessageTest {
-  private final LocalMessageDispatcher localMessageDispatcher = new LocalMessageDispatcher();
+  private static final Tang TANG = Tang.Factory.getTang();
 
   @Test
   public void testLocalMessages() throws Exception {
@@ -42,9 +41,19 @@ public class LocalMessageTest {
     final String secondListenerIdToDriver = "SecondToDriver";
     final String listenerIdBetweenExecutors = "BetweenExecutors";
 
-    final MessageEnvironment driverEnv = new LocalMessageEnvironment(driverNodeId, localMessageDispatcher);
-    final MessageEnvironment executorOneEnv = new LocalMessageEnvironment(executorOneNodeId, localMessageDispatcher);
-    final MessageEnvironment executorTwoEnv = new LocalMessageEnvironment(executorTwoNodeId, localMessageDispatcher);
+    final Injector injector = TANG.newInjector(TANG.newConfigurationBuilder()
+        .bindImplementation(MessageEnvironment.class, LocalMessageEnvironment.class).build());
+    injector.getInstance(LocalMessageDispatcher.class);
+
+    final MessageEnvironment driverEnv = injector.forkInjector(TANG.newConfigurationBuilder()
+        .bindNamedParameter(MessageParameters.SenderId.class, driverNodeId).build())
+        .getInstance(MessageEnvironment.class);
+    final MessageEnvironment executorOneEnv = injector.forkInjector(TANG.newConfigurationBuilder()
+        .bindNamedParameter(MessageParameters.SenderId.class, executorOneNodeId).build())
+        .getInstance(MessageEnvironment.class);
+    final MessageEnvironment executorTwoEnv = injector.forkInjector(TANG.newConfigurationBuilder()
+        .bindNamedParameter(MessageParameters.SenderId.class, executorTwoNodeId).build())
+        .getInstance(MessageEnvironment.class);
 
     final AtomicInteger toDriverMessageUsingSend = new AtomicInteger();
 
diff --git a/runtime/driver/src/main/java/edu/snu/nemo/driver/UserApplicationRunner.java b/runtime/driver/src/main/java/edu/snu/nemo/driver/UserApplicationRunner.java
index de2dcb0..4ab57a2 100644
--- a/runtime/driver/src/main/java/edu/snu/nemo/driver/UserApplicationRunner.java
+++ b/runtime/driver/src/main/java/edu/snu/nemo/driver/UserApplicationRunner.java
@@ -59,6 +59,7 @@ public final class UserApplicationRunner {
   private UserApplicationRunner(@Parameter(JobConf.DAGDirectory.class) final String dagDirectory,
                                 @Parameter(JobConf.OptimizationPolicy.class) final String optimizationPolicy,
                                 @Parameter(JobConf.MaxTaskAttempt.class) final int maxScheduleAttempt,
+                                final NemoBackend backend,
                                 final PubSubEventHandlerWrapper pubSubEventHandlerWrapper,
                                 final Injector injector,
                                 final RuntimeMaster runtimeMaster) {
@@ -67,7 +68,7 @@ public final class UserApplicationRunner {
     this.maxScheduleAttempt = maxScheduleAttempt;
     this.injector = injector;
     this.runtimeMaster = runtimeMaster;
-    this.backend = new NemoBackend();
+    this.backend = backend;
     this.pubSubWrapper = pubSubEventHandlerWrapper;
   }
 
diff --git a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/Executor.java b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/Executor.java
index 25f6224..52e153e 100644
--- a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/Executor.java
+++ b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/Executor.java
@@ -74,12 +74,12 @@ public final class Executor {
   private final MetricMessageSender metricMessageSender;
 
   @Inject
-  public Executor(@Parameter(JobConf.ExecutorId.class) final String executorId,
-                  final PersistentConnectionToMasterMap persistentConnectionToMasterMap,
-                  final MessageEnvironment messageEnvironment,
-                  final SerializerManager serializerManager,
-                  final DataTransferFactory dataTransferFactory,
-                  final MetricManagerWorker metricMessageSender) {
+  private Executor(@Parameter(JobConf.ExecutorId.class) final String executorId,
+                   final PersistentConnectionToMasterMap persistentConnectionToMasterMap,
+                   final MessageEnvironment messageEnvironment,
+                   final SerializerManager serializerManager,
+                   final DataTransferFactory dataTransferFactory,
+                   final MetricManagerWorker metricMessageSender) {
     this.executorId = executorId;
     this.executorService = Executors.newCachedThreadPool(new BasicThreadFactory.Builder()
         .namingPattern("TaskExecutor thread-%d")
diff --git a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/SerializerManager.java b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/SerializerManager.java
index 5ea1bec..c415965 100644
--- a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/SerializerManager.java
+++ b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/SerializerManager.java
@@ -40,7 +40,7 @@ public final class SerializerManager {
    * Constructor.
    */
   @Inject
-  public SerializerManager() {
+  private SerializerManager() {
   }
 
   /**
diff --git a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/data/BlockStoreTest.java b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/data/BlockStoreTest.java
index 2af771e..8f1b7e5 100644
--- a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/data/BlockStoreTest.java
+++ b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/data/BlockStoreTest.java
@@ -77,7 +77,7 @@ public final class BlockStoreTest {
       Collections.singletonList(new DecompressionStreamChainer(CompressionProperty.Value.LZ4)));
   private static final SerializerManager serializerManager = mock(SerializerManager.class);
   private BlockManagerMaster blockManagerMaster;
-  private LocalMessageDispatcher messageDispatcher;
+  private Injector baseInjector;
   // Variables for shuffle test
   private static final int NUM_WRITE_VERTICES = 3;
   private static final int NUM_READ_VERTICES = 3;
@@ -104,11 +104,9 @@ public final class BlockStoreTest {
    */
   @Before
   public void setUp() throws Exception {
-    messageDispatcher = new LocalMessageDispatcher();
-    final LocalMessageEnvironment messageEnvironment =
-        new LocalMessageEnvironment(MessageEnvironment.MASTER_COMMUNICATION_ID, messageDispatcher);
-    final Injector injector = Tang.Factory.getTang().newInjector();
-    injector.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
+    baseInjector = LocalMessageDispatcher.getInjector();
+    final Injector injector = LocalMessageEnvironment
+        .forkInjector(baseInjector, MessageEnvironment.MASTER_COMMUNICATION_ID);
     blockManagerMaster = injector.getInstance(BlockManagerMaster.class);
     when(serializerManager.getSerializer(any())).thenReturn(SERIALIZER);
 
@@ -280,14 +278,11 @@ public final class BlockStoreTest {
 
   private GlusterFileStore createGlusterFileStore(final String executorId)
       throws InjectionException {
-    final LocalMessageEnvironment localMessageEnvironment =
-        new LocalMessageEnvironment(executorId, messageDispatcher);
-    final Injector injector = Tang.Factory.getTang().newInjector();
+    final Injector injector = LocalMessageEnvironment.forkInjector(baseInjector, executorId);
     injector.bindVolatileParameter(JobConf.GlusterVolumeDirectory.class, TMP_FILE_DIRECTORY);
     injector.bindVolatileParameter(JobConf.JobId.class, "GFS test");
     injector.bindVolatileParameter(JobConf.ExecutorId.class, executorId);
     injector.bindVolatileInstance(SerializerManager.class, serializerManager);
-    injector.bindVolatileInstance(MessageEnvironment.class, localMessageEnvironment);
     return injector.getInstance(GlusterFileStore.class);
   }
 
diff --git a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
index f801685..66bc001 100644
--- a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
+++ b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
@@ -40,7 +40,6 @@ import edu.snu.nemo.runtime.common.plan.RuntimeEdge;
 import edu.snu.nemo.runtime.common.plan.Stage;
 import edu.snu.nemo.runtime.common.plan.StageEdge;
 import edu.snu.nemo.runtime.executor.Executor;
-import edu.snu.nemo.runtime.executor.MetricManagerWorker;
 import edu.snu.nemo.runtime.executor.data.BlockManagerWorker;
 import edu.snu.nemo.runtime.executor.data.SerializerManager;
 import edu.snu.nemo.runtime.master.*;
@@ -109,7 +108,6 @@ public final class DataTransferTest {
   private static final DecoderFactory DECODER_FACTORY =
       PairDecoderFactory.of(IntDecoderFactory.of(), IntDecoderFactory.of());
   private static final Tang TANG = Tang.Factory.getTang();
-  private static final int HASH_RANGE_MULTIPLIER = 10;
 
   private BlockManagerMaster master;
   private BlockManagerWorker worker1;
@@ -119,51 +117,37 @@ public final class DataTransferTest {
 
   @Before
   public void setUp() throws InjectionException {
-    final LocalMessageDispatcher messageDispatcher = new LocalMessageDispatcher();
-    final LocalMessageEnvironment messageEnvironment =
-        new LocalMessageEnvironment(MessageEnvironment.MASTER_COMMUNICATION_ID, messageDispatcher);
     final Configuration configuration = Tang.Factory.getTang().newConfigurationBuilder()
         .bindNamedParameter(JobConf.ScheduleSerThread.class, "1")
         .build();
-    final Injector injector = Tang.Factory.getTang().newInjector(configuration);
-    injector.bindVolatileInstance(EvaluatorRequestor.class, mock(EvaluatorRequestor.class));
-    injector.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
-    final ContainerManager containerManager = injector.getInstance(ContainerManager.class);
-    final ExecutorRegistry executorRegistry = injector.getInstance(ExecutorRegistry.class);
-
-    final MetricMessageHandler metricMessageHandler = mock(MetricMessageHandler.class);
-    final PubSubEventHandlerWrapper pubSubEventHandler = mock(PubSubEventHandlerWrapper.class);
-    final UpdatePhysicalPlanEventHandler updatePhysicalPlanEventHandler = mock(UpdatePhysicalPlanEventHandler.class);
-    final SchedulingConstraintRegistry schedulingConstraint = injector.getInstance(SchedulingConstraintRegistry.class);
-    final SchedulingPolicy schedulingPolicy = injector.getInstance(SchedulingPolicy.class);
-    final PendingTaskCollectionPointer taskQueue = new PendingTaskCollectionPointer();
-    final SchedulerRunner schedulerRunner = new SchedulerRunner(schedulingConstraint, schedulingPolicy, taskQueue, executorRegistry);
-    final Scheduler scheduler = new BatchSingleJobScheduler(
-        schedulerRunner, taskQueue, master, pubSubEventHandler, updatePhysicalPlanEventHandler, executorRegistry);
+    final Injector baseInjector = Tang.Factory.getTang().newInjector(configuration);
+    baseInjector.bindVolatileInstance(EvaluatorRequestor.class, mock(EvaluatorRequestor.class));
+    final Injector dispatcherInjector = LocalMessageDispatcher.forkInjector(baseInjector);
+    final Injector injector = LocalMessageEnvironment.forkInjector(dispatcherInjector,
+        MessageEnvironment.MASTER_COMMUNICATION_ID);
+
+    injector.bindVolatileInstance(PubSubEventHandlerWrapper.class, mock(PubSubEventHandlerWrapper.class));
+    injector.bindVolatileInstance(UpdatePhysicalPlanEventHandler.class, mock(UpdatePhysicalPlanEventHandler.class));
     final AtomicInteger executorCount = new AtomicInteger(0);
-    final ClientRPC clientRPC = mock(ClientRPC.class);
-    final MetricManagerMaster metricManagerMaster = mock(MetricManagerMaster.class);
+    injector.bindVolatileInstance(ClientRPC.class, mock(ClientRPC.class));
+    injector.bindVolatileInstance(MetricManagerMaster.class, mock(MetricManagerMaster.class));
+    injector.bindVolatileInstance(MetricMessageHandler.class, mock(MetricMessageHandler.class));
+    injector.bindVolatileParameter(JobConf.DAGDirectory.class, EMPTY_DAG_DIRECTORY);
 
     // Necessary for wiring up the message environments
-    final RuntimeMaster runtimeMaster =
-        new RuntimeMaster(scheduler, containerManager, master,
-            metricMessageHandler, messageEnvironment, clientRPC, metricManagerMaster, EMPTY_DAG_DIRECTORY);
-
-    final Injector injector1 = Tang.Factory.getTang().newInjector();
-    injector1.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
-    injector1.bindVolatileInstance(RuntimeMaster.class, runtimeMaster);
-    final BlockManagerMaster master = injector1.getInstance(BlockManagerMaster.class);
+    injector.getInstance(RuntimeMaster.class);
+    final BlockManagerMaster master = injector.getInstance(BlockManagerMaster.class);
 
-    final Injector injector2 = createNameClientInjector();
-    injector2.bindVolatileParameter(JobConf.JobId.class, "data transfer test");
+    final Injector nameClientInjector = createNameClientInjector();
+    nameClientInjector.bindVolatileParameter(JobConf.JobId.class, "data transfer test");
 
     this.master = master;
-    final Pair<BlockManagerWorker, DataTransferFactory> pair1 =
-        createWorker(EXECUTOR_ID_PREFIX + executorCount.getAndIncrement(), messageDispatcher, injector2);
+    final Pair<BlockManagerWorker, DataTransferFactory> pair1 = createWorker(
+        EXECUTOR_ID_PREFIX + executorCount.getAndIncrement(), dispatcherInjector, nameClientInjector);
     this.worker1 = pair1.left();
     this.transferFactory = pair1.right();
-    this.worker2 = createWorker(EXECUTOR_ID_PREFIX + executorCount.getAndIncrement(), messageDispatcher,
-        injector2).left();
+    this.worker2 = createWorker(EXECUTOR_ID_PREFIX + executorCount.getAndIncrement(), dispatcherInjector,
+        nameClientInjector).left();
   }
 
   @After
@@ -174,10 +158,12 @@ public final class DataTransferTest {
 
   private Pair<BlockManagerWorker, DataTransferFactory> createWorker(
       final String executorId,
-      final LocalMessageDispatcher messageDispatcher,
-      final Injector nameClientInjector) {
-    final LocalMessageEnvironment messageEnvironment = new LocalMessageEnvironment(executorId, messageDispatcher);
-    final PersistentConnectionToMasterMap conToMaster = new PersistentConnectionToMasterMap(messageEnvironment);
+      final Injector dispatcherInjector,
+      final Injector nameClientInjector) throws InjectionException {
+    final Injector messageEnvironmentInjector = LocalMessageEnvironment.forkInjector(dispatcherInjector, executorId);
+    final MessageEnvironment messageEnvironment = messageEnvironmentInjector.getInstance(MessageEnvironment.class);
+    final PersistentConnectionToMasterMap conToMaster = messageEnvironmentInjector
+        .getInstance(PersistentConnectionToMasterMap.class);
     final Configuration executorConfiguration = TANG.newConfigurationBuilder()
         .bindNamedParameter(JobConf.ExecutorId.class, executorId)
         .bindNamedParameter(MessageParameters.SenderId.class, executorId)
@@ -188,12 +174,10 @@ public final class DataTransferTest {
     injector.bindVolatileParameter(JobConf.FileDirectory.class, TMP_LOCAL_FILE_DIRECTORY);
     injector.bindVolatileParameter(JobConf.GlusterVolumeDirectory.class, TMP_REMOTE_FILE_DIRECTORY);
     final BlockManagerWorker blockManagerWorker;
-    final MetricManagerWorker metricManagerWorker;
     final SerializerManager serializerManager;
     final DataTransferFactory dataTransferFactory;
     try {
       blockManagerWorker = injector.getInstance(BlockManagerWorker.class);
-      metricManagerWorker = injector.getInstance(MetricManagerWorker.class);
       serializerManager = injector.getInstance(SerializerManager.class);
       serializerManagers.put(blockManagerWorker, serializerManager);
       dataTransferFactory = injector.getInstance(DataTransferFactory.class);
@@ -202,14 +186,7 @@ public final class DataTransferTest {
     }
 
     // Unused, but necessary for wiring up the message environments
-    final Executor executor = new Executor(
-        executorId,
-        conToMaster,
-        messageEnvironment,
-        serializerManager,
-        dataTransferFactory,
-        metricManagerWorker);
-    injector.bindVolatileInstance(Executor.class, executor);
+    injector.getInstance(Executor.class);
 
     return Pair.of(blockManagerWorker, dataTransferFactory);
   }
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
index 806f000..454e91b 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
@@ -96,14 +96,14 @@ public final class RuntimeMaster {
   private final Server metricServer;
 
   @Inject
-  public RuntimeMaster(final Scheduler scheduler,
-                       final ContainerManager containerManager,
-                       final BlockManagerMaster blockManagerMaster,
-                       final MetricMessageHandler metricMessageHandler,
-                       final MessageEnvironment masterMessageEnvironment,
-                       final ClientRPC clientRPC,
-                       final MetricManagerMaster metricManagerMaster,
-                       @Parameter(JobConf.DAGDirectory.class) final String dagDirectory) {
+  private RuntimeMaster(final Scheduler scheduler,
+                        final ContainerManager containerManager,
+                        final BlockManagerMaster blockManagerMaster,
+                        final MetricMessageHandler metricMessageHandler,
+                        final MessageEnvironment masterMessageEnvironment,
+                        final ClientRPC clientRPC,
+                        final MetricManagerMaster metricManagerMaster,
+                        @Parameter(JobConf.DAGDirectory.class) final String dagDirectory) {
     // We would like to use a single thread for runtime master operations
     // since the processing logic in master takes a very short amount of time
     // compared to the job completion times of executed jobs
@@ -198,9 +198,6 @@ public final class RuntimeMaster {
       metricMessageHandler.terminate();
       containerManager.terminate();
 
-      // TODO #?: parameterize file path using Tang
-      metricStore.dumpAllMetricToFile("/tmp/dump");
-
       try {
         metricServer.stop();
       } catch (final Exception e) {
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobScheduler.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobScheduler.java
index 531fced..3780902 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobScheduler.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobScheduler.java
@@ -77,12 +77,12 @@ public final class BatchSingleJobScheduler implements Scheduler {
   private List<List<Stage>> sortedScheduleGroups;
 
   @Inject
-  public BatchSingleJobScheduler(final SchedulerRunner schedulerRunner,
-                                 final PendingTaskCollectionPointer pendingTaskCollectionPointer,
-                                 final BlockManagerMaster blockManagerMaster,
-                                 final PubSubEventHandlerWrapper pubSubEventHandlerWrapper,
-                                 final UpdatePhysicalPlanEventHandler updatePhysicalPlanEventHandler,
-                                 final ExecutorRegistry executorRegistry) {
+  private BatchSingleJobScheduler(final SchedulerRunner schedulerRunner,
+                                  final PendingTaskCollectionPointer pendingTaskCollectionPointer,
+                                  final BlockManagerMaster blockManagerMaster,
+                                  final PubSubEventHandlerWrapper pubSubEventHandlerWrapper,
+                                  final UpdatePhysicalPlanEventHandler updatePhysicalPlanEventHandler,
+                                  final ExecutorRegistry executorRegistry) {
     this.schedulerRunner = schedulerRunner;
     this.pendingTaskCollectionPointer = pendingTaskCollectionPointer;
     this.blockManagerMaster = blockManagerMaster;
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
index a91e996..55c3824 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
@@ -15,7 +15,6 @@
  */
 package edu.snu.nemo.runtime.master.scheduler;
 
-import com.google.common.annotations.VisibleForTesting;
 import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
 import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
 import edu.snu.nemo.runtime.common.plan.Task;
@@ -29,9 +28,8 @@ import javax.inject.Inject;
 @AssociatedProperty(ExecutorPlacementProperty.class)
 public final class ContainerTypeAwareSchedulingConstraint implements SchedulingConstraint {
 
-  @VisibleForTesting
   @Inject
-  public ContainerTypeAwareSchedulingConstraint() {
+  private ContainerTypeAwareSchedulingConstraint() {
   }
 
   @Override
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ExecutorRegistry.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ExecutorRegistry.java
index 98af0a1..6360316 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ExecutorRegistry.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ExecutorRegistry.java
@@ -47,7 +47,7 @@ public final class ExecutorRegistry {
   private final Map<String, Pair<ExecutorRepresenter, ExecutorState>> executors;
 
   @Inject
-  public ExecutorRegistry() {
+  private ExecutorRegistry() {
     this.executors = new HashMap<>();
   }
 
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicy.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicy.java
index 7023961..1261d44 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicy.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicy.java
@@ -15,7 +15,6 @@
  */
 package edu.snu.nemo.runtime.master.scheduler;
 
-import com.google.common.annotations.VisibleForTesting;
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
 import org.apache.reef.annotations.audience.DriverSide;
@@ -31,9 +30,8 @@ import java.util.*;
 @DriverSide
 public final class MinOccupancyFirstSchedulingPolicy implements SchedulingPolicy {
 
-  @VisibleForTesting
   @Inject
-  public MinOccupancyFirstSchedulingPolicy() {
+  private MinOccupancyFirstSchedulingPolicy() {
   }
 
   @Override
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointer.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointer.java
index 3fb08c6..15deb00 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointer.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointer.java
@@ -33,7 +33,7 @@ public final class PendingTaskCollectionPointer {
   private Collection<Task> curTaskCollection;
 
   @Inject
-  public PendingTaskCollectionPointer() {
+  private PendingTaskCollectionPointer() {
   }
 
   /**
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SchedulerRunner.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SchedulerRunner.java
index 0b0524e..defc91b 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SchedulerRunner.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SchedulerRunner.java
@@ -15,7 +15,6 @@
  */
 package edu.snu.nemo.runtime.master.scheduler;
 
-import com.google.common.annotations.VisibleForTesting;
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.runtime.common.state.TaskState;
 import edu.snu.nemo.runtime.master.JobStateManager;
@@ -58,12 +57,11 @@ public final class SchedulerRunner {
   private final SchedulingConstraintRegistry schedulingConstraintRegistry;
   private final SchedulingPolicy schedulingPolicy;
 
-  @VisibleForTesting
   @Inject
-  public SchedulerRunner(final SchedulingConstraintRegistry schedulingConstraintRegistry,
-                         final SchedulingPolicy schedulingPolicy,
-                         final PendingTaskCollectionPointer pendingTaskCollectionPointer,
-                         final ExecutorRegistry executorRegistry) {
+  private SchedulerRunner(final SchedulingConstraintRegistry schedulingConstraintRegistry,
+                          final SchedulingPolicy schedulingPolicy,
+                          final PendingTaskCollectionPointer pendingTaskCollectionPointer,
+                          final ExecutorRegistry executorRegistry) {
     this.jobStateManagers = new HashMap<>();
     this.pendingTaskCollectionPointer = pendingTaskCollectionPointer;
     this.schedulerThread = Executors.newSingleThreadExecutor(runnable ->
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
index cb74f05..e0d79a0 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
@@ -15,7 +15,6 @@
  */
 package edu.snu.nemo.runtime.master.scheduler;
 
-import com.google.common.annotations.VisibleForTesting;
 import edu.snu.nemo.common.ir.Readable;
 import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
 import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedulingProperty;
@@ -37,9 +36,8 @@ import java.util.*;
 @AssociatedProperty(SourceLocationAwareSchedulingProperty.class)
 public final class SourceLocationAwareSchedulingConstraint implements SchedulingConstraint {
 
-  @VisibleForTesting
   @Inject
-  public SourceLocationAwareSchedulingConstraint() {
+  private SourceLocationAwareSchedulingConstraint() {
   }
 
   /**
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/BlockManagerMasterTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/BlockManagerMasterTest.java
index a465374..e5b362e 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/BlockManagerMasterTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/BlockManagerMasterTest.java
@@ -40,11 +40,8 @@ public final class BlockManagerMasterTest {
 
   @Before
   public void setUp() throws Exception {
-    final LocalMessageDispatcher messageDispatcher = new LocalMessageDispatcher();
-    final LocalMessageEnvironment messageEnvironment =
-        new LocalMessageEnvironment(MessageEnvironment.MASTER_COMMUNICATION_ID, messageDispatcher);
-    final Injector injector = Tang.Factory.getTang().newInjector();
-    injector.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
+    final Injector injector = LocalMessageEnvironment.forkInjector(LocalMessageDispatcher.getInjector(),
+        MessageEnvironment.MASTER_COMMUNICATION_ID);
     blockManagerMaster = injector.getInstance(BlockManagerMaster.class);
   }
 
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/JobStateManagerTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/JobStateManagerTest.java
index 6860893..803fa40 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/JobStateManagerTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/JobStateManagerTest.java
@@ -21,14 +21,12 @@ import edu.snu.nemo.runtime.common.message.MessageEnvironment;
 import edu.snu.nemo.runtime.common.message.local.LocalMessageDispatcher;
 import edu.snu.nemo.runtime.common.message.local.LocalMessageEnvironment;
 import edu.snu.nemo.runtime.common.plan.PhysicalPlan;
-import edu.snu.nemo.runtime.common.plan.PhysicalPlanGenerator;
 import edu.snu.nemo.runtime.common.plan.Stage;
 import edu.snu.nemo.runtime.common.state.JobState;
 import edu.snu.nemo.runtime.common.state.StageState;
 import edu.snu.nemo.runtime.common.state.TaskState;
 import edu.snu.nemo.runtime.plangenerator.TestPlanGenerator;
 import org.apache.reef.tang.Injector;
-import org.apache.reef.tang.Tang;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -50,21 +48,14 @@ import static org.mockito.Mockito.mock;
 @PrepareForTest(MetricMessageHandler.class)
 public final class JobStateManagerTest {
   private static final int MAX_SCHEDULE_ATTEMPT = 2;
-  private BlockManagerMaster blockManagerMaster;
   private MetricMessageHandler metricMessageHandler;
-  private PhysicalPlanGenerator physicalPlanGenerator;
 
   @Before
   public void setUp() throws Exception {
-    final LocalMessageDispatcher messageDispatcher = new LocalMessageDispatcher();
-    final LocalMessageEnvironment messageEnvironment =
-        new LocalMessageEnvironment(MessageEnvironment.MASTER_COMMUNICATION_ID, messageDispatcher);
-    final Injector injector = Tang.Factory.getTang().newInjector();
-    injector.bindVolatileInstance(MessageEnvironment.class, messageEnvironment);
-    blockManagerMaster = injector.getInstance(BlockManagerMaster.class);
+    final Injector injector = LocalMessageEnvironment.forkInjector(LocalMessageDispatcher.getInjector(),
+        MessageEnvironment.MASTER_COMMUNICATION_ID);
     metricMessageHandler = mock(MetricMessageHandler.class);
     injector.bindVolatileParameter(JobConf.DAGDirectory.class, "");
-    physicalPlanGenerator = injector.getInstance(PhysicalPlanGenerator.class);
   }
 
   /**
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
index e4491b6..44fb64c 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
@@ -63,15 +63,8 @@ import static org.mockito.Mockito.mock;
 public final class BatchSingleJobSchedulerTest {
   private static final Logger LOG = LoggerFactory.getLogger(BatchSingleJobSchedulerTest.class.getName());
   private Scheduler scheduler;
-  private SchedulingConstraintRegistry schedulingConstraint;
-  private SchedulingPolicy schedulingPolicy;
-  private SchedulerRunner schedulerRunner;
   private ExecutorRegistry executorRegistry;
-  private MetricMessageHandler metricMessageHandler;
-  private PendingTaskCollectionPointer pendingTaskCollectionPointer;
-  private PubSubEventHandlerWrapper pubSubEventHandler;
-  private UpdatePhysicalPlanEventHandler updatePhysicalPlanEventHandler;
-  private BlockManagerMaster blockManagerMaster = mock(BlockManagerMaster.class);
+  private final MetricMessageHandler metricMessageHandler = mock(MetricMessageHandler.class);
   private final MessageSender<ControlMessage.Message> mockMsgSender = mock(MessageSender.class);
 
   private static final int EXECUTOR_CAPACITY = 20;
@@ -85,16 +78,10 @@ public final class BatchSingleJobSchedulerTest {
     injector.bindVolatileParameter(JobConf.DAGDirectory.class, "");
 
     executorRegistry = injector.getInstance(ExecutorRegistry.class);
-    metricMessageHandler = mock(MetricMessageHandler.class);
-    pendingTaskCollectionPointer = new PendingTaskCollectionPointer();
-    schedulingConstraint = injector.getInstance(SchedulingConstraintRegistry.class);
-    schedulingPolicy = injector.getInstance(SchedulingPolicy.class);
-    schedulerRunner = new SchedulerRunner(schedulingConstraint, schedulingPolicy, pendingTaskCollectionPointer, executorRegistry);
-    pubSubEventHandler = mock(PubSubEventHandlerWrapper.class);
-    updatePhysicalPlanEventHandler = mock(UpdatePhysicalPlanEventHandler.class);
-    scheduler =
-        new BatchSingleJobScheduler(schedulerRunner, pendingTaskCollectionPointer,
-            blockManagerMaster, pubSubEventHandler, updatePhysicalPlanEventHandler, executorRegistry);
+    injector.bindVolatileInstance(BlockManagerMaster.class, mock(BlockManagerMaster.class));
+    injector.bindVolatileInstance(PubSubEventHandlerWrapper.class, mock(PubSubEventHandlerWrapper.class));
+    injector.bindVolatileInstance(UpdatePhysicalPlanEventHandler.class, mock(UpdatePhysicalPlanEventHandler.class));
+    scheduler = injector.getInstance(BatchSingleJobScheduler.class);
 
     final ActiveContext activeContext = mock(ActiveContext.class);
     Mockito.doThrow(new RuntimeException()).when(activeContext).close();
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
index 4aa2ecc..574d912 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
@@ -18,6 +18,8 @@ package edu.snu.nemo.runtime.master.scheduler;
 import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -43,8 +45,9 @@ public final class ContainerTypeAwareSchedulingConstraintTest {
   }
 
   @Test
-  public void testContainerTypeAware() {
-    final SchedulingConstraint schedulingConstraint = new ContainerTypeAwareSchedulingConstraint();
+  public void testContainerTypeAware() throws InjectionException {
+    final SchedulingConstraint schedulingConstraint = Tang.Factory.getTang().newInjector()
+        .getInstance(ContainerTypeAwareSchedulingConstraint.class);
     final ExecutorRepresenter a0 = mockExecutorRepresenter(ExecutorPlacementProperty.TRANSIENT);
     final ExecutorRepresenter a1 = mockExecutorRepresenter(ExecutorPlacementProperty.RESERVED);
     final ExecutorRepresenter a2 = mockExecutorRepresenter(ExecutorPlacementProperty.NONE);
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
index a4df785..e387c70 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
@@ -19,6 +19,7 @@ import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotCompliancePro
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
 import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicyTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicyTest.java
index 8831e6c..ca8d307 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicyTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/MinOccupancyFirstSchedulingPolicyTest.java
@@ -17,6 +17,8 @@ package edu.snu.nemo.runtime.master.scheduler;
 
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -50,8 +52,9 @@ public final class MinOccupancyFirstSchedulingPolicyTest {
   }
 
   @Test
-  public void test() {
-    final SchedulingPolicy schedulingPolicy = new MinOccupancyFirstSchedulingPolicy();
+  public void test() throws InjectionException {
+    final SchedulingPolicy schedulingPolicy = Tang.Factory.getTang().newInjector()
+        .getInstance(MinOccupancyFirstSchedulingPolicy.class);
     final ExecutorRepresenter a0 = mockExecutorRepresenter(1);
     final ExecutorRepresenter a1 = mockExecutorRepresenter(2);
     final ExecutorRepresenter a2 = mockExecutorRepresenter(2);
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointerTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointerTest.java
index 46cf822..469746c 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointerTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/PendingTaskCollectionPointerTest.java
@@ -16,6 +16,8 @@
 package edu.snu.nemo.runtime.master.scheduler;
 
 import edu.snu.nemo.runtime.common.plan.Task;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -46,8 +48,9 @@ public final class PendingTaskCollectionPointerTest {
   }
 
   @Before
-  public void setUp() {
-    this.pendingTaskCollectionPointer = new PendingTaskCollectionPointer();
+  public void setUp() throws InjectionException {
+    this.pendingTaskCollectionPointer = Tang.Factory.getTang().newInjector()
+        .getInstance(PendingTaskCollectionPointer.class);
   }
 
   @Test
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
index 99abf88..81bd65d 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
@@ -19,6 +19,8 @@ import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedu
 import edu.snu.nemo.runtime.common.plan.Task;
 import edu.snu.nemo.common.ir.Readable;
 import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -53,8 +55,9 @@ public final class SourceLocationAwareSchedulingConstraintTest {
    * there are no executors in appropriate location(s).
    */
   @Test
-  public void testSourceLocationAwareSchedulingNotAvailable() {
-    final SchedulingConstraint schedulingConstraint = new SourceLocationAwareSchedulingConstraint();
+  public void testSourceLocationAwareSchedulingNotAvailable() throws InjectionException {
+    final SchedulingConstraint schedulingConstraint = Tang.Factory.getTang().newInjector()
+        .getInstance(SourceLocationAwareSchedulingConstraint.class);
 
     // Prepare test scenario
     final Task task = CreateTask.withReadablesWithSourceLocations(
@@ -72,8 +75,9 @@ public final class SourceLocationAwareSchedulingConstraintTest {
    * with multiple source locations.
    */
   @Test
-  public void testSourceLocationAwareSchedulingWithMultiSource() {
-    final SchedulingConstraint schedulingConstraint = new SourceLocationAwareSchedulingConstraint();
+  public void testSourceLocationAwareSchedulingWithMultiSource() throws InjectionException {
+    final SchedulingConstraint schedulingConstraint = Tang.Factory.getTang().newInjector()
+        .getInstance(SourceLocationAwareSchedulingConstraint.class);
     // Prepare test scenario
     final Task task0 = CreateTask.withReadablesWithSourceLocations(
         Collections.singletonList(Collections.singletonList(SITE_1)));
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
index 4b6af1a..fe5ee17 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
@@ -85,16 +85,11 @@ public final class TaskRetryTest {
     executorRegistry = injector.getInstance(ExecutorRegistry.class);
 
     // Get scheduler
-    final PubSubEventHandlerWrapper pubSubEventHandler = mock(PubSubEventHandlerWrapper.class);
-    final UpdatePhysicalPlanEventHandler updatePhysicalPlanEventHandler = mock(UpdatePhysicalPlanEventHandler.class);
-    final SchedulingConstraintRegistry constraintRegistry = mock(SchedulingConstraintRegistry.class);
-    final SchedulingPolicy schedulingPolicy = injector.getInstance(MinOccupancyFirstSchedulingPolicy.class);
-    final PendingTaskCollectionPointer pendingTaskCollectionPointer = new PendingTaskCollectionPointer();
-    final SchedulerRunner schedulerRunner = new SchedulerRunner(
-        constraintRegistry, schedulingPolicy, pendingTaskCollectionPointer, executorRegistry);
-    final BlockManagerMaster blockManagerMaster = mock(BlockManagerMaster.class);
-    scheduler =  new BatchSingleJobScheduler(schedulerRunner, pendingTaskCollectionPointer, blockManagerMaster,
-        pubSubEventHandler, updatePhysicalPlanEventHandler, executorRegistry);
+    injector.bindVolatileInstance(PubSubEventHandlerWrapper.class, mock(PubSubEventHandlerWrapper.class));
+    injector.bindVolatileInstance(UpdatePhysicalPlanEventHandler.class, mock(UpdatePhysicalPlanEventHandler.class));
+    injector.bindVolatileInstance(SchedulingConstraintRegistry.class, mock(SchedulingConstraintRegistry.class));
+    injector.bindVolatileInstance(BlockManagerMaster.class, mock(BlockManagerMaster.class));
+    scheduler = injector.getInstance(Scheduler.class);
 
     // Get JobStateManager
     jobStateManager = runPhysicalPlan(TestPlanGenerator.PlanType.TwoVerticesJoined);
diff --git a/tests/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java b/tests/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
index 3c74fec..2427232 100644
--- a/tests/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
+++ b/tests/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
@@ -24,21 +24,19 @@ import edu.snu.nemo.common.ir.vertex.IRVertex;
 import edu.snu.nemo.common.ir.vertex.OperatorVertex;
 import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
 import edu.snu.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty;
-import edu.snu.nemo.common.ir.vertex.transform.Transform;
-import edu.snu.nemo.common.test.EmptyComponents;
 import org.apache.reef.tang.Injector;
 import org.apache.reef.tang.Tang;
 import org.junit.Test;
 
 import java.util.Iterator;
 
+import static edu.snu.nemo.common.test.EmptyComponents.EMPTY_TRANSFORM;
 import static org.junit.Assert.assertNotEquals;
 
 /**
  * Tests {@link PhysicalPlanGenerator}.
  */
 public final class PhysicalPlanGeneratorTest {
-  private static final Transform EMPTY_TRANSFORM = new EmptyComponents.EmptyTransform("");
 
   /**
    * Test splitting ScheduleGroups by Pull StageEdges.
diff --git a/tests/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java b/tests/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
index bcdc96a..b9f6c74 100644
--- a/tests/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
+++ b/tests/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
@@ -38,6 +38,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import static edu.snu.nemo.common.test.EmptyComponents.EMPTY_TRANSFORM;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
@@ -45,8 +46,6 @@ import static org.junit.Assert.assertNotEquals;
  * Tests {@link StagePartitioner}.
  */
 public final class StagePartitionerTest {
-  private static final Transform EMPTY_TRANSFORM = new EmptyComponents.EmptyTransform("empty");
-
   private StagePartitioner stagePartitioner;
 
   @Before
diff --git a/tests/src/test/java/edu/snu/nemo/runtime/master/MetricFlushTest.java b/tests/src/test/java/edu/snu/nemo/runtime/master/MetricFlushTest.java
index 75f964f..ea8237a 100644
--- a/tests/src/test/java/edu/snu/nemo/runtime/master/MetricFlushTest.java
+++ b/tests/src/test/java/edu/snu/nemo/runtime/master/MetricFlushTest.java
@@ -16,10 +16,7 @@
 package edu.snu.nemo.runtime.master;
 
 import edu.snu.nemo.runtime.common.comm.ControlMessage;
-import edu.snu.nemo.runtime.common.message.MessageContext;
-import edu.snu.nemo.runtime.common.message.MessageEnvironment;
-import edu.snu.nemo.runtime.common.message.MessageListener;
-import edu.snu.nemo.runtime.common.message.MessageSender;
+import edu.snu.nemo.runtime.common.message.*;
 import edu.snu.nemo.runtime.common.message.local.LocalMessageDispatcher;
 import edu.snu.nemo.runtime.common.message.local.LocalMessageEnvironment;
 import edu.snu.nemo.runtime.executor.MetricManagerWorker;
@@ -52,40 +49,29 @@ import static org.mockito.Mockito.mock;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({ExecutorRepresenter.class, ExecutorRegistry.class})
 public final class MetricFlushTest {
-  private static final Tang TANG = Tang.Factory.getTang();
   private static final String MASTER = "MASTER";
   private static final String WORKER = "WORKER";
   private static final int EXECUTOR_NUM = 5;
-  private static MessageSender masterToWorkerSender;
 
   @Test(timeout = 10000)
   public void test() throws InjectionException, ExecutionException, InterruptedException {
     final CountDownLatch latch = new CountDownLatch(EXECUTOR_NUM);
 
-    final LocalMessageDispatcher localMessagedispatcher = new LocalMessageDispatcher();
+    final Injector injector = LocalMessageDispatcher.getInjector();
 
-    final Configuration configuration = TANG.newConfigurationBuilder()
-        .build();
-    final Injector injector = TANG.newInjector(configuration);
+    final Injector masterInjector = LocalMessageEnvironment.forkInjector(injector, MASTER);
+    final Injector workerInjector = LocalMessageEnvironment.forkInjector(injector, WORKER);
 
-    final Injector masterInjector = injector.forkInjector();
-    final Injector workerInjector = injector.forkInjector();
+    final MessageEnvironment masterMessageEnvironment = masterInjector.getInstance(MessageEnvironment.class);
+    final MessageEnvironment workerMessageEnvironment = workerInjector.getInstance(MessageEnvironment.class);
 
-    final LocalMessageEnvironment masterMessageEnvironment = new LocalMessageEnvironment(MASTER,
-        localMessagedispatcher);
-    masterInjector.bindVolatileInstance(MessageEnvironment.class, masterMessageEnvironment);
-
-    final LocalMessageEnvironment workerMessageEnvironment = new LocalMessageEnvironment(WORKER,
-        localMessagedispatcher);
-    workerInjector.bindVolatileInstance(MessageEnvironment.class, workerMessageEnvironment);
-
-    masterToWorkerSender = masterMessageEnvironment
+    final MessageSender masterToWorkerSender = masterMessageEnvironment
         .asyncConnect(WORKER, MessageEnvironment.EXECUTOR_MESSAGE_LISTENER_ID).get();
 
     final Set<ExecutorRepresenter> executorRepresenterSet = new HashSet<>();
 
     for (int i = 0; i < EXECUTOR_NUM; i++) {
-      executorRepresenterSet.add(newWorker());
+      executorRepresenterSet.add(newWorker(masterToWorkerSender));
     }
 
     final ExecutorRegistry executorRegistry = mock(ExecutorRegistry.class);
@@ -129,7 +115,7 @@ public final class MetricFlushTest {
     latch.await();
   }
 
-  private ExecutorRepresenter newWorker() {
+  private static ExecutorRepresenter newWorker(final MessageSender masterToWorkerSender) {
     final ExecutorRepresenter workerRepresenter = mock(ExecutorRepresenter.class);
     doAnswer((Answer<Void>) invocationOnMock -> {
       final ControlMessage.Message msg = (ControlMessage.Message) invocationOnMock.getArguments()[0];
diff --git a/tests/src/test/java/edu/snu/nemo/tests/compiler/backend/nemo/NemoBackendTest.java b/tests/src/test/java/edu/snu/nemo/tests/compiler/backend/nemo/NemoBackendTest.java
index 0f24f55..1ffd866 100644
--- a/tests/src/test/java/edu/snu/nemo/tests/compiler/backend/nemo/NemoBackendTest.java
+++ b/tests/src/test/java/edu/snu/nemo/tests/compiler/backend/nemo/NemoBackendTest.java
@@ -30,6 +30,7 @@ import edu.snu.nemo.runtime.common.plan.PhysicalPlan;
 import edu.snu.nemo.runtime.common.plan.PhysicalPlanGenerator;
 import org.apache.reef.tang.Injector;
 import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -71,8 +72,8 @@ public final class NemoBackendTest<I, O> {
    * @throws Exception during the Execution Plan generation.
    */
   @Test
-  public void testExecutionPlanGeneration() {
-    final NemoBackend backend = new NemoBackend();
+  public void testExecutionPlanGeneration() throws InjectionException {
+    final NemoBackend backend = Tang.Factory.getTang().newInjector().getInstance(NemoBackend.class);
     final PhysicalPlan executionPlan = backend.compile(dag, physicalPlanGenerator);
 
     assertEquals(2, executionPlan.getStageDAG().getVertices().size());
diff --git a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
index 33e4988..7d1f588 100644
--- a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
+++ b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
@@ -39,6 +39,7 @@ import org.powermock.modules.junit4.PowerMockRunner;
 
 import java.util.*;
 
+import static edu.snu.nemo.common.test.EmptyComponents.EMPTY_TRANSFORM;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -49,8 +50,6 @@ import static org.junit.Assert.assertTrue;
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobLauncher.class)
 public final class DefaultScheduleGroupPassTest {
-  private static final Transform EMPTY_TRANSFORM = new EmptyComponents.EmptyTransform("empty");
-
   @Test
   public void testAnnotatingPass() {
     final AnnotatingPass scheduleGroupPass = new DefaultScheduleGroupPass();
diff --git a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
index 6861793..827e326 100644
--- a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
+++ b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
@@ -71,7 +71,7 @@ public class CommonSubexpressionEliminationPassTest {
   }
 
   @Test
-  public void testCommonSubexpressionEliminationPass() throws Exception {
+  public void testCommonSubexpressionEliminationPass() {
     final long originalVerticesNum = dagNotToOptimize.getVertices().size();
     final long optimizedVerticesNum = dagToOptimize.getVertices().size();
 
diff --git a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
index 7fc3752..4568f68 100644
--- a/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
+++ b/tests/src/test/java/edu/snu/nemo/tests/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPassTest.java
@@ -43,7 +43,7 @@ public class LoopExtractionPassTest {
   }
 
   @Test
-  public void testLoopGrouping() throws Exception {
+  public void testLoopGrouping() {
     final DAG<IRVertex, IREdge> processedDAG = new LoopExtractionPass().apply(compiledDAG);
 
     assertEquals(9, processedDAG.getTopologicalSort().size());