You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2013/12/02 05:23:50 UTC

[03/10] git commit: DRILL-312: Modularize org.apache.drill.exec.physical.impl.ImplCreator using operator creator registry

DRILL-312: Modularize org.apache.drill.exec.physical.impl.ImplCreator using operator creator registry


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

Branch: refs/heads/master
Commit: b91f2e8a837d7079c305442b41a3f3ef20b9846f
Parents: ba5e652
Author: Aditya Kishore <ad...@maprtech.com>
Authored: Sun Dec 1 19:37:40 2013 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Dec 1 19:37:40 2013 -0800

----------------------------------------------------------------------
 .../exceptions/ExecutionSetupException.java     |  10 ++
 .../drill/exec/physical/impl/ImplCreator.java   | 175 ++-----------------
 .../physical/impl/OperatorCreatorRegistry.java  | 100 +++++++++++
 .../drill/exec/server/DrillbitContext.java      |  17 +-
 .../apache/drill/exec/client/DumpCatTest.java   |   6 +-
 .../exec/fn/impl/TestRepeatedFunction.java      |   2 +
 .../physical/impl/TestComparisonFunctions.java  |   1 +
 .../exec/physical/impl/TestSimpleFunctions.java |   5 +
 .../drill/exec/physical/impl/agg/TestAgg.java   |   2 +
 .../physical/impl/filter/TestSimpleFilter.java  |   3 +
 .../exec/physical/impl/join/TestMergeJoin.java  |   5 +
 .../physical/impl/limit/TestSimpleLimit.java    |   6 +
 .../impl/project/TestSimpleProjection.java      |   2 +
 .../exec/physical/impl/sort/TestSimpleSort.java |   3 +
 .../physical/impl/svremover/TestSVRemover.java  |   2 +
 .../impl/trace/TestTraceMultiRecordBatch.java   |   4 +-
 .../impl/trace/TestTraceOutputDump.java         |   4 +-
 .../physical/impl/union/TestSimpleUnion.java    |   4 +
 18 files changed, 177 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java b/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
index 2e50ae5..ae647f1 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
@@ -17,9 +17,19 @@
  */
 package org.apache.drill.common.exceptions;
 
