You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by js...@apache.org on 2015/09/02 23:52:54 UTC

[3/5] drill git commit: DRILL-3598: use a factory to create the root allocator. - made the constructor for TopLevelAllocator package private to enforce this

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/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 ffa8765..7e1da44 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
@@ -40,7 +40,7 @@ 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.memory.BufferAllocator;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -69,19 +69,16 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
 public class TestCastFunctions extends PopUnitTestBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class);
-
-  DrillConfig c = DrillConfig.create();
-
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class);
+  private final DrillConfig c = DrillConfig.create();
 
   @Test
   // cast to bigint.
   public void testCastBigInt(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    final BufferAllocator allocator = new TopLevelAllocator();
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
 
-    new NonStrictExpectations(){{
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -89,19 +86,18 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastBigInt.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastBigInt.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      BigIntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), BigIntVector.class);
-      BigIntVector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final BigIntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), BigIntVector.class);
+      final BigIntVector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
           BigIntHolder holder0 = new BigIntHolder();
           a0.get(i, holder0);
           assertEquals(1256, holder0.value);
@@ -116,7 +112,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -125,11 +121,11 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //cast to int
   public void testCastInt(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
 
-    final BufferAllocator allocator = new TopLevelAllocator();
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
 
-    new NonStrictExpectations(){{
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -137,24 +133,22 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastInt.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastInt.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      IntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), IntVector.class);
-      IntVector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final IntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), IntVector.class);
+      final IntVector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          IntHolder holder0 = new IntHolder();
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
+          final IntHolder holder0 = new IntHolder();
           a0.get(i, holder0);
           assertEquals(1256, holder0.value);
           ++count;
-
       }
       assertEquals(5, count);
     }
