You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2017/09/17 05:43:05 UTC

[2/4] drill git commit: DRILL-5723: Added System Internal Options That can be Modified at Runtime Changes include:

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/ZookeeperHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ZookeeperHelper.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ZookeeperHelper.java
index 42247ea..8a07fd2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ZookeeperHelper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ZookeeperHelper.java
@@ -21,9 +21,11 @@ import static com.google.common.base.Throwables.propagate;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
 import java.util.Properties;
 
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.util.MiniZooKeeperCluster;
 
 /**
@@ -32,7 +34,7 @@ import org.apache.drill.exec.util.MiniZooKeeperCluster;
  * <p>Tests that need a Zookeeper instance can initialize a static instance of this class in
  * their {@link org.junit.BeforeClass} section to set up Zookeeper.
  * <p>
- * Modified to also work in the {@link ClusterFixture} class. The "bare" use sets up a
+ * Modified to also work in the {@link org.apache.drill.test.ClusterFixture} class. The "bare" use sets up a
  * Drill config. The use in the cluster fixture delegates to the cluster fixture the task
  * of setting up the Drill config. In the "bare" case, the port number comes from the
  * Drill config. In the cluster fixture case, we let ZK choose the port and we learn
@@ -43,7 +45,7 @@ import org.apache.drill.exec.util.MiniZooKeeperCluster;
 public class ZookeeperHelper {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZookeeperHelper.class);
 
-  private final File testDir = new File("target/test-data");
+  private final File testDir = new File("target/test-data/" + ManagementFactory.getRuntimeMXBean().getName());
   private final DrillConfig config;
   private String zkUrl;
   private MiniZooKeeperCluster zkCluster;
@@ -54,7 +56,7 @@ public class ZookeeperHelper {
    * <p>Will create a "test-data" directory for Zookeeper's use if one doesn't already exist.
    */
   public ZookeeperHelper() {
-    this(false);
+    this(false, true);
   }
 
   /**
@@ -63,7 +65,7 @@ public class ZookeeperHelper {
    * <p>Will create a "test-data" directory for Zookeeper's use if one doesn't already exist.
    * @param failureInCancelled pass true if you want failures in cancelled fragments to be reported as failures
    */
-  public ZookeeperHelper(boolean failureInCancelled) {
+  public ZookeeperHelper(boolean failureInCancelled, boolean autoPort) {
     final Properties overrideProps = new Properties();
     // Forced to disable this, because currently we leak memory which is a known issue for query cancellations.
     // Setting this causes unittests to fail.
@@ -71,21 +73,13 @@ public class ZookeeperHelper {
       overrideProps.setProperty(ExecConstants.RETURN_ERROR_FOR_FAILURE_IN_CANCELLED_FRAGMENTS, "true");
     }
     config = DrillConfig.create(overrideProps);
-    zkUrl = config.getString(ExecConstants.ZK_CONNECTION);
 
-    testDir.mkdirs();
-  }
-
-  /**
-   * Constructor for the cluster fixture case. Don't create a Drill config.
-   * Let ZK choose the port.
-   */
+    if (!autoPort) {
+      zkUrl = config.getString(ExecConstants.ZK_CONNECTION);
+    }
 
-  public ZookeeperHelper(String dummy) {
-    zkUrl = null;
-    config = null;
     testDir.mkdirs();
- }
+  }
 
   /**
    * Start the Zookeeper instance.
@@ -101,9 +95,12 @@ public class ZookeeperHelper {
 
     try {
       zkCluster = new MiniZooKeeperCluster();
+      zkCluster.setDefaultClientPort(MiniZooKeeperCluster.DEFAULT_PORT);
+
       if (zkUrl != null) {
         zkCluster.setDefaultClientPort(Integer.parseInt(zkUrl.split(":")[1]));
       }
+
       zkCluster.startup(testDir, numServers);
       if (zkUrl == null) {
         zkUrl = "localhost:" + zkCluster.getClientPort();
@@ -125,8 +122,13 @@ public class ZookeeperHelper {
     } catch (IOException e) {
       // since this is meant to be used in a test's cleanup, we don't propagate the exception
       final String message = "Unable to shutdown Zookeeper";
-      System.err.println(message + '.');
-      logger.warn(message, e);
+      logger.error(message, e);
+    } finally {
+      try {
+        org.apache.commons.io.FileUtils.deleteDirectory(testDir);
+      } catch (IOException e) {
+        logger.error("Unable to delete zookeeper directory", e);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/client/ConnectTriesPropertyTestClusterBits.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/ConnectTriesPropertyTestClusterBits.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/ConnectTriesPropertyTestClusterBits.java
index e083af2..5c28af1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/ConnectTriesPropertyTestClusterBits.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/ConnectTriesPropertyTestClusterBits.java
@@ -84,6 +84,7 @@ public class ConnectTriesPropertyTestClusterBits {
   @AfterClass
   public static void testCleanUp() throws Exception {
     AutoCloseables.close(drillbits);
+    zkHelper.stopZookeeper();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index 85757b1..e27ce13 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -38,8 +38,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
   private static String plan;
 
   @BeforeClass
-  public static void setUp() throws Exception {
-    DrillSystemTestBase.setUp();
+  public void setUp() throws Exception {
+    this.setUp();
     plan = Resources.toString(Resources.getResource("simple_plan.json"), Charsets.UTF_8);
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
index 3c8ca01..b941816 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
@@ -27,9 +27,6 @@ import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.rpc.user.UserSession;
-import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
-import org.apache.drill.exec.server.options.OptionValue.OptionScope;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.codehaus.commons.compiler.CompileException;
 import org.junit.Assert;
@@ -59,7 +56,7 @@ public class TestClassTransformation extends BaseTestQuery {
   @Test
   public void testJaninoClassCompiler() throws Exception {
     logger.debug("Testing JaninoClassCompiler");
-    sessionOptions.setOption(OptionValue.createString(OptionType.SESSION, ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JANINO.name(), OptionScope.SESSION));
+    sessionOptions.setLocalOption(ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JANINO.name());
     for (int i = 0; i < ITERATION_COUNT; i++) {
       compilationInnerClass(false); // Traditional byte-code manipulation
       compilationInnerClass(true); // Plain-old Java
@@ -69,8 +66,7 @@ public class TestClassTransformation extends BaseTestQuery {
   @Test
   public void testJDKClassCompiler() throws Exception {
     logger.debug("Testing JDKClassCompiler");
-    OptionType type = OptionType.SESSION;
-    sessionOptions.setOption(OptionValue.createString(type, ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JDK.name(), OptionScope.SESSION));
+    sessionOptions.setLocalOption(ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JDK.name());
     for (int i = 0; i < ITERATION_COUNT; i++) {
       compilationInnerClass(false); // Traditional byte-code manipulation
       compilationInnerClass(true); // Plain-old Java
@@ -82,10 +78,9 @@ public class TestClassTransformation extends BaseTestQuery {
     CodeGenerator<ExampleInner> cg = newCodeGenerator(ExampleInner.class, ExampleTemplateWithInner.class);
     ClassSet classSet = new ClassSet(null, cg.getDefinition().getTemplateClassName(), cg.getMaterializedClassName());
     String sourceCode = cg.generateAndGet();
-    OptionType type = OptionType.SESSION;
-    sessionOptions.setOption(OptionValue.createString(type, ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JDK.name(), OptionScope.SESSION));
+    sessionOptions.setLocalOption(ClassCompilerSelector.JAVA_COMPILER_OPTION, ClassCompilerSelector.CompilerPolicy.JDK.name());
 
-    sessionOptions.setOption(OptionValue.createBoolean(type, ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION, false, OptionScope.SESSION));
+    sessionOptions.setLocalOption(ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION, false);
     @SuppressWarnings("resource")
     QueryClassLoader loader = new QueryClassLoader(config, sessionOptions);
     final byte[][] codeWithoutDebug = loader.getClassByteCode(classSet.generated, sourceCode);
@@ -95,7 +90,7 @@ public class TestClassTransformation extends BaseTestQuery {
       sizeWithoutDebug += bs.length;
     }
 
-    sessionOptions.setOption(OptionValue.createBoolean(type, ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION, true, OptionScope.SESSION));
+    sessionOptions.setLocalOption(ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION, true);
     loader = new QueryClassLoader(config, sessionOptions);
     final byte[][] codeWithDebug = loader.getClassByteCode(classSet.generated, sourceCode);
     loader.close();

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
index f9a6358..f452669 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
@@ -22,7 +22,9 @@ import com.google.common.io.Files;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
+import org.apache.drill.exec.server.options.OptionDefinition;
 import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -46,10 +48,11 @@ public class TestInboundImpersonationPrivileges extends BaseTestImpersonation {
   }
 
   private static boolean checkPrivileges(final String proxyName, final String targetName) {
+    OptionDefinition optionDefinition = SystemOptionManager.createDefaultOptionDefinitions().get(ExecConstants.IMPERSONATION_POLICIES_KEY);
     ExecConstants.IMPERSONATION_POLICY_VALIDATOR.validate(
-        OptionValue.createString(OptionValue.OptionType.SYSTEM,
+        OptionValue.create(optionDefinition.getMetaData().getType(),
             ExecConstants.IMPERSONATION_POLICIES_KEY,
-            IMPERSONATION_POLICIES,OptionValue.OptionScope.SYSTEM), null);
+            IMPERSONATION_POLICIES,OptionValue.OptionScope.SYSTEM), optionDefinition.getMetaData(),null);
     try {
       return InboundImpersonationManager.hasImpersonationPrivileges(proxyName, targetName, IMPERSONATION_POLICIES);
     } catch (final Exception e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 02e047e..ed1c06e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -119,7 +119,7 @@ public class TestConvertFunctions extends BaseTestQuery {
           .run();
     } finally {
       // restore the system option
-      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption.string_val);
       test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
     }
   }
@@ -547,7 +547,7 @@ public class TestConvertFunctions extends BaseTestQuery {
       testBigIntVarCharReturnTripConvertLogical();
     } finally {
       // restore the system option
-      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption.string_val);
     }
   }
 
@@ -564,7 +564,7 @@ public class TestConvertFunctions extends BaseTestQuery {
     } catch(RpcException e) {
       caughtException = true;
     } finally {
-      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption.string_val);
     }
 
     // Yes: sometimes this works, sometimes it does not...
@@ -579,7 +579,7 @@ public class TestConvertFunctions extends BaseTestQuery {
       testBigIntVarCharReturnTripConvertLogical();
     } finally {
       // restore the system option
-      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption.string_val);
     }
   }
 
@@ -650,7 +650,7 @@ public class TestConvertFunctions extends BaseTestQuery {
 
     } finally {
       // restore the system option
-      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption.string_val);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
index fb43f3e..7dce887 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
@@ -19,7 +19,6 @@
 package org.apache.drill.exec.physical.impl.agg;
 
 import ch.qos.logback.classic.Level;
-import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.physical.impl.aggregate.HashAggTemplate;
@@ -42,7 +41,7 @@ import static org.junit.Assert.assertTrue;
 /**
  *  Test spilling for the Hash Aggr operator (using the mock reader)
  */
