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

[05/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/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
index 972b56a..7865b53 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.work.batch;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.proto.BitControl.CustomMessage;
 import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -152,7 +152,7 @@ public class ControlMessageHandler implements RequestHandler<ControlConnection>
     logger.debug("Received remote fragment start instruction", fragment);
 
     try {
-      final FragmentContext fragmentContext = new FragmentContext(drillbitContext, fragment,
+      final FragmentContextImpl fragmentContext = new FragmentContextImpl(drillbitContext, fragment,
           drillbitContext.getFunctionImplementationRegistry());
       final FragmentStatusReporter statusReporter = new FragmentStatusReporter(fragmentContext);
       final FragmentExecutor fragmentExecutor = new FragmentExecutor(fragmentContext, fragment, statusReporter);

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
index 74762a4..508d934 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
@@ -223,7 +223,7 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
 
   @Override
   protected void upkeep(RawFragmentBatch batch) {
-    if (context.isOverMemoryLimit()) {
+    if (context.getAllocator().isOverLimit()) {
       outOfMemory.set(true);
     }
 
@@ -300,11 +300,11 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
           try {
             batch.writeToStream(outputStream);
           } catch (IOException e) {
-            context.fail(e);
+            context.getExecutorState().fail(e);
           }
         }
       } catch (Throwable e) {
-        context.fail(e);
+        context.getExecutorState().fail(e);
       } finally {
         logger.info("Spooler thread exiting");
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
index b177770..0b0ae26 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
@@ -31,7 +31,7 @@ public class UnlimitedRawBatchBuffer extends BaseRawBatchBuffer<RawFragmentBatch
   private final int softlimit;
   private final int startlimit;
 
-  public UnlimitedRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId) {
+  public UnlimitedRawBatchBuffer(FragmentContext context, int fragmentCount) {
     super(context, fragmentCount);
     this.softlimit = bufferSizePerSocket * fragmentCount;
     this.startlimit = Math.max(softlimit/2, 1);
@@ -65,7 +65,7 @@ public class UnlimitedRawBatchBuffer extends BaseRawBatchBuffer<RawFragmentBatch
 
     @Override
     public boolean checkForOutOfMemory() {
-      return context.isOverMemoryLimit();
+      return context.getAllocator().isOverLimit();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
index ce04848..2e5f2dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
@@ -24,7 +24,7 @@ import io.netty.buffer.ByteBuf;
 import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.proto.BitControl.InitializeFragments;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
@@ -123,7 +123,7 @@ public class FragmentsRunner {
    */
   private void setupRootFragment(final PlanFragment rootFragment, final FragmentRoot rootOperator) throws ExecutionSetupException {
     QueryManager queryManager = foreman.getQueryManager();
-    final FragmentContext rootContext = new FragmentContext(drillbitContext, rootFragment, foreman.getQueryContext(),
+    final FragmentContextImpl rootContext = new FragmentContextImpl(drillbitContext, rootFragment, foreman.getQueryContext(),
         initiatingClient, drillbitContext.getFunctionImplementationRegistry());
     final FragmentStatusReporter statusReporter = new FragmentStatusReporter(rootContext);
     final FragmentExecutor rootRunner = new FragmentExecutor(rootContext, rootFragment, statusReporter, rootOperator);
@@ -322,7 +322,7 @@ public class FragmentsRunner {
   private void startLocalFragment(final PlanFragment fragment) throws ExecutionSetupException {
     logger.debug("Received local fragment start instruction", fragment);
 
-    final FragmentContext fragmentContext = new FragmentContext(drillbitContext, fragment, drillbitContext.getFunctionImplementationRegistry());
+    final FragmentContextImpl fragmentContext = new FragmentContextImpl(drillbitContext, fragment, drillbitContext.getFunctionImplementationRegistry());
     final FragmentStatusReporter statusReporter = new FragmentStatusReporter(fragmentContext);
     final FragmentExecutor fragmentExecutor = new FragmentExecutor(fragmentContext, fragment, statusReporter);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractFragmentManager.java
index f427a84..d6ea501 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractFragmentManager.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.work.fragment;
 
 import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
@@ -28,8 +29,6 @@ import org.apache.drill.exec.work.batch.IncomingBuffers;
 import java.io.IOException;
 
 public abstract class AbstractFragmentManager implements FragmentManager {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFragmentManager.class);
-
 
   protected final IncomingBuffers buffers;
 
@@ -37,7 +36,7 @@ public abstract class AbstractFragmentManager implements FragmentManager {
 
   protected final FragmentHandle fragmentHandle;
 
-  protected final FragmentContext fragmentContext;
+  protected final ExecutorFragmentContext fragmentContext;
 
   protected volatile boolean cancel = false;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index e97a382..7b9d524 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -29,8 +29,9 @@ import org.apache.drill.common.EventProcessor;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.FragmentContext.ExecutorState;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.RootExec;
@@ -41,7 +42,6 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.ImpersonationUtil;
@@ -58,7 +58,7 @@ public class FragmentExecutor implements Runnable {
 
   private final AtomicBoolean hasCloseoutThread = new AtomicBoolean(false);
   private final String fragmentName;
-  private final FragmentContext fragmentContext;
+  private final ExecutorFragmentContext fragmentContext;
   private final FragmentStatusReporter statusReporter;
   private final DeferredException deferredException = new DeferredException();
   private final PlanFragment fragment;
@@ -78,8 +78,8 @@ public class FragmentExecutor implements Runnable {
    * @param fragment
    * @param statusReporter
    */
-  public FragmentExecutor(final FragmentContext context, final PlanFragment fragment,
-      final FragmentStatusReporter statusReporter) {
+  public FragmentExecutor(final ExecutorFragmentContext context, final PlanFragment fragment,
+                          final FragmentStatusReporter statusReporter) {
     this(context, fragment, statusReporter, null);
   }
 
@@ -91,7 +91,7 @@ public class FragmentExecutor implements Runnable {
    * @param statusReporter
    * @param rootOperator
    */
-  public FragmentExecutor(final FragmentContext context, final PlanFragment fragment,
+  public FragmentExecutor(final ExecutorFragmentContext context, final PlanFragment fragment,
                           final FragmentStatusReporter statusReporter, final FragmentRoot rootOperator) {
     this.fragmentContext = context;
     this.statusReporter = statusReporter;
@@ -191,8 +191,7 @@ public class FragmentExecutor implements Runnable {
     myThreadRef.set(myThread);
     final String originalThreadName = myThread.getName();
     final FragmentHandle fragmentHandle = fragmentContext.getHandle();
-    final DrillbitContext drillbitContext = fragmentContext.getDrillbitContext();
-    final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator();
+    final ClusterCoordinator clusterCoordinator = fragmentContext.getClusterCoordinator();
     final DrillbitStatusListener drillbitStatusListener = new FragmentDrillbitStatusListener();
     final String newThreadName = QueryIdHelper.getExecutorThreadName(fragmentHandle);
 
@@ -202,7 +201,7 @@ public class FragmentExecutor implements Runnable {
 
       // if we didn't get the root operator when the executor was created, create it now.
       final FragmentRoot rootOperator = this.rootOperator != null ? this.rootOperator :
-          drillbitContext.getPlanReader().readFragmentRoot(fragment.getFragmentJson());
+          fragmentContext.getPlanReader().readFragmentRoot(fragment.getFragmentJson());
 
           root = ImplCreator.getExec(fragmentContext, rootOperator);
           if (root == null) {
@@ -215,7 +214,7 @@ public class FragmentExecutor implements Runnable {
       eventProcessor.start();
       injector.injectPause(fragmentContext.getExecutionControls(), "fragment-running", logger);
 
-      final DrillbitEndpoint endpoint = drillbitContext.getEndpoint();
+      final DrillbitEndpoint endpoint = fragmentContext.getEndpoint();
       logger.debug("Starting fragment {}:{} on {}:{}",
           fragmentHandle.getMajorFragmentId(), fragmentHandle.getMinorFragmentId(),
           endpoint.getAddress(), endpoint.getUserPort());
@@ -296,7 +295,7 @@ public class FragmentExecutor implements Runnable {
     if (outcome == FragmentState.FAILED) {
       final FragmentHandle handle = getContext().getHandle();
       final UserException uex = UserException.systemError(deferredException.getAndClear())
-          .addIdentity(getContext().getIdentity())
+          .addIdentity(getContext().getEndpoint())
           .addContext("Fragment", handle.getMajorFragmentId() + ":" + handle.getMinorFragmentId())
           .build(logger);
       statusReporter.fail(uex);
@@ -414,11 +413,11 @@ public class FragmentExecutor implements Runnable {
     updateState(FragmentState.FAILED);
   }
 
-  public FragmentContext getContext() {
+  public ExecutorFragmentContext getContext() {
     return fragmentContext;
   }
 
-  private class ExecutorStateImpl implements ExecutorState {
+  private class ExecutorStateImpl implements FragmentContext.ExecutorState {
     public boolean shouldContinue() {
       return FragmentExecutor.this.shouldContinue();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentStatusReporter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentStatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentStatusReporter.java
index c095edf..baccd55 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentStatusReporter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentStatusReporter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.work.fragment;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
@@ -36,21 +36,21 @@ import java.util.concurrent.atomic.AtomicReference;
 public class FragmentStatusReporter implements AutoCloseable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatusReporter.class);
 
-  protected final FragmentContext context;
+  protected final ExecutorFragmentContext context;
 
   protected final AtomicReference<DrillbitEndpoint> foremanDrillbit;
 
   protected final DrillbitEndpoint localDrillbit;
 
-  public FragmentStatusReporter(final FragmentContext context) {
+  public FragmentStatusReporter(final ExecutorFragmentContext context) {
     this.context = context;
     this.foremanDrillbit = new AtomicReference<>(context.getForemanEndpoint());
-    this.localDrillbit = context.getIdentity();
+    this.localDrillbit = context.getEndpoint();
   }
 
   /**
    * Returns a {@link FragmentStatus} with the given state. {@link FragmentStatus} has additional information like
-   * metrics, etc. that is gathered from the {@link FragmentContext}.
+   * metrics, etc. that is gathered from the {@link ExecutorFragmentContext}.
    *
    * @param state the state to include in the status
    * @return the status
@@ -77,7 +77,7 @@ public class FragmentStatusReporter implements AutoCloseable {
 
   /**
    * Reports the state change to the Foreman. The state is wrapped in a {@link FragmentStatus} that has additional
-   * information like metrics, etc. This additional information is gathered from the {@link FragmentContext}.
+   * information like metrics, etc. This additional information is gathered from the {@link ExecutorFragmentContext}.
    * NOTE: Use {@link #fail} to report state change to {@link FragmentState#FAILED}.
    *
    * @param newState the new state
@@ -124,7 +124,7 @@ public class FragmentStatusReporter implements AutoCloseable {
       context.getWorkEventbus().statusUpdate(status);
     } else {
       // Send the status via Control Tunnel to remote foreman node
-      final ControlTunnel tunnel = context.getControlTunnel(foremanNode);
+      final ControlTunnel tunnel = context.getController().getTunnel(foremanNode);
       tunnel.sendFragmentStatus(status);
     }
   }
@@ -132,7 +132,7 @@ public class FragmentStatusReporter implements AutoCloseable {
   /**
    * {@link FragmentStatus} with the {@link FragmentState#FAILED} state is reported to the Foreman. The
    * {@link FragmentStatus} has additional information like metrics, etc. that is gathered from the
-   * {@link FragmentContext}.
+   * {@link ExecutorFragmentContext}.
    *
    * @param ex the exception related to the failure
    */

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index 8a8e8d2..472419a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -20,9 +20,6 @@ package org.apache.drill;
 import java.io.IOException;
 import java.net.URL;
 
-import mockit.Mocked;
-import mockit.NonStrictExpectations;
-
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.config.DrillConfig;
@@ -58,16 +55,14 @@ import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableList;
 import com.google.common.io.Resources;
 
-public class PlanningBase extends ExecTest{
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningBase.class);
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
+public class PlanningBase extends ExecTest {
   @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(10000);
 
-  @Mocked DrillbitContext dbContext;
   private final DrillConfig config = DrillConfig.create();
 
-  @Mocked QueryContext context;
-
   BufferAllocator allocator = RootAllocatorFactory.newRoot(config);
 
   protected void testSqlPlanFromFile(String file) throws Exception {
@@ -75,6 +70,9 @@ public class PlanningBase extends ExecTest{
   }
 
   protected void testSqlPlan(String sqlCommands) throws Exception {
+    final DrillbitContext dbContext = mock(DrillbitContext.class);
+    final QueryContext context = mock(QueryContext.class);
+
     final String[] sqlStrings = sqlCommands.split(";");
     final LocalPersistentStoreProvider provider = new LocalPersistentStoreProvider(config);
     provider.start();
@@ -84,28 +82,17 @@ public class PlanningBase extends ExecTest{
     systemOptions.init();
     @SuppressWarnings("resource")
     final UserSession userSession = UserSession.Builder.newBuilder().withOptionManager(systemOptions).build();
-    final SessionOptionManager sessionOptions = (SessionOptionManager) userSession.getOptions();
+    final SessionOptionManager sessionOptions = userSession.getOptions();
     final QueryOptionManager queryOptions = new QueryOptionManager(sessionOptions);
     final ExecutionControls executionControls = new ExecutionControls(queryOptions, DrillbitEndpoint.getDefaultInstance());
 
-    new NonStrictExpectations() {
-      {
-        dbContext.getMetrics();
-        result = new MetricRegistry();
-        dbContext.getAllocator();
-        result = allocator;
-        dbContext.getConfig();
-        result = config;
-        dbContext.getOptionManager();
-        result = systemOptions;
-        dbContext.getStoreProvider();
-        result = provider;
-        dbContext.getClasspathScan();
-        result = scanResult;
-        dbContext.getLpPersistence();
-        result = logicalPlanPersistence;
-      }
-    };
+    when(dbContext.getMetrics()).thenReturn(new MetricRegistry());
+    when(dbContext.getAllocator()).thenReturn(allocator);
+    when(dbContext.getConfig()).thenReturn(config);
+    when(dbContext.getOptionManager()).thenReturn(systemOptions);
+    when(dbContext.getStoreProvider()).thenReturn(provider);
+    when(dbContext.getClasspathScan()).thenReturn(scanResult);
+    when(dbContext.getLpPersistence()).thenReturn(logicalPlanPersistence);
 
     final StoragePluginRegistry registry = new StoragePluginRegistryImpl(dbContext);
     registry.init();
@@ -114,38 +101,20 @@ public class PlanningBase extends ExecTest{
     final SchemaPlus root = CalciteSchema.createRootSchema(false, false).plus();
     registry.getSchemaFactory().registerSchemas(SchemaConfig.newBuilder("foo", context).build(), root);
 
-    new NonStrictExpectations() {
-      {
-        context.getNewDefaultSchema();
-        result = root;
-        context.getLpPersistence();
-        result = new LogicalPlanPersistence(config, ClassPathScanner.fromPrescan(config));
-        context.getStorage();
-        result = registry;
-        context.getFunctionRegistry();
-        result = functionRegistry;
-        context.getSession();
-        result = UserSession.Builder.newBuilder().setSupportComplexTypes(true).build();
-        context.getCurrentEndpoint();
-        result = DrillbitEndpoint.getDefaultInstance();
-        context.getActiveEndpoints();
-        result = ImmutableList.of(DrillbitEndpoint.getDefaultInstance());
-        context.getPlannerSettings();
-        result = new PlannerSettings(queryOptions, functionRegistry);
-        context.getOptions();
-        result = queryOptions;
-        context.getConfig();
-        result = config;
-        context.getDrillOperatorTable();
-        result = table;
-        context.getAllocator();
-        result = allocator;
-        context.getExecutionControls();
-        result = executionControls;
-        dbContext.getLpPersistence();
-        result = logicalPlanPersistence;
-      }
-    };
+    when(context.getNewDefaultSchema()).thenReturn(root);
+    when(context.getLpPersistence()).thenReturn(new LogicalPlanPersistence(config, ClassPathScanner.fromPrescan(config)));
+    when(context.getStorage()).thenReturn(registry);
+    when(context.getFunctionRegistry()).thenReturn(functionRegistry);
+    when(context.getSession()).thenReturn(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build());
+    when(context.getCurrentEndpoint()).thenReturn(DrillbitEndpoint.getDefaultInstance());
+    when(context.getActiveEndpoints()).thenReturn(ImmutableList.of(DrillbitEndpoint.getDefaultInstance()));
+    when(context.getPlannerSettings()).thenReturn(new PlannerSettings(queryOptions, functionRegistry));
+    when(context.getOptions()).thenReturn(queryOptions);
+    when(context.getConfig()).thenReturn(config);
+    when(context.getDrillOperatorTable()).thenReturn(table);
+    when(context.getAllocator()).thenReturn(allocator);
+    when(context.getExecutionControls()).thenReturn(executionControls);
+    when(context.getLpPersistence()).thenReturn(logicalPlanPersistence);
 
     for (final String sql : sqlStrings) {
       if (sql.trim().isEmpty()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
index 0786b66..d68dc0d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
@@ -18,9 +18,9 @@
 package org.apache.drill;
 
 import com.google.common.collect.Lists;
-import mockit.Deencapsulation;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.reflect.FieldUtils;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.common.config.CommonConstants;
@@ -481,8 +481,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
         .baselineValues("a")
         .go();
 
-    Path localUdfDirPath = hadoopToJavaPath((org.apache.hadoop.fs.Path) Deencapsulation.getField(
-        getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir"));
+    Path localUdfDirPath = hadoopToJavaPath((org.apache.hadoop.fs.Path)FieldUtils.readField(
+      getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir", true));
 
     assertTrue("Binary should exist in local udf directory",
       localUdfDirPath.resolve(default_binary_name).toFile().exists());
@@ -548,8 +548,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     test("create function using jar '%s'", default_binary_name);
     test("select custom_lower('A') from (values(1))");
 
-    Path localUdfDirPath = hadoopToJavaPath((org.apache.hadoop.fs.Path)Deencapsulation.getField(
-      getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir"));
+    Path localUdfDirPath = hadoopToJavaPath((org.apache.hadoop.fs.Path)FieldUtils.readField(
+        getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir", true));
 
     assertTrue("Binary should exist in local udf directory",
       localUdfDirPath.resolve(default_binary_name).toFile().exists());
@@ -896,8 +896,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     thread2.join();
 
     verify(functionImplementationRegistry, times(2)).syncWithRemoteRegistry(anyLong());
-    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(
-        functionImplementationRegistry, "localFunctionRegistry");
+    LocalFunctionRegistry localFunctionRegistry = (LocalFunctionRegistry)FieldUtils.readField(
+        functionImplementationRegistry, "localFunctionRegistry", true);
     assertEquals("Sync function registry version should match", 1L, localFunctionRegistry.getVersion());
   }
 
@@ -934,8 +934,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     }
 
     verify(functionImplementationRegistry, times(2)).syncWithRemoteRegistry(anyLong());
-    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(
-        functionImplementationRegistry, "localFunctionRegistry");
+    LocalFunctionRegistry localFunctionRegistry = (LocalFunctionRegistry)FieldUtils.readField(
+        functionImplementationRegistry, "localFunctionRegistry", true);
     assertEquals("Sync function registry version should match", 1L, localFunctionRegistry.getVersion());
   }
 
@@ -962,19 +962,19 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     FileUtils.copyFile(src.resolve(name).toFile(), destFile);
   }
 
-  private RemoteFunctionRegistry spyRemoteFunctionRegistry() {
+  private RemoteFunctionRegistry spyRemoteFunctionRegistry() throws IllegalAccessException {
     FunctionImplementationRegistry functionImplementationRegistry =
         getDrillbitContext().getFunctionImplementationRegistry();
     RemoteFunctionRegistry remoteFunctionRegistry = functionImplementationRegistry.getRemoteFunctionRegistry();
     RemoteFunctionRegistry spy = spy(remoteFunctionRegistry);
-    Deencapsulation.setField(functionImplementationRegistry, "remoteFunctionRegistry", spy);
+    FieldUtils.writeField(functionImplementationRegistry, "remoteFunctionRegistry", spy, true);
     return spy;
   }
 
-  private FunctionImplementationRegistry spyFunctionImplementationRegistry() {
+  private FunctionImplementationRegistry spyFunctionImplementationRegistry() throws IllegalAccessException {
     DrillbitContext drillbitContext = getDrillbitContext();
     FunctionImplementationRegistry spy = spy(drillbitContext.getFunctionImplementationRegistry());
-    Deencapsulation.setField(drillbitContext, "functionRegistry", spy);
+    FieldUtils.writeField(drillbitContext, "functionRegistry", spy, true);
     return spy;
   }
 
@@ -1013,5 +1013,4 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index e60533b..fe8bbb6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec;
 
 import com.codahale.metrics.MetricRegistry;
+import com.google.common.io.Files;
 import mockit.Mock;
 import mockit.MockUp;
-import mockit.NonStrictExpectations;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
@@ -48,10 +48,13 @@ import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 
+import java.io.File;
 import java.io.IOException;
 import java.text.DateFormatSymbols;
 import java.util.Locale;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class ExecTest extends DrillTest {
 
@@ -91,15 +94,28 @@ public class ExecTest extends DrillTest {
     return FileSystem.get(configuration);
   }
 
-  protected void mockDrillbitContext(final DrillbitContext bitContext) throws Exception {
-    new NonStrictExpectations() {{
-      bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
-      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(ClassPathScanner.fromPrescan(c));
-      bitContext.getConfig(); result = c;
-      bitContext.getOptionManager(); result = optionManager;
-      bitContext.getCompiler(); result = CodeCompilerTestFactory.getTestCompiler(c);
-    }};
+  /**
+   * Create a temp directory to store the given <i>dirName</i>.
+   * Directory will be deleted on exit.
+   * @param dirName directory name
+   * @return Full path including temp parent directory and given directory name.
+   */
+  public static String getTempDir(final String dirName) {
+    final File dir = Files.createTempDir();
+    return dir.getAbsolutePath() + File.separator + dirName;
+  }
+
+  protected DrillbitContext mockDrillbitContext() throws Exception {
+    final DrillbitContext context = mock(DrillbitContext.class);
+
+    when(context.getMetrics()).thenReturn(new MetricRegistry());
+    when(context.getAllocator()).thenReturn(RootAllocatorFactory.newRoot(c));
+    when(context.getOperatorCreatorRegistry()).thenReturn(new OperatorCreatorRegistry(ClassPathScanner.fromPrescan(c)));
+    when(context.getConfig()).thenReturn(c);
+    when(context.getOptionManager()).thenReturn(optionManager);
+    when(context.getCompiler()).thenReturn(CodeCompilerTestFactory.getTestCompiler(c));
+
+    return context;
   }
 
   protected LogicalExpression parseExpr(String expr) throws RecognitionException {

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/RunRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/RunRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/RunRootExec.java
index 8d6cc0c..a3fd2e7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/RunRootExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/RunRootExec.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.scanner.ClassPathScanner;
 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,7 +52,7 @@ public class RunRootExec {
     PhysicalPlanReader reader = bitContext.getPlanReader();
     PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(new File(path), Charsets.UTF_8));
     FunctionImplementationRegistry registry = bitContext.getFunctionImplementationRegistry();
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), null, registry);
+    FragmentContextImpl context = new FragmentContextImpl(bitContext, PlanFragment.getDefaultInstance(), null, registry);
     SimpleRootExec exec;
     for (int i = 0; i < iterations; i ++) {
       Stopwatch w = Stopwatch.createStarted();

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java
index 63957f4..8aba833 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.ExecConstants;
 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;
@@ -46,7 +46,7 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 /**
  * The unit test case will read a physical plan in json format. The physical plan contains a "trace" operator,
@@ -54,28 +54,27 @@ import mockit.Injectable;
  */
 
 public class DumpCatTest  extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DumpCatTest.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testDumpCat(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable
+  public void testDumpCat() throws Throwable
   {
-
-      mockDrillbitContext(bitContext);
+      final DrillbitContext bitContext = mockDrillbitContext();
+      final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
       final PhysicalPlanReader reader = 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()) {
       }
 
-      if(context.getFailureCause() != null) {
-          throw context.getFailureCause();
+      if(context.getExecutorState().getFailureCause() != null) {
+          throw context.getExecutorState().getFailureCause();
       }
-      assertTrue(!context.isFailed());
+      assertTrue(!context.getExecutorState().isFailed());
 
       exec.close();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index fbbb4c8..a5a5389 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -18,10 +18,9 @@
 package org.apache.drill.exec.expr;
 
 import static org.junit.Assert.assertEquals;
-import mockit.Expectations;
-import mockit.Injectable;
-import mockit.NonStrict;
-import mockit.NonStrictExpectations;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExpressionParsingException;
@@ -42,7 +41,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 public class ExpressionTest extends ExecTest {
@@ -52,68 +50,61 @@ public class ExpressionTest extends ExecTest {
   private final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
 
   @Test
-  public void testBasicExpression(@Injectable RecordBatch batch) throws Exception {
-    System.out.println(getExpressionCode("if(true) then 1 else 0 end", batch));
+  public void testBasicExpression() throws Exception {
+    System.out.println(getExpressionCode("if(true) then 1 else 0 end"));
   }
 
   @Test
-  public void testExprParseUpperExponent(@Injectable RecordBatch batch) throws Exception {
-    getExpressionCode("multiply(`$f0`, 1.0E-4)", batch);
+  public void testExprParseUpperExponent() throws Exception {
+    getExpressionCode("multiply(`$f0`, 1.0E-4)");
   }
 
   @Test
-  public void testExprParseLowerExponent(@Injectable RecordBatch batch) throws Exception {
-    getExpressionCode("multiply(`$f0`, 1.0e-4)", batch);
+  public void testExprParseLowerExponent() throws Exception {
+    getExpressionCode("multiply(`$f0`, 1.0e-4)");
   }
 
   @Test
-  public void testSpecial(final @Injectable RecordBatch batch, @Injectable ValueVector vector) throws Exception {
+  public void testSpecial() throws Exception {
+    final RecordBatch batch = mock(RecordBatch.class);
+    final VectorWrapper wrapper = mock(VectorWrapper.class);
     final TypeProtos.MajorType type = Types.optional(MinorType.INT);
     final TypedFieldId tfid = new TypedFieldId(type, false, 0);
 
-    new NonStrictExpectations() {
-      @NonStrict VectorWrapper<?> wrapper;
-      {
-        batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
-        result = tfid;
-        batch.getValueAccessorById(IntVector.class, tfid.getFieldIds());
-        result = wrapper;
-        wrapper.getValueVector();
-        result = new IntVector(MaterializedField.create("result", type), RootAllocatorFactory.newRoot(c));
-      }
-
-    };
+    when(wrapper.getValueVector()).thenReturn(new IntVector(MaterializedField.create("result", type), RootAllocatorFactory.newRoot(c)));
+
+    when(batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN))).thenReturn(tfid);
+    when(batch.getValueAccessorById(IntVector.class, tfid.getFieldIds())).thenReturn(wrapper);
+
     System.out.println(getExpressionCode("1 + 1", batch));
   }
 
   @Test
-  public void testSchemaExpression(final @Injectable RecordBatch batch) throws Exception {
-    final TypedFieldId tfid = new TypedFieldId(Types.optional(MinorType.BIGINT), false, 0);
-
-    new Expectations() {
-      {
-        batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
-        result = tfid;
-        // batch.getValueVectorById(tfid); result = new Fixed4(null, null);
-      }
+  public void testSchemaExpression() throws Exception {
+    final RecordBatch batch = mock(RecordBatch.class);
+    when(batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN)))
+      .thenReturn(new TypedFieldId(Types.optional(MinorType.BIGINT), false, 0));
 
-    };
     System.out.println(getExpressionCode("1 + alpha", batch));
-
   }
 
   @Test(expected = ExpressionParsingException.class)
-  public void testExprParseError(@Injectable RecordBatch batch) throws Exception {
-    getExpressionCode("less than(1, 2)", batch);
+  public void testExprParseError() throws Exception {
+    getExpressionCode("less than(1, 2)");
   }
 
   @Test
-  public void testExprParseNoError(@Injectable RecordBatch batch) throws Exception {
-    getExpressionCode("equal(1, 2)", batch);
+  public void testExprParseNoError() throws Exception {
+    getExpressionCode("equal(1, 2)");
   }
 
   // HELPER METHODS //
 
+  private String getExpressionCode(String expression) throws Exception {
+    final RecordBatch batch = mock(RecordBatch.class);
+    return getExpressionCode(expression, batch);
+  }
+
   private String getExpressionCode(String expression, RecordBatch batch) throws Exception {
     final LogicalExpression expr = parseExpr(expression);
     final ErrorCollector error = new ErrorCollectorImpl();

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
index 56181e6..31d3d91 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.expr.fn.impl;
 
 import static org.junit.Assert.assertTrue;
 
-import mockit.integration.junit4.JMockit;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlFunctionTest;
@@ -29,13 +28,11 @@ import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
 
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 
-@RunWith(JMockit.class)
 @Category({SqlFunctionTest.class, UnlikelyTest.class})
 public class TestStringFunctions extends BaseTestQuery {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
index 5a85319..ffaa9e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
@@ -28,7 +28,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;
 
 @Category(OperatorTest.class)
 public class TestMathFunctions extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class);
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testBasicMathFunctions(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable
+  public void testBasicMathFunctions() throws Throwable
   {
-    mockDrillbitContext(bitContext);
+    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/simple_math_functions.json"), Charsets.UTF_8));
     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 SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while(exec.next()) {
@@ -76,9 +76,9 @@ public class TestMathFunctions extends ExecTest {
       assertEquals(floatAddVector.getAccessor().get(0), (1.1 + 2.2), 0);
     }
 
-    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/fn/impl/TestMultiInputAdd.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
index 237b01a..472c76a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -22,8 +22,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.List;
 
-import mockit.Injectable;
-
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
@@ -32,9 +30,7 @@ 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;
@@ -45,41 +41,35 @@ import org.junit.experimental.categories.Category;
 
 @Category(OperatorTest.class)
 public class TestMultiInputAdd extends PopUnitTestBase {
+  private final DrillConfig c = DrillConfig.create();
 
-//    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class);
-
-    DrillConfig c = DrillConfig.create();
-
-
-    @Test
-    public void testMultiInputAdd(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable
-    {
-        try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
-
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    Files.toString(DrillFileUtils.getResourceAsFile("/functions/multi_input_add_test.json"), Charsets.UTF_8));
+  @Test
+  public void testMultiInputAdd() throws Throwable {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+        Files.toString(DrillFileUtils.getResourceAsFile("/functions/multi_input_add_test.json"), Charsets.UTF_8));
 
-            RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
-            for (VectorWrapper<?> v : batchLoader) {
+      for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = v.getValueVector().getAccessor();
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-                assertTrue((accessor.getObject(0)).equals(10));
-            }
+        assertTrue((accessor.getObject(0)).equals(10));
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-                b.release();
-            }
-        }
+      batchLoader.clear();
+      for (QueryDataBatch b : results) {
+        b.release();
+      }
     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
index 8862f8c..59188d4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
@@ -28,7 +28,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.physical.impl.ImplCreator;
@@ -45,16 +45,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, OperatorTest.class})
 public class TestNewMathFunctions extends ExecTest {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestNewMathFunctions.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;
@@ -74,9 +73,10 @@ public class TestNewMathFunctions 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) {
@@ -86,7 +86,7 @@ public class TestNewMathFunctions extends ExecTest {
       registry = new FunctionImplementationRegistry(c);
     }
     if (context == null) {
-      context =  new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry); //new FragmentContext(bitContext, ExecProtos.FragmentHandle.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()));
@@ -101,40 +101,36 @@ public class TestNewMathFunctions 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 testTrigoMathFunc(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testTrigoMathFunc() throws Throwable {
     final Object [] expected = new Object[] {Math.sin(45), Math.cos(45), Math.tan(45),Math.asin(45), Math.acos(45), Math.atan(45),Math.sinh(45), Math.cosh(45), Math.tanh(45)};
-    runTest(bitContext, connection, expected, "functions/testTrigoMathFunctions.json");
+    runTest(expected, "functions/testTrigoMathFunctions.json");
   }
 
   @Test
-  public void testExtendedMathFunc(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable {
+  public void testExtendedMathFunc() throws Throwable {
     final BigDecimal d = new BigDecimal("100111111111111111111111111111111111.00000000000000000000000000000000000000000000000000001");
     final Object [] expected = new Object[] {Math.cbrt(1000), Math.log(10), (Math.log(64.0)/Math.log(2.0)), Math.exp(10), Math.toDegrees(0.5), Math.toRadians(45.0), Math.PI, Math.cbrt(d.doubleValue()), Math.log(d.doubleValue()), (Math.log(d.doubleValue())/Math.log(2)), Math.exp(d.doubleValue()), Math.toDegrees(d.doubleValue()), Math.toRadians(d.doubleValue())};
 
-    runTest(bitContext, connection, expected, "functions/testExtendedMathFunctions.json");
+    runTest(expected, "functions/testExtendedMathFunctions.json");
   }
 
   @Test
-  public void testTruncDivMod(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable{
+  public void testTruncDivMod() throws Throwable{
     final Object [] expected = new Object[] {101.0, 0, 101, 1010.0, 101, 481.0, 0.001099999999931267};
-    runTest(bitContext, connection, expected, "functions/testDivModTruncFunctions.json");
+    runTest(expected, "functions/testDivModTruncFunctions.json");
   }
 
  @Test
- public void testIsNumeric(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserClientConnection connection) throws Throwable{
+ public void testIsNumeric() throws Throwable{
    final Object [] expected = new Object[] {1, 1, 1, 0};
-   runTest(bitContext, connection, expected, "functions/testIsNumericFunction.json");
+   runTest(expected, "functions/testIsNumericFunction.json");
  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java
index fedd30a..952a1a3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.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,23 +44,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 TestRepeatedFunction extends ExecTest{
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRepeatedFunction.class);
+public class TestRepeatedFunction extends ExecTest {
   private final DrillConfig c = DrillConfig.create();
 
   @Test
-  public void testRepeated(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable {
-//    System.out.println(System.getProperty("java.class.path"));
-    mockDrillbitContext(bitContext);
+  public void testRepeated() 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("/physical_repeated_1.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()));
 
     boolean oneIsOne = false;
@@ -93,9 +92,9 @@ public class TestRepeatedFunction 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());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index eaf5e02..493a3b6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -35,7 +35,7 @@ import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextImpl;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.BitControl;
@@ -198,9 +198,9 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     final MockScanBatchCreator creator = new MockScanBatchCreator();
 
     try {
-      final FragmentContext context =
-          new FragmentContext(bit.getContext(), planFragment, null, bit.getContext().getFunctionImplementationRegistry());
-      return (ScanBatch) creator.getBatch(context, scanPOP, children);
+      final FragmentContextImpl context =
+          new FragmentContextImpl(bit.getContext(), planFragment, null, bit.getContext().getFunctionImplementationRegistry());
+      return creator.getBatch(context,scanPOP, children);
     } catch (Exception ex) {
       throw new DrillRuntimeException("Error when setup fragment context" + ex);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
index ee57ad9..aa92b08 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
@@ -66,8 +66,7 @@ public class TestInboundImpersonation extends BaseTestImpersonation {
         .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
             ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE))
         .withValue(ExecConstants.IMPERSONATION_ENABLED,
-            ConfigValueFactory.fromAnyRef(true)),
-        false);
+            ConfigValueFactory.fromAnyRef(true)));
 
     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/memory/TestAllocators.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
index e903166..b4a655c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
@@ -33,7 +33,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.DrillFileUtils;
 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.OpProfileDef;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.ops.OperatorStats;
@@ -198,8 +198,8 @@ public class TestAllocators extends DrillTest {
       pfBuilder2.setMemInitial(500000);
       BitControl.PlanFragment pf2 = pfBuilder1.build();
 
-      FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry);
-      FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry);
+      FragmentContextImpl fragmentContext1 = new FragmentContextImpl(bitContext, pf1, null, functionRegistry);
+      FragmentContextImpl fragmentContext2 = new FragmentContextImpl(bitContext, pf2, null, functionRegistry);
 
       // Get a few physical operators. Easiest way is to read a physical plan.
       PhysicalPlanReader planReader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(bitContext,
@@ -246,7 +246,7 @@ public class TestAllocators extends DrillTest {
       pfBuilder3.setMemInitial(1000000);
       BitControl.PlanFragment pf3 = pfBuilder3.build();
 
-      FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry);
+      FragmentContextImpl fragmentContext3 = new FragmentContextImpl(bitContext, pf3, null, functionRegistry);
 
       // New fragment starts an operator that allocates an amount within the limit
       def = new OpProfileDef(physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE,

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index 42d2193..da3a6d8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.drill.common.DeferredException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.FragmentContext.ExecutorState;
+import org.apache.drill.exec.ops.RootFragmentContext;
 import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.record.RecordBatch;
@@ -37,8 +37,6 @@ import com.google.common.collect.Lists;
 
 @Deprecated
 public class SimpleRootExec implements RootExec, Iterable<ValueVector> {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
-
   private final RecordBatch incoming;
   private final ScreenRoot screenRoot;
 
@@ -49,10 +47,11 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector> {
     } else {
       throw new UnsupportedOperationException();
     }
-    incoming.getContext().setExecutorState(new DummyExecutorState());
+
+    screenRoot.getContext().setExecutorState(new DummyExecutorState());
   }
 
-  private class DummyExecutorState implements ExecutorState {
+  private class DummyExecutorState implements FragmentContext.ExecutorState {
     final DeferredException ex = new DeferredException();
 
     @Override
@@ -77,9 +76,8 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector> {
 
   }
 
-
-  public FragmentContext getContext() {
-    return incoming.getContext();
+  public RootFragmentContext getContext() {
+    return screenRoot.getContext();
   }
 
   public SelectionVector2 getSelectionVector2() {

http://git-wip-us.apache.org/repos/asf/drill/blob/186536d5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
index 11f739d..96d2944 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
@@ -34,7 +34,7 @@ import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
-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;
@@ -60,20 +60,19 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-import mockit.Injectable;
+import org.mockito.Mockito;
 
 public class TestCastFunctions extends PopUnitTestBase {
   @Test
   // cast to bigint.
-  public void testCastBigInt(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void testCastBigInt() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastBigInt.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -95,23 +94,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //cast to int
-  public void testCastInt(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void testCastInt() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastInt.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -132,22 +130,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //cast to float4
-  public void testCastFloat4(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testCastFloat4() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastFloat4.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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,22 +167,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //cast to float8
-  public void testCastFloat8(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testCastFloat8() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastFloat8.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -206,22 +204,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //cast to varchar(length)
-  public void testCastVarChar(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testCastVarChar() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastVarChar.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -242,22 +240,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //cast to varbinary(length)
-  public void testCastVarBinary(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-    mockDrillbitContext(bitContext);
+  public void testCastVarBinary() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastVarBinary.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -278,23 +276,22 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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
   //nested: cast is nested in another cast, or another function.
-  public void testCastNested(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void testCastNested() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastNested.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -315,24 +312,23 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     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(expected = NumberFormatException.class)
-  public void testCastNumException(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserClientConnection connection) throws Throwable {
-
-    mockDrillbitContext(bitContext);
+  public void testCastNumException() throws Throwable {
+    final DrillbitContext bitContext = mockDrillbitContext();
+    final UserClientConnection connection = Mockito.mock(UserClientConnection.class);
 
     final PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG);
     final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(DrillFileUtils.getResourceAsFile("/functions/cast/testCastNumException.json"), Charsets.UTF_8));
     final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(CONFIG);
-    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()) {
@@ -342,10 +338,10 @@ public class TestCastFunctions extends PopUnitTestBase {
 
     context.close();
 
-    assertTrue(context.isFailed());
+    assertTrue(context.getExecutorState().isFailed());
 
-    if(context.getFailureCause() != null) {
-      throw context.getFailureCause();
+    if(context.getExecutorState().getFailureCause() != null) {
+      throw context.getExecutorState().getFailureCause();
     }
   }