@@ -164,7 +158,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -173,9 +167,9 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //cast to float4
   public void testCastFloat4(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-    final BufferAllocator allocator = new TopLevelAllocator();
-    new NonStrictExpectations(){{
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -183,20 +177,19 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat4.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat4.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      Float4Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float4Vector.class);
-      Float4Vector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final Float4Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float4Vector.class);
+      final Float4Vector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          Float4Holder holder0 = new Float4Holder();
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
+          final Float4Holder holder0 = new Float4Holder();
           a0.get(i, holder0);
           assertEquals(12.56, holder0.value, 0.001);
           ++count;
@@ -210,7 +203,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -219,10 +212,9 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //cast to float8
   public void testCastFloat8(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    final BufferAllocator allocator = new TopLevelAllocator();
-    new NonStrictExpectations(){{
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -230,20 +222,19 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat8.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat8.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      Float8Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float8Vector.class);
-      Float8Vector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final Float8Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float8Vector.class);
+      final Float8Vector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
       for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          Float8Holder holder0 = new Float8Holder();
+          final Float8Holder holder0 = new Float8Holder();
           a0.get(i, holder0);
           assertEquals(12.56, holder0.value, 0.001);
           ++count;
@@ -257,7 +248,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -266,10 +257,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //cast to varchar(length)
   public void testCastVarChar(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
 
-    final BufferAllocator allocator = new TopLevelAllocator();
-    new NonStrictExpectations(){{
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -277,24 +268,22 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarChar.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarChar.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      VarCharVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarCharVector.class);
-      VarCharVector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final VarCharVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarCharVector.class);
+      final VarCharVector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          VarCharHolder holder0 = new VarCharHolder();
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
+          final VarCharHolder holder0 = new VarCharHolder();
           a0.get(i, holder0);
           assertEquals("123", StringFunctionHelpers.toStringFromUTF8(holder0.start, holder0.end, holder0.buffer));
           ++count;
-
       }
       assertEquals(5, count);
     }
@@ -304,7 +293,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -313,11 +302,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //cast to varbinary(length)
   public void testCastVarBinary(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    final BufferAllocator allocator = new TopLevelAllocator();
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
 
-    new NonStrictExpectations(){{
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -325,20 +313,19 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarBinary.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarBinary.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      VarBinaryVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarBinaryVector.class);
-      VarBinaryVector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final VarBinaryVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarBinaryVector.class);
+      final VarBinaryVector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          VarBinaryHolder holder0 = new VarBinaryHolder();
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
+          final VarBinaryHolder holder0 = new VarBinaryHolder();
           a0.get(i, holder0);
           assertEquals("123", StringFunctionHelpers.toStringFromUTF8(holder0.start, holder0.end, holder0.buffer));
           ++count;
@@ -351,7 +338,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
@@ -360,10 +347,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   @Test
   //nested: cast is nested in another cast, or another function.
   public void testCastNested(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
 
-    final BufferAllocator allocator = new TopLevelAllocator();
-    new NonStrictExpectations(){{
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -371,20 +358,19 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNested.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNested.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      IntVector c0 = exec.getValueVectorById(new SchemaPath("add_cast", ExpressionPosition.UNKNOWN),IntVector.class);
-      IntVector.Accessor a0;
-      a0 = c0.getAccessor();
+    while(exec.next()) {
+      final IntVector c0 = exec.getValueVectorById(new SchemaPath("add_cast", ExpressionPosition.UNKNOWN),IntVector.class);
+      final IntVector.Accessor a0 = c0.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
-          IntHolder holder0 = new IntHolder();
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++) {
+          final IntHolder holder0 = new IntHolder();
           a0.get(i, holder0);
           assertEquals(300, holder0.value);
           ++count;
@@ -397,7 +383,7 @@ public class TestCastFunctions extends PopUnitTestBase{
     context.close();
     allocator.close();
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
 
@@ -407,11 +393,10 @@ public class TestCastFunctions extends PopUnitTestBase{
 
   @Test(expected = NumberFormatException.class)
   public void testCastNumException(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    final BufferAllocator allocator = new TopLevelAllocator();
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(c);
 
-    new NonStrictExpectations(){{
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
@@ -419,13 +404,13 @@ public class TestCastFunctions extends PopUnitTestBase{
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNumException.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNumException.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
+    while(exec.next()) {
     }
 
     exec.close();
@@ -435,32 +420,30 @@ public class TestCastFunctions extends PopUnitTestBase{
 
     assertTrue(context.isFailed());
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
-
   }
 
   @Test
   public void testCastFromNullablCol() throws Throwable {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try(Drillbit bit = new Drillbit(CONFIG, serviceSet);
-        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+    try(final Drillbit bit = new Drillbit(CONFIG, serviceSet);
+        final DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
       bit.run();
 
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarCharNull.json"), Charsets.UTF_8).replace("#{TEST_FILE}", "/jsoninput/input1.json"));
 
-      QueryDataBatch batch = results.get(0);
+      final QueryDataBatch batch = results.get(0);
 
-      RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+      final RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
       batchLoader.load(batch.getHeader().getDef(), batch.getData());
 
-      Object [][] result = getRunResult(batchLoader);
+      final Object [][] result = getRunResult(batchLoader);
 
-      Object [][] expected = new Object[2][2];
+      final Object [][] expected = new Object[2][2];
 
       expected[0][0] = new String("2001");
       expected[0][1] = new String("1.2");
@@ -477,24 +460,23 @@ public class TestCastFunctions extends PopUnitTestBase{
         }
       }
       batchLoader.clear();
-      for(QueryDataBatch b : results){
+      for(final QueryDataBatch b : results){
         b.release();
       }
-
     }
   }
 
   private Object[][] getRunResult(VectorAccessible va) {
     int size = 0;
-    for (VectorWrapper v : va) {
+    for (final VectorWrapper v : va) {
       size++;
     }
 
-    Object[][] res = new Object [va.getRecordCount()][size];
+    final Object[][] res = new Object [va.getRecordCount()][size];
     for (int j = 0; j < va.getRecordCount(); j++) {
       int i = 0;
-      for (VectorWrapper v : va) {
-        Object o =  v.getValueVector().getAccessor().getObject(j);
+      for (final VectorWrapper v : va) {
+        final Object o =  v.getValueVector().getAccessor().getObject(j);
         if (o instanceof byte[]) {
           res[j][i++] =  new String((byte[]) o);
         } else {
@@ -504,5 +486,4 @@ public class TestCastFunctions extends PopUnitTestBase{
     }
     return res;
  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/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 c69c6f5..eb2e3c0 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
@@ -26,7 +26,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.compile.CodeCompiler;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -51,16 +51,15 @@ public class TestComparisonFunctions extends ExecTest {
 
   public void runTest(@Injectable final DrillbitContext bitContext,
                       @Injectable UserServer.UserClientConnection connection, String expression, int expectedResults) throws Throwable {
-
     new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       bitContext.getConfig(); result = c;
       bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c);
     }};
 
-    String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);
+    final String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);
     if (reader == null) {
       reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     }
@@ -69,8 +68,8 @@ public class TestComparisonFunctions extends ExecTest {
     }
     final FragmentContext context =
         new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    PhysicalPlan plan = reader.readPhysicalPlan(planString);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlan plan = reader.readPhysicalPlan(planString);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
     while(exec.next()) {
       assertEquals(String.format("Expression: %s;", expression), expectedResults,
@@ -92,7 +91,7 @@ public class TestComparisonFunctions extends ExecTest {
 
   @Test
   public void testInt(@Injectable final DrillbitContext bitContext,
-                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                           @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "intColumn == intColumn", 100);
     runTest(bitContext, connection, "intColumn != intColumn", 0);
     runTest(bitContext, connection, "intColumn > intColumn", 0);
@@ -103,7 +102,7 @@ public class TestComparisonFunctions extends ExecTest {
 
   @Test
   public void testBigInt(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                      @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "bigIntColumn == bigIntColumn", 100);
     runTest(bitContext, connection, "bigIntColumn != bigIntColumn", 0);
     runTest(bitContext, connection, "bigIntColumn > bigIntColumn", 0);
@@ -114,7 +113,7 @@ public class TestComparisonFunctions extends ExecTest {
 
   @Test
   public void testFloat4(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "float4Column == float4Column", 100);
     runTest(bitContext, connection, "float4Column != float4Column", 0);
     runTest(bitContext, connection, "float4Column > float4Column", 0);
@@ -125,7 +124,7 @@ public class TestComparisonFunctions extends ExecTest {
 
   @Test
   public void testFloat8(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "float8Column == float8Column", 100);
     runTest(bitContext, connection, "float8Column != float8Column", 0);
     runTest(bitContext, connection, "float8Column > float8Column", 0);
@@ -136,7 +135,7 @@ public class TestComparisonFunctions extends ExecTest {
 
   @Test
   public void testIntNullable(@Injectable final DrillbitContext bitContext,
-                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                      @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "intNullableColumn == intNullableColumn", 50);
     runTest(bitContext, connection, "intNullableColumn != intNullableColumn", 0);
     runTest(bitContext, connection, "intNullableColumn > intNullableColumn", 0);
@@ -144,9 +143,10 @@ public class TestComparisonFunctions extends ExecTest {
     runTest(bitContext, connection, "intNullableColumn >= intNullableColumn", 50);
     runTest(bitContext, connection, "intNullableColumn <= intNullableColumn", 50);
   }
+
   @Test
   public void testBigIntNullable(@Injectable final DrillbitContext bitContext,
-                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable {
     runTest(bitContext, connection, "bigIntNullableColumn == bigIntNullableColumn", 50);
     runTest(bitContext, connection, "bigIntNullableColumn != bigIntNullableColumn", 0);
     runTest(bitContext, connection, "bigIntNullableColumn > bigIntNullableColumn", 0);
@@ -154,5 +154,4 @@ public class TestComparisonFunctions extends ExecTest {
     runTest(bitContext, connection, "bigIntNullableColumn >= bigIntNullableColumn", 50);
     runTest(bitContext, connection, "bigIntNullableColumn <= bigIntNullableColumn", 50);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
index 03c6f41..7f24c4c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java
@@ -26,7 +26,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.compile.CodeCompiler;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -43,22 +43,22 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
 public class TestImplicitCastFunctions extends ExecTest {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class);
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class);
 
-  DrillConfig c = DrillConfig.create();
-  PhysicalPlanReader reader;
-  FunctionImplementationRegistry registry;
-  FragmentContext context;
+  private final DrillConfig c = DrillConfig.create();
+  private PhysicalPlanReader reader;
+  private FunctionImplementationRegistry registry;
+  private FragmentContext context;
 
   public Object[] getRunResult(SimpleRootExec exec) {
     int size = 0;
-    for (ValueVector v : exec) {
+    for (final ValueVector v : exec) {
       size++;
     }
 
-    Object[] res = new Object [size];
+    final Object[] res = new Object [size];
     int i = 0;
-    for (ValueVector v : exec) {
+    for (final ValueVector v : exec) {
       res[i++] = v.getAccessor().getObject(0);
     }
     return res;
@@ -69,13 +69,13 @@ public class TestImplicitCastFunctions extends ExecTest {
 
     new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       bitContext.getConfig(); result = c;
       bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c);
     }};
 
-    String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
+    final String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
     if (reader == null) {
       reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     }
@@ -85,16 +85,16 @@ public class TestImplicitCastFunctions extends ExecTest {
     if (context == null) {
       context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     }
-    PhysicalPlan plan = reader.readPhysicalPlan(planString);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlan plan = reader.readPhysicalPlan(planString);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
 
     exec.next(); // skip schema batch
     while (exec.next()) {
-      Object [] res = getRunResult(exec);
+      final Object [] res = getRunResult(exec);
       assertEquals("return count does not match", res.length, expectedResults.length);
 
-      for (int i = 0; i<res.length; i++) {
+      for (int i = 0; i < res.length; i++) {
         assertEquals(String.format("column %s does not match", i),  res[i], expectedResults[i]);
       }
     }
@@ -109,7 +109,7 @@ public class TestImplicitCastFunctions extends ExecTest {
   @Test
   public void testImplicitCastWithConstant(@Injectable final DrillbitContext bitContext,
                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-    Object [] expected = new Object[21];
+    final Object [] expected = new Object[21];
     expected [0] = new Double (30.1);
     expected [1] = new Double (30.1);
     expected [2] = new Double (30.1);
@@ -141,7 +141,7 @@ public class TestImplicitCastFunctions extends ExecTest {
   @Test
   public void testImplicitCastWithMockColumn(@Injectable final DrillbitContext bitContext,
                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-    Object [] expected = new Object[5];
+    final Object [] expected = new Object[5];
     expected [0] = new Integer (0);
     expected [1] = new Integer (0);
     expected [2] = new Float (-2.14748365E9);
@@ -154,7 +154,7 @@ public class TestImplicitCastFunctions extends ExecTest {
   @Test
   public void testImplicitCastWithNullExpression(@Injectable final DrillbitContext bitContext,
                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-    Object [] expected = new Object[10];
+    final Object [] expected = new Object[10];
 
     expected [0] = Boolean.TRUE;
     expected [1] = Boolean.FALSE;
@@ -170,5 +170,4 @@ public class TestImplicitCastFunctions extends ExecTest {
 
     runTest(bitContext, connection, expected, "functions/cast/testICastNullExp.json");
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
index dc37071..f2f2f0d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.opt.BasicOptimizer;
@@ -66,28 +66,28 @@ import com.google.common.io.Resources;
 
 @Ignore
 public class TestOptiqPlans extends ExecTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class);
-  DrillConfig c = DrillConfig.create();
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class);
+  private final DrillConfig config = DrillConfig.create();
 
   @Test
   public void orderBy(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
       @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
       @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_order.json", coord, com, controller, workBus);
+    final SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_order.json", coord, com, controller, workBus);
   }
 
   @Test
   public void stringFilter(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
       @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
       @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_string_filter.json", coord, com, controller, workBus);
+    final SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_string_filter.json", coord, com, controller, workBus);
   }
 
   @Test
   public void groupBy(@Injectable final BootStrapContext bitContext, @Injectable UserClientConnection connection,
       @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com,
       @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
-    SimpleRootExec exec = doLogicalTest(bitContext, connection, "/logical_group.json", coord, com, controller, workBus);
+    final SimpleRootExec exec = doLogicalTest(bitContext, connection, "/logical_group.json", coord, com, controller, workBus);
   }
 
   private SimpleRootExec doLogicalTest(final BootStrapContext context, UserClientConnection connection, String file,
@@ -97,24 +97,23 @@ public class TestOptiqPlans extends ExecTest {
         context.getMetrics();
         result = new MetricRegistry();
         context.getAllocator();
-        result = new TopLevelAllocator();
+        result = RootAllocatorFactory.newRoot(config);
         context.getConfig();
-        result = c;
+        result = config;
       }
     };
-    RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet();
-    DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, controller,
-        com, workBus, new LocalPStoreProvider(DrillConfig.create()), null);
-    QueryContext qc = new QueryContext(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build(),
+    final RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet();
+    final DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, controller,
+        com, workBus, new LocalPStoreProvider(config), null);
+    final QueryContext qc = new QueryContext(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build(),
         bitContext);
-    PhysicalPlanReader reader = bitContext.getPlanReader();
-    LogicalPlan plan = reader.readLogicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
-    PhysicalPlan pp = new BasicOptimizer(qc, connection).optimize(new BasicOptimizer.BasicOptimizationContext(qc), plan);
+    final PhysicalPlanReader reader = bitContext.getPlanReader();
+    final LogicalPlan plan = reader.readLogicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
+    final PhysicalPlan pp = new BasicOptimizer(qc, connection).optimize(new BasicOptimizer.BasicOptimizationContext(qc), plan);
 
-
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext fctxt = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(fctxt, (FragmentRoot) pp.getSortedOperators(false)
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
+    final FragmentContext fctxt = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(fctxt, (FragmentRoot) pp.getSortedOperators(false)
         .iterator().next()));
     return exec;
 
@@ -122,24 +121,23 @@ public class TestOptiqPlans extends ExecTest {
 
   @Test
   public void testFilterPlan() throws Exception {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try (Drillbit bit1 = new Drillbit(config, serviceSet);
-        DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
+    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
+        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_filter.json"), Charsets.UTF_8));
-      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryDataBatch b : results) {
+      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (final QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
-        for (VectorWrapper<?> vw : loader) {
+        for (final VectorWrapper<?> vw : loader) {
           System.out.println(vw.getValueVector().getField().toExpr());
-          ValueVector vv = vw.getValueVector();
+          final ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            Object o = vv.getAccessor().getObject(i);
+            final Object o = vv.getAccessor().getObject(i);
             System.out.println(vv.getAccessor().getObject(i));
           }
         }
@@ -152,24 +150,23 @@ public class TestOptiqPlans extends ExecTest {
 
   @Test
   public void testJoinPlan() throws Exception {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try (Drillbit bit1 = new Drillbit(config, serviceSet);
-        DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
+    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
+        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_join.json"), Charsets.UTF_8));
-      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryDataBatch b : results) {
+      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (final QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
-        for (VectorWrapper<?> vw : loader) {
+        for (final VectorWrapper<?> vw : loader) {
           System.out.println(vw.getValueVector().getField().toExpr());
-          ValueVector vv = vw.getValueVector();
+          final ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            Object o = vv.getAccessor().getObject(i);
+            final Object o = vv.getAccessor().getObject(i);
             System.out.println(vv.getAccessor().getObject(i));
           }
         }
@@ -182,27 +179,26 @@ public class TestOptiqPlans extends ExecTest {
 
   @Test
   public void testFilterString() throws Exception {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try (Drillbit bit1 = new Drillbit(config, serviceSet);
-        DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
+    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
+        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_string_filter.json"), Charsets.UTF_8));
-      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryDataBatch b : results) {
+      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (final QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
-        for (VectorWrapper<?> vw : loader) {
+        for (final VectorWrapper<?> vw : loader) {
           System.out.println(vw.getValueVector().getField().toExpr());
-          ValueVector vv = vw.getValueVector();
+          final ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            Object o = vv.getAccessor().getObject(i);
+            final Object o = vv.getAccessor().getObject(i);
             if (vv instanceof VarBinaryVector) {
-              VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              VarBinaryHolder vbh = new VarBinaryHolder();
+              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
+              final VarBinaryHolder vbh = new VarBinaryHolder();
               x.get(i, vbh);
               System.out.printf("%d..%d", vbh.start, vbh.end);
 
@@ -222,27 +218,26 @@ public class TestOptiqPlans extends ExecTest {
 
   @Test
   public void testLogicalJsonScan() throws Exception {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try (Drillbit bit1 = new Drillbit(config, serviceSet);
-        DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
+    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
+        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_json_scan.json"), Charsets.UTF_8));
-      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryDataBatch b : results) {
+      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (final QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
-        for (VectorWrapper vw : loader) {
+        for (final VectorWrapper vw : loader) {
           System.out.println(vw.getValueVector().getField().toExpr());
-          ValueVector vv = vw.getValueVector();
+          final ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            Object o = vv.getAccessor().getObject(i);
+            final Object o = vv.getAccessor().getObject(i);
             if (vv instanceof VarBinaryVector) {
-              VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              VarBinaryHolder vbh = new VarBinaryHolder();
+              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
+              final VarBinaryHolder vbh = new VarBinaryHolder();
               x.get(i, vbh);
               System.out.printf("%d..%d", vbh.start, vbh.end);
 
@@ -262,27 +257,26 @@ public class TestOptiqPlans extends ExecTest {
 
   @Test
   public void testOrderVarbinary() throws Exception {
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
+    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try (Drillbit bit1 = new Drillbit(config, serviceSet);
-        DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
+    try (final Drillbit bit1 = new Drillbit(config, serviceSet);
+        final DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      final List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_order_varbinary.json"), Charsets.UTF_8));
-      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryDataBatch b : results) {
+      final RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (final QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
-        for (VectorWrapper vw : loader) {
+        for (final VectorWrapper vw : loader) {
           System.out.println(vw.getValueVector().getField().toExpr());
-          ValueVector vv = vw.getValueVector();
+          final ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-            Object o = vv.getAccessor().getObject(i);
+            final Object o = vv.getAccessor().getObject(i);
             if (vv instanceof VarBinaryVector) {
-              VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
-              VarBinaryHolder vbh = new VarBinaryHolder();
+              final VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
+              final VarBinaryHolder vbh = new VarBinaryHolder();
               x.get(i, vbh);
               System.out.printf("%d..%d", vbh.start, vbh.end);
 
@@ -307,22 +301,21 @@ public class TestOptiqPlans extends ExecTest {
         bitContext.getMetrics();
         result = new MetricRegistry();
         bitContext.getAllocator();
-        result = new TopLevelAllocator();
+        result = RootAllocatorFactory.newRoot(config);
         bitContext.getConfig();
-        result = c;
+        result = config;
       }
     };
 
-    StoragePluginRegistry reg = new StoragePluginRegistry(bitContext);
+    final StoragePluginRegistry reg = new StoragePluginRegistry(bitContext);
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(),
+    final PhysicalPlanReader reader = new PhysicalPlanReader(config, config.getMapper(),
         CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), reg);
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false)
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false)
         .iterator().next()));
     return exec;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index 09ba1a5..5d0ea2d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -21,12 +21,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
@@ -41,34 +40,32 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
 public class TestSimpleFragmentRun extends PopUnitTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class);
-
-  private static final Charset UTF_8 = Charset.forName("UTF-8");
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class);
 
   @Test
   public void runNoExchangeFragment() throws Exception {
-    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-        Drillbit bit = new Drillbit(CONFIG, serviceSet);
-        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
+    try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+        final Drillbit bit = new Drillbit(CONFIG, serviceSet);
+        final DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
 
     // run query.
     bit.run();
     client.connect();
-    String path = "/physical_test2.json";
+    final String path = "/physical_test2.json";
 //      String path = "/filter/test1.json";
-    List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(path), Charsets.UTF_8));
+    final List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(path), Charsets.UTF_8));
 
     // look at records
-    RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
+    final RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
     int recordCount = 0;
-    for (QueryDataBatch batch : results) {
-      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+    for (final QueryDataBatch batch : results) {
+      final boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
       boolean firstColumn = true;
 
       // print headers.
       if (schemaChanged) {
         System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (VectorWrapper<?> value : batchLoader) {
+        for (final VectorWrapper<?> value : batchLoader) {
 
           if (firstColumn) {
             firstColumn = false;
@@ -86,7 +83,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       for (int i = 0; i < batchLoader.getRecordCount(); i++) {
         boolean first = true;
         recordCount++;
-        for (VectorWrapper<?> value : batchLoader) {
+        for (final VectorWrapper<?> value : batchLoader) {
           if (first) {
             first = false;
           } else {
@@ -108,20 +105,20 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
 
   @Test
   public void runJSONScanPopFragment() throws Exception {
-    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-         Drillbit bit = new Drillbit(CONFIG, serviceSet);
-         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
+    try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         final Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         final DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
 
       // run query.
       bit.run();
       client.connect();
-      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL,
+      final List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/physical_json_scan_test1.json"), Charsets.UTF_8)
               .replace("#{TEST_FILE}", FileUtils.getResourceAsFile("/scan_json_test_1.json").toURI().toString())
       );
 
       // look at records
-      RecordBatchLoader batchLoader = new RecordBatchLoader(new TopLevelAllocator(CONFIG));
+      final RecordBatchLoader batchLoader = new RecordBatchLoader(RootAllocatorFactory.newRoot(CONFIG));
       int recordCount = 0;
 
       //int expectedBatchCount = 2;
@@ -129,7 +126,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       //assertEquals(expectedBatchCount, results.size());
 
       for (int i = 0; i < results.size(); ++i) {
-        QueryDataBatch batch = results.get(i);
+        final QueryDataBatch batch = results.get(i);
         if (i == 0) {
           assertTrue(batch.hasData());
         } else {
@@ -143,7 +140,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
 
         // print headers.
         System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (VectorWrapper<?> v : batchLoader) {
+        for (final VectorWrapper<?> v : batchLoader) {
 
           if (firstColumn) {
             firstColumn = false;
@@ -162,14 +159,14 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
         for (int r = 0; r < batchLoader.getRecordCount(); r++) {
           boolean first = true;
           recordCount++;
-          for (VectorWrapper<?> v : batchLoader) {
+          for (final VectorWrapper<?> v : batchLoader) {
             if (first) {
               first = false;
             } else {
               System.out.print("\t");
             }
 
-            ValueVector.Accessor accessor = v.getValueVector().getAccessor();
+            final ValueVector.Accessor accessor = v.getValueVector().getAccessor();
             System.out.print(accessor.getObject(r));
           }
           if (!first) {
@@ -183,5 +180,4 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       assertEquals(2, recordCount);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bfe6cfad/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 d551319..4cbd4a9 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
@@ -44,7 +44,7 @@ import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers;
 import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -65,13 +65,12 @@ import com.google.common.io.Files;
 import com.sun.codemodel.JClassAlreadyExistsException;
 
 public class TestSimpleFunctions extends ExecTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class);
-
-  DrillConfig c = DrillConfig.create();
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class);
+  private final DrillConfig c = DrillConfig.create();
 
   @Test
   public void testHashFunctionResolution(@Injectable DrillConfig config) throws JClassAlreadyExistsException, IOException {
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config);
     // test required vs nullable Int input
     resolveHash(config,
         new TypedNullConstant(Types.optional(TypeProtos.MinorType.INT)),
@@ -136,46 +135,44 @@ public class TestSimpleFunctions extends ExecTest {
   public void resolveHash(DrillConfig config, LogicalExpression arg, TypeProtos.MajorType expectedArg,
                                     TypeProtos.MajorType expectedOut, TypeProtos.DataMode expectedBestInputMode,
                                     FunctionImplementationRegistry registry) throws JClassAlreadyExistsException, IOException {
-    List<LogicalExpression> args = new ArrayList<>();
+    final List<LogicalExpression> args = new ArrayList<>();
     args.add(arg);
-    String[] registeredNames = { "hash" };
+    final String[] registeredNames = { "hash" };
     FunctionCall call = new FunctionCall(
         "hash",
         args,
         ExpressionPosition.UNKNOWN
     );
-    FunctionResolver resolver = FunctionResolverFactory.getResolver(call);
-    DrillFuncHolder matchedFuncHolder = registry.findDrillFunction(resolver, call);
+    final FunctionResolver resolver = FunctionResolverFactory.getResolver(call);
+    final DrillFuncHolder matchedFuncHolder = registry.findDrillFunction(resolver, call);
     assertEquals( expectedBestInputMode, matchedFuncHolder.getParmMajorType(0).getMode());
   }
 
   @Test
   public void testSubstring(@Injectable final DrillbitContext bitContext,
-                            @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
+                            @Injectable UserServer.UserClientConnection connection) throws Throwable {
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       bitContext.getConfig(); result = c;
       bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstring.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstring.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      NullableVarCharVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarCharVector.class);
-      NullableVarCharVector.Accessor a1;
-      a1 = c1.getAccessor();
+    while(exec.next()) {
+      final NullableVarCharVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarCharVector.class);
+      final NullableVarCharVector.Accessor a1 = c1.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c1.getAccessor().getValueCount(); i++){
+      for(int i = 0; i < c1.getAccessor().getValueCount(); i++) {
         if (!a1.isNull(i)) {
-          NullableVarCharHolder holder = new NullableVarCharHolder();
+          final NullableVarCharHolder holder = new NullableVarCharHolder();
           a1.get(i, holder);
           assertEquals("aaaa", StringFunctionHelpers.toStringFromUTF8(holder.start,  holder.end,  holder.buffer));
           ++count;
@@ -184,40 +181,37 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
-
   }
 
   @Test
   public void testSubstringNegative(@Injectable final DrillbitContext bitContext,
-                                    @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    new NonStrictExpectations(){{
+                                    @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       bitContext.getConfig(); result = c;
       bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstringNegative.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstringNegative.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      NullableVarCharVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarCharVector.class);
-      NullableVarCharVector.Accessor a1;
-      a1 = c1.getAccessor();
+    while(exec.next()) {
+      final NullableVarCharVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarCharVector.class);
+      final NullableVarCharVector.Accessor a1 = c1.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c1.getAccessor().getValueCount(); i++){
+      for(int i = 0; i < c1.getAccessor().getValueCount(); i++) {
         if (!a1.isNull(i)) {
-          NullableVarCharHolder holder = new NullableVarCharHolder();
+          final NullableVarCharHolder holder = new NullableVarCharHolder();
           a1.get(i, holder);
           //when offset is negative, substring return empty string.
           assertEquals("", StringFunctionHelpers.toStringFromUTF8(holder.start,  holder.end,  holder.buffer));
@@ -227,40 +221,37 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
-
   }
 
   @Test
   public void testByteSubstring(@Injectable final DrillbitContext bitContext,
-                                  @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-    new NonStrictExpectations(){{
+                                  @Injectable UserServer.UserClientConnection connection) throws Throwable {
+    new NonStrictExpectations() {{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = RootAllocatorFactory.newRoot(c);
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
       bitContext.getConfig(); result = c;
       bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c);
     }};
 
-    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testByteSubstring.json"), Charsets.UTF_8));
-    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-    FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
-    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    final PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    final PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testByteSubstring.json"), Charsets.UTF_8));
+    final FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+    final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
-    while(exec.next()){
-      NullableVarBinaryVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarBinaryVector.class);
-      NullableVarBinaryVector.Accessor a1;
-      a1 = c1.getAccessor();
+    while(exec.next()) {
+      final NullableVarBinaryVector c1 = exec.getValueVectorById(new SchemaPath("col3", ExpressionPosition.UNKNOWN), NullableVarBinaryVector.class);
+      final NullableVarBinaryVector.Accessor a1 = c1.getAccessor();
 
       int count = 0;
-      for(int i = 0; i < c1.getAccessor().getValueCount(); i++){
+      for(int i = 0; i < c1.getAccessor().getValueCount(); i++) {
         if (!a1.isNull(i)) {
-          NullableVarBinaryHolder holder = new NullableVarBinaryHolder();
+          final NullableVarBinaryHolder holder = new NullableVarBinaryHolder();
           a1.get(i, holder);
           assertEquals("aa", StringFunctionHelpers.toStringFromUTF8(holder.start,  holder.end,  holder.buffer));
           ++count;
@@ -269,11 +260,9 @@ public class TestSimpleFunctions extends ExecTest {
       assertEquals(50, count);
     }
 
-    if(context.getFailureCause() != null){
+    if(context.getFailureCause() != null) {
       throw context.getFailureCause();
     }
     assertTrue(!context.isFailed());
-
   }
-
 }