-public class TestHashAggrSpill extends BaseTestQuery {
+public class TestHashAggrSpill {
 
     private void runAndDump(ClientFixture client, String sql, long expectedRows, long spillCycle, long spilledPartitions) throws Exception {
         String plan = client.queryBuilder().sql(sql).explainJson();

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
index c83a30f..e3842be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
@@ -21,6 +21,7 @@ import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.test.OperatorFixture;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -102,10 +103,7 @@ public class TestMergeJoinAdvanced extends BaseTestQuery {
       test("select * from dfs_test.`%s/join/j1` j1 left outer join dfs_test.`%s/join/j2` j2 on (j1.c_varchar = j2.c_varchar)",
         TEST_RES_PATH, TEST_RES_PATH);
     } finally {
-      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(PlannerSettings.BROADCAST.getDefault().bool_val));
-      setSessionOption(PlannerSettings.HASHJOIN.getOptionName(), String.valueOf(PlannerSettings.HASHJOIN.getDefault().bool_val));
-      setSessionOption(ExecConstants.SLICE_TARGET, String.valueOf(ExecConstants.SLICE_TARGET_DEFAULT));
-      setSessionOption(ExecConstants.MAX_WIDTH_PER_NODE_KEY, String.valueOf(ExecConstants.MAX_WIDTH_PER_NODE.getDefault().num_val));
+      test("ALTER SESSION RESET ALL");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java
index ae7c57b..b5846bd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java
@@ -22,6 +22,7 @@ import org.apache.drill.PlanTestBase;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.test.OperatorFixture;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -71,7 +72,8 @@ public class TestLimitWithExchanges extends BaseTestQuery {
       final String[] expectedPlan5 = {"(?s)Limit\\(fetch=\\[1\\].*UnionExchange.*Limit\\(fetch=\\[1\\]\\).*Join"};
       testLimitHelper(sql5, expectedPlan5, excludedPlan, 1);
     } finally {
-      test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_OPTION.getDefault().getValue());
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `%s` = %s", ExecConstants.SLICE_TARGET, testOptionSet.getDefault(ExecConstants.SLICE_TARGET).getValue());
     }
   }
 
@@ -92,7 +94,8 @@ public class TestLimitWithExchanges extends BaseTestQuery {
 
       testLimitHelper(sql2, expectedPlan, excludedPlan2, 5);
     } finally {
-      test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_OPTION.getDefault().getValue());
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `planner.slice_target` = " + testOptionSet.getDefault(ExecConstants.SLICE_TARGET).getValue());
     }
   }
 
