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:28 UTC

[04/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/TestComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
index b26e0cf..010acbd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.drill.common.config.DrillConfig;
 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.planner.PhysicalPlanReader;
@@ -36,19 +36,17 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 public class TestComparisonFunctions extends ExecTest {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestComparisonFunctions.class);
-
   private final DrillConfig c = DrillConfig.create();
   private final String COMPARISON_TEST_PHYSICAL_PLAN = "functions/comparisonTest.json";
   private PhysicalPlanReader reader;
   private FunctionImplementationRegistry registry;
 
-  public void runTest(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserClientConnection connection, String expression, int expectedResults) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void runTest(String expression, int expectedResults) throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);
     if (reader == null) {
@@ -57,92 +55,83 @@ public class TestComparisonFunctions extends ExecTest {
     if (registry == null) {
       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 PhysicalPlan plan = reader.readPhysicalPlan(planString);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while(exec.next()) {
       assertEquals(String.format("Expression: %s;", expression), expectedResults,
           exec.getSelectionVector2().getCount());
-//      for (ValueVector vv: exec) {
-//        vv.close();
-//      }
     }
 
     exec.close();
     context.close();
 
-    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 testInt(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "intColumn == intColumn", 100);
-    runTest(bitContext, connection, "intColumn != intColumn", 0);
-    runTest(bitContext, connection, "intColumn > intColumn", 0);
-    runTest(bitContext, connection, "intColumn < intColumn", 0);
-    runTest(bitContext, connection, "intColumn >= intColumn", 100);
-    runTest(bitContext, connection, "intColumn <= intColumn", 100);
+  public void testInt() throws Throwable {
+    runTest("intColumn == intColumn", 100);
+    runTest("intColumn != intColumn", 0);
+    runTest("intColumn > intColumn", 0);
+    runTest("intColumn < intColumn", 0);
+    runTest("intColumn >= intColumn", 100);
+    runTest("intColumn <= intColumn", 100);
   }
 
   @Test
-  public void testBigInt(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "bigIntColumn == bigIntColumn", 100);
-    runTest(bitContext, connection, "bigIntColumn != bigIntColumn", 0);
-    runTest(bitContext, connection, "bigIntColumn > bigIntColumn", 0);
-    runTest(bitContext, connection, "bigIntColumn < bigIntColumn", 0);
-    runTest(bitContext, connection, "bigIntColumn >= bigIntColumn", 100);
-    runTest(bitContext, connection, "bigIntColumn <= bigIntColumn", 100);
+  public void testBigInt() throws Throwable {
+    runTest("bigIntColumn == bigIntColumn", 100);
+    runTest("bigIntColumn != bigIntColumn", 0);
+    runTest("bigIntColumn > bigIntColumn", 0);
+    runTest("bigIntColumn < bigIntColumn", 0);
+    runTest("bigIntColumn >= bigIntColumn", 100);
+    runTest("bigIntColumn <= bigIntColumn", 100);
   }
 
   @Test
-  public void testFloat4(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "float4Column == float4Column", 100);
-    runTest(bitContext, connection, "float4Column != float4Column", 0);
-    runTest(bitContext, connection, "float4Column > float4Column", 0);
-    runTest(bitContext, connection, "float4Column < float4Column", 0);
-    runTest(bitContext, connection, "float4Column >= float4Column", 100);
-    runTest(bitContext, connection, "float4Column <= float4Column", 100);
+  public void testFloat4() throws Throwable {
+    runTest("float4Column == float4Column", 100);
+    runTest("float4Column != float4Column", 0);
+    runTest("float4Column > float4Column", 0);
+    runTest("float4Column < float4Column", 0);
+    runTest("float4Column >= float4Column", 100);
+    runTest("float4Column <= float4Column", 100);
   }
 
   @Test
-  public void testFloat8(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "float8Column == float8Column", 100);
-    runTest(bitContext, connection, "float8Column != float8Column", 0);
-    runTest(bitContext, connection, "float8Column > float8Column", 0);
-    runTest(bitContext, connection, "float8Column < float8Column", 0);
-    runTest(bitContext, connection, "float8Column >= float8Column", 100);
-    runTest(bitContext, connection, "float8Column <= float8Column", 100);
+  public void testFloat8() throws Throwable {
+    runTest("float8Column == float8Column", 100);
+    runTest("float8Column != float8Column", 0);
+    runTest("float8Column > float8Column", 0);
+    runTest("float8Column < float8Column", 0);
+    runTest("float8Column >= float8Column", 100);
+    runTest("float8Column <= float8Column", 100);
   }
 
   @Test
-  public void testIntNullable(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "intNullableColumn == intNullableColumn", 50);
-    runTest(bitContext, connection, "intNullableColumn != intNullableColumn", 0);
-    runTest(bitContext, connection, "intNullableColumn > intNullableColumn", 0);
-    runTest(bitContext, connection, "intNullableColumn < intNullableColumn", 0);
-    runTest(bitContext, connection, "intNullableColumn >= intNullableColumn", 50);
-    runTest(bitContext, connection, "intNullableColumn <= intNullableColumn", 50);
+  public void testIntNullable() throws Throwable {
+    runTest("intNullableColumn == intNullableColumn", 50);
+    runTest("intNullableColumn != intNullableColumn", 0);
+    runTest("intNullableColumn > intNullableColumn", 0);
+    runTest("intNullableColumn < intNullableColumn", 0);
+    runTest("intNullableColumn >= intNullableColumn", 50);
+    runTest("intNullableColumn <= intNullableColumn", 50);
   }
 
   @Test
-  public void testBigIntNullable(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserClientConnection connection) throws Throwable {
-    runTest(bitContext, connection, "bigIntNullableColumn == bigIntNullableColumn", 50);
-    runTest(bitContext, connection, "bigIntNullableColumn != bigIntNullableColumn", 0);
-    runTest(bitContext, connection, "bigIntNullableColumn > bigIntNullableColumn", 0);
-    runTest(bitContext, connection, "bigIntNullableColumn < bigIntNullableColumn", 0);
-    runTest(bitContext, connection, "bigIntNullableColumn >= bigIntNullableColumn", 50);
-    runTest(bitContext, connection, "bigIntNullableColumn <= bigIntNullableColumn", 50);
+  public void testBigIntNullable() throws Throwable {
+    runTest("bigIntNullableColumn == bigIntNullableColumn", 50);
+    runTest("bigIntNullableColumn != bigIntNullableColumn", 0);
+    runTest("bigIntNullableColumn > bigIntNullableColumn", 0);
+    runTest("bigIntNullableColumn < bigIntNullableColumn", 0);
+    runTest("bigIntNullableColumn >= bigIntNullableColumn", 50);
+    runTest("bigIntNullableColumn <= bigIntNullableColumn", 50);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index c1951a3..06d761f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -39,8 +39,6 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.apache.drill.exec.rpc.UserClientConnection;
-import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.util.ByteBufUtil.HadoopWritables;
 import org.apache.drill.exec.util.VectorUtil;
@@ -55,7 +53,6 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
 import io.netty.buffer.DrillBuf;
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
 
 @Category(UnlikelyTest.class)
@@ -478,32 +475,27 @@ public class TestConvertFunctions extends BaseTestQuery {
   }
 
   @Test
-  public void testFloats5(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testFloats5() throws Throwable {
     verifyPhysicalPlan("convert_from(convert_to(cast(77 as float8), 'DOUBLE'), 'DOUBLE')", 77.0);
   }
 
   @Test
-  public void testFloats5be(@Injectable final DrillbitContext bitContext,
-                          @Injectable UserClientConnection connection) throws Throwable {
+  public void testFloats5be() throws Throwable {
     verifyPhysicalPlan("convert_from(convert_to(cast(77 as float8), 'DOUBLE_BE'), 'DOUBLE_BE')", 77.0);
   }
 
   @Test
-  public void testFloats6(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testFloats6() throws Throwable {
     verifyPhysicalPlan("convert_to(cast(77 as float8), 'DOUBLE')", new byte[] {0, 0, 0, 0, 0, 64, 83, 64});
   }
 
   @Test
-  public void testFloats7(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testFloats7() throws Throwable {
     verifyPhysicalPlan("convert_to(4.9e-324, 'DOUBLE')", new byte[] {1, 0, 0, 0, 0, 0, 0, 0});
   }
 
   @Test
-  public void testFloats8(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testFloats8() throws Throwable {
     verifyPhysicalPlan("convert_to(1.7976931348623157e+308, 'DOUBLE')", new byte[] {-1, -1, -1, -1, -1, -1, -17, 127});
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
index c0c3aae..b40bee8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.drill.common.config.DrillConfig;
 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.planner.PhysicalPlanReader;
@@ -37,15 +37,13 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 public class TestImplicitCastFunctions extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class);
-
   private final DrillConfig c = DrillConfig.create();
   private PhysicalPlanReader reader;
   private FunctionImplementationRegistry registry;
-  private FragmentContext context;
+  private FragmentContextImpl context;
 
   public Object[] getRunResult(SimpleRootExec exec) {
     int size = 0;
@@ -61,10 +59,9 @@ public class TestImplicitCastFunctions extends ExecTest {
     return res;
  }
 
-  public void runTest(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserClientConnection connection, Object[] expectedResults, String planPath) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void runTest(Object[] expectedResults, String planPath) throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
     if (reader == null) {
@@ -74,7 +71,7 @@ public class TestImplicitCastFunctions extends ExecTest {
       registry = new FunctionImplementationRegistry(c);
     }
     if (context == null) {
-      context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+      context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     }
     final PhysicalPlan plan = reader.readPhysicalPlan(planString);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
@@ -90,16 +87,15 @@ public class TestImplicitCastFunctions extends ExecTest {
       }
     }
 
-    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 testImplicitCastWithConstant(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable{
+  public void testImplicitCastWithConstant() throws Throwable{
     final Object [] expected = new Object[21];
     expected [0] = new Double (30.1);
     expected [1] = new Double (30.1);
@@ -126,12 +122,11 @@ public class TestImplicitCastFunctions extends ExecTest {
     expected [19] = Boolean.TRUE;
     expected [20] = Boolean.TRUE;
 
-    runTest(bitContext, connection, expected, "functions/cast/testICastConstant.json");
+    runTest(expected, "functions/cast/testICastConstant.json");
   }
 
   @Test
-  public void testImplicitCastWithMockColumn(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable{
+  public void testImplicitCastWithMockColumn() throws Throwable{
     final Object [] expected = new Object[5];
     expected [0] = new Integer (0);
     expected [1] = new Integer (0);
@@ -139,12 +134,11 @@ public class TestImplicitCastFunctions extends ExecTest {
     expected [3] = new Float (-2.14748365E9);
     expected [4] = new Double (-9.223372036854776E18);
 
-    runTest(bitContext, connection, expected, "functions/cast/testICastMockCol.json");
+    runTest(expected, "functions/cast/testICastMockCol.json");
   }
 
   @Test
-  public void testImplicitCastWithNullExpression(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable{
+  public void testImplicitCastWithNullExpression() throws Throwable{
     final Object [] expected = new Object[10];
 
     expected [0] = Boolean.TRUE;
@@ -159,6 +153,6 @@ public class TestImplicitCastFunctions extends ExecTest {
     expected [8] = null;
     expected [9] = null;
 
-    runTest(bitContext, connection, expected, "functions/cast/testICastNullExp.json");
+    runTest(expected, "functions/cast/testICastNullExp.json");
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
deleted file mode 100644
index b6bc497..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ /dev/null
@@ -1,322 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl;
-
-import java.util.List;
-
-import mockit.Injectable;
-import mockit.NonStrictExpectations;
-
-import org.apache.drill.categories.PlannerTest;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.util.DrillFileUtils;
-import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.coord.ClusterCoordinator;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.holders.VarBinaryHolder;
-import org.apache.drill.exec.memory.RootAllocatorFactory;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.opt.BasicOptimizer;
-import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.control.Controller;
-import org.apache.drill.exec.rpc.control.WorkEventBus;
-import org.apache.drill.exec.rpc.data.DataConnectionCreator;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.apache.drill.exec.rpc.UserClientConnection;
-import org.apache.drill.exec.rpc.user.UserSession;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.StoragePluginRegistryImpl;
-import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VarBinaryVector;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import com.codahale.metrics.MetricRegistry;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import com.google.common.io.Resources;
-import org.junit.experimental.categories.Category;
-
-@Ignore
-@Category(PlannerTest.class)
-public class TestOptiqPlans extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class);
-  private final DrillConfig config = DrillConfig.create();
-
-  @Test
-  public void orderBy(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
-      @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
-      @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    final SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_order.json", coord, com, controller, workBus);
-  }
-
-  @Test
-  public void stringFilter(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
-      @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
-      @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    final SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_string_filter.json", coord, com, controller, workBus);
-  }
-
-  @Test
-  public void groupBy(@Injectable final BootStrapContext bitContext, @Injectable UserClientConnection connection,
-      @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
-      @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    final SimpleRootExec exec = doLogicalTest(bitContext, connection, "/logical_group.json", coord, com, controller, workBus);
-  }
-
-  private SimpleRootExec doLogicalTest(final BootStrapContext context, UserClientConnection connection, String file,
-      ClusterCoordinator coord, DataConnectionCreator com, Controller controller, WorkEventBus workBus) throws Exception {
-    new NonStrictExpectations() {
-      {
-        context.getMetrics();
-        result = new MetricRegistry();
-        context.getAllocator();
-        result = RootAllocatorFactory.newRoot(config);
-        context.getConfig();
-        result = config;
-      }
-    };
-    final RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet();
-    final DrillbitContext bitContext = new DrillbitContext(
-        DrillbitEndpoint.getDefaultInstance(),
-        context,
-        coord,
-        controller,
-        com,
-        workBus,
-        new LocalPersistentStoreProvider(config));
-    final QueryContext qc = new QueryContext(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build(),
-        bitContext, QueryId.getDefaultInstance());
-    final PhysicalPlanReader reader = bitContext.getPlanReader();
-    final LogicalPlan plan = reader.readLogicalPlan(Files.toString(DrillFileUtils.getResourceAsFile(file), Charsets.UTF_8));
-    final PhysicalPlan pp = new BasicOptimizer(qc, connection).optimize(new BasicOptimizer.BasicOptimizationContext(qc), plan);
-
-    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
-    final FragmentContext fctxt = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(fctxt, (FragmentRoot) pp.getSortedOperators(false)
-        .iterator().next()));
-    return exec;
-
-  }
-
-  @Test
-  public void testFilterPlan() throws Exception {
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
-        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
-      bit1.run();
-      client.connect();
-      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-          Resources.toString(Resources.getResource("physical_filter.json"), Charsets.UTF_8));
-      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (final QueryDataBatch b : results) {
-        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
-        loader.load(b.getHeader().getDef(), b.getData());
-        for (final VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
-          final ValueVector vv = vw.getValueVector();
-          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            final Object o = vv.getAccessor().getObject(i);
-            System.out.println(o);
-          }
-        }
-        loader.clear();
-        b.release();
-      }
-      client.close();
-    }
-  }
-
-  @Test
-  public void testJoinPlan() throws Exception {
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
-        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
-      bit1.run();
-      client.connect();
-      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-          Resources.toString(Resources.getResource("physical_join.json"), Charsets.UTF_8));
-      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (final QueryDataBatch b : results) {
-        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
-        loader.load(b.getHeader().getDef(), b.getData());
-        for (final VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
-          final ValueVector vv = vw.getValueVector();
-          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            final Object o = vv.getAccessor().getObject(i);
-            System.out.println(o);
-          }
-        }
-        loader.clear();
-        b.release();
-      }
-      client.close();
-    }
-  }
-
-  @Test
-  public void testFilterString() throws Exception {
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
-        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
-      bit1.run();
-      client.connect();
-      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
-          Resources.toString(Resources.getResource("logical_string_filter.json"), Charsets.UTF_8));
-      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (final QueryDataBatch b : results) {
-        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
-        loader.load(b.getHeader().getDef(), b.getData());
-        for (final VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
-          final ValueVector vv = vw.getValueVector();
-          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            final Object o = vv.getAccessor().getObject(i);
-            if (vv instanceof VarBinaryVector) {
-              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              final VarBinaryHolder vbh = new VarBinaryHolder();
-              x.get(i, vbh);
-              System.out.printf("%d..%d", vbh.start, vbh.end);
-
-              System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
-            } else {
-              System.out.println(vv.getAccessor().getObject(i));
-            }
-
-          }
-        }
-        loader.clear();
-        b.release();
-      }
-      client.close();
-    }
-  }
-
-  @Test
-  public void testLogicalJsonScan() throws Exception {
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
-        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
-      bit1.run();
-      client.connect();
-      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
-          Resources.toString(Resources.getResource("logical_json_scan.json"), Charsets.UTF_8));
-      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (final QueryDataBatch b : results) {
-        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
-        loader.load(b.getHeader().getDef(), b.getData());
-        for (final VectorWrapper vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
-          final ValueVector vv = vw.getValueVector();
-          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            final Object o = vv.getAccessor().getObject(i);
-            if (vv instanceof VarBinaryVector) {
-              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              final VarBinaryHolder vbh = new VarBinaryHolder();
-              x.get(i, vbh);
-              System.out.printf("%d..%d", vbh.start, vbh.end);
-
-              System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
-            } else {
-              System.out.println(vv.getAccessor().getObject(i));
-            }
-
-          }
-        }
-        loader.clear();
-        b.release();
-      }
-      client.close();
-    }
-  }
-
-  @Test
-  public void testOrderVarbinary() throws Exception {
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
-        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
-      bit1.run();
-      client.connect();
-      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-          Resources.toString(Resources.getResource("physical_order_varbinary.json"), Charsets.UTF_8));
-      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (final QueryDataBatch b : results) {
-        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
-        loader.load(b.getHeader().getDef(), b.getData());
-        for (final VectorWrapper vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
-          final ValueVector vv = vw.getValueVector();
-          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            final Object o = vv.getAccessor().getObject(i);
-            if (vv instanceof VarBinaryVector) {
-              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              final VarBinaryHolder vbh = new VarBinaryHolder();
-              x.get(i, vbh);
-              System.out.printf("%d..%d", vbh.start, vbh.end);
-
-              System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
-            } else {
-              System.out.println(vv.getAccessor().getObject(i));
-            }
-
-          }
-        }
-        loader.clear();
-        b.release();
-      }
-      client.close();
-    }
-  }
-
-  private SimpleRootExec doPhysicalTest(final DrillbitContext bitContext, UserClientConnection connection, String file)
-      throws Exception {
-    mockDrillbitContext(bitContext);
-
-    final StoragePluginRegistry reg = new StoragePluginRegistryImpl(bitContext);
-
-    final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(config, reg);
-    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile(file), Charsets.UTF_8));
-    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
-    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false)
-        .iterator().next()));
-    return exec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
index 84d68f5..36cbfc4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
@@ -23,17 +23,13 @@ import static org.junit.Assert.assertTrue;
 import java.util.Iterator;
 import java.util.List;
 