+import java.lang.reflect.InvocationTargetException;
+
 
 public class ExecutionSetupException extends DrillException{
+  private static final long serialVersionUID = -6943409010231014085L;
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionSetupException.class);
+
+  public static ExecutionSetupException fromThrowable(String message, Throwable cause) {
+    Throwable t = cause instanceof InvocationTargetException 
+        ? ((InvocationTargetException)cause).getTargetException() : cause;
+    if(t instanceof ExecutionSetupException) return ((ExecutionSetupException) t);
+    return new ExecutionSetupException(message, t);
+  }
   
   public ExecutionSetupException() {
     super();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 3e4c1eb..2dc5f16 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl;
 
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -25,43 +24,8 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.SubScan;
-import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.HashPartitionSender;
-import org.apache.drill.exec.physical.config.IteratorValidator;
-import org.apache.drill.exec.physical.config.Limit;
-import org.apache.drill.exec.physical.config.MergeJoinPOP;
-import org.apache.drill.exec.physical.config.MergingReceiverPOP;
-import org.apache.drill.exec.physical.config.OrderedPartitionSender;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.physical.config.RandomReceiver;
-import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.physical.config.SelectionVectorRemover;
-import org.apache.drill.exec.physical.config.SingleSender;
-import org.apache.drill.exec.physical.config.Sort;
-import org.apache.drill.exec.physical.config.StreamingAggregate;
-import org.apache.drill.exec.physical.config.Trace;
-import org.apache.drill.exec.physical.config.Union;
-import org.apache.drill.exec.physical.impl.aggregate.AggBatchCreator;
-import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
-import org.apache.drill.exec.physical.impl.join.MergeJoinCreator;
-import org.apache.drill.exec.physical.impl.limit.LimitBatchCreator;
-import org.apache.drill.exec.physical.impl.orderedpartitioner.OrderedPartitionSenderCreator;
-import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderCreator;
-import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
-import org.apache.drill.exec.physical.impl.sort.SortBatchCreator;
-import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
-import org.apache.drill.exec.physical.impl.trace.TraceBatchCreator;
-import org.apache.drill.exec.physical.impl.union.UnionBatchCreator;
-import org.apache.drill.exec.physical.impl.validate.IteratorValidatorCreator;
 import org.apache.drill.exec.physical.impl.validate.IteratorValidatorInjector;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.store.json.JSONScanBatchCreator;
-import org.apache.drill.exec.store.json.JSONSubScan;
-import org.apache.drill.exec.store.mock.MockScanBatchCreator;
-import org.apache.drill.exec.store.mock.MockSubScanPOP;
-import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;
-import org.apache.drill.exec.store.parquet.ParquetScanBatchCreator;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -69,138 +33,38 @@ import com.google.common.collect.Lists;
 /**
  * Implementation of the physical operator visitor
  */
-public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
+public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
 
-  // Element creators supported by this visitor
-  private MockScanBatchCreator msc = new MockScanBatchCreator();
-  private ParquetScanBatchCreator parquetScan = new ParquetScanBatchCreator();
-  private ScreenCreator sc = new ScreenCreator();
-  private MergingReceiverCreator mrc = new MergingReceiverCreator();
-  private RandomReceiverCreator rrc = new RandomReceiverCreator();
-  private PartitionSenderCreator hsc = new PartitionSenderCreator();
-  private OrderedPartitionSenderCreator opsc = new OrderedPartitionSenderCreator();
-  private SingleSenderCreator ssc = new SingleSenderCreator();
-  private ProjectBatchCreator pbc = new ProjectBatchCreator();
-  private FilterBatchCreator fbc = new FilterBatchCreator();
-  private LimitBatchCreator lbc = new LimitBatchCreator();
-  private UnionBatchCreator unionbc = new UnionBatchCreator();
-  private SVRemoverCreator svc = new SVRemoverCreator();
-  private SortBatchCreator sbc = new SortBatchCreator();
-  private AggBatchCreator abc = new AggBatchCreator();
-  private MergeJoinCreator mjc = new MergeJoinCreator();
-  private IteratorValidatorCreator ivc = new IteratorValidatorCreator();
   private RootExec root = null;
-  private TraceBatchCreator tbc = new TraceBatchCreator();
 
   private ImplCreator(){}
 
-  public RootExec getRoot(){
+  private RootExec getRoot(){
     return root;
   }
 
   @Override
-  public RecordBatch visitProject(Project op, FragmentContext context) throws ExecutionSetupException {
-    return pbc.getBatch(context, op, getChildren(op, context));
-  }
-
-  @Override
-  public RecordBatch visitTrace(Trace op, FragmentContext context) throws ExecutionSetupException {
-    return tbc.getBatch(context, op, getChildren(op, context));
-  }
-  @Override
-  public RecordBatch visitSubScan(SubScan subScan, FragmentContext context) throws ExecutionSetupException {
-    Preconditions.checkNotNull(subScan);
+  @SuppressWarnings("unchecked")
+  public RecordBatch visitOp(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkNotNull(op);
     Preconditions.checkNotNull(context);
 
-    if (subScan instanceof MockSubScanPOP) {
-      return msc.getBatch(context, (MockSubScanPOP) subScan, Collections.<RecordBatch> emptyList());
-    } else if (subScan instanceof JSONSubScan) {
-      return new JSONScanBatchCreator().getBatch(context, (JSONSubScan) subScan, Collections.<RecordBatch> emptyList());
-    } else if (subScan instanceof ParquetRowGroupScan) {
-      return parquetScan.getBatch(context, (ParquetRowGroupScan) subScan, Collections.<RecordBatch> emptyList());
-    } else {
-      return super.visitSubScan(subScan, context);
-    }
-
-  }
-
-  @Override
-  public RecordBatch visitOp(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
-    if (op instanceof SelectionVectorRemover) {
-      return svc.getBatch(context, (SelectionVectorRemover) op, getChildren(op, context));
+    Object opCreator = context.getDrillbitContext().getOperatorCreatorRegistry().getOperatorCreator(op.getClass());
+    if (opCreator != null) {
+      if (op instanceof FragmentRoot ) {
+        root = ((RootCreator<PhysicalOperator>)opCreator).getRoot(context, op, getChildren(op, context));
+        return null;
+      } else {
+        return ((BatchCreator<PhysicalOperator>)opCreator).getBatch(context, op, getChildren(op, context));
+      }
     } else {
-      return super.visitOp(op, context);
+      throw new UnsupportedOperationException(String.format(
+          "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.",
+          this.getClass().getCanonicalName(), op.getClass().getCanonicalName()));
     }
   }
 
-  @Override
-  public RecordBatch visitSort(Sort sort, FragmentContext context) throws ExecutionSetupException {
-    return sbc.getBatch(context, sort, getChildren(sort, context));
-  }
-
-  @Override
-  public RecordBatch visitLimit(Limit limit, FragmentContext context) throws ExecutionSetupException {
-    return lbc.getBatch(context, limit, getChildren(limit, context));
-  }
-
-  @Override
-  public RecordBatch visitMergeJoin(MergeJoinPOP op, FragmentContext context) throws ExecutionSetupException {
-    return mjc.getBatch(context, op, getChildren(op, context));
-  }
-
-  @Override
-  public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
-    Preconditions.checkArgument(root == null);
-    root = sc.getRoot(context, op, getChildren(op, context));
-    return null;
-  }
-
-  @Override
-  public RecordBatch visitHashPartitionSender(HashPartitionSender op, FragmentContext context) throws ExecutionSetupException {
-    root = hsc.getRoot(context, op, getChildren(op, context));
-    return null;
-  }
-
-  @Override
-  public RecordBatch visitOrderedPartitionSender(OrderedPartitionSender op, FragmentContext context) throws ExecutionSetupException {
-    root = opsc.getRoot(context, op, getChildren(op, context));
-    return null;
-  }
-  
-  @Override
-  public RecordBatch visitFilter(Filter filter, FragmentContext context) throws ExecutionSetupException {
-    return fbc.getBatch(context, filter, getChildren(filter, context));
-  }
-
-
-  @Override
-  public RecordBatch visitStreamingAggregate(StreamingAggregate config, FragmentContext context)
-      throws ExecutionSetupException {
-    return abc.getBatch(context, config, getChildren(config, context));
-  }
-
-  @Override
-  public RecordBatch visitUnion(Union union, FragmentContext context) throws ExecutionSetupException {
-    return unionbc.getBatch(context, union, getChildren(union, context));
-  }
-
-  @Override
-  public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
-    root = ssc.getRoot(context, op, getChildren(op, context));
-    return null;
-  }
-
-  @Override
-  public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
-    return rrc.getBatch(context, op, null);
-  }
-
-  @Override
-  public RecordBatch visitMergingReceiver(MergingReceiverPOP op, FragmentContext context) throws ExecutionSetupException {
-    return mrc.getBatch(context, op, null);
-  }
-
   private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
     List<RecordBatch> children = Lists.newArrayList();
     for (PhysicalOperator child : op) {
@@ -209,11 +73,6 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     return children;
   }
 
-  @Override
-  public RecordBatch visitIteratorValidator(IteratorValidator op, FragmentContext context) throws ExecutionSetupException {
-    return ivc.getBatch(context, op, getChildren(op, context));
-  }
-  
   public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException {
     ImplCreator i = new ImplCreator();
     boolean isAssertEnabled = false;
@@ -227,6 +86,4 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
           "The provided fragment did not have a root node that correctly created a RootExec value.");
     return i.getRoot();
   }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