@@ -121,7 +124,8 @@ public class TestLimitWithExchanges extends BaseTestQuery {
       testLimitHelper(sql3, expectedPlan2, excludedPlan2, 10);
       testLimitHelper(sql4, expectedPlan2, excludedPlan2, 10);
     } finally {
-      test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_OPTION.getDefault().getValue());
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `planner.slice_target` = " + testOptionSet.getDefault(ExecConstants.SLICE_TARGET).getValue());
     }
   }
 
@@ -142,5 +146,4 @@ public class TestLimitWithExchanges extends BaseTestQuery {
     final int actualRecordCount = testSql(sql);
     assertEquals(String.format("Received unexpected number of rows in output: expected=%d, received=%s", expectedRecordCount, actualRecordCount), expectedRecordCount, actualRecordCount);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
index 8d690d3..40de4b1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
@@ -35,6 +35,7 @@ import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.IntVector;
@@ -82,7 +83,7 @@ public class TestOrderedPartitionExchange extends PopUnitTestBase {
           int rows = b.getHeader().getRowCount();
           count += rows;
           DrillConfig config = DrillConfig.create();
-          RecordBatchLoader loader = new RecordBatchLoader(new BootStrapContext(config, ClassPathScanner.fromPrescan(config)).getAllocator());
+          RecordBatchLoader loader = new RecordBatchLoader(new BootStrapContext(config, SystemOptionManager.createDefaultOptionDefinitions(), ClassPathScanner.fromPrescan(config)).getAllocator());
           loader.load(b.getHeader().getDef(), b.getData());
           BigIntVector vv1 = (BigIntVector)loader.getValueAccessorById(BigIntVector.class, loader.getValueVectorId(
                   new SchemaPath("col1", ExpressionPosition.UNKNOWN)).getFieldIds()).getValueVector();

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
index 4114a04..9578332 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
@@ -62,7 +62,7 @@ import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.server.options.OptionValue.AccessibleScopes;
 import org.apache.drill.exec.server.options.OptionValue.OptionScope;
 import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.work.QueryWorkUnit;
@@ -181,20 +181,20 @@ public class TestPartitionSender extends PlanTestBase {
 
     final OptionList options = new OptionList();
     // try multiple scenarios with different set of options
-    options.add(OptionValue.createLong(OptionType.SESSION, "planner.slice_target", 1, OptionScope.SESSION));
+    options.add(OptionValue.create(OptionValue.AccessibleScopes.SESSION, "planner.slice_target", 1, OptionScope.SESSION));
     testThreadsHelper(hashToRandomExchange, drillbitContext, options,
         incoming, registry, planReader, planningSet, rootFragment, 1);
 
     options.clear();
-    options.add(OptionValue.createLong(OptionType.SESSION, "planner.slice_target", 1, OptionScope.SESSION));
-    options.add(OptionValue.createLong(OptionType.SESSION, "planner.partitioner_sender_max_threads", 10, OptionScope.SESSION));
+    options.add(OptionValue.create(AccessibleScopes.SESSION, "planner.slice_target", 1, OptionScope.SESSION));
+    options.add(OptionValue.create(OptionValue.AccessibleScopes.SESSION, "planner.partitioner_sender_max_threads", 10, OptionScope.SESSION));
     hashToRandomExchange.setCost(1000);
     testThreadsHelper(hashToRandomExchange, drillbitContext, options,
         incoming, registry, planReader, planningSet, rootFragment, 10);
 
     options.clear();
-    options.add(OptionValue.createLong(OptionType.SESSION, "planner.slice_target", 1000, OptionScope.SESSION));
-    options.add(OptionValue.createLong(OptionType.SESSION, "planner.partitioner_sender_threads_factor",2, OptionScope.SESSION));
+    options.add(OptionValue.create(AccessibleScopes.SESSION, "planner.slice_target", 1000, OptionScope.SESSION));
+    options.add(OptionValue.create(AccessibleScopes.SESSION, "planner.partitioner_sender_threads_factor",2, OptionScope.SESSION));
     hashToRandomExchange.setCost(14000);
     testThreadsHelper(hashToRandomExchange, drillbitContext, options,
         incoming, registry, planReader, planningSet, rootFragment, 2);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index fd19378..f85cada 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -35,12 +35,14 @@ import java.util.Map;
 
 import com.google.common.base.Joiner;
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.DrillVersionInfo;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.fn.interp.TestConstantFolding;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -264,7 +266,9 @@ public class TestParquetWriter extends BaseTestQuery {
       String inputTable = "cp.`tpch/lineitem.parquet`";
       runTestAndValidate(selection, validationSelection, inputTable, "lineitem_parquet_converted");
     } finally {
-      test("alter session set `%s` = %b", ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING, ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR.getDefault().bool_val);
+      OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `%s` = %b", ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING,
+        optionSet.getDefault(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING).bool_val);
     }
   }
 
@@ -318,8 +322,11 @@ public class TestParquetWriter extends BaseTestQuery {
       String inputTable = "cp.`tpch/supplier.parquet`";
       runTestAndValidate("*", "*", inputTable, "supplier_parquet_no_dict_uncompressed");
     } finally {
-      test(String.format("alter session set `%s` = %b", ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING, ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR.getDefault().bool_val));
-      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test(String.format("alter session set `%s` = %b", ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING,
+        optionSet.getDefault(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING).bool_val));
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE,
+        optionSet.getDefault(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE).string_val));
     }
   }
 
@@ -330,7 +337,9 @@ public class TestParquetWriter extends BaseTestQuery {
       String inputTable = "cp.`tpch/supplier.parquet`";
       runTestAndValidate("*", "*", inputTable, "supplier_parquet_dict_gzip");
     } finally {
-      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE,
+        optionSet.getDefault(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE).string_val));
     }
   }
 
@@ -452,8 +461,9 @@ public class TestParquetWriter extends BaseTestQuery {
         .optionSettingQueriesForBaseline("alter system set `store.parquet.use_new_reader` = true")
         .build().run();
     } finally {
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
       test("alter system set `%s` = %b", ExecConstants.PARQUET_NEW_RECORD_READER,
-          ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR.getDefault().bool_val);
+        optionSet.getDefault(ExecConstants.PARQUET_NEW_RECORD_READER).bool_val);
     }
   }
 
@@ -472,10 +482,10 @@ public class TestParquetWriter extends BaseTestQuery {
             "alter system set `store.parquet.use_new_reader` = true")
         .build().run();
     } finally {
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
       test("alter system set `%s` = %b",
           ExecConstants.PARQUET_NEW_RECORD_READER,
-          ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR
-              .getDefault().bool_val);
+          optionSet.getDefault(ExecConstants.PARQUET_NEW_RECORD_READER).bool_val);
     }
   }
 
@@ -947,7 +957,8 @@ public class TestParquetWriter extends BaseTestQuery {
       String inputTable = "cp.`tpch/supplier.parquet`";
         runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_gzip");
     } finally {
-      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, optionSet.getDefault(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE).string_val));
     }
   }
 
