You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2015/06/08 21:07:28 UTC

[3/5] drill git commit: DRILL-3035: Created ControlsInjector interface to enforce method implementations + DRILL-2867: Add ControlsValidator to VALIDATORS only if assertions are enabled + return in ExecutionControls ctor if assertions are not enabled + a

DRILL-3035: Created ControlsInjector interface to enforce method implementations
+ DRILL-2867: Add ControlsValidator to VALIDATORS only if assertions are enabled
+ return in ExecutionControls ctor if assertions are not enabled
+ added InjectorFactory class to align with the logger pattern


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/2af6340d
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/2af6340d
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/2af6340d

Branch: refs/heads/master
Commit: 2af6340da24cf40cc3512b728a460b84eb610dce
Parents: d9452d9
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Thu May 21 14:25:34 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Fri Jun 5 07:49:54 2015 -0700

----------------------------------------------------------------------
 .../drill/exec/memory/TopLevelAllocator.java    |   6 +-
 .../drill/exec/ops/AccountingDataTunnel.java    |   5 +-
 .../drill/exec/physical/impl/ScanBatch.java     |   5 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   7 +-
 .../exec/physical/impl/SingleSenderCreator.java |   6 +-
 .../impl/mergereceiver/MergingRecordBatch.java  |   6 +-
 .../partitionsender/PartitionerDecorator.java   |   6 +-
 .../UnorderedReceiverBatch.java                 |   7 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |   7 +-
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   5 +-
 .../apache/drill/exec/rpc/data/DataTunnel.java  |   5 +-
 .../server/options/SystemOptionManager.java     |  19 ++--
 .../drill/exec/store/pojo/PojoRecordReader.java |   6 +-
 .../drill/exec/testing/ControlsInjector.java    | 108 +++++++++++++++++++
 .../exec/testing/ControlsInjectorFactory.java   |  38 +++++++
 .../drill/exec/testing/ExecutionControls.java   |   8 +-
 .../exec/testing/ExecutionControlsInjector.java | 103 ++++--------------
 .../exec/testing/NoOpControlsInjector.java      |  30 ++++--
 .../apache/drill/exec/work/foreman/Foreman.java |   6 +-
 .../exec/work/fragment/FragmentExecutor.java    |   5 +-
 .../testing/TestCountDownLatchInjection.java    |   2 +-
 .../exec/testing/TestExceptionInjection.java    |   2 +-
 .../drill/exec/testing/TestPauseInjection.java  |   2 +-
 23 files changed, 256 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
index e2d5b18..b4386a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
@@ -31,14 +31,14 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.AssertionUtil;
 import org.apache.drill.exec.util.Pointer;
 
 public class TopLevelAllocator implements BufferAllocator {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopLevelAllocator.class);