-import mockit.Injectable;
-
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.apache.drill.exec.rpc.UserClientConnection;
 import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
@@ -44,8 +40,7 @@ import com.google.common.io.Files;
 public class TestReverseImplicitCast extends PopUnitTestBase {
 
   @Test
-  public void twoWayCast(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserClientConnection connection) throws Throwable {
+  public void twoWayCast() throws Throwable {
 
     // Function checks for casting from Float, Double to Decimal data types
     try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
index 1f83fc4..192c1ad 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -40,7 +39,7 @@ import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers;
 import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
-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.planner.PhysicalPlanReader;
@@ -58,7 +57,7 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import com.sun.codemodel.JClassAlreadyExistsException;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 public class TestSimpleFunctions extends ExecTest {
 
@@ -67,69 +66,44 @@ public class TestSimpleFunctions extends ExecTest {
     @SuppressWarnings("resource")
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     // test required vs nullable Int input
-    resolveHash(c,
-        new TypedNullConstant(Types.optional(TypeProtos.MinorType.INT)),
-        Types.optional(TypeProtos.MinorType.INT),
-        Types.required(TypeProtos.MinorType.INT),
+    resolveHash(new TypedNullConstant(Types.optional(TypeProtos.MinorType.INT)),
         TypeProtos.DataMode.OPTIONAL,
         registry);
 
-    resolveHash(c,
-        new ValueExpressions.IntExpression(1, ExpressionPosition.UNKNOWN),
-        Types.required(TypeProtos.MinorType.INT),
-        Types.required(TypeProtos.MinorType.INT),
+    resolveHash(new ValueExpressions.IntExpression(1, ExpressionPosition.UNKNOWN),
         TypeProtos.DataMode.REQUIRED,
         registry);
 
     // test required vs nullable float input
-    resolveHash(c,
-        new TypedNullConstant(Types.optional(TypeProtos.MinorType.FLOAT4)),
-        Types.optional(TypeProtos.MinorType.FLOAT4),
-        Types.required(TypeProtos.MinorType.FLOAT4),
+    resolveHash(new TypedNullConstant(Types.optional(TypeProtos.MinorType.FLOAT4)),
         TypeProtos.DataMode.OPTIONAL,
         registry);
 
-    resolveHash(c,
-        new ValueExpressions.FloatExpression(5.0f, ExpressionPosition.UNKNOWN),
-        Types.required(TypeProtos.MinorType.FLOAT4),
-        Types.required(TypeProtos.MinorType.FLOAT4),
+    resolveHash(new ValueExpressions.FloatExpression(5.0f, ExpressionPosition.UNKNOWN),
         TypeProtos.DataMode.REQUIRED,
         registry);
 
     // test required vs nullable long input
-    resolveHash(c,
-        new TypedNullConstant(Types.optional(TypeProtos.MinorType.BIGINT)),
-        Types.optional(TypeProtos.MinorType.BIGINT),
-        Types.required(TypeProtos.MinorType.BIGINT),
+    resolveHash(new TypedNullConstant(Types.optional(TypeProtos.MinorType.BIGINT)),
         TypeProtos.DataMode.OPTIONAL,
         registry);
 
-    resolveHash(c,
-        new ValueExpressions.LongExpression(100L, ExpressionPosition.UNKNOWN),
-        Types.required(TypeProtos.MinorType.BIGINT),
-        Types.required(TypeProtos.MinorType.BIGINT),
+    resolveHash(new ValueExpressions.LongExpression(100L, ExpressionPosition.UNKNOWN),
         TypeProtos.DataMode.REQUIRED,
         registry);
 
     // test required vs nullable double input
-    resolveHash(c,
-        new TypedNullConstant(Types.optional(TypeProtos.MinorType.FLOAT8)),
-        Types.optional(TypeProtos.MinorType.FLOAT8),
-        Types.required(TypeProtos.MinorType.FLOAT8),
+    resolveHash(new TypedNullConstant(Types.optional(TypeProtos.MinorType.FLOAT8)),
         TypeProtos.DataMode.OPTIONAL,
         registry);
 
-    resolveHash(c,
-        new ValueExpressions.DoubleExpression(100.0, ExpressionPosition.UNKNOWN),
-        Types.required(TypeProtos.MinorType.FLOAT8),
-        Types.required(TypeProtos.MinorType.FLOAT8),
+    resolveHash(new ValueExpressions.DoubleExpression(100.0, ExpressionPosition.UNKNOWN),
         TypeProtos.DataMode.REQUIRED,
         registry);
   }
 
-  public void resolveHash(DrillConfig config, LogicalExpression arg, TypeProtos.MajorType expectedArg,
-                                    TypeProtos.MajorType expectedOut, TypeProtos.DataMode expectedBestInputMode,
-                                    FunctionImplementationRegistry registry) throws JClassAlreadyExistsException, IOException {
+  public void resolveHash(LogicalExpression arg, TypeProtos.DataMode expectedBestInputMode,
+                          FunctionImplementationRegistry registry) throws JClassAlreadyExistsException, IOException {
     final List<LogicalExpression> args = new ArrayList<>();
     args.add(arg);
     FunctionCall call = new FunctionCall(
@@ -143,14 +117,14 @@ public class TestSimpleFunctions extends ExecTest {
   }
 
   @Test
-  public void testSubstring(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testSubstring() 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("/functions/testSubstring.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()) {
@@ -169,21 +143,21 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    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 testSubstringNegative(@Injectable final DrillbitContext bitContext,
-                                    @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testSubstringNegative() 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("/functions/testSubstringNegative.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()) {
@@ -203,21 +177,21 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    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 testByteSubstring(@Injectable final DrillbitContext bitContext,
-                                  @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testByteSubstring() 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("/functions/testByteSubstring.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()) {
@@ -236,9 +210,9 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    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/TestStringFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java
index 84d837b..27ef5e1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java
@@ -24,7 +24,7 @@ import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.config.DrillConfig;
 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.planner.PhysicalPlanReader;
@@ -39,16 +39,15 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Category({UnlikelyTest.class})
 public class TestStringFunctions extends ExecTest {
-
   private final DrillConfig c = DrillConfig.create();
   private PhysicalPlanReader reader;
   private FunctionImplementationRegistry registry;
-  private FragmentContext context;
+  private FragmentContextImpl context;
 
   public Object[] getRunResult(SimpleRootExec exec) {
     int size = 0;
@@ -68,10 +67,9 @@ public class TestStringFunctions extends ExecTest {
     return res;
  }
 
-  public void runTest(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserClientConnection connection, Object[] expectedResults, String planPath) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void runTest(Object[] expectedResults, String planPath) throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
     if (reader == null) {
@@ -81,7 +79,7 @@ public class TestStringFunctions extends ExecTest {
       registry = new FunctionImplementationRegistry(c);
     }
     if (context == null) {
-      context =  new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry); //new FragmentContext(bitContext, ExecProtos.FragmentHandle.getInstance(), connection, registry);
+      context =  new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     }
     final PhysicalPlan plan = reader.readPhysicalPlan(planString);
     final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
@@ -96,138 +94,121 @@ public class TestStringFunctions extends ExecTest {
       }
     }
 
-    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 testCharLength(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testCharLength() throws Throwable {
     Object [] expected = new Object[] {new Long(8), new Long(0), new Long(5), new Long(5),
                                        new Long(8), new Long(0), new Long(5), new Long(5),
                                        new Long(8), new Long(0), new Long(5), new Long(5),};
-    runTest(bitContext, connection, expected, "functions/string/testCharLength.json");
+    runTest(expected, "functions/string/testCharLength.json");
   }
 
   @Test
-  public void testLike(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testLike() throws Throwable {
     final Object [] expected = new Object[] {Boolean.TRUE, Boolean.TRUE, Boolean.TRUE, Boolean.FALSE};
-    runTest(bitContext, connection, expected, "functions/string/testLike.json");
+    runTest(expected, "functions/string/testLike.json");
   }
 
   @Test
-  public void testSimilar(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testSimilar() throws Throwable {
     final Object [] expected = new Object[] {Boolean.TRUE, Boolean.FALSE, Boolean.TRUE, Boolean.FALSE};
-    runTest(bitContext, connection, expected, "functions/string/testSimilar.json");
+    runTest(expected, "functions/string/testSimilar.json");
   }
 
   @Test
-  public void testLtrim(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testLtrim() throws Throwable {
     final Object [] expected = new Object[] {"def", "abcdef", "dabc", "", "", ""};
-    runTest(bitContext, connection, expected, "functions/string/testLtrim.json");
+    runTest(expected, "functions/string/testLtrim.json");
   }
 
   @Test
-  public void testTrim(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testTrim() throws Throwable {
     final Object [] expected = new Object[] {"fghI", "", "", "!", " aaa "};
-    runTest(bitContext, connection, expected, "functions/string/testTrim.json");
+    runTest(expected, "functions/string/testTrim.json");
   }
 
   @Test
-  public void testReplace(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testReplace() throws Throwable {
     final Object [] expected = new Object[] {"aABABcdf", "ABABbABbcdf", "aababcdf", "acdf", "ABCD", "abc"};
-    runTest(bitContext, connection, expected, "functions/string/testReplace.json");
+    runTest(expected, "functions/string/testReplace.json");
   }
 
   @Test
-  public void testRtrim(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testRtrim() throws Throwable {
     final Object [] expected = new Object[] {"abc", "abcdef", "ABd", "", "", ""};
-    runTest(bitContext, connection, expected, "functions/string/testRtrim.json");
+    runTest(expected, "functions/string/testRtrim.json");
   }
 
   @Test
-  public void testConcat(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testConcat() throws Throwable {
     final Object [] expected = new Object[] {"abcABC", "abc", "ABC", ""};
-    runTest(bitContext, connection, expected, "functions/string/testConcat.json");
+    runTest(expected, "functions/string/testConcat.json");
   }
 
   @Test
-  public void testLower(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testLower() throws Throwable {
     final Object [] expected = new Object[] {"abcefgh", "abc", ""};
-    runTest(bitContext, connection, expected, "functions/string/testLower.json");
+    runTest(expected, "functions/string/testLower.json");
   }
 
   @Test
-  public void testPosition(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testPosition() throws Throwable {
     final Object [] expected = new Object[] {new Long(2), new Long(0), new Long(0), new Long(0),
                                        new Long(2), new Long(0), new Long(0), new Long(0)};
-    runTest(bitContext, connection, expected, "functions/string/testPosition.json");
+    runTest(expected, "functions/string/testPosition.json");
   }
 
   @Test
-  public void testRight(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testRight() throws Throwable {
     final Object [] expected = new Object[] {"ef", "abcdef", "abcdef", "cdef", "f", "", ""};
-    runTest(bitContext, connection, expected, "functions/string/testRight.json");
+    runTest(expected, "functions/string/testRight.json");
   }
 
 
   @Test
-  public void testSubstr(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testSubstr() throws Throwable {
     final Object [] expected = new Object[] {"abc", "bcd", "bcdef", "bcdef", "", "", "", "", "भारत", "वर्ष", "वर्ष", "cdef", "", "", "", "ड्रिल"};
-    runTest(bitContext, connection, expected, "functions/string/testSubstr.json");
+    runTest(expected, "functions/string/testSubstr.json");
   }
 
   @Test
-  public void testLeft(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testLeft() throws Throwable {
     final Object [] expected = new Object[] {"ab", "abcdef", "abcdef", "abcd", "a", "", ""};
-    runTest(bitContext, connection, expected, "functions/string/testLeft.json");
+    runTest(expected, "functions/string/testLeft.json");
   }
 
   @Test
-  public void testLpad(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testLpad() throws Throwable {
     final Object [] expected = new Object[] {"", "", "abcdef", "ab", "ab", "abcdef", "AAAAabcdef", "ABABabcdef", "ABCAabcdef", "ABCDabcdef"};
-    runTest(bitContext, connection, expected, "functions/string/testLpad.json");
+    runTest(expected, "functions/string/testLpad.json");
   }
 
   @Test
-  public void testRegexpReplace(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testRegexpReplace() throws Throwable {
     final Object [] expected = new Object[] {"ThM", "Th", "Thomas"};
-    runTest(bitContext, connection, expected, "functions/string/testRegexpReplace.json");
+    runTest(expected, "functions/string/testRegexpReplace.json");
   }
 
   @Test
-  public void testRpad(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testRpad() throws Throwable {
     final Object [] expected = new Object[] {"", "", "abcdef", "ab", "ab", "abcdef", "abcdefAAAA", "abcdefABAB", "abcdefABCA", "abcdefABCD"};
-    runTest(bitContext, connection, expected, "functions/string/testRpad.json");
+    runTest(expected, "functions/string/testRpad.json");
   }
 
   @Test
-  public void testUpper(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testUpper() throws Throwable {
     final Object [] expected = new Object[] {"ABCEFGH", "ABC", ""};
-    runTest(bitContext, connection, expected, "functions/string/testUpper.json");
+    runTest(expected, "functions/string/testUpper.json");
   }
 
   @Test
-  public void testNewStringFuncs(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
+  public void testNewStringFuncs() throws Throwable {
     final Object [] expected = new Object[] {97, 65, -32, "A", "btrim", "Peace Peace Peace ", "हकुना मताता हकुना मताता ", "katcit", "\u00C3\u00A2pple", "नदम"};
-    runTest(bitContext, connection, expected, "functions/string/testStringFuncs.json");
+    runTest(expected, "functions/string/testStringFuncs.json");
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
index f07352c..e4b1376 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
@@ -26,7 +26,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 TestAgg extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAgg.class);
   private final DrillConfig c = DrillConfig.create();
 
-  private SimpleRootExec doTest(final DrillbitContext bitContext, UserClientConnection connection, String file) throws Exception {
-
-    mockDrillbitContext(bitContext);
+  private SimpleRootExec doTest(String file) throws Exception {
+    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(file), 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()));
     return exec;
   }
 
   @Test
-  public void oneKeyAgg(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    final SimpleRootExec exec = doTest(bitContext, connection, "/agg/test1.json");
+  public void oneKeyAgg() throws Throwable {
+    final SimpleRootExec exec = doTest("/agg/test1.json");
 
     while(exec.next()) {
       final BigIntVector cnt = exec.getValueVectorById(SchemaPath.getSimplePath("cnt"), BigIntVector.class);
@@ -80,16 +79,16 @@ public class TestAgg extends ExecTest {
       }
     }
 
-    if(exec.getContext().getFailureCause() != null) {
-      throw exec.getContext().getFailureCause();
+    if(exec.getContext().getExecutorState().getFailureCause() != null) {
+      throw exec.getContext().getExecutorState().getFailureCause();
     }
-    assertTrue(!exec.getContext().isFailed());
 
+    assertTrue(!exec.getContext().getExecutorState().isFailed());
   }
 
   @Test
-  public void twoKeyAgg(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    SimpleRootExec exec = doTest(bitContext, connection, "/agg/twokey.json");
+  public void twoKeyAgg() throws Throwable {
+    SimpleRootExec exec = doTest("/agg/twokey.json");
 
     while(exec.next()) {
       final IntVector key1 = exec.getValueVectorById(SchemaPath.getSimplePath("key1"), IntVector.class);
@@ -102,14 +101,6 @@ public class TestAgg extends ExecTest {
       final long[] totalArr = {0,34,68,0,34,68};
 
       for(int i = 0; i < exec.getRecordCount(); i++) {
-//        System.out.print(key1.getAccessor().getObject(i));
-//        System.out.print("\t");
-//        System.out.print(key2.getAccessor().getObject(i));
-//        System.out.print("\t");
-//        System.out.print(cnt.getAccessor().getObject(i));
-//        System.out.print("\t");
-//        System.out.print(total.getAccessor().getObject(i));
-//        System.out.println();
         assertEquals((Long) cntArr[i], cnt.getAccessor().getObject(i));
         assertEquals(keyArr1[i], key1.getAccessor().getObject(i));
         assertEquals((Long) keyArr2[i], key2.getAccessor().getObject(i));
@@ -117,9 +108,9 @@ public class TestAgg extends ExecTest {
       }
     }
 
-    if(exec.getContext().getFailureCause() != null){
-      throw exec.getContext().getFailureCause();
+    if(exec.getContext().getExecutorState().getFailureCause() != null){
+      throw exec.getContext().getExecutorState().getFailureCause();
     }
-    assertTrue(!exec.getContext().isFailed());
+    assertTrue(!exec.getContext().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/filter/TestSimpleFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
index 750276c..354343e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.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;
@@ -41,23 +41,22 @@ 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 TestSimpleFilter extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFilter.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-//    System.out.println(System.getProperty("java.class.path"));
-    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("/filter/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()) {
       assertEquals(50, exec.getRecordCount());
@@ -65,21 +64,22 @@ public class TestSimpleFilter 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());
   }
 
   @Test
   @Ignore ("Filter does not support SV4")
-  public void testSV4Filter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testSV4Filter() 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("/filter/test_sv4.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 recordCount = 0;
     while(exec.next()) {
@@ -91,9 +91,9 @@ public class TestSimpleFilter extends ExecTest {
     exec.close();
     assertEquals(50, recordCount);
 
-    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/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
index cee84dc..1abd4e5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
@@ -27,10 +27,10 @@ import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.test.TestTools;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -55,24 +55,24 @@ import org.junit.rules.TestRule;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 @Category({SlowTest.class, OperatorTest.class})
 public class TestHashJoin extends PopUnitTestBase {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class);
-
   @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(100000);
 
   private final DrillConfig c = DrillConfig.create();
 
-  private void testHJMockScanCommon(final DrillbitContext bitContext, UserClientConnection connection, String physicalPlan, int expectedRows) throws Throwable {
+  private void testHJMockScanCommon(String physicalPlan, int expectedRows) throws Throwable {
+
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
-    mockDrillbitContext(bitContext);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile(physicalPlan), 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 totalRecordCount = 0;
@@ -82,31 +82,28 @@ public class TestHashJoin extends PopUnitTestBase {
     exec.close();
     assertEquals(expectedRows, totalRecordCount);
     System.out.println("Total Record Count: " + totalRecordCount);
-    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 multiBatchEqualityJoin(@Injectable final DrillbitContext bitContext,
-                                 @Injectable UserClientConnection connection) throws Throwable {
+  public void multiBatchEqualityJoin() throws Throwable {
 
-    testHJMockScanCommon(bitContext, connection, "/join/hash_join_multi_batch.json", 200000);
+    testHJMockScanCommon("/join/hash_join_multi_batch.json", 200000);
   }
 
   @Test
-  public void multiBatchRightOuterJoin(@Injectable final DrillbitContext bitContext,
-                                       @Injectable UserClientConnection connection) throws Throwable {
+  public void multiBatchRightOuterJoin() throws Throwable {
 
-    testHJMockScanCommon(bitContext, connection, "/join/hj_right_outer_multi_batch.json", 100000);
+    testHJMockScanCommon("/join/hj_right_outer_multi_batch.json", 100000);
   }
 
   @Test
-  public void multiBatchLeftOuterJoin(@Injectable final DrillbitContext bitContext,
-                                      @Injectable UserClientConnection connection) throws Throwable {
+  public void multiBatchLeftOuterJoin() throws Throwable {
 
-    testHJMockScanCommon(bitContext, connection, "/join/hj_left_outer_multi_batch.json", 100000);
+    testHJMockScanCommon("/join/hj_left_outer_multi_batch.json", 100000);
   }
 
   @Test
@@ -151,9 +148,7 @@ public class TestHashJoin extends PopUnitTestBase {
   }
 
   @Test
-  public void hjWithExchange(@Injectable final DrillbitContext bitContext,
-                             @Injectable UserClientConnection connection) throws Throwable {
-
+  public void hjWithExchange() throws Throwable {
     // Function tests with hash join with exchanges
     try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
       final Drillbit bit = new Drillbit(CONFIG, serviceSet);
@@ -179,9 +174,7 @@ public class TestHashJoin extends PopUnitTestBase {
   }
 
   @Test
-  public void multipleConditionJoin(@Injectable final DrillbitContext bitContext,
-                                    @Injectable UserClientConnection connection) throws Throwable {
-
+  public void multipleConditionJoin() throws Throwable {
     // Function tests hash join with multiple join conditions
     try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
       final Drillbit bit = new Drillbit(CONFIG, serviceSet);
@@ -225,9 +218,7 @@ public class TestHashJoin extends PopUnitTestBase {
   }
 
   @Test
-  public void hjWithExchange1(@Injectable final DrillbitContext bitContext,
-                              @Injectable UserClientConnection connection) throws Throwable {
-
+  public void hjWithExchange1() throws Throwable {
     // Another test for hash join with exchanges
     try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
          final Drillbit bit = new Drillbit(CONFIG, serviceSet);

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 1c16fa4..56e04fe 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -28,7 +28,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.client.DrillClient;
 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;
@@ -52,25 +52,23 @@ import com.google.common.base.Charsets;
 import com.google.common.collect.Lists;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Category({SlowTest.class, OperatorTest.class})
 public class TestMergeJoin extends PopUnitTestBase {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
   @Ignore // this doesn't have a sort.  it also causes an infinite loop.  these may or may not be related.
-  public void simpleEqualityJoin(@Injectable final DrillbitContext bitContext,
-                                 @Injectable UserClientConnection connection) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void simpleEqualityJoin() 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("/join/merge_join.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 totalRecordCount = 0;
@@ -102,17 +100,17 @@ public class TestMergeJoin extends PopUnitTestBase {
     }
     assertEquals(100, totalRecordCount);
     System.out.println("Total Record Count: " + totalRecordCount);
-    if (context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if (context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 
   @Test
   @Ignore
-  public void orderedEqualityLeftJoin(@Injectable final DrillbitContext bitContext,
-                                      @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void orderedEqualityLeftJoin() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c,
         new StoragePluginRegistryImpl(bitContext));
@@ -122,7 +120,7 @@ public class TestMergeJoin extends PopUnitTestBase {
             .replace("#{LEFT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
             .replace("#{RIGHT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString()));
     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 totalRecordCount = 0;
@@ -153,17 +151,17 @@ public class TestMergeJoin extends PopUnitTestBase {
     System.out.println("Total Record Count: " + totalRecordCount);
     assertEquals(25, totalRecordCount);
 
-    if (context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if (context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 
   @Test
   @Ignore
-  public void orderedEqualityInnerJoin(@Injectable final DrillbitContext bitContext,
-                                       @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void orderedEqualityInnerJoin() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c,
         new StoragePluginRegistryImpl(bitContext));
@@ -173,7 +171,7 @@ public class TestMergeJoin extends PopUnitTestBase {
             .replace("#{LEFT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
             .replace("#{RIGHT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString()));
     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 totalRecordCount = 0;
@@ -204,17 +202,17 @@ public class TestMergeJoin extends PopUnitTestBase {
     System.out.println("Total Record Count: " + totalRecordCount);
     assertEquals(23, totalRecordCount);
 
-    if (context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if (context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
-    assertTrue(!context.isFailed());
+    assertTrue(!context.getExecutorState().isFailed());
   }
 
   @Test
   @Ignore
-  public void orderedEqualityMultiBatchJoin(@Injectable final DrillbitContext bitContext,
-                                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void orderedEqualityMultiBatchJoin() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c,
         new StoragePluginRegistryImpl(bitContext));
@@ -224,7 +222,7 @@ public class TestMergeJoin extends PopUnitTestBase {
             .replace("#{LEFT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_multi_batch.left.json").toURI().toString())
             .replace("#{RIGHT_FILE}", DrillFileUtils.getResourceAsFile("/join/merge_multi_batch.right.json").toURI().toString()));
     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 totalRecordCount = 0;
@@ -254,30 +252,31 @@ public class TestMergeJoin extends PopUnitTestBase {
     System.out.println("Total Record Count: " + totalRecordCount);
     assertEquals(25, totalRecordCount);
 
-    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 testJoinBatchSize(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{
-    mockDrillbitContext(bitContext);
+  public void testJoinBatchSize() 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("/join/join_batchsize.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()));
     exec.next(); // skip schema batch
     while (exec.next()) {
       assertEquals(100, 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());
   }
 
   @Test