new file mode 100644
index 0000000..8c768e5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OperatorCreatorRegistry.java
@@ -0,0 +1,100 @@
+/**
+ * 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.lang.reflect.Constructor;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.common.config.CommonConstants;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.util.PathScanner;
+
+public class OperatorCreatorRegistry {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCreatorRegistry.class);
+
+  private volatile Map<Class<?>, Constructor<?>> constructorRegistry = new HashMap<Class<?>, Constructor<?>>();
+  private volatile Map<Class<?>, Object> instanceRegistry = new HashMap<Class<?>, Object>();
+
+  public OperatorCreatorRegistry(DrillConfig config) {
+    addImplemntorsToMap(config, BatchCreator.class);
+    addImplemntorsToMap(config, RootCreator.class);
+    logger.debug("Adding Operator Creator map: {}", constructorRegistry);
+  }
+
+  public synchronized Object getOperatorCreator(Class<?> operator) throws ExecutionSetupException {
+    Object opCreator = instanceRegistry.get(operator);
+    if (opCreator != null) return opCreator;
+
+    Constructor<?> c = constructorRegistry.get(operator);
+    if(c == null) {
+      throw new ExecutionSetupException(
+          String.format("Failure finding OperatorCreator constructor for config %s", operator.getCanonicalName()));
+    }
+    try {
+      opCreator = c.newInstance();
+      instanceRegistry.put(operator, opCreator);
+      return opCreator;
+    } catch (Throwable t) {
+      throw ExecutionSetupException.fromThrowable(
+          String.format("Failure creating OperatorCreator for Operator %s", operator.getCanonicalName()), t);
+    }
+  }
+
+  private <T> void addImplemntorsToMap(DrillConfig config, Class<T> baseInterface) {
+    Class<?>[] providerClasses = PathScanner.scanForImplementationsArr(baseInterface,
+        config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
+    for (Class<?> c : providerClasses) {
+      Class<?> operatorClass = c;
+      boolean interfaceFound = false;
+      while (!interfaceFound && !(c.equals(java.lang.Object.class))) {
+        Type[] ifaces = c.getGenericInterfaces(); // never returns null
+        for (Type iface : ifaces) {
+          if (!(iface instanceof ParameterizedType
+              && ((ParameterizedType)iface).getRawType().equals(baseInterface))) {
+            continue;
+          }
+          Type[] args = ((ParameterizedType)iface).getActualTypeArguments();
+          interfaceFound = true;
+          boolean constructorFound = false;
+          for(Constructor<?> constructor : operatorClass.getConstructors()){
+            Class<?>[] params = constructor.getParameterTypes();
+            if(params.length == 0){
+              Constructor<?> old = constructorRegistry.put((Class<?>) args[0], constructor);
+              if (old != null) {
+                throw new RuntimeException(
+                    String.format("Duplicate OperatorCreator [%s, %s] found for PhysicalOperator %s",
+                    old.getDeclaringClass().getCanonicalName(), operatorClass.getCanonicalName(),
+                    ((Class<?>) args[0]).getCanonicalName()));
+              }
+              constructorFound = true;
+            }
+          }
+          if(!constructorFound){
+            logger.debug("Skipping registration of OperatorCreator {} as it doesn't have a default constructor",
+                operatorClass.getCanonicalName());
+          }
+        }
+        c = c.getSuperclass();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index 080fd70..4656590 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -24,21 +24,18 @@ import java.util.Collection;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
-import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.store.StorageEngine;
+import org.apache.drill.exec.store.StorageEngineRegistry;
 
-import com.google.common.base.Preconditions;
 import com.codahale.metrics.MetricRegistry;
-
-import org.apache.drill.exec.store.StorageEngineRegistry;
+import com.google.common.base.Preconditions;
 
 public class DrillbitContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
@@ -51,6 +48,7 @@ public class DrillbitContext {
   private final DistributedCache cache;
   private final DrillbitEndpoint endpoint;
   private final StorageEngineRegistry storageEngineRegistry;
+  private final OperatorCreatorRegistry operatorCreatorRegistry;
   
   public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, BitCom com, DistributedCache cache) {
     super();
@@ -65,6 +63,7 @@ public class DrillbitContext {
     this.endpoint = endpoint;
     this.storageEngineRegistry = new StorageEngineRegistry(this);
     this.reader = new PhysicalPlanReader(context.getConfig(), context.getConfig().getMapper(), endpoint, storageEngineRegistry);
+    this.operatorCreatorRegistry = new OperatorCreatorRegistry(context.getConfig());
   }
   
   public DrillbitEndpoint getEndpoint(){
@@ -82,7 +81,11 @@ public class DrillbitContext {
   public BufferAllocator getAllocator(){
     return context.getAllocator();
   }
-  
+
+  public OperatorCreatorRegistry getOperatorCreatorRegistry() {
+    return operatorCreatorRegistry;
+  }
+
   public StorageEngine getStorageEngine(StorageEngineConfig config) throws ExecutionSetupException {
     return storageEngineRegistry.getEngine(config);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 a78ffc3..0ecab3a 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,24 +27,21 @@ import mockit.NonStrictExpectations;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.ExecConstants;
-
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
-
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
-
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.AfterClass;
@@ -71,6 +68,7 @@ public class DumpCatTest {
           bitContext.getMetrics(); result = new MetricRegistry();
           bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
           bitContext.getConfig(); result = c;
+          bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       }};
 
       PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 3ec9492..aa67ba5 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
@@ -30,6 +30,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -59,6 +60,7 @@ public class TestRepeatedFunction {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 54bf0fd..4dea33d 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
@@ -56,6 +56,7 @@ public class TestComparisonFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 1776d8d..411f21c 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
@@ -59,6 +59,7 @@ public class TestSimpleFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
@@ -86,6 +87,7 @@ public class TestSimpleFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
@@ -113,6 +115,7 @@ public class TestSimpleFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
@@ -152,6 +155,7 @@ public class TestSimpleFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
@@ -191,6 +195,7 @@ public class TestSimpleFunctions {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 b18ef71..4eae66f 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
@@ -30,6 +30,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -54,6 +55,7 @@ public class TestAgg {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 14d68f3..d1c756f 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
@@ -28,6 +28,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -56,6 +57,7 @@ public class TestSimpleFilter {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     
@@ -80,6 +82,7 @@ public class TestSimpleFilter {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/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 09b7ebe..41cb034 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
@@ -33,6 +33,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -63,6 +64,7 @@ public class TestMergeJoin {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
@@ -112,6 +114,7 @@ public class TestMergeJoin {
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
       bitContext.getConfig(); result = c;
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(),CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), new StorageEngineRegistry(bitContext));
@@ -164,6 +167,7 @@ public class TestMergeJoin {
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
       bitContext.getConfig(); result = c;
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(),CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), new StorageEngineRegistry(bitContext));
@@ -216,6 +220,7 @@ public class TestMergeJoin {
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
       bitContext.getConfig(); result = c;
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(),CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), new StorageEngineRegistry(bitContext));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
index 89d909d..b254fc0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java
@@ -20,9 +20,11 @@ package org.apache.drill.exec.physical.impl.limit;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import com.codahale.metrics.MetricRegistry;
+
 import junit.framework.Assert;
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -30,6 +32,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -50,6 +53,7 @@ public class TestSimpleLimit {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     verifyLimitCount(bitContext, connection, "test1.json", 5);
@@ -60,6 +64,7 @@ public class TestSimpleLimit {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     verifyLimitCount(bitContext, connection, "test3.json", 95);
@@ -70,6 +75,7 @@ public class TestSimpleLimit {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
     verifyLimitCount(bitContext, connection, "test2.json", 69999);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
index ca38d9c..72dbe6e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -59,6 +60,7 @@ public class TestSimpleProjection {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
index 426aa3a..4472668 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java
@@ -30,6 +30,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -58,6 +59,7 @@ public class TestSimpleSort {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     
@@ -105,6 +107,7 @@ public class TestSimpleSort {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
index 2da96d7..5d2b67c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
@@ -29,6 +29,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -58,6 +59,7 @@ public class TestSVRemover {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
     
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
index dd7d006..a3d923e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.physical.impl.trace;
 
 import static org.junit.Assert.*;
-
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
 
@@ -30,6 +29,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -37,7 +37,6 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
-
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
@@ -65,6 +64,7 @@ public class TestTraceMultiRecordBatch {
             bitContext.getMetrics(); result = new MetricRegistry();
             bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
             bitContext.getConfig(); result = c;
+            bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
         }};
 
         PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
index f4e6180..7643cf8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.physical.impl.trace;
 
 import static org.junit.Assert.*;
-
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
 
@@ -31,6 +30,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -40,7 +40,6 @@ import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -79,6 +78,7 @@ public class TestTraceOutputDump {
             bitContext.getMetrics(); result = new MetricRegistry();
             bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
             bitContext.getConfig(); result = c;
+            bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
         }};
 
         PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b91f2e8a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
index f74c66f..fe90ad4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
@@ -20,8 +20,10 @@ package org.apache.drill.exec.physical.impl.union;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import com.codahale.metrics.MetricRegistry;
+
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -29,6 +31,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 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.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -54,6 +57,7 @@ public class TestSimpleUnion {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};