@@ -958,7 +969,8 @@ public class TestParquetWriter extends BaseTestQuery {
       String inputTable = "cp.`supplier_snappy.parquet`";
       runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_snappy");
     } finally {
-      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, optionSet.getDefault(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE).string_val));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
index 4f92c6f..b4173d3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.writer;
 
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
@@ -92,7 +93,8 @@ public class TestParquetWriterEmptyFiles extends BaseTestQuery {
         .go();
     } finally {
       // restore the session option
-      test("ALTER SESSION SET `store.parquet.block-size` = %d", ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR.getDefault().num_val);
+      final OperatorFixture.TestOptionSet optionSet = new OperatorFixture.TestOptionSet();
+      test("ALTER SESSION SET `store.parquet.block-size` = %d", optionSet.getDefault(ExecConstants.PARQUET_BLOCK_SIZE).num_val);
       deleteTableIfExists(outputFile);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
index a3ea198..8165b0d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
@@ -51,6 +51,7 @@ import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.security.KerberosHelper;
 import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
@@ -119,7 +120,7 @@ public class TestBitBitKerberos extends BaseTestQuery {
     updateTestCluster(1, newConfig);
 
     ScanResult result = ClassPathScanner.fromPrescan(newConfig);
-    c1 = new BootStrapContext(newConfig, result);
+    c1 = new BootStrapContext(newConfig, SystemOptionManager.createDefaultOptionDefinitions(), result);
     setupFragmentContextAndManager();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
index bdc3230..8c53915 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitRpc.java
@@ -45,6 +45,7 @@ import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
@@ -65,9 +66,9 @@ public class TestBitRpc extends ExecTest {
   public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus) throws Exception {
 
     DrillConfig config1 = DrillConfig.create();
-    final BootStrapContext c = new BootStrapContext(config1, ClassPathScanner.fromPrescan(config1));
+    final BootStrapContext c = new BootStrapContext(config1, SystemOptionManager.createDefaultOptionDefinitions(), ClassPathScanner.fromPrescan(config1));
     DrillConfig config2 = DrillConfig.create();
-    BootStrapContext c2 = new BootStrapContext(config2, ClassPathScanner.fromPrescan(config2));
+    BootStrapContext c2 = new BootStrapContext(config2, SystemOptionManager.createDefaultOptionDefinitions(), ClassPathScanner.fromPrescan(config2));
 
     final FragmentContext fcon = new MockUp<FragmentContext>(){
       BufferAllocator getAllocator(){

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
index f525020..52f5fb9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
@@ -81,6 +81,7 @@ import org.apache.drill.exec.work.foreman.ForemanException;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -185,7 +186,7 @@ public class TestDrillbitResilience extends DrillTest {
     System.setProperty(ExecConstants.HTTP_ENABLE, "false");
 
     // turn on error for failure in cancelled fragments
-    zkHelper = new ZookeeperHelper(true);
+    zkHelper = new ZookeeperHelper(true, true);
     zkHelper.startZookeeper(1);
 
     // use a non-null service set so that the drillbits can use port hunting
@@ -805,8 +806,9 @@ public class TestDrillbitResilience extends DrillTest {
       final String query = "SELECT sales_city, COUNT(*) cnt FROM cp.`region.json` GROUP BY sales_city";
       assertCancelledWithoutException(control, new ListenerThatCancelsQueryAfterFirstBatchOfData(), query);
     } finally {
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
       setSessionOption(SLICE_TARGET, Long.toString(SLICE_TARGET_DEFAULT));
-      setSessionOption(HASHAGG.getOptionName(), HASHAGG.getDefault().bool_val.toString());
+      setSessionOption(HASHAGG.getOptionName(), testOptionSet.getDefault(HASHAGG.getOptionName()).bool_val.toString());
     }
   }
 
@@ -836,10 +838,11 @@ public class TestDrillbitResilience extends DrillTest {
       final long after = countAllocatedMemory();
       assertEquals(String.format("We are leaking %d bytes", after - before), before, after);
     } finally {
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
       setSessionOption(SLICE_TARGET, Long.toString(SLICE_TARGET_DEFAULT));
-      setSessionOption(HASHAGG.getOptionName(), HASHAGG.getDefault().bool_val.toString());
+      setSessionOption(HASHAGG.getOptionName(), testOptionSet.getDefault(HASHAGG.getOptionName()).bool_val.toString());
       setSessionOption(PARTITION_SENDER_SET_THREADS.getOptionName(),
-          Long.toString(PARTITION_SENDER_SET_THREADS.getDefault().num_val));
+          Long.toString(testOptionSet.getDefault(PARTITION_SENDER_SET_THREADS.getOptionName()).num_val));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
index 2761faa..f707bad 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
@@ -26,7 +26,7 @@ import static org.apache.drill.exec.ExecConstants.ENABLE_VERBOSE_ERRORS_KEY;
 import static org.apache.drill.exec.ExecConstants.SLICE_TARGET;
 import static org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.VALIDATION;
 
-public class TestOptions extends BaseTestQuery{
+public class TestOptions extends BaseTestQuery {
 //  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptions.class);
 
   @Test
@@ -56,7 +56,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER session SET `%s` = %d;", SLICE_TARGET,
       ExecConstants.SLICE_TARGET_DEFAULT);
     testBuilder()
-        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
+        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
         .unOrdered()
         .baselineColumns("status")
         .baselineValues("DEFAULT")
@@ -68,7 +68,7 @@ public class TestOptions extends BaseTestQuery{
   public void setAndResetSessionOption() throws Exception {
     // check unchanged
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
       .unOrdered()
       .expectsEmptyResultSet()
       .build()
@@ -77,9 +77,9 @@ public class TestOptions extends BaseTestQuery{
     // change option
     test("SET `%s` = %d;", SLICE_TARGET, 10);
     // check changed
-    test("SELECT status, type, name FROM sys.options WHERE type = 'SESSION';");
+    test("SELECT status, accessibleScopes, name FROM sys.options WHERE optionScope = 'SESSION';");
     testBuilder()
-      .sqlQuery("SELECT num_val FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
+      .sqlQuery("SELECT num_val FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
       .unOrdered()
       .baselineColumns("num_val")
       .baselineValues((long) 10)
@@ -90,7 +90,7 @@ public class TestOptions extends BaseTestQuery{
     test("RESET `%s`;", SLICE_TARGET);
     // check reverted
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
       .unOrdered()
       .expectsEmptyResultSet()
       .build()
@@ -101,7 +101,7 @@ public class TestOptions extends BaseTestQuery{
   public void setAndResetSystemOption() throws Exception {
     // check unchanged
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'BOOT'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("status")
       .baselineValues("DEFAULT")
@@ -112,7 +112,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER system SET `%s` = %b;", ENABLE_VERBOSE_ERRORS_KEY, true);
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE name = '%s' AND optionScope = 'SYSTEM'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -123,7 +123,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER system RESET `%s`;", ENABLE_VERBOSE_ERRORS_KEY);
     // check reverted
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'BOOT'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("status")
       .baselineValues("DEFAULT")
@@ -137,7 +137,7 @@ public class TestOptions extends BaseTestQuery{
     test("SET `%s` = %b;", ENABLE_VERBOSE_ERRORS_KEY, true);
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -148,7 +148,7 @@ public class TestOptions extends BaseTestQuery{
     test("RESET ALL;");
     // check no session options changed
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE status <> 'DEFAULT' AND type = 'SESSION'")
+      .sqlQuery("SELECT status FROM sys.options WHERE status <> 'DEFAULT' AND optionScope = 'SESSION'")
       .unOrdered()
       .expectsEmptyResultSet()
       .build()
@@ -162,7 +162,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER SYSTEM SET `%s` = %b;", ENABLE_VERBOSE_ERRORS_KEY, true);
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -170,7 +170,7 @@ public class TestOptions extends BaseTestQuery{
       .run();
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -181,14 +181,14 @@ public class TestOptions extends BaseTestQuery{
     test("RESET `%s`;", ENABLE_VERBOSE_ERRORS_KEY);
     // check reverted
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .expectsEmptyResultSet()
       .build()
       .run();
     // check unchanged
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -205,7 +205,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER SYSTEM SET `%s` = %b;", ENABLE_VERBOSE_ERRORS_KEY, true);
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -213,7 +213,7 @@ public class TestOptions extends BaseTestQuery{
       .run();
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -224,14 +224,14 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER SESSION RESET ALL;");
     // check no session options changed
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE status <> 'DEFAULT' AND type = 'SESSION'")
+      .sqlQuery("SELECT status FROM sys.options WHERE status <> 'DEFAULT' AND optionScope = 'SESSION'")
       .unOrdered()
       .expectsEmptyResultSet()
       .build()
       .run();
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -246,7 +246,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER SYSTEM SET `%s` = %b;", ENABLE_VERBOSE_ERRORS_KEY, true);
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -254,7 +254,7 @@ public class TestOptions extends BaseTestQuery{
       .run();
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SYSTEM' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)
@@ -265,7 +265,7 @@ public class TestOptions extends BaseTestQuery{
     test("ALTER system RESET `%s`;", ENABLE_VERBOSE_ERRORS_KEY);
     // check reverted
     testBuilder()
-      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'BOOT'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("status")
       .baselineValues("DEFAULT")
@@ -273,7 +273,7 @@ public class TestOptions extends BaseTestQuery{
       .run();
     // check changed
     testBuilder()
-      .sqlQuery("SELECT bool_val FROM sys.options WHERE type = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
+      .sqlQuery("SELECT bool_val FROM sys.options WHERE optionScope = 'SESSION' AND name = '%s'", ENABLE_VERBOSE_ERRORS_KEY)
       .unOrdered()
       .baselineColumns("bool_val")
       .baselineValues(true)

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
index 7bd558f..f660a27 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
@@ -90,7 +90,7 @@ public class TestOptionsAuthEnabled extends BaseTestQuery {
     updateClient(ADMIN_USER, ADMIN_USER_PASSWORD);
     final String setOptionQuery =
         String.format("ALTER SESSION SET `%s`='%s,%s'", ExecConstants.ADMIN_USERS_KEY, ADMIN_USER, PROCESS_USER);
-    errorMsgTestHelper(setOptionQuery, "Admin related settings can only be set at SYSTEM level scope");
+    errorMsgTestHelper(setOptionQuery, "PERMISSION ERROR: Cannot change option security.admin.users in scope SESSION");
   }
 
   @Test
@@ -98,14 +98,14 @@ public class TestOptionsAuthEnabled extends BaseTestQuery {
     updateClient(TEST_USER_2, TEST_USER_2_PASSWORD);
     final String setOptionQuery =
         String.format("ALTER SESSION SET `%s`='%s,%s'", ExecConstants.ADMIN_USERS_KEY, ADMIN_USER, PROCESS_USER);
-    errorMsgTestHelper(setOptionQuery, "Admin related settings can only be set at SYSTEM level scope");
+    errorMsgTestHelper(setOptionQuery, "PERMISSION ERROR: Cannot change option security.admin.users in scope SESSION");
   }
 
   private void setOptHelper() throws Exception {
     try {
       test(setSysOptionQuery);
       testBuilder()
-          .sqlQuery(String.format("SELECT num_val FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'",
+          .sqlQuery(String.format("SELECT num_val FROM sys.options WHERE name = '%s' AND optionScope = 'SYSTEM'",
               ExecConstants.SLICE_TARGET))
           .unOrdered()
           .baselineColumns("num_val")

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java
new file mode 100644
index 0000000..27ad1d8
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.server.rest;
+
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.options.OptionDefinition;
+import org.apache.drill.exec.server.options.OptionValidator;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.FixtureBuilder;
+import org.apache.drill.test.RestClientFixture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.drill.test.TestConfigLinkage.MOCK_PROPERTY;
+import static org.apache.drill.test.TestConfigLinkage.createMockPropOptionDefinition;
+
+public class StatusResourcesTest {
+  @Test
+  public void testRetrieveInternalOption() throws Exception {
+    OptionDefinition optionDefinition = createMockPropOptionDefinition();
+
+    FixtureBuilder builder = ClusterFixture.builder().
+      configProperty(ExecConstants.HTTP_ENABLE, true).
+      configProperty(OptionValidator.OPTION_DEFAULTS_ROOT + MOCK_PROPERTY, "a").
+      configProperty(ExecConstants.HTTP_PORT_HUNT, true).
+      putDefinition(optionDefinition);
+
+    try (ClusterFixture cluster = builder.build();
+         ClientFixture client = cluster.clientFixture();
+         RestClientFixture restClientFixture = cluster.restClientFixture()) {
+      Assert.assertNull(restClientFixture.getStatusOption(MOCK_PROPERTY));
+      StatusResources.OptionWrapper option = restClientFixture.getStatusInternalOption(MOCK_PROPERTY);
+      Assert.assertEquals("a", option.getValueAsString());
+
+      client.alterSystem(MOCK_PROPERTY, "c");
+
+      Assert.assertNull(restClientFixture.getStatusOption(MOCK_PROPERTY));
+      option = restClientFixture.getStatusInternalOption(MOCK_PROPERTY);
+      Assert.assertEquals("c", option.getValueAsString());
+    }
+  }
+
+  @Test
+  public void testRetrievePublicOption() throws Exception {
+    FixtureBuilder builder = ClusterFixture.builder().
+      configProperty(ExecConstants.HTTP_ENABLE, true).
+      configProperty(ExecConstants.HTTP_PORT_HUNT, true).
+      systemOption(ExecConstants.SLICE_TARGET, 20);
+    try (ClusterFixture cluster = builder.build();
+         ClientFixture client = cluster.clientFixture();
+         RestClientFixture restClientFixture = cluster.restClientFixture()) {
+      Assert.assertNull(restClientFixture.getStatusInternalOption(ExecConstants.SLICE_TARGET));
+      StatusResources.OptionWrapper option = restClientFixture.getStatusOption(ExecConstants.SLICE_TARGET);
+      Assert.assertEquals(20, option.getValue());
+
+      client.alterSystem(ExecConstants.SLICE_TARGET, 30);
+
+      Assert.assertNull(restClientFixture.getStatusInternalOption(ExecConstants.SLICE_TARGET));
+      option = restClientFixture.getStatusOption(ExecConstants.SLICE_TARGET);
+      Assert.assertEquals(30, option.getValue());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
index ff04f8c..99b7c8f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
@@ -26,7 +26,7 @@ public class TestWithClause extends BaseTestQuery {
 
   @Test
   public void withClause() throws Exception {
-    test("with alpha as (select * from sys.options where type = 'SYSTEM') \n" +
+    test("with alpha as (select * from sys.options where optionScope = 'SYSTEM') \n" +
         "\n" +
         "select * from alpha");
   }
@@ -34,7 +34,7 @@ public class TestWithClause extends BaseTestQuery {
   @Test
   @Ignore
   public void withClauseWithAliases() throws Exception {
-    test("with alpha (x,y) as (select name, kind from sys.options where type = 'SYSTEM') \n" +
+    test("with alpha (x,y) as (select name, kind from sys.options where optionScope = 'SYSTEM') \n" +
         "\n" +
         "select x, y from alpha");
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index f9add51..fa5c8b2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.BitControl;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -291,7 +292,9 @@ public class TestParquetFilterPushDown extends PlanTestBase {
       testParquetFilterPD(query1, 9, 3, false);
 
     } finally {
-      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY  + "` = " + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING.getDefault().bool_val);
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY  + "` = " +
+        testOptionSet.getDefault(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY).bool_val);
       deleteTableIfExists(tableName);
     }
   }
@@ -314,7 +317,9 @@ public class TestParquetFilterPushDown extends PlanTestBase {
       testParquetFilterPD(query1, 9, 3, false);
 
     } finally {
-      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY  + "` = " + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD.getDefault().num_val);
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY + "` = " +
+        testOptionSet.getDefault(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY).num_val);
       deleteTableIfExists(tableName);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
index 93e2497..0504bb7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
@@ -30,8 +30,6 @@ import org.junit.Test;
 public class TestPStoreProviders extends TestWithZookeeper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestPStoreProviders.class);
 
-  static LocalPersistentStoreProvider provider;
-
   @Test
   public void verifyLocalStore() throws Exception {
     try(LocalPersistentStoreProvider provider = new LocalPersistentStoreProvider(DrillConfig.create())){
@@ -41,10 +39,11 @@ public class TestPStoreProviders extends TestWithZookeeper {
 
   @Test
   public void verifyZkStore() throws Exception {
-    DrillConfig config = getConfig();
-    String connect = config.getString(ExecConstants.ZK_CONNECTION);
+    String connect = zkHelper.getConnectionString();
+    DrillConfig config = zkHelper.getConfig();
+
     CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder()
-    .namespace(config.getString(ExecConstants.ZK_ROOT))
+    .namespace(zkHelper.getConfig().getString(ExecConstants.ZK_ROOT))
     .retryPolicy(new RetryNTimes(1, 100))
     .connectionTimeoutMs(config.getInt(ExecConstants.ZK_TIMEOUT))
     .connectString(connect);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
index 4c29dbe..632ffeb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
@@ -34,7 +34,7 @@ public class TestSystemTable extends BaseTestQuery {
   public void alterSessionOption() throws Exception {
 
     newTest() //
-      .sqlQuery("select bool_val as bool from sys.options where name = '%s' order by type desc", ExecConstants.JSON_ALL_TEXT_MODE)
+      .sqlQuery("select bool_val as bool from sys.options where name = '%s' order by accessibleScopes desc", ExecConstants.JSON_ALL_TEXT_MODE)
       .baselineColumns("bool")
       .ordered()
       .baselineValues(false)
@@ -43,7 +43,7 @@ public class TestSystemTable extends BaseTestQuery {
     test("alter session set `%s` = true", ExecConstants.JSON_ALL_TEXT_MODE);
 
     newTest() //
-      .sqlQuery("select bool_val as bool from sys.options where name = '%s' order by type desc ", ExecConstants.JSON_ALL_TEXT_MODE)
+      .sqlQuery("select bool_val as bool from sys.options where name = '%s' order by accessibleScopes desc ", ExecConstants.JSON_ALL_TEXT_MODE)
       .baselineColumns("bool")
       .ordered()
       .baselineValues(false)

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java
index 39c9ea3..c4b725c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.testing;
 
-import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROLS_VALIDATOR;
 import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROL_INJECTIONS;
 import static org.junit.Assert.fail;
 
@@ -30,9 +29,7 @@ import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.rpc.user.UserSession.QueryCountIncrementer;
-import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.server.options.OptionValue.OptionScope;
+import org.apache.drill.exec.server.options.SessionOptionManager;
 
 /**
  * Static methods for constructing exception and pause injections for testing purposes.
@@ -71,13 +68,10 @@ public class ControlsInjectionUtil {
 
   public static void setControls(final UserSession session, final String controls) {
     validateControlsString(controls);
-    final OptionValue opValue = OptionValue.createString(OptionValue.OptionType.SESSION,
-      DRILLBIT_CONTROL_INJECTIONS, controls, OptionScope.SESSION);
 
-    final OptionManager options = session.getOptions();
+    final SessionOptionManager options = session.getOptions();
     try {
-      DRILLBIT_CONTROLS_VALIDATOR.validate(opValue, null);
-      options.setOption(opValue);
+      options.setLocalOption(DRILLBIT_CONTROL_INJECTIONS, controls);
     } catch (final Exception e) {
       fail("Could not set controls options: " + e.getMessage());
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
index 84a7320..22da20b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
@@ -216,79 +216,83 @@ public class TestExceptionInjection extends BaseTestQuery {
     final ZookeeperHelper zkHelper = new ZookeeperHelper();
     zkHelper.startZookeeper(1);
 
-    // Creating two drillbits
-    final Drillbit drillbit1, drillbit2;
-    final DrillConfig drillConfig = zkHelper.getConfig();
     try {
-      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
-      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
-    } catch (DrillbitStartupException e) {
-      throw new RuntimeException("Failed to start drillbits.", e);
-    }
-
-    final DrillbitContext drillbitContext1 = drillbit1.getContext();
-    final DrillbitContext drillbitContext2 = drillbit2.getContext();
+      // Creating two drillbits
+      final Drillbit drillbit1, drillbit2;
+      final DrillConfig drillConfig = zkHelper.getConfig();
+      try {
+        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
+        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
+      } catch (DrillbitStartupException e) {
+        throw new RuntimeException("Failed to start drillbits.", e);
+      }
 
-    final UserSession session = UserSession.Builder.newBuilder()
-        .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
-        .withUserProperties(UserProperties.getDefaultInstance())
-        .withOptionManager(drillbitContext1.getOptionManager())
+      final DrillbitContext drillbitContext1 = drillbit1.getContext();
+      final DrillbitContext drillbitContext2 = drillbit2.getContext();
+
+      final UserSession session = UserSession.Builder.newBuilder()
+          .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
+          .withUserProperties(UserProperties.getDefaultInstance())
+          .withOptionManager(drillbitContext1.getOptionManager())
+          .build();
+
+      final String passthroughDesc = "<<injected from descPassthrough>>";
+      final int nSkip = 7;
+      final int nFire = 3;
+      final Class<? extends Throwable> exceptionClass = RuntimeException.class;
+      // only drillbit1's (address, port)
+      final String controls = Controls.newBuilder()
+      .addExceptionOnBit(DummyClass.class, passthroughDesc, exceptionClass, drillbitContext1.getEndpoint(), nSkip, nFire)
         .build();
 
-    final String passthroughDesc = "<<injected from descPassthrough>>";
-    final int nSkip = 7;
-    final int nFire = 3;
-    final Class<? extends Throwable> exceptionClass = RuntimeException.class;
-    // only drillbit1's (address, port)
-    final String controls = Controls.newBuilder()
-    .addExceptionOnBit(DummyClass.class, passthroughDesc, exceptionClass, drillbitContext1.getEndpoint(), nSkip, nFire)
-      .build();
+      ControlsInjectionUtil.setControls(session, controls);
 
-    ControlsInjectionUtil.setControls(session, controls);
+      {
+        final QueryContext queryContext1 = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
+        final DummyClass class1 = new DummyClass(queryContext1);
 
-    {
-      final QueryContext queryContext1 = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
-      final DummyClass class1 = new DummyClass(queryContext1);
+        // these shouldn't throw
+        for (int i = 0; i < nSkip; ++i) {
+          class1.descPassthroughMethod(passthroughDesc);
+        }
 
-      // these shouldn't throw
-      for (int i = 0; i < nSkip; ++i) {
+        // these should throw
+        for (int i = 0; i < nFire; ++i) {
+          assertPassthroughThrows(class1, exceptionClass.getName(), passthroughDesc);
+        }
+
+        // this shouldn't throw
         class1.descPassthroughMethod(passthroughDesc);
+        try {
+          queryContext1.close();
+        } catch (Exception e) {
+          fail();
+        }
       }
+      {
+        final QueryContext queryContext2 = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
+        final DummyClass class2 = new DummyClass(queryContext2);
 
-      // these should throw
-      for (int i = 0; i < nFire; ++i) {
-        assertPassthroughThrows(class1, exceptionClass.getName(), passthroughDesc);
-      }
+        // these shouldn't throw
+        for (int i = 0; i < nSkip; ++i) {
+          class2.descPassthroughMethod(passthroughDesc);
+        }
 
-      // this shouldn't throw
-      class1.descPassthroughMethod(passthroughDesc);
-      try {
-        queryContext1.close();
-      } catch (Exception e) {
-        fail();
-      }
-    }
-    {
-      final QueryContext queryContext2 = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
-      final DummyClass class2 = new DummyClass(queryContext2);
+        // these shouldn't throw
+        for (int i = 0; i < nFire; ++i) {
+          class2.descPassthroughMethod(passthroughDesc);
+        }
 
-      // these shouldn't throw
-      for (int i = 0; i < nSkip; ++i) {
+        // this shouldn't throw
         class2.descPassthroughMethod(passthroughDesc);
+        try {
+          queryContext2.close();
+        } catch (Exception e) {
+          fail();
+        }
       }
-
-      // these shouldn't throw
-      for (int i = 0; i < nFire; ++i) {
-        class2.descPassthroughMethod(passthroughDesc);
-      }
-
-      // this shouldn't throw
-      class2.descPassthroughMethod(passthroughDesc);
-      try {
-        queryContext2.close();
-      } catch (Exception e) {
-        fail();
-      }
+    } finally {
+      zkHelper.stopZookeeper();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
index 54f851a..c618739 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
@@ -150,66 +150,69 @@ public class TestPauseInjection extends BaseTestQuery {
     final ZookeeperHelper zkHelper = new ZookeeperHelper();
     zkHelper.startZookeeper(1);
 
-    // Creating two drillbits
-    final Drillbit drillbit1, drillbit2;
-    final DrillConfig drillConfig = zkHelper.getConfig();
     try {
-      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
-      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
-    } catch (final DrillbitStartupException e) {
-      throw new RuntimeException("Failed to start two drillbits.", e);
-    }
-
-    final DrillbitContext drillbitContext1 = drillbit1.getContext();
-    final DrillbitContext drillbitContext2 = drillbit2.getContext();
-
-    final UserSession session = UserSession.Builder.newBuilder()
-      .withCredentials(UserCredentials.newBuilder()
-        .setUserName("foo")
-        .build())
-      .withUserProperties(UserProperties.getDefaultInstance())
-      .withOptionManager(drillbitContext1.getOptionManager())
-      .build();
-
-    final DrillbitEndpoint drillbitEndpoint1 = drillbitContext1.getEndpoint();
-    final String controls = Controls.newBuilder()
-      .addPauseOnBit(DummyClass.class, DummyClass.PAUSES, drillbitEndpoint1)
-      .build();
-
-    ControlsInjectionUtil.setControls(session, controls);
-
-    {
-      final long expectedDuration = 1000L;
-      final ExtendedLatch trigger = new ExtendedLatch(1);
-      final Pointer<Exception> ex = new Pointer<>();
-      final QueryContext queryContext = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
-      (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
-
-      // test that the pause happens
-      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
-      final long actualDuration = dummyClass.pauses();
-      assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration),
-        expectedDuration <= actualDuration);
-      assertTrue("No exception should be thrown.", ex.value == null);
+      // Creating two drillbits
+      final Drillbit drillbit1, drillbit2;
+      final DrillConfig drillConfig = zkHelper.getConfig();
       try {
-        queryContext.close();
-      } catch (final Exception e) {
-        fail("Failed to close query context: " + e);
+        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
+        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
+      } catch (final DrillbitStartupException e) {
+        throw new RuntimeException("Failed to start two drillbits.", e);
       }
-    }
 
-    {
-      final ExtendedLatch trigger = new ExtendedLatch(1);
-      final QueryContext queryContext = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
+      final DrillbitContext drillbitContext1 = drillbit1.getContext();
+      final DrillbitContext drillbitContext2 = drillbit2.getContext();
+
+      final UserSession session = UserSession.Builder.newBuilder()
+        .withCredentials(UserCredentials.newBuilder()
+          .setUserName("foo")
+          .build())
+        .withUserProperties(UserProperties.getDefaultInstance())
+        .withOptionManager(drillbitContext1.getOptionManager())
+        .build();
+
+      final DrillbitEndpoint drillbitEndpoint1 = drillbitContext1.getEndpoint();
+      final String controls = Controls.newBuilder()
+        .addPauseOnBit(DummyClass.class, DummyClass.PAUSES, drillbitEndpoint1)
+        .build();
+
+      ControlsInjectionUtil.setControls(session, controls);
+
+      {
+        final long expectedDuration = 1000L;
+        final ExtendedLatch trigger = new ExtendedLatch(1);
+        final Pointer<Exception> ex = new Pointer<>();
+        final QueryContext queryContext = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
+        (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
+
+        // test that the pause happens
+        final DummyClass dummyClass = new DummyClass(queryContext, trigger);
+        final long actualDuration = dummyClass.pauses();
+        assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration), expectedDuration <= actualDuration);
+        assertTrue("No exception should be thrown.", ex.value == null);
+        try {
+          queryContext.close();
+        } catch (final Exception e) {
+          fail("Failed to close query context: " + e);
+        }
+      }
 
-      // if the resume did not happen, the test would hang
-      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
-      dummyClass.pauses();
-      try {
-        queryContext.close();
-      } catch (final Exception e) {
-        fail("Failed to close query context: " + e);
+      {
+        final ExtendedLatch trigger = new ExtendedLatch(1);
+        final QueryContext queryContext = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
+
+        // if the resume did not happen, the test would hang
+        final DummyClass dummyClass = new DummyClass(queryContext, trigger);
+        dummyClass.pauses();
+        try {
+          queryContext.close();
+        } catch (final Exception e) {
+          fail("Failed to close query context: " + e);
+        }
       }
+    } finally {
+      zkHelper.stopZookeeper();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/util/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/MiniZooKeeperCluster.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/MiniZooKeeperCluster.java
index a502c32..f9abe7f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/MiniZooKeeperCluster.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/MiniZooKeeperCluster.java
@@ -29,7 +29,9 @@ import java.net.Socket;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileUtil;
@@ -44,7 +46,8 @@ public class MiniZooKeeperCluster {
   private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
 
   private static final int TICK_TIME = 2000;
-  private static final int CONNECTION_TIMEOUT = 30000;
+  private static final int CONNECTION_TIMEOUT = 10000;
+  public static final int DEFAULT_PORT = 2181;
 
   private boolean started;
 
@@ -146,27 +149,41 @@ public class MiniZooKeeperCluster {
       } else {
         tickTimeToUse = TICK_TIME;
       }
+
       ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
       NIOServerCnxnFactory standaloneServerFactory;
+
       while (true) {
+        System.out.println("Starting zookeeper " + tentativePort);
+
         try {
           standaloneServerFactory = new NIOServerCnxnFactory();
-          standaloneServerFactory.configure(
-            new InetSocketAddress(tentativePort), 1000);
+          standaloneServerFactory.configure(new InetSocketAddress(tentativePort), 1000);
         } catch (BindException e) {
-          LOG.debug("Failed binding ZK Server to client port: " +
-            tentativePort);
+          LOG.debug("Failed binding ZK Server to client port: " + tentativePort);
           // This port is already in use, try to use another.
           tentativePort++;
           continue;
         }
-        break;
-      }
 
-      // Start up this ZK server
-      standaloneServerFactory.startup(server);
-      if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
-        throw new IOException("Waiting for startup of standalone server");
+        // Start up this ZK server
+
+        try {
+          standaloneServerFactory.startup(server);
+        } catch (IOException e) {
+          LOG.error("Zookeeper startupt error", e);
+          tentativePort++;
+          continue;
+        }
+
+        if (!waitForServerUp(server, CONNECTION_TIMEOUT)) {
+          server.shutdown();
+          server = new ZooKeeperServer(dir, dir, tickTimeToUse);
+          tentativePort++;
+          continue;
+        }
+
+        break;
       }
 
       // We have selected this port as a client port.
@@ -326,32 +343,13 @@ public class MiniZooKeeperCluster {
   }
 
   // XXX: From o.a.zk.t.ClientBase
-  private static boolean waitForServerUp(int port, long timeout) {
+  private static boolean waitForServerUp(ZooKeeperServer server, long timeout) {
     long start = System.currentTimeMillis();
     while (true) {
-      try {
-        Socket sock = new Socket("localhost", port);
-        BufferedReader reader = null;
-        try {
-          OutputStream outstream = sock.getOutputStream();
-          outstream.write("stat".getBytes());
-          outstream.flush();
+      // Doing it this way instead of openning a connection to zookeeper because of ZOOKEEPER-2383
 
-          Reader isr = new InputStreamReader(sock.getInputStream());
-          reader = new BufferedReader(isr);
-          String line = reader.readLine();
-          if (line != null && line.startsWith("Zookeeper version:")) {
-            return true;
-          }
-        } finally {
-          sock.close();
-          if (reader != null) {
-            reader.close();
-          }
-        }
-      } catch (IOException e) {
-        // ignore as this is expected
-        LOG.info("server localhost:" + port + " not up " + e);
+      if (server.isRunning()) {
+        return true;
       }
 
       if (System.currentTimeMillis() > start + timeout) {

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
index 6d6f35a..4d2da29 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
@@ -29,6 +29,7 @@ import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.test.OperatorFixture;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -59,12 +60,13 @@ public class TestExtendedTypes extends BaseTestQuery {
           + "/0_0_0.json"));
       assertEquals(new String(originalData), new String(newData));
     } finally {
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
       testNoResult(String.format("ALTER SESSION SET `%s` = '%s'",
           ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName(),
-          ExecConstants.OUTPUT_FORMAT_VALIDATOR.getDefault().getValue()));
+          testOptionSet.getDefault(ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName()).getValue()));
       testNoResult(String.format("ALTER SESSION SET `%s` = %s",
           ExecConstants.JSON_EXTENDED_TYPES.getOptionName(),
-          ExecConstants.JSON_EXTENDED_TYPES.getDefault().getValue()));
+          testOptionSet.getDefault(ExecConstants.JSON_EXTENDED_TYPES.getOptionName()).getValue()));
     }
   }
 
@@ -89,12 +91,13 @@ public class TestExtendedTypes extends BaseTestQuery {
       String expected = "drill_timestamp_millies,bin,bin1\n2015-07-07T03:59:43.488,drill,drill\n";
       Assert.assertEquals(expected, actual);
     } finally {
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
       testNoResult(String.format("ALTER SESSION SET `%s` = '%s'",
           ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName(),
-          ExecConstants.OUTPUT_FORMAT_VALIDATOR.getDefault().getValue()));
+          testOptionSet.getDefault(ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName()).getValue()));
       testNoResult(String.format("ALTER SESSION SET `%s` = %s",
           ExecConstants.JSON_EXTENDED_TYPES.getOptionName(),
-          ExecConstants.JSON_EXTENDED_TYPES.getDefault().getValue()));
+          testOptionSet.getDefault(ExecConstants.JSON_EXTENDED_TYPES.getOptionName()).getValue()));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
index 271f29f..389bb80 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;