-
-  private static final ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(TopLevelAllocator.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(TopLevelAllocator.class);
   public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer";
 
   public static long MAXIMUM_DIRECT_MEMORY;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AccountingDataTunnel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AccountingDataTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AccountingDataTunnel.java
index 2659464..22923bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AccountingDataTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AccountingDataTunnel.java
@@ -21,6 +21,7 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.data.DataTunnel;
+import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.testing.ExecutionControlsInjector;
 import org.slf4j.Logger;
@@ -46,9 +47,9 @@ public class AccountingDataTunnel {
   }
 
   /**
-   * See {@link DataTunnel#setTestInjectionControls(ExecutionControlsInjector, ExecutionControls, Logger)}.
+   * See {@link DataTunnel#setTestInjectionControls(ControlsInjector, ExecutionControls, Logger)}.
    */
-  public void setTestInjectionControls(final ExecutionControlsInjector testInjector,
+  public void setTestInjectionControls(final ControlsInjector testInjector,
       final ExecutionControls testControls, final org.slf4j.Logger testLogger) {
     tunnel.setTestInjectionControls(testInjector, testControls, testLogger);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 6176f77..da73185 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -49,7 +49,8 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.store.RecordReader;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
@@ -63,7 +64,7 @@ import com.google.common.collect.Maps;
  */
 public class ScanBatch implements CloseableRecordBatch {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(ScanBatch.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ScanBatch.class);
 
   private final Map<MaterializedField.Key, ValueVector> fieldVectorMap = Maps.newHashMap();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 76dc91c..363205c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -35,13 +35,12 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 
 import com.google.common.base.Preconditions;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 
 public class ScreenCreator implements RootCreator<Screen>{
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(ScreenCreator.class);
-
-
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ScreenCreator.class);
 
   @Override
   public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) throws ExecutionSetupException {

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index 1f6767c..f382759 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -31,7 +31,8 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 
 public class SingleSenderCreator implements RootCreator<SingleSender>{
 
@@ -44,8 +45,7 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
 
   public static class SingleSenderRootExec extends BaseRootExec {
     private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
-    private static final ExecutionControlsInjector injector =
-        ExecutionControlsInjector.getInjector(SingleSenderRootExec.class);
+    private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(SingleSenderRootExec.class);
 
     private final FragmentHandle oppositeHandle;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index baf9bda..3ca11f1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -70,7 +70,8 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.CopyUtil;
 import org.apache.drill.exec.vector.FixedWidthVector;
@@ -83,13 +84,12 @@ import com.google.common.collect.Lists;
 import com.sun.codemodel.JConditional;
 import com.sun.codemodel.JExpr;
 
-
 /**
  * The MergingRecordBatch merges pre-sorted record batches from remote senders.
  */
 public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP> implements RecordBatch {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergingRecordBatch.class);
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(MergingRecordBatch.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(MergingRecordBatch.class);
 
   private static final int OUTGOING_BATCH_SIZE = 32 * 1024;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
index e210514..b1468a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
@@ -29,8 +29,9 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.testing.CountDownLatchInjection;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
 
 /**
  * Decorator class to hide multiple Partitioner existence from the caller
@@ -42,8 +43,7 @@ import org.apache.drill.exec.testing.ExecutionControlsInjector;
  */
 public class PartitionerDecorator {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionerDecorator.class);
-  private static final ExecutionControlsInjector injector =
-      ExecutionControlsInjector.getInjector(PartitionerDecorator.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(PartitionerDecorator.class);
 
   private List<Partitioner> partitioners;
   private final OperatorStats stats;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
index 684f715..caabfce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.MetricDef;
-import org.apache.drill.exec.ops.OpProfileDef;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.MinorFragmentEndpoint;
@@ -49,12 +48,12 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 
 public class UnorderedReceiverBatch implements CloseableRecordBatch {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnorderedReceiverBatch.class);
-  private final static ExecutionControlsInjector injector =
-      ExecutionControlsInjector.getInjector(UnorderedReceiverBatch.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(UnorderedReceiverBatch.class);
 
   private final RecordBatchLoader batchLoader;
   private final RawFragmentBatchProvider fragProvider;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 8871a5f..5ce63fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -62,8 +62,8 @@ import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.store.ischema.Records;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.vector.CopyUtil;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
@@ -80,6 +80,7 @@ import com.sun.codemodel.JExpr;
 
 public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ExternalSortBatch.class);
 
   private static final long MAX_SORT_BYTES = 1L * 1024 * 1024 * 1024;
   private static final GeneratorMapping COPIER_MAPPING = new GeneratorMapping("doSetup", "doCopy", null, null);
@@ -115,7 +116,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
   public static final String INTERRUPTION_AFTER_SORT = "after-sort";
   public static final String INTERRUPTION_AFTER_SETUP = "after-setup";
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(ExternalSortBatch.class);
+
 
   public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException {
     super(popConfig, context, true);

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index e385600..73aeec6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -48,7 +48,8 @@ import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
 import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.rel.RelCollationTraitDef;
@@ -66,7 +67,7 @@ import org.apache.hadoop.security.AccessControlException;
 
 public class DrillSqlWorker {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DrillSqlWorker.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(DrillSqlWorker.class);
 
   private final Planner planner;
   private final HepPlanner hepPlanner;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
index 7a6477e..c38138c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.rpc.ListeningCommand;
 import org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.testing.ExecutionControlsInjector;
 
@@ -43,7 +44,7 @@ public class DataTunnel {
 
   // Needed for injecting a test pause
   private boolean isInjectionControlSet;
-  private ExecutionControlsInjector testInjector;
+  private ControlsInjector testInjector;
   private ExecutionControls testControls;
   private org.slf4j.Logger testLogger;
 
@@ -63,7 +64,7 @@ public class DataTunnel {
    * @param testControls
    * @param testLogger
    */
-  public void setTestInjectionControls(final ExecutionControlsInjector testInjector,
+  public void setTestInjectionControls(final ControlsInjector testInjector,
       final ExecutionControls testControls, final org.slf4j.Logger testLogger) {
     isInjectionControlSet = true;
     this.testInjector = testInjector;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index e268e64..abd2212 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
@@ -30,13 +31,10 @@ import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.QueryClassLoader;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
-import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
-import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
-import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
-import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.store.sys.PStore;
 import org.apache.drill.exec.store.sys.PStoreConfig;
 import org.apache.drill.exec.store.sys.PStoreProvider;
+import org.apache.drill.exec.util.AssertionUtil;
 import org.apache.calcite.sql.SqlLiteral;
 
 import com.google.common.collect.Maps;
@@ -44,7 +42,10 @@ import com.google.common.collect.Maps;
 public class SystemOptionManager extends BaseOptionManager {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
 
-  private static final OptionValidator[] VALIDATORS = {
+  private static final ImmutableList<OptionValidator> VALIDATORS;
+
+  static {
+    final ImmutableList.Builder<OptionValidator> builder = ImmutableList.<OptionValidator>builder().add(
       PlannerSettings.CONSTANT_FOLDING,
       PlannerSettings.EXCHANGE,
       PlannerSettings.HASHAGG,
@@ -111,10 +112,14 @@ public class SystemOptionManager extends BaseOptionManager {
       QueryClassLoader.JAVA_COMPILER_DEBUG,
       ExecConstants.ENABLE_VERBOSE_ERRORS,
       ExecConstants.ENABLE_WINDOW_FUNCTIONS_VALIDATOR,
-      ExecConstants.DRILLBIT_CONTROLS_VALIDATOR,
       ClassTransformer.SCALAR_REPLACEMENT_VALIDATOR,
       ExecConstants.ENABLE_NEW_TEXT_READER
-  };
+    );
+    if (AssertionUtil.isAssertionsEnabled()) {
+      builder.add(ExecConstants.DRILLBIT_CONTROLS_VALIDATOR);
+    }
+    VALIDATORS = builder.build();
+  }
 
   private final PStoreConfig<OptionValue> config;
   private PStore<OptionValue> options;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
index a893da1..543121f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
@@ -42,7 +42,8 @@ import org.apache.drill.exec.store.pojo.Writers.NDoubleWriter;
 import org.apache.drill.exec.store.pojo.Writers.NIntWriter;
 import org.apache.drill.exec.store.pojo.Writers.NTimeStampWriter;
 import org.apache.drill.exec.store.pojo.Writers.StringWriter;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -50,8 +51,7 @@ import com.google.common.collect.Lists;
 
 public class PojoRecordReader<T> extends AbstractRecordReader {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoRecordReader.class);
-  private static final ExecutionControlsInjector injector =
-    ExecutionControlsInjector.getInjector(PojoRecordReader.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(PojoRecordReader.class);
 
   public final int forJsonIgnore = 1;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjector.java
new file mode 100644
index 0000000..c7bdbad
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjector.java
@@ -0,0 +1,108 @@
+/**
+ * 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.drill.exec.testing;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.slf4j.Logger;
+
+public interface ControlsInjector {
+
+  /**
+   * Get the injector's owning class.
+   *
+   * @return the injector's owning class
+   */
+  Class<?> getSiteClass();
+
+  /**
+   * Inject (throw) an unchecked exception at this point, if an injection is specified, and it is time
+   * for it to be thrown.
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate an exception
+   * during testing.
+   *
+   * @param executionControls the controls in the current context
+   * @param desc              the site descriptor
+   *                          throws the exception specified by the injection, if it is time
+   */
+  void injectUnchecked(ExecutionControls executionControls, String desc);
+
+  /**
+   * Inject (throw) an unchecked exception at this point, if the fragmentContext is not null,
+   * an injection is specified, and it is time for it to be thrown.
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate an exception
+   * during testing.
+   *
+   * @param fragmentContext   fragmentContext used to retrieve the controls, can be null
+   * @param desc              the site description
+   *                          throws the exception specified by the injection, if it is time
+   */
+  void injectUnchecked(FragmentContext fragmentContext, String desc);
+
+  /**
+   * Inject (throw) a checked exception at this point, if an injection is specified, and it is time
+   * for it to be thrown.
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate an exception
+   * during testing.
+   *
+   * @param executionControls the controls in the current context
+   * @param desc              the site descriptor
+   * @param exceptionClass    the expected class of the exception (or a super class of it)
+   * @throws T the exception specified by the injection, if it is time
+   */
+  <T extends Throwable> void injectChecked(ExecutionControls executionControls, String desc, Class<T> exceptionClass)
+    throws T;
+
+  /**
+   * Pauses at this point, if such an injection is specified (i.e. matches the site description).
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate a pause
+   * during testing.
+   *
+   * @param executionControls the controls in the current context
+   * @param desc              the site descriptor
+   * @param logger            logger of the class containing the injection site
+   */
+  void injectPause(ExecutionControls executionControls, String desc, Logger logger);
+
+  /**
+   * Insert a pause that can be interrupted using {@link Thread#interrupt()} at the given site point, if such an
+   * injection is specified (i.e. matches the site description).
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate a interruptible pause
+   * during testing.
+   *
+   * @param executionControls the controls in the current context
+   * @param desc              the site descriptor
+   * @param logger            logger of the class containing the injection site
+   * @throws InterruptedException if interrupted using {@link Thread#interrupt()}
+   */
+  void injectInterruptiblePause(ExecutionControls executionControls, String desc, Logger logger)
+    throws InterruptedException;
+
+  /**
+   * Used to insert a latch in site class. See {@link CountDownLatchInjection} and
+   * {@link org.apache.drill.exec.testing.TestCountDownLatchInjection} for usage.
+   * @param executionControls the controls in the current context
+   * @param desc              the site descriptor
+   */
+  CountDownLatchInjection getLatch(final ExecutionControls executionControls, final String desc);
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjectorFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjectorFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjectorFactory.java
new file mode 100644
index 0000000..45dbcae
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ControlsInjectorFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.testing;
+
+import org.apache.drill.exec.util.AssertionUtil;
+
+public class ControlsInjectorFactory {
+
+  /**
+   * Returns an {@link ExecutionControlsInjector} if assertions are not enabled, or {@link NoOpControlsInjector}
+   * otherwise.
+   *
+   * @param clazz the owning class
+   * @return the newly created injector
+   */
+  public static ControlsInjector getInjector(final Class<?> clazz) {
+    if (AssertionUtil.isAssertionsEnabled()) {
+      return new ExecutionControlsInjector(clazz);
+    } else {
+      return new NoOpControlsInjector(clazz);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
index 639802f..836fa50 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
 import org.apache.drill.exec.server.options.TypeValidators.TypeValidator;
 import org.apache.drill.exec.testing.InjectionSite.InjectionSiteKeyDeserializer;
+import org.apache.drill.exec.util.AssertionUtil;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -73,6 +74,7 @@ public final class ExecutionControls {
 
     /**
      * Constructor for controls option validator.
+     *
      * @param name the name of the validator
      * @param def  the default JSON, specified as string
      * @param ttl  the number of queries for which this option should be valid
@@ -85,7 +87,7 @@ public final class ExecutionControls {
 
     @Override
     public int getTtl() {
-      return  ttl;
+      return ttl;
     }
 
     @Override
@@ -130,6 +132,10 @@ public final class ExecutionControls {
   public ExecutionControls(final OptionManager options, final DrillbitEndpoint endpoint) {
     this.endpoint = endpoint;
 
+    if (!AssertionUtil.isAssertionsEnabled()) {
+      return;
+    }
+
     final OptionValue optionValue = options.getOption(ExecConstants.DRILLBIT_CONTROL_INJECTIONS);
     if (optionValue == null) {
       return;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
index e3a4ba6..d8979d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.testing;
 
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.util.AssertionUtil;
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 
 /**
@@ -27,14 +27,15 @@ import org.slf4j.Logger;
  * of loggers). Injection site either use {@link org.apache.drill.exec.ops.FragmentContext} or
  * {@link org.apache.drill.exec.ops.QueryContext}. See {@link org.apache.drill.exec.testing.TestExceptionInjection} and
  * {@link org.apache.drill.exec.testing.TestPauseInjection} for examples of use.
+ * See {@link ControlsInjector} for documentation.
  */
-public class ExecutionControlsInjector {
-//  private static final Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionControlsInjector.class);
+public class ExecutionControlsInjector implements ControlsInjector {
 
   private final Class<?> clazz; // the class that owns this injector
 
   /**
-   * Constructor. Classes should use the static {@link #getInjector} method to obtain their injector.
+   * Constructor. Classes should use the static {@link ControlsInjectorFactory#getInjector} method to obtain their
+   * injector.
    *
    * @param clazz the owning class
    */
@@ -42,96 +43,40 @@ public class ExecutionControlsInjector {
     this.clazz = clazz;
   }
 
-  /**
-   * Create an injector if assertions are enabled
-   *
-   * @param clazz the owning class
-   * @return the newly created injector
-   */
-  public static ExecutionControlsInjector getInjector(final Class<?> clazz) {
-    if (AssertionUtil.isAssertionsEnabled()) {
-      return new ExecutionControlsInjector(clazz);
-    } else {
-      return new NoOpControlsInjector(clazz);
-    }
-  }
-
-  /**
-   * Get the injector's owning class.
-   *
-   * @return the injector's owning class
-   */
+  @Override
   public Class<?> getSiteClass() {
     return clazz;
   }
 
-  /**
-   * Inject (throw) an unchecked exception at this point, if an injection is specified, and it is time
-   * for it to be thrown.
-   * <p/>
-   * <p>Implementors use this in their code at a site where they want to simulate an exception
-   * during testing.
-   *
-   * @param executionControls the controls in the current context
-   * @param desc              the site description
-   *                          throws the exception specified by the injection, if it is time
-   */
+  @Override
   public void injectUnchecked(final ExecutionControls executionControls, final String desc) {
+    Preconditions.checkNotNull(executionControls);
     final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc);
     if (exceptionInjection != null) {
       exceptionInjection.throwUnchecked();
     }
   }
 
-  /**
-   * Inject (throw) an unchecked exception at this point, if the fragmentContext is not null,
-   * an injection is specified, and it is time for it to be thrown.
-   * <p/>
-   * <p>Implementors use this in their code at a site where they want to simulate an exception
-   * during testing.
-   *
-   * @param fragmentContext   fragmentContext used to retrieve the controls, can be null
-   * @param desc              the site description
-   *                          throws the exception specified by the injection, if it is time
-   */
+  @Override
   public void injectUnchecked(final FragmentContext fragmentContext, final String desc) {
     if (fragmentContext != null) {
       injectUnchecked(fragmentContext.getExecutionControls(), desc);
     }
   }
 
-  /**
-   * Inject (throw) a checked exception at this point, if an injection is specified, and it is time
-   * for it to be thrown.
-   * <p/>
-   * <p>Implementors use this in their code at a site where they want to simulate an exception
-   * during testing.
-   *
-   * @param executionControls the controls in the current context
-   * @param desc              the site description
-   * @param exceptionClass    the expected class of the exception (or a super class of it)
-   * @throws T the exception specified by the injection, if it is time
-   */
-  public <T extends Throwable> void injectChecked(
-    final ExecutionControls executionControls, final String desc, final Class<T> exceptionClass) throws T {
+  @Override
+  public <T extends Throwable> void injectChecked(final ExecutionControls executionControls, final String desc,
+                                                  final Class<T> exceptionClass) throws T {
+    Preconditions.checkNotNull(executionControls);
     final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc);
     if (exceptionInjection != null) {
       exceptionInjection.throwChecked(exceptionClass);
     }
   }
 
-  /**
-   * Pauses at this point, if such an injection is specified (i.e. matches the site description).
-   * <p/>
-   * <p>Implementors use this in their code at a site where they want to simulate a pause
-   * during testing.
-   *
-   * @param executionControls the controls in the current context
-   * @param desc              the site description
-   * @param logger            logger of the class containing the injection site
-   */
-  public void injectPause(final ExecutionControls executionControls, final String desc,
-                                               final Logger logger) {
+  @Override
+  public void injectPause(final ExecutionControls executionControls, final String desc, final Logger logger) {
+    Preconditions.checkNotNull(executionControls);
     final PauseInjection pauseInjection =
       executionControls.lookupPauseInjection(this, desc);
 
@@ -142,20 +87,10 @@ public class ExecutionControlsInjector {
     }
   }
 
-  /**
-   * Insert a pause that can be interrupted using {@link Thread#interrupt()} at the given site point, if such an
-   * injection is specified (i.e. matches the site description).
-   * <p/>
-   * <p>Implementors use this in their code at a site where they want to simulate a interruptible pause
-   * during testing.
-   *
-   * @param executionControls the controls in the current context
-   * @param desc              the site description
-   * @param logger            logger of the class containing the injection site
-   * @throws InterruptedException if interrupted using {@link Thread#interrupt()}
-   */
+  @Override
   public void injectInterruptiblePause(final ExecutionControls executionControls, final String desc,
       final Logger logger) throws InterruptedException {
+    Preconditions.checkNotNull(executionControls);
     final PauseInjection pauseInjection = executionControls.lookupPauseInjection(this, desc);
 
     if (pauseInjection != null) {
@@ -170,7 +105,9 @@ public class ExecutionControlsInjector {
     }
   }
 
+  @Override
   public CountDownLatchInjection getLatch(final ExecutionControls executionControls, final String desc) {
+    Preconditions.checkNotNull(executionControls);
     return executionControls.lookupCountDownLatchInjection(this, desc);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
index bf4221e..b8ec44d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
@@ -17,15 +17,30 @@
  */
 package org.apache.drill.exec.testing;
 
+import org.apache.drill.exec.ops.FragmentContext;
 import org.slf4j.Logger;
 
 /**
  * An injector that does not inject any controls, useful when not testing (i.e. assertions are not enabled).
+ * See {@link ControlsInjector} for documentation.
  */
-public final class NoOpControlsInjector extends ExecutionControlsInjector {
+public final class NoOpControlsInjector implements ControlsInjector {
 
+  private final Class<?> clazz;
+
+  /**
+   * Constructor. Classes should use the static {@link ControlsInjectorFactory#getInjector} method to obtain their
+   * injector.
+   *
+   * @param clazz the owning class
+   */
   protected NoOpControlsInjector(final Class<?> clazz) {
-    super(clazz);
+    this.clazz = clazz;
+  }
+
+  @Override
+  public Class<?> getSiteClass() {
+    return clazz;
   }
 
   @Override
@@ -33,18 +48,21 @@ public final class NoOpControlsInjector extends ExecutionControlsInjector {
   }
 
   @Override
+  public void injectUnchecked(final FragmentContext fragmentContext, final String desc) {
+  }
+
+  @Override
   public <T extends Throwable> void injectChecked(
     final ExecutionControls executionControls, final String desc, final Class<T> exceptionClass) throws T {
   }
 
   @Override
-  public void injectPause(final ExecutionControls executionControls, final String desc,
-                                               final Logger logger) {
+  public void injectPause(final ExecutionControls executionControls, final String desc, final Logger logger) {
   }
 
   @Override
-  public void injectInterruptiblePause(ExecutionControls executionControls, String desc, Logger logger)
-      throws InterruptedException {
+  public void injectInterruptiblePause(final ExecutionControls executionControls, final String desc,
+                                       final Logger logger) throws InterruptedException {
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 5d07b49..78c438b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -73,7 +73,8 @@ import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.EndpointListener;
 import org.apache.drill.exec.work.QueryWorkUnit;
@@ -107,8 +108,9 @@ import com.google.common.collect.Sets;
 public class Foreman implements Runnable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
   private static final org.slf4j.Logger queryLogger = org.slf4j.LoggerFactory.getLogger("query.logger");
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(Foreman.class);
+
   private static final ObjectMapper MAPPER = new ObjectMapper();
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(Foreman.class);
   private static final long RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
 
   private final QueryId queryId;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index e5e0700..a9c2b6d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -41,7 +41,8 @@ import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.testing.ExecutionControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -52,7 +53,7 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
 public class FragmentExecutor implements Runnable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentExecutor.class);
-  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(FragmentExecutor.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(FragmentExecutor.class);
 
   private final AtomicBoolean hasCloseoutThread = new AtomicBoolean(false);
   private final String fragmentName;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
index c98f54c..c0d9fd4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
@@ -47,7 +47,7 @@ public class TestCountDownLatchInjection extends BaseTestQuery {
    * {@link org.apache.drill.exec.ops.FragmentContext}.
    */
   private static class DummyClass {
-    private static final ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DummyClass.class);
+    private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(DummyClass.class);
 
     private final QueryContext context;
     private final CountDownLatch latch;

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
index e3558a1..305f0b7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
@@ -51,7 +51,7 @@ public class TestExceptionInjection extends BaseTestQuery {
    * For instance, these are accessible from {@link org.apache.drill.exec.ops.QueryContext}.
    */
   private static class DummyClass {
-    private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DummyClass.class);
+    private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(DummyClass.class);
     private final QueryContext context;
 
     public DummyClass(final QueryContext context) {

http://git-wip-us.apache.org/repos/asf/drill/blob/2af6340d/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
index ba29c58..2af6d95 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
@@ -56,7 +56,7 @@ public class TestPauseInjection extends BaseTestQuery {
    */
   private static class DummyClass {
     private static final Logger logger = org.slf4j.LoggerFactory.getLogger(DummyClass.class);
-    private static final ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DummyClass.class);
+    private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(DummyClass.class);
 
     private final QueryContext context;
     private final CountDownLatch latch;