You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2018/01/26 12:44:27 UTC

[03/11] drill git commit: DRILL-5730: Mock testing improvements and interface improvements

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
index 2032afc..e7a14f8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -42,23 +42,26 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Category(OperatorTest.class)
 public class TestSimpleLimit extends ExecTest {
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testLimit(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
+  public void testLimit() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
-    mockDrillbitContext(bitContext);
     verifyLimitCount(bitContext, connection, "test1.json", 5);
   }
 
   @Test
-  public void testLimitNoEnd(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testLimitNoEnd() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
+
     verifyLimitCount(bitContext, connection, "test3.json", 95);
   }
 
@@ -68,24 +71,23 @@ public class TestSimpleLimit extends ExecTest {
   // However, when evaluate the increasingBitInt(0), if the outgoing batch could not hold the new value, doEval() return false, and start the
   // next batch. But the value has already been increased by 1 in the prior failed try. Therefore, the sum of the generated number could be different,
   // depending on the size of each outgoing batch, and when the batch could not hold any more values.
-  public void testLimitAcrossBatches(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
+  public void testLimitAcrossBatches() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
-    mockDrillbitContext(bitContext);
     verifyLimitCount(bitContext, connection, "test2.json", 69999);
     final long start = 30000;
     final long end = 100000;
     final long expectedSum = (end - start) * (end + start - 1) / 2; //Formula for sum of series
 
     verifySum(bitContext, connection, "test4.json", 70000, expectedSum);
-
-
   }
 
   private void verifyLimitCount(DrillbitContext bitContext, UserClientConnection connection, String testPlan, int expectedCount) throws Throwable {
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/limit/" + testPlan), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
     int recordCount = 0;
     while(exec.next()) {
@@ -94,18 +96,18 @@ public class TestSimpleLimit extends ExecTest {
 
     assertEquals(expectedCount, recordCount);
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
 
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 
   private void verifySum(DrillbitContext bitContext, UserClientConnection connection, String testPlan, int expectedCount, long expectedSum) throws Throwable {
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/limit/" + testPlan), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
     int recordCount = 0;
     long sum = 0;
@@ -120,9 +122,9 @@ public class TestSimpleLimit extends ExecTest {
     assertEquals(expectedCount, recordCount);
     assertEquals(expectedSum, sum);
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
index 544ed49..c431fea 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
@@ -33,7 +33,7 @@ import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.MinorFragmentEndpoint;
 import org.apache.drill.exec.physical.PhysicalPlan;
@@ -218,9 +218,9 @@ public class TestPartitionSender extends PlanTestBase {
         continue;
       }
       MockPartitionSenderRootExec partionSenderRootExec = null;
-      FragmentContext context = null;
+      FragmentContextImpl context = null;
       try {
-        context = new FragmentContext(drillbitContext, planFragment, null, registry);
+        context = new FragmentContextImpl(drillbitContext, planFragment, null, registry);
         final int majorFragmentId = planFragment.getHandle().getMajorFragmentId();
         final HashPartitionSender partSender = new HashPartitionSender(majorFragmentId, hashToRandomExchange, hashToRandomExchange.getExpression(), mfEndPoints);
         partionSenderRootExec = new MockPartitionSenderRootExec(context, incoming, partSender);
@@ -347,7 +347,7 @@ public class TestPartitionSender extends PlanTestBase {
    */
   private static class MockPartitionSenderRootExec extends PartitionSenderRootExec {
 
-    public MockPartitionSenderRootExec(FragmentContext context,
+    public MockPartitionSenderRootExec(FragmentContextImpl context,
         RecordBatch incoming, HashPartitionSender operator)
         throws OutOfMemoryException {
       super(context, incoming, operator);

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
index 022d153..2c360fc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -44,29 +44,28 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Category(OperatorTest.class)
 public class TestSimpleProjection extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleProjection.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void project() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/project/test1.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while (exec.next()) {
       VectorUtil.showVectorAccessibleContent(exec.getIncoming(), "\t");
       final NullableBigIntVector c1 = exec.getValueVectorById(new SchemaPath("col1", ExpressionPosition.UNKNOWN), NullableBigIntVector.class);
       final NullableBigIntVector c2 = exec.getValueVectorById(new SchemaPath("col2", ExpressionPosition.UNKNOWN), NullableBigIntVector.class);
-      int x = 0;
       final NullableBigIntVector.Accessor a1 = c1.getAccessor();
       final NullableBigIntVector.Accessor a2 = c2.getAccessor();
 
@@ -74,13 +73,13 @@ public class TestSimpleProjection extends ExecTest {
         if (!a1.isNull(i)) {
           assertEquals(a1.get(i)+1, a2.get(i));
         }
-        x += a1.isNull(i) ? 0 : a1.get(i);
       }
     }
 
-    if (context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if (context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+
+    assertTrue(!context.getExecutorState().isFailed());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
index 98c997f..4492bde 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -45,23 +45,23 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Ignore
 @Category(OperatorTest.class)
 public class TestSimpleSort extends ExecTest {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleSort.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void sortOneKeyAscending(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void sortOneKeyAscending() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/sort/one_key_sort.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     int previousInt = Integer.MIN_VALUE;
@@ -86,20 +86,21 @@ public class TestSimpleSort extends ExecTest {
 
     System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 
   @Test
-  public void sortTwoKeysOneAscendingOneDescending(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void sortTwoKeysOneAscendingOneDescending() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/sort/two_key_sort.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     int previousInt = Integer.MIN_VALUE;
@@ -126,15 +127,14 @@ public class TestSimpleSort extends ExecTest {
         }
 
         assertTrue(previousLong >= a2.get(i));
-        //System.out.println(previousInt + "\t" + a2.get(i));
       }
     }
 
     System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
index 51d1e7b..9d0877b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
@@ -24,7 +24,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -40,8 +40,8 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 /*
  * This test uses a physical plan with the mock scan that generates 100k records.
@@ -53,17 +53,17 @@ import org.junit.experimental.categories.Category;
  */
 @Category(OperatorTest.class)
 public class TestTraceMultiRecordBatch extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testFilter() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/trace/multi_record_batch_trace.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while(exec.next()) {
@@ -74,9 +74,9 @@ public class TestTraceMultiRecordBatch extends ExecTest {
 
     exec.close();
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
index 780c83d..f3382a2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
@@ -26,7 +26,7 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.cache.VectorAccessibleSerializable;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -48,8 +48,8 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 /*
  * This test uses a simple physical plan with a mock-scan that
@@ -67,17 +67,17 @@ import org.junit.experimental.categories.Category;
  */
 @Category(OperatorTest.class)
 public class TestTraceOutputDump extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testFilter() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/trace/simple_trace.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while(exec.next()) {
@@ -85,10 +85,10 @@ public class TestTraceOutputDump extends ExecTest {
 
     exec.close();
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
 
     final FragmentHandle handle = context.getHandle();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
index 4f49355..49a42cc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -40,23 +40,23 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 // See also TestUnionDistinct for a test that does not need JMockit
 @Category(OperatorTest.class)
 public class TestSimpleUnion extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleUnion.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testUnion(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testUnion() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/union/test1.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     final int[] counts = new int[]{0, 100,50}; // first batch : 0-row schema-only batch.
@@ -66,9 +66,9 @@ public class TestSimpleUnion extends ExecTest {
       assertEquals(counts[i++], exec.getRecordCount());
     }
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
index 7c3c4cf..cdca30e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -40,7 +40,6 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.OperatorFixture;
-import org.apache.drill.test.OperatorFixture.OperatorFixtureBuilder;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.HyperRowSetImpl;
 import org.apache.drill.test.rowSet.IndirectRowSet;
@@ -190,7 +189,6 @@ public class TestSortImpl extends DrillTest {
    * row set format. For historical reasons, the sort dumps its output into a vector
    * container (normally attached to the external sort batch, here used stand-alone.)
    *
-   * @param fixture operator test fixture
    * @param results sort results iterator
    * @param dest container that holds the sort results
    * @return
@@ -582,7 +580,7 @@ public class TestSortImpl extends DrillTest {
 
   @Test
   public void testSpill() throws Exception {
-    OperatorFixtureBuilder builder = OperatorFixture.builder();
+    OperatorFixture.Builder builder = OperatorFixture.builder();
     builder.configBuilder()
       .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 2);
     try (OperatorFixture fixture = builder.build()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
index 754d858..e6e72e7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.physical.unit;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import mockit.NonStrictExpectations;
 import org.apache.drill.test.DrillTestWrapper;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -29,12 +28,12 @@ import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.parquet.ParquetDirectByteBufferAllocator;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.CodecFactory;
 import org.apache.parquet.hadoop.ParquetFileReader;
@@ -47,8 +46,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 
 import static org.apache.drill.exec.physical.base.AbstractBase.INIT_ALLOCATION;
 import static org.apache.drill.exec.physical.base.AbstractBase.MAX_ALLOCATION;
@@ -63,9 +60,6 @@ import static org.apache.drill.exec.physical.base.AbstractBase.MAX_ALLOCATION;
  */
 
 public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
-
-  private final ExecutorService scanExecutor =  Executors.newFixedThreadPool(2, new NamedThreadFactory("scan-"));
-
   public static class MiniPlanTestBuilder {
     protected List<Map<String, Object>> baselineRecords;
     protected RecordBatch root;
@@ -386,14 +380,12 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
 
     @Override
     public PopBuilder buildAddAsInput() throws Exception {
-      mockOpContext(popConfig, this.initReservation, this.maxAllocation);
       RecordBatch scanBatch = getScanBatch();
       return parent.addInput(scanBatch);
     }
 
     @Override
     public RecordBatch build() throws Exception {
-      mockOpContext(popConfig, this.initReservation, this.maxAllocation);
       return getScanBatch();
     }
 
@@ -411,7 +403,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
         readerList.add(readers.next());
       }
 
-      RecordBatch scanBatch = new ScanBatch(null, fragContext, readerList);
+      RecordBatch scanBatch = new ScanBatch(new MockPhysicalOperator(), fragContext, readerList);
       return scanBatch;
     }
   }
@@ -467,7 +459,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
         }
       }
 
-      RecordBatch scanBatch = new ScanBatch(null, fragContext, readers);
+      RecordBatch scanBatch = new ScanBatch(new MockPhysicalOperator(), fragContext, readers);
       return scanBatch;
     }
   } // end of ParquetScanBuilder
@@ -475,12 +467,5 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
   @Override
   protected void mockOpContext(PhysicalOperator popConfig, long initReservation, long maxAllocation) throws Exception {
     super.mockOpContext(popConfig, initReservation, maxAllocation);
-
-    // mock ScanExecutor used by parquet reader.
-    new NonStrictExpectations() {
-      {
-        opContext.getScanExecutor();result = scanExecutor;
-      }
-    };
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
index 088994f..b01dd3e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
@@ -21,15 +21,27 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import mockit.Delegate;
-import mockit.Injectable;
-import mockit.NonStrictExpectations;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
 import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.ops.AccountingDataTunnel;
+import org.apache.drill.exec.ops.AccountingUserConnection;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentStats;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.user.UserServer;
+import org.apache.drill.exec.server.QueryProfileStoreContext;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.easy.json.JSONRecordReader;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
 import org.apache.drill.test.DrillTestWrapper;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -45,67 +57,70 @@ import org.apache.drill.common.logical.data.Order;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.compile.CodeCompiler;
-import org.apache.drill.exec.compile.TemplateClassDefinition;
-import org.apache.drill.exec.exception.ClassTransformationException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.ClassGenerator;
-import org.apache.drill.exec.expr.CodeGenerator;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.RootAllocatorFactory;
-import org.apache.drill.exec.ops.BufferManagerImpl;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.base.AbstractBase;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ScanBatch;
-import org.apache.drill.exec.physical.impl.project.Projector;
-import org.apache.drill.exec.physical.impl.project.ProjectorTemplate;
-import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
-import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
+import org.apache.drill.test.OperatorFixture;
+import org.junit.After;
 import org.junit.Before;
+import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
-/**
- * Look! Doesn't extend BaseTestQuery!!
- */
 public class PhysicalOpUnitTestBase extends ExecTest {
-  @Injectable FragmentContext fragContext;
-  @Injectable DrillbitContext drillbitContext;
-  @Injectable OperatorContext opContext;
-  @Injectable OperatorStats opStats;
-  @Injectable PhysicalOperator popConf;
-  @Injectable ExecutionControls executionControls;
+  protected MockExecutorFragmentContext fragContext;
+  protected DrillbitContext drillbitContext;
+  protected OperatorFixture.MockOperatorContext opContext;
+  protected OperatorFixture operatorFixture;
+  protected ExecutorService scanExecutor;
+  protected ExecutorService scanDecodeExecutor;
 
   private final DrillConfig drillConf = DrillConfig.create();
-  private final BufferAllocator allocator = RootAllocatorFactory.newRoot(drillConf);
-  private final BufferManagerImpl bufManager = new BufferManagerImpl(allocator);
   private final ScanResult classpathScan = ClassPathScanner.fromPrescan(drillConf);
-  private final FunctionImplementationRegistry funcReg = new FunctionImplementationRegistry(drillConf, classpathScan);
-  private final TemplateClassDefinition<Projector> templateClassDefinition = new TemplateClassDefinition<Projector>(Projector.class, ProjectorTemplate.class);
   private final OperatorCreatorRegistry opCreatorReg = new OperatorCreatorRegistry(classpathScan);
 
   @Before
   public void setup() throws Exception {
+    scanExecutor = Executors.newFixedThreadPool(2, new NamedThreadFactory("scan-"));
+    scanDecodeExecutor = Executors.newFixedThreadPool(2, new NamedThreadFactory("scanDecoder-"));
+
+    drillbitContext = Mockito.mock(DrillbitContext.class);
+    Mockito.when(drillbitContext.getScanExecutor()).thenReturn(scanExecutor);
+    Mockito.when(drillbitContext.getScanDecodeExecutor()).thenReturn(scanDecodeExecutor);
+
+    final OperatorFixture.Builder builder = new OperatorFixture.Builder();
+    builder.configBuilder().configProps(drillConf);
+    operatorFixture = builder
+      .setScanExecutor(scanExecutor)
+      .setScanDecoderExecutor(scanDecodeExecutor)
+      .build();
     mockFragmentContext();
   }
 
+  @After
+  public void teardown() {
+    scanExecutor.shutdownNow();
+    scanDecodeExecutor.shutdownNow();
+  }
+
   @Override
   protected LogicalExpression parseExpr(String expr) {
     ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
@@ -207,7 +222,7 @@ public class PhysicalOpUnitTestBase extends ExecTest {
         List<RecordBatch> incomingStreams = Lists.newArrayList();
         if (inputStreamsJSON != null) {
           for (List<String> batchesJson : inputStreamsJSON) {
-            incomingStreams.add(new ScanBatch(null, fragContext,
+            incomingStreams.add(new ScanBatch(popConfig, fragContext,
                 getReaderListForJsonBatches(batchesJson, fragContext)));
           }
         }
@@ -222,8 +237,6 @@ public class PhysicalOpUnitTestBase extends ExecTest {
         throw new RuntimeException(e);
       } catch (UnsupportedEncodingException e) {
         throw new RuntimeException(e);
-      } catch (SchemaChangeException e) {
-        throw new RuntimeException(e);
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -285,69 +298,146 @@ public class PhysicalOpUnitTestBase extends ExecTest {
     }
   }
 
-  protected void mockFragmentContext() throws Exception{
-    final CodeCompiler compiler = new CodeCompiler(drillConf, optionManager);
-//    final BufferAllocator allocator = this.allocator.newChildAllocator("allocator_for_operator_test", initReservation, maxAllocation);
-    new NonStrictExpectations() {
-      {
-//        optManager.getOption(withAny(new TypeValidators.BooleanValidator("", false))); result = false;
-//        // TODO(DRILL-4450) - Probably want to just create a default option manager, this is a hack to prevent
-//        // the code compilation from failing when trying to decide of scalar replacement is turned on
-//        // this will cause other code paths to fail because this return value won't be valid for most
-//        // string options
-//        optManager.getOption(withAny(new TypeValidators.StringValidator("", "try"))); result = "try";
-//        optManager.getOption(withAny(new TypeValidators.PositiveLongValidator("", 1l, 1l))); result = 10;
-        drillbitContext.getCompiler(); result = new CodeCompiler(drillConf, optionManager);
-        fragContext.getOptions(); result = optionManager;
-        fragContext.getOptions(); result = optionManager;
-        fragContext.getManagedBuffer(); result = bufManager.getManagedBuffer();
-        fragContext.shouldContinue(); result = true;
-        fragContext.getExecutionControls(); result = executionControls;
-        fragContext.getFunctionRegistry(); result = funcReg;
-        fragContext.getConfig(); result = drillConf;
-        fragContext.getHandle(); result = ExecProtos.FragmentHandle.getDefaultInstance();
-        fragContext.getFunctionRegistry(); result = funcReg;
-        fragContext.getDrillbitContext(); result = drillbitContext;
-        try {
-          CodeGenerator<?> cg = CodeGenerator.get(templateClassDefinition);
-          cg.plainJavaCapable(true);
-          // cg.saveCodeForDebugging(true);
-          fragContext.getImplementationClass(withAny(cg));
-          result = new Delegate<Object>()
-          {
-            @SuppressWarnings("unused")
-            Object getImplementationClass(CodeGenerator<Object> gen) throws IOException, ClassTransformationException {
-              return compiler.createInstance(gen);
-            }
-          };
-          fragContext.getImplementationClass(withAny(CodeGenerator.get(templateClassDefinition).getRoot()));
-          result = new Delegate<Object>()
-          {
-            @SuppressWarnings("unused")
-            Object getImplementationClass(ClassGenerator<Object> gen) throws IOException, ClassTransformationException {
-              return compiler.createInstance(gen.getCodeGenerator());
-            }
-          };
-        } catch (ClassTransformationException e) {
-          throw new RuntimeException(e);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
+  /**
+   * <h2>Note</h2>
+   * <p>
+   *   The {@link MockExecutorFragmentContext} should only be used in {@link PhysicalOpUnitTestBase} because {@link PhysicalOpUnitTestBase}
+   *   needs a dummy {@link ExecutorFragmentContext} to be passed to batch creators. If you are unit testing operators and need a mock fragment context
+   *   please use {@link OperatorFixture.MockFragmentContext}.
+   * </p>
+   */
+  protected static class MockExecutorFragmentContext extends OperatorFixture.MockFragmentContext implements ExecutorFragmentContext {
+
+    public MockExecutorFragmentContext(final FragmentContext fragmentContext) {
+      super(fragmentContext.getConfig(), fragmentContext.getOptions(), fragmentContext.getAllocator(),
+        fragmentContext.getScanExecutor(), fragmentContext.getScanDecodeExecutor());
+    }
+
+    @Override
+    public PhysicalPlanReader getPlanReader() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ClusterCoordinator getClusterCoordinator() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CoordinationProtos.DrillbitEndpoint getForemanEndpoint() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CoordinationProtos.DrillbitEndpoint getEndpoint() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Collection<CoordinationProtos.DrillbitEndpoint> getBits() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public OperatorCreatorRegistry getOperatorCreatorRegistry() {
+      return null;
+    }
+
+    @Override
+    public void setBuffers(IncomingBuffers buffers) {
+    }
+
+    @Override
+    public QueryProfileStoreContext getProfileStoreContext() {
+      return null;
+    }
+
+    @Override
+    public WorkEventBus getWorkEventbus() {
+      return null;
+    }
+
+    @Override
+    public Set<Map.Entry<UserServer.BitToUserConnection, UserServer.BitToUserConnectionConfig>> getUserConnections() {
+      return null;
+    }
+
+    @Override
+    public void waitForSendComplete() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public AccountingDataTunnel getDataTunnel(CoordinationProtos.DrillbitEndpoint endpoint) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public AccountingUserConnection getUserDataTunnel() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Controller getController() {
+      return null;
+    }
+
+    @Override
+    public IncomingBuffers getBuffers() {
+      return null;
+    }
+
+    @Override
+    public FragmentStats getStats() {
+      return null;
+    }
+
+    @Override
+    public void setExecutorState(ExecutorState executorState) {
+    }
+
+    @Override
+    public boolean isUserAuthenticationEnabled() {
+      return false;
+    }
   }
 
-  protected void mockOpContext(final PhysicalOperator popConfig, long initReservation, long maxAllocation) throws Exception{
-    final BufferAllocator allocator = this.allocator.newChildAllocator("allocator_for_operator_test", initReservation, maxAllocation);
-    new NonStrictExpectations() {
-      {
-        opContext.getStats();result = opStats;
-        opContext.getAllocator(); result = allocator;
-        opContext.getFragmentContext(); result = fragContext;
-        opContext.getOperatorDefn(); result = popConfig;
-        fragContext.newOperatorContext(withAny(popConf)); result = opContext;
-      }
-    };
+  /**
+   * <h2>Note</h2>
+   * <p>
+   *   The {@link MockPhysicalOperator} should only be used in {@link PhysicalOpUnitTestBase} because {@link PhysicalOpUnitTestBase}
+   *   needs a dummy {@link MockPhysicalOperator} to be passed to Scanners.
+   * </p>
+   */
+  protected static class MockPhysicalOperator extends AbstractBase
+  {
+    @Override
+    public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+      return null;
+    }
+
+    @Override
+    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+      return null;
+    }
+
+    @Override
+    public int getOperatorType() {
+      return 0;
+    }
+
+    @Override
+    public Iterator<PhysicalOperator> iterator() {
+      return null;
+    }
+  }
+
+  protected void mockFragmentContext() throws Exception {
+    fragContext = new MockExecutorFragmentContext(operatorFixture.getFragmentContext());
+  }
+
+  protected void mockOpContext(final PhysicalOperator popConfig, long initReservation, long maxAllocation) throws Exception {
+    opContext = (OperatorFixture.MockOperatorContext)operatorFixture.operatorContext(popConfig);
   }
 
   protected OperatorCreatorRegistry getOpCreatorReg() {
@@ -367,7 +457,6 @@ public class PhysicalOpUnitTestBase extends ExecTest {
     return readerList;
   }
 
-
   /**
    * Create JSONRecordReader from input strings.
    * @param jsonBatches : list of input strings, each element represent a batch. Each string could either

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/planner/fragment/TestHardAffinityFragmentParallelizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/fragment/TestHardAffinityFragmentParallelizer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/fragment/TestHardAffinityFragmentParallelizer.java
index 9d94e9a..95363a7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/fragment/TestHardAffinityFragmentParallelizer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/fragment/TestHardAffinityFragmentParallelizer.java
@@ -19,8 +19,6 @@ package org.apache.drill.exec.planner.fragment;
 
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.ImmutableList;
-import mockit.Mocked;
-import mockit.NonStrictExpectations;
 import org.apache.drill.categories.PlannerTest;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -37,6 +35,8 @@ import static org.apache.drill.exec.planner.fragment.HardAffinityFragmentParalle
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 @Category(PlannerTest.class)
 public class TestHardAffinityFragmentParallelizer {
@@ -50,9 +50,6 @@ public class TestHardAffinityFragmentParallelizer {
   private static final DrillbitEndpoint N3_EP2 = newDrillbitEndpoint("node3", 30011);
   private static final DrillbitEndpoint N4_EP2 = newDrillbitEndpoint("node4", 30011);
 
-  @Mocked private Fragment fragment;
-  @Mocked private PhysicalOperator root;
-
   private static final DrillbitEndpoint newDrillbitEndpoint(String address, int port) {
     return DrillbitEndpoint.newBuilder().setAddress(address).setControlPort(port).build();
   }
@@ -87,11 +84,10 @@ public class TestHardAffinityFragmentParallelizer {
   }
 
   private final Wrapper newWrapper(double cost, int minWidth, int maxWidth, List<EndpointAffinity> epAffs) {
-    new NonStrictExpectations() {
-      {
-        fragment.getRoot(); result = root;
-      }
-    };
+    final Fragment fragment = mock(Fragment.class);
+    final PhysicalOperator root = mock(PhysicalOperator.class);
+
+    when(fragment.getRoot()).thenReturn(root);
 
     final Wrapper fragmentWrapper = new Wrapper(fragment, 1);
     final Stats stats = fragmentWrapper.getStats();

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
index 7f6929f..31c25a3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
@@ -20,10 +20,8 @@ package org.apache.drill.exec.record;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import mockit.Injectable;
-import mockit.Mock;
-import mockit.MockUp;
-import mockit.NonStrictExpectations;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import org.apache.drill.categories.VectorTest;
 import org.apache.drill.common.config.DrillConfig;
@@ -45,7 +43,6 @@ import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.fn.registry.RemoteFunctionRegistry;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
@@ -54,22 +51,18 @@ import org.junit.experimental.categories.Category;
 
 @Category(VectorTest.class)
 public class ExpressionTreeMaterializerTest extends ExecTest {
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializerTest.class);
-
-  final MajorType boolConstant = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BIT).build();
   final MajorType bigIntType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BIGINT).build();
-  final MajorType intType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.INT).build();
 
   DrillConfig c = DrillConfig.create();
   FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
 
-  private MaterializedField getField(int fieldId, String name, MajorType type) {
+  private MaterializedField getField(String name, MajorType type) {
     return MaterializedField.create(name, type);
   }
 
   @Test
-  public void testMaterializingConstantTree(@Injectable RecordBatch batch) throws SchemaChangeException {
+  public void testMaterializingConstantTree() throws SchemaChangeException {
+    final RecordBatch batch = mock(RecordBatch.class);
 
     ErrorCollector ec = new ErrorCollectorImpl();
     LogicalExpression expr = ExpressionTreeMaterializer.materialize(new ValueExpressions.LongExpression(1L,
@@ -80,18 +73,15 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
   }
 
   @Test
-  public void testMaterializingLateboundField(final @Injectable RecordBatch batch) throws SchemaChangeException {
+  public void testMaterializingLateboundField() throws SchemaChangeException {
+    final RecordBatch batch = mock(RecordBatch.class);
+    when(batch.getValueVectorId(new SchemaPath("test", ExpressionPosition.UNKNOWN)))
+      .thenReturn(new TypedFieldId(Types.required(MinorType.BIGINT), -5));
+
     final SchemaBuilder builder = BatchSchema.newBuilder();
-    builder.addField(getField(2, "test", bigIntType));
+    builder.addField(getField("test", bigIntType));
     final BatchSchema schema = builder.build();
 
-    new NonStrictExpectations() {
-      {
-        batch.getValueVectorId(new SchemaPath("test", ExpressionPosition.UNKNOWN));
-        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
-      }
-    };
-
     ErrorCollector ec = new ErrorCollectorImpl();
     LogicalExpression expr = ExpressionTreeMaterializer.materialize(new FieldReference("test",
         ExpressionPosition.UNKNOWN), batch, ec, registry);
@@ -100,18 +90,13 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
   }
 
   @Test
-  public void testMaterializingLateboundTree(final @Injectable RecordBatch batch) throws SchemaChangeException {
-    new NonStrictExpectations() {
-      {
-        batch.getValueVectorId(SchemaPath.getSimplePath("test"));
-        result = new TypedFieldId(Types.required(MinorType.BIT), -4);
-        batch.getValueVectorId(SchemaPath.getSimplePath("test1"));
-        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
-      }
-    };
+  public void testMaterializingLateboundTree() throws SchemaChangeException {
+    final RecordBatch batch = mock(RecordBatch.class);
 
-    ErrorCollector ec = new ErrorCollectorImpl();
+    when(batch.getValueVectorId(SchemaPath.getSimplePath("test"))).thenReturn(new TypedFieldId(Types.required(MinorType.BIT), -4));
+    when(batch.getValueVectorId(SchemaPath.getSimplePath("test1"))).thenReturn(new TypedFieldId(Types.required(MinorType.BIGINT), -5));
 
+    ErrorCollector ec = new ErrorCollectorImpl();
 
     LogicalExpression elseExpression = new IfExpression.Builder().setElse(new ValueExpressions.LongExpression(1L, ExpressionPosition.UNKNOWN))
         .setIfCondition(new IfExpression.IfCondition(new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN),
@@ -125,11 +110,8 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
     LogicalExpression newExpr = ExpressionTreeMaterializer.materialize(expr, batch, ec, registry);
     assertTrue(newExpr instanceof IfExpression);
     IfExpression newIfExpr = (IfExpression) newExpr;
-    //assertEquals(1, newIfExpr.conditions.size());
     IfExpression.IfCondition ifCondition = newIfExpr.ifCondition;
     assertTrue(newIfExpr.elseExpression instanceof IfExpression);
-    //assertEquals(1, newIfExpr.conditions.size());
-    //ifCondition = newIfExpr.conditions.get(0);
     assertEquals(bigIntType, ifCondition.expression.getMajorType());
     assertEquals(true, ((ValueExpressions.BooleanExpression) ((IfExpression)(newIfExpr.elseExpression)).ifCondition.condition).value);
     if (ec.hasErrors()) {
@@ -139,7 +121,12 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
   }
 
   @Test
-  public void testMaterializingLateboundTreeValidated(final @Injectable RecordBatch batch) throws SchemaChangeException {
+  public void testMaterializingLateboundTreeValidated() throws SchemaChangeException {
+    final RecordBatch batch = mock(RecordBatch.class);
+
+    when(batch.getValueVectorId(new SchemaPath("test", ExpressionPosition.UNKNOWN)))
+      .thenReturn(new TypedFieldId(Types.required(MinorType.BIGINT), -5));
+
     ErrorCollector ec = new ErrorCollector() {
       int errorCount = 0;
 
@@ -195,20 +182,6 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
       }
     };
 
-    new NonStrictExpectations() {
-      {
-        batch.getValueVectorId(new SchemaPath("test", ExpressionPosition.UNKNOWN));
-        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
-      }
-    };
-
-    new MockUp<RemoteFunctionRegistry>() {
-      @Mock
-      long getRegistryVersion() {
-        return 0L;
-      }
-    };
-
     LogicalExpression functionCallExpr = new FunctionCall("testFunc",
       ImmutableList.of((LogicalExpression) new FieldReference("test", ExpressionPosition.UNKNOWN) ),
       ExpressionPosition.UNKNOWN);
@@ -217,5 +190,4 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
     assertEquals(1, ec.getErrorCount());
     System.out.println(ec.toErrorString());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java
index c07dbaa..e88bb41 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java
@@ -22,13 +22,12 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.Lists;
-import mockit.Injectable;
 
 import org.apache.drill.categories.VectorTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.ops.OpProfileDef;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.ops.OperatorUtilities;
@@ -50,18 +49,18 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 import java.util.List;
 
 @Category(VectorTest.class)
 public class TestRecordIterator extends PopUnitTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRecordIterator.class);
   DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testSimpleIterator(@Injectable final DrillbitContext bitContext,
-                                  @Injectable UserClientConnection connection) throws Throwable{
-    mockDrillbitContext(bitContext);
+  public void testSimpleIterator() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
 
@@ -69,7 +68,7 @@ public class TestRecordIterator extends PopUnitTestBase {
 
     final PhysicalPlan plan = reader.readPhysicalPlan(planStr);
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, BitControl.PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), connection, registry);
     final List<PhysicalOperator> operatorList = plan.getSortedOperators(false);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) operatorList.iterator().next()));
 
@@ -116,17 +115,16 @@ public class TestRecordIterator extends PopUnitTestBase {
   }
 
   @Test
-  public void testMarkResetIterator(@Injectable final DrillbitContext bitContext,
-                                 @Injectable UserClientConnection connection) throws Throwable{
-    mockDrillbitContext(bitContext);
+  public void testMarkResetIterator() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
-
     final String planStr = Files.toString(DrillFileUtils.getResourceAsFile("/record/test_recorditerator.json"), Charsets.UTF_8);
 
     final PhysicalPlan plan = reader.readPhysicalPlan(planStr);
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    final FragmentContext context = new FragmentContext(bitContext, BitControl.PlanFragment.getDefaultInstance(), connection, registry);
+    final FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), connection, registry);
     final List<PhysicalOperator> operatorList = plan.getSortedOperators(false);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) operatorList.iterator().next()));
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
new file mode 100644
index 0000000..ef79780
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
@@ -0,0 +1,128 @@
+/*
+ * 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.record;
+
+import static org.junit.Assert.*;
+
+import org.apache.drill.categories.VectorTest;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(VectorTest.class)
+public class TestVectorContainer extends DrillTest {
+
+  // TODO: Replace the following with an extension of SubOperatorTest class
+  // once that is available.
+
+  protected volatile static OperatorFixture fixture;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    fixture = OperatorFixture.standardFixture();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    fixture.close();
+  }
+
+  /**
+   * Test of the ability to merge two schemas and to merge
+   * two vector containers. The merge is "horizontal", like
+   * a row-by-row join. Since each container is a list of
+   * vectors, we just combine the two lists to create the
+   * merged result.
+   */
+  @Test
+  public void testContainerMerge() {
+
+    // Simulated data from a reader
+
+    BatchSchema leftSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.VARCHAR)
+        .build();
+    SingleRowSet left = fixture.rowSetBuilder(leftSchema)
+        .addRow(10, "fred")
+        .addRow(20, "barney")
+        .addRow(30, "wilma")
+        .build();
+
+    // Simulated "implicit" coumns: row number and file name
+
+    BatchSchema rightSchema = new SchemaBuilder()
+        .add("x", MinorType.SMALLINT)
+        .add("y", MinorType.VARCHAR)
+        .build();
+    SingleRowSet right = fixture.rowSetBuilder(rightSchema)
+        .addRow(1, "foo.txt")
+        .addRow(2, "bar.txt")
+        .addRow(3, "dino.txt")
+        .build();
+
+    // The merge batch we expect to see
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.VARCHAR)
+        .add("x", MinorType.SMALLINT)
+        .add("y", MinorType.VARCHAR)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, "fred", 1, "foo.txt")
+        .addRow(20, "barney", 2, "bar.txt")
+        .addRow(30, "wilma", 3, "dino.txt")
+        .build();
+
+    // Merge containers without selection vector
+
+    RowSet merged = fixture.wrap(
+        left.container().merge(right.container()));
+
+    RowSetComparison comparison = new RowSetComparison(expected);
+    comparison.verify(merged);
+
+    // Merge containers via row set facade
+
+    RowSet mergedRs = DirectRowSet.fromContainer(left.container().merge(right.container()));
+    comparison.verifyAndClearAll(mergedRs);
+
+    // Add a selection vector. Merging is forbidden, in the present code,
+    // for batches that have a selection vector.
+
+    SingleRowSet leftIndirect = left.toIndirect();
+    try {
+      leftIndirect.container().merge(right.container());
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+    leftIndirect.clear();
+    right.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
index 834f108..0b00824 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
@@ -17,15 +17,13 @@
  */
 package org.apache.drill.exec.rpc.data;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigValueFactory;
 import io.netty.buffer.ByteBuf;
-import mockit.Injectable;
-import mockit.Mock;
-import mockit.MockUp;
-import mockit.NonStrictExpectations;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SecurityTest;
 import org.apache.drill.common.config.DrillConfig;
@@ -60,6 +58,7 @@ import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
@@ -68,6 +67,7 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -78,12 +78,12 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static junit.framework.TestCase.fail;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 @Ignore("See DRILL-5387")
 @Category(SecurityTest.class)
 public class TestBitBitKerberos extends BaseTestQuery {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitBitKerberos.class);
-
   private static KerberosHelper krbHelper;
   private static DrillConfig newConfig;
 
@@ -111,8 +111,7 @@ public class TestBitBitKerberos extends BaseTestQuery {
         .withValue(BootStrapContext.SERVICE_PRINCIPAL,
             ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
         .withValue(BootStrapContext.SERVICE_KEYTAB_LOCATION,
-            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())),
-        false);
+            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
 
     // Ignore the compile time warning caused by the code below.
 
@@ -134,40 +133,9 @@ public class TestBitBitKerberos extends BaseTestQuery {
   }
 
   private static void setupFragmentContextAndManager() {
-    final FragmentContext fcontext = new MockUp<FragmentContext>(){
-      @SuppressWarnings("unused")
-      BufferAllocator getAllocator(){
-        return c1.getAllocator();
-      }
-    }.getMockInstance();
-
-    manager = new MockUp<FragmentManager>(){
-      int v = 0;
-
-      @Mock
-      boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException {
-        try {
-          v++;
-          if (v % 10 == 0) {
-            System.out.println("sleeping.");
-            Thread.sleep(3000);
-          }
-        } catch (InterruptedException e) {
-
-        }
-        RawFragmentBatch rfb = batch.newRawFragmentBatch(c1.getAllocator());
-        rfb.sendOk();
-        rfb.release();
-
-        return true;
-      }
-
-      @SuppressWarnings("unused")
-      public FragmentContext getFragmentContext(){
-        return fcontext;
-      }
-
-    }.getMockInstance();
+    final FragmentContextImpl fcontext = mock(FragmentContextImpl.class);
+    when(fcontext.getAllocator()).thenReturn(c1.getAllocator());
+    manager = new MockFragmentManager(fcontext);
   }
 
   private static WritableBatch getRandomBatch(BufferAllocator allocator, int records) {
@@ -222,11 +190,11 @@ public class TestBitBitKerberos extends BaseTestQuery {
   }
 
   @Test
-  public void success(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus) throws Exception {
+  public void success() throws Exception {
+    final WorkerBee bee = mock(WorkerBee.class);
+    final WorkEventBus workBus = mock(WorkEventBus.class);
 
-    new NonStrictExpectations() {{
-      workBus.getFragmentManager( (FragmentHandle) any); result = manager;
-    }};
+    when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
 
     DataConnectionConfig config = new DataConnectionConfig(c1.getAllocator(), c1,
         new DataServerRequestHandler(workBus, bee));
@@ -242,7 +210,6 @@ public class TestBitBitKerberos extends BaseTestQuery {
       tunnel.sendRecordBatch(new TimingOutcome(max), new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1,
           1, 1, 1, getRandomBatch(c1.getAllocator(), 5000)));
       System.out.println(System.currentTimeMillis() - t1);
-      // System.out.println("sent.");
     }
     System.out.println(String.format("Max time: %d", max.get()));
     assertTrue(max.get() > 2700);
@@ -250,7 +217,11 @@ public class TestBitBitKerberos extends BaseTestQuery {
   }
 
   @Test
-  public void successEncryption(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus) throws Exception {
+  public void successEncryption() throws Exception {
+    final WorkerBee bee = mock(WorkerBee.class);
+    final WorkEventBus workBus = mock(WorkEventBus.class);
+
+    when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
 
     newConfig = new DrillConfig(
       config.withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
@@ -266,15 +237,10 @@ public class TestBitBitKerberos extends BaseTestQuery {
         .withValue(BootStrapContext.SERVICE_PRINCIPAL,
           ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
         .withValue(BootStrapContext.SERVICE_KEYTAB_LOCATION,
-          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())),
-      false);
+          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
 
     updateTestCluster(1, newConfig);
 
-    new NonStrictExpectations() {{
-      workBus.getFragmentManager( (FragmentHandle) any); result = manager;
-    }};
-
     DataConnectionConfig config = new DataConnectionConfig(c1.getAllocator(), c1,
       new DataServerRequestHandler(workBus, bee));
     DataServer server = new DataServer(config);
@@ -296,8 +262,14 @@ public class TestBitBitKerberos extends BaseTestQuery {
   }
 
   @Test
-  public void successEncryptionChunkMode(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus)
+  public void successEncryptionChunkMode()
     throws Exception {
+
+    final WorkerBee bee = mock(WorkerBee.class);
+    final WorkEventBus workBus = mock(WorkEventBus.class);
+
+    when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
+
     newConfig = new DrillConfig(
       config.withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
         ConfigValueFactory.fromIterable(Lists.newArrayList("kerberos")))
@@ -314,15 +286,10 @@ public class TestBitBitKerberos extends BaseTestQuery {
         .withValue(BootStrapContext.SERVICE_PRINCIPAL,
           ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
         .withValue(BootStrapContext.SERVICE_KEYTAB_LOCATION,
-          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())),
-      false);
+          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
 
     updateTestCluster(1, newConfig);
 
-    new NonStrictExpectations() {{
-      workBus.getFragmentManager( (FragmentHandle) any); result = manager;
-    }};
-
     DataConnectionConfig config = new DataConnectionConfig(c1.getAllocator(), c1,
       new DataServerRequestHandler(workBus, bee));
     DataServer server = new DataServer(config);
@@ -360,8 +327,7 @@ public class TestBitBitKerberos extends BaseTestQuery {
           .withValue(BootStrapContext.SERVICE_PRINCIPAL,
             ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
           .withValue(BootStrapContext.SERVICE_KEYTAB_LOCATION,
-            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())),
-        false);
+            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
 
       updateTestCluster(1, newConfig);
       fail();
@@ -403,8 +369,7 @@ public class TestBitBitKerberos extends BaseTestQuery {
         .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
             ConfigValueFactory.fromAnyRef("kerberos"))
         .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(false))
-        ,false);
+            ConfigValueFactory.fromAnyRef(false)));
 
     updateTestCluster(1, newConfig, connectionProps);
 
@@ -447,8 +412,7 @@ public class TestBitBitKerberos extends BaseTestQuery {
           .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
               ConfigValueFactory.fromAnyRef("kerberos"))
           .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-              ConfigValueFactory.fromAnyRef(false))
-          ,false);
+              ConfigValueFactory.fromAnyRef(false)));
 
       updateTestCluster(2, newConfig, connectionProps);
 
@@ -466,4 +430,72 @@ public class TestBitBitKerberos extends BaseTestQuery {
   public static void cleanTest() throws Exception {
     krbHelper.stopKdc();
   }
+
+  public static class MockFragmentManager implements FragmentManager
+  {
+    private int v = 0;
+    private final FragmentContextImpl fragmentContext;
+
+    public MockFragmentManager(final FragmentContextImpl fragmentContext)
+    {
+      this.fragmentContext = Preconditions.checkNotNull(fragmentContext);
+    }
+
+    @Override
+    public boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException {
+      try {
+        v++;
+        if (v % 10 == 0) {
+          System.out.println("sleeping.");
+          Thread.sleep(3000);
+        }
+      } catch (InterruptedException e) {
+
+      }
+      RawFragmentBatch rfb = batch.newRawFragmentBatch(c1.getAllocator());
+      rfb.sendOk();
+      rfb.release();
+
+      return true;
+    }
+
+    @Override
+    public FragmentExecutor getRunnable() {
+      return null;
+    }
+
+    @Override
+    public void cancel() {
+
+    }
+
+    @Override
+    public boolean isCancelled() {
+      return false;
+    }
+
+    @Override
+    public void unpause() {
+
+    }
+
+    @Override
+    public boolean isWaiting() {
+      return false;
+    }
+
+    @Override
+    public FragmentHandle getHandle() {
+      return null;
+    }
+
+    @Override
+    public FragmentContext getFragmentContext() {
+      return fragmentContext;
+    }
+
+    @Override
+    public void receivingFragmentFinished(FragmentHandle handle) {
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
index 1e8318f..bd3e60f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
@@ -20,10 +20,6 @@ package org.apache.drill.exec.rpc.data;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import io.netty.buffer.ByteBuf;
-import mockit.Injectable;
-import mockit.Mock;
-import mockit.MockUp;
-import mockit.NonStrictExpectations;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -33,6 +29,7 @@ import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
@@ -49,6 +46,7 @@ import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 import org.junit.Test;
 
@@ -58,55 +56,23 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestBitRpc extends ExecTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
-
   @Test
-  public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus) throws Exception {
-
-    DrillConfig config1 = DrillConfig.create();
+  public void testConnectionBackpressure() throws Exception {
+    final WorkerBee bee = mock(WorkerBee.class);
+    final WorkEventBus workBus = mock(WorkEventBus.class);
+    final DrillConfig config1 = DrillConfig.create();
     final BootStrapContext c = new BootStrapContext(config1, SystemOptionManager.createDefaultOptionDefinitions(), ClassPathScanner.fromPrescan(config1));
-    DrillConfig config2 = DrillConfig.create();
-    BootStrapContext c2 = new BootStrapContext(config2, SystemOptionManager.createDefaultOptionDefinitions(), ClassPathScanner.fromPrescan(config2));
+    final FragmentContextImpl fcon = mock(FragmentContextImpl.class);
+    when(fcon.getAllocator()).thenReturn(c.getAllocator());
 
-    final FragmentContext fcon = new MockUp<FragmentContext>(){
-      BufferAllocator getAllocator(){
-        return c.getAllocator();
-      }
-    }.getMockInstance();
-
-    final FragmentManager fman = new MockUp<FragmentManager>(){
-      int v = 0;
-
-      @Mock
-      boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException {
-        try {
-          v++;
-          if (v % 10 == 0) {
-            System.out.println("sleeping.");
-            Thread.sleep(3000);
-          }
-        } catch (InterruptedException e) {
+    final FragmentManager fman = new MockFragmentManager(c);
 
-        }
-        RawFragmentBatch rfb = batch.newRawFragmentBatch(c.getAllocator());
-        rfb.sendOk();
-        rfb.release();
-
-        return true;
-      }
-
-      public FragmentContext getFragmentContext(){
-        return fcon;
-      }
-
-    }.getMockInstance();
-
-
-    new NonStrictExpectations() {{
-      workBus.getFragmentManager( (FragmentHandle) any); result = fman;
-    }};
+    when(workBus.getFragmentManager(any(FragmentHandle.class))).thenReturn(fman);
 
     int port = 1234;
 
@@ -124,7 +90,6 @@ public class TestBitRpc extends ExecTest {
       tunnel.sendRecordBatch(new TimingOutcome(max), new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1,
           1, 1, 1, getRandomBatch(c.getAllocator(), 5000)));
       System.out.println(System.currentTimeMillis() - t1);
-      // System.out.println("sent.");
     }
     System.out.println(String.format("Max time: %d", max.get()));
     assertTrue(max.get() > 2700);
@@ -180,4 +145,72 @@ public class TestBitRpc extends ExecTest {
     }
   }
 
+  public static class MockFragmentManager implements FragmentManager
+  {
+    private final BootStrapContext c;
+    private int v;
+
+    public MockFragmentManager(BootStrapContext c)
+    {
+      this.c = c;
+    }
+
+    @Override
+    public boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException {
+      try {
+        v++;
+        if (v % 10 == 0) {
+          System.out.println("sleeping.");
+          Thread.sleep(3000);
+        }
+      } catch (InterruptedException e) {
+
+      }
+      RawFragmentBatch rfb = batch.newRawFragmentBatch(c.getAllocator());
+      rfb.sendOk();
+      rfb.release();
+
+      return true;
+    }
+
+    @Override
+    public FragmentExecutor getRunnable() {
+      return null;
+    }
+
+    @Override
+    public void cancel() {
+
+    }
+
+    @Override
+    public boolean isCancelled() {
+      return false;
+    }
+
+    @Override
+    public void unpause() {
+
+    }
+
+    @Override
+    public boolean isWaiting() {
+      return false;
+    }
+
+    @Override
+    public FragmentHandle getHandle() {
+      return null;
+    }
+
+    @Override
+    public FragmentContext getFragmentContext() {
+      return null;
+    }
+
+    @Override
+    public void receivingFragmentFinished(FragmentHandle handle) {
+
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
index 97360d3..1c017dc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
@@ -52,8 +52,7 @@ public class TestCustomUserAuthenticator extends BaseTestQuery {
             .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
                 ConfigValueFactory.fromAnyRef("true"))
             .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
-                ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)),
-        false);
+                ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)));
 
     final Properties connectionProps = new Properties();
     connectionProps.setProperty(DrillProperties.USER, "anonymous");

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
index 065746c..dbdbe3c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
@@ -71,8 +71,7 @@ public class TestUserBitKerberos extends BaseTestQuery {
       .withValue(BootStrapContext.SERVICE_KEYTAB_LOCATION,
         ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString()))
       .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-        ConfigValueFactory.fromIterable(Lists.newArrayList("plain", "kerberos"))),
-      false);
+        ConfigValueFactory.fromIterable(Lists.newArrayList("plain", "kerberos"))));
 
     final Properties connectionProps = new Properties();
     connectionProps.setProperty(DrillProperties.USER, "anonymous");