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

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

DRILL-5723: Added System Internal Options That can be Modified at Runtime
Changes include:

   1. Addition of internal options.
   2. Refactoring of OptionManagers and OptionValidators.
   3. Fixed ambiguity in the meaning of an option type, and changed its name to accessibleScopes.
   4. Updated javadocs in the Option System classes.
   5. Added RestClientFixture for testing the Rest API.
   6. Fixed flakey test in TestExceptionInjection caused by race condition.
   7. Fixed various tests which started zookeeper but failed to shut it down at the end of tests.
   8. Added port hunting to the Drill Webserver for testing
   9. Fixed various flaky tests
  10. Fix compile issue

closes #923


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

Branch: refs/heads/master
Commit: 6adeb986016a769755fd5e8fc66244ee1e8d18e1
Parents: df54cc2
Author: Timothy Farkas <ti...@apache.org>
Authored: Thu Aug 17 16:49:51 2017 -0700
Committer: Paul Rogers <pr...@maprtech.com>
Committed: Sat Sep 16 18:32:00 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/test/DrillTest.java   |   2 +-
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java |   2 +-
 .../apache/drill/exec/hive/TestHiveStorage.java |   4 +-
 contrib/storage-jdbc/pom.xml                    |  47 +--
 contrib/storage-mongo/pom.xml                   |  12 +-
 .../drill/exec/store/mongo/MongoTestBase.java   |   3 +
 .../drill/exec/store/mongo/MongoTestSuit.java   |   8 +
 exec/java-exec/pom.xml                          |   5 +
 .../org/apache/drill/exec/ExecConstants.java    |  12 +-
 .../exec/compile/ClassCompilerSelector.java     |   5 +-
 .../org/apache/drill/exec/ops/QueryContext.java |   5 +-
 .../exec/planner/physical/PlannerSettings.java  |   4 +-
 .../planner/sql/handlers/DefaultSqlHandler.java |   5 +-
 .../planner/sql/handlers/SetOptionHandler.java  |  44 +-
 .../rpc/user/InboundImpersonationManager.java   |   7 +-
 .../apache/drill/exec/rpc/user/UserSession.java |  12 +-
 .../drill/exec/server/BootStrapContext.java     |  11 +-
 .../org/apache/drill/exec/server/Drillbit.java  |  56 ++-
 .../drill/exec/server/DrillbitContext.java      |   2 +-
 .../exec/server/options/BaseOptionManager.java  | 112 +++--
 .../exec/server/options/BaseOptionSet.java      |  57 +++
 .../server/options/DrillConfigIterator.java     |   8 +-
 .../server/options/FallbackOptionManager.java   |  91 +---
 .../server/options/FragmentOptionManager.java   |  23 +-
 .../server/options/InMemoryOptionManager.java   |  42 +-
 .../exec/server/options/OptionDefinition.java   |  47 +++
 .../exec/server/options/OptionManager.java      | 106 ++++-
 .../exec/server/options/OptionMetaData.java     |  68 +++
 .../drill/exec/server/options/OptionSet.java    |   8 +-
 .../exec/server/options/OptionValidator.java    |  33 +-
 .../drill/exec/server/options/OptionValue.java  |  82 ++--
 .../exec/server/options/QueryOptionManager.java |  29 +-
 .../server/options/SessionOptionManager.java    |  35 +-
 .../server/options/SystemOptionManager.java     | 419 +++++++++++--------
 .../exec/server/options/TypeValidators.java     | 124 ++----
 .../drill/exec/server/rest/StatusResources.java | 119 ++++--
 .../drill/exec/server/rest/WebServer.java       |  85 ++--
 .../exec/store/sys/ExtendedOptionIterator.java  |  20 +-
 .../drill/exec/store/sys/OptionIterator.java    |  45 +-
 .../drill/exec/store/sys/SystemTable.java       |  22 +-
 .../drill/exec/testing/ExecutionControls.java   |  15 +-
 .../src/main/resources/drill-module.conf        |   1 +
 .../java/org/apache/drill/BaseTestQuery.java    |  12 +
 .../java/org/apache/drill/QueryTestUtil.java    |  12 +-
 .../java/org/apache/drill/TestStarQueries.java  |   2 +-
 .../apache/drill/exec/DrillSystemTestBase.java  |   2 +-
 .../apache/drill/exec/TestWithZookeeper.java    |  16 +-
 .../org/apache/drill/exec/ZookeeperHelper.java  |  38 +-
 .../ConnectTriesPropertyTestClusterBits.java    |   1 +
 .../exec/client/DrillClientSystemTest.java      |   4 +-
 .../exec/compile/TestClassTransformation.java   |  15 +-
 .../TestInboundImpersonationPrivileges.java     |   7 +-
 .../physical/impl/TestConvertFunctions.java     |  10 +-
 .../physical/impl/agg/TestHashAggrSpill.java    |   3 +-
 .../impl/join/TestMergeJoinAdvanced.java        |   6 +-
 .../impl/limit/TestLimitWithExchanges.java      |  11 +-
 .../TestOrderedPartitionExchange.java           |   3 +-
 .../partitionsender/TestPartitionSender.java    |  12 +-
 .../physical/impl/writer/TestParquetWriter.java |  30 +-
 .../writer/TestParquetWriterEmptyFiles.java     |   4 +-
 .../drill/exec/rpc/data/TestBitBitKerberos.java |   3 +-
 .../apache/drill/exec/rpc/data/TestBitRpc.java  |   5 +-
 .../exec/server/TestDrillbitResilience.java     |  11 +-
 .../apache/drill/exec/server/TestOptions.java   |  46 +-
 .../exec/server/TestOptionsAuthEnabled.java     |   6 +-
 .../exec/server/rest/StatusResourcesTest.java   |  80 ++++
 .../apache/drill/exec/sql/TestWithClause.java   |   4 +-
 .../parquet/TestParquetFilterPushDown.java      |   9 +-
 .../exec/store/sys/TestPStoreProviders.java     |   9 +-
 .../drill/exec/store/sys/TestSystemTable.java   |   4 +-
 .../exec/testing/ControlsInjectionUtil.java     |  12 +-
 .../exec/testing/TestExceptionInjection.java    | 122 +++---
 .../drill/exec/testing/TestPauseInjection.java  | 111 ++---
 .../drill/exec/util/MiniZooKeeperCluster.java   |  66 ++-
 .../complex/writer/TestExtendedTypes.java       |  11 +-
 .../exec/work/batch/TestSpoolingBuffer.java     |   1 -
 .../work/metadata/TestMetadataProvider.java     |  24 +-
 .../org/apache/drill/test/ClusterFixture.java   |  20 +-
 .../org/apache/drill/test/ConfigBuilder.java    |  19 +-
 .../org/apache/drill/test/FixtureBuilder.java   |   6 +
 .../org/apache/drill/test/OperatorFixture.java  |  44 +-
 .../apache/drill/test/RestClientFixture.java    | 117 ++++++
 .../apache/drill/test/TestConfigLinkage.java    | 134 +++++-
 .../org/apache/drill/jdbc/ITTestShadedJar.java  |  12 +-
 .../org/apache/drill/exec/util/Pointer.java     |   2 +-
 pom.xml                                         |   4 +
 86 files changed, 1849 insertions(+), 1058 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/common/src/test/java/org/apache/drill/test/DrillTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index bb051d7..e70a9cf 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -57,7 +57,7 @@ public class DrillTest {
   static MemWatcher memWatcher;
   static String className;
 
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(1000000);
   @Rule public final TestLogReporter logOutcome = LOG_OUTCOME;
 
   @Rule public final TestRule REPEAT_RULE = TestTools.getRepeatRule(false);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index 9ca2dbd..88200ec 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -134,7 +134,7 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
 
     } 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/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
index 082a019..4e7e21c 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
@@ -28,6 +28,7 @@ import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.joda.time.DateTime;
@@ -73,9 +74,10 @@ public class TestHiveStorage extends HiveTestBase {
           .baselineValues(200l)
           .go();
     } finally {
+      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
       test(String.format("alter session set `%s` = %s",
           ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS,
-              ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS_VALIDATOR.getDefault().bool_val ? "true" : "false"));
+          Boolean.toString(testOptionSet.getDefault(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS).bool_val)));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/contrib/storage-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/pom.xml b/contrib/storage-jdbc/pom.xml
index 9d8f5f0..f784c09 100755
--- a/contrib/storage-jdbc/pom.xml
+++ b/contrib/storage-jdbc/pom.xml
@@ -6,9 +6,7 @@
  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.
@@ -31,6 +29,7 @@
     <mysql.connector.version>5.1.36</mysql.connector.version>
     <derby.database.name>drill_derby_test</derby.database.name>
     <mysql.database.name>drill_mysql_test</mysql.database.name>
+    <skipTests>false</skipTests>
   </properties>
 
   <dependencies>
@@ -39,7 +38,7 @@
       <artifactId>drill-java-exec</artifactId>
       <version>${project.version}</version>
     </dependency>
-    
+
     <!-- Test dependencies -->
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
@@ -92,6 +91,11 @@
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
+          <!--
+          Forking multiple processes can cause race conditions with the initialization of
+          the test databases.
+          -->
+          <forkCount combine.self="override">1</forkCount>
           <excludes>
             <exclude>**/*</exclude>
           </excludes>
@@ -102,12 +106,21 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-failsafe-plugin</artifactId>
         <version>2.18.1</version>
+        <configuration>
+          <forkCount combine.self="override">1</forkCount>
+          <systemPropertyVariables>
+            <derby.port>${derby.reserved.port}</derby.port>
+            <mysql.port>${mysql.reserved.port}</mysql.port>
+            <mysql.name>${mysql.database.name}</mysql.name>
+          </systemPropertyVariables>
+          <includes>
+            <include>**/*IT.java</include>
+          </includes>
+        </configuration>
         <executions>
           <execution>
-            <goals>
-              <goal>integration-test</goal>
-              <goal>verify</goal>
-            </goals>
+            <id>run-IT-Tests</id>
+            <phase>integration-test</phase>
           </execution>
         </executions>
       </plugin>
@@ -258,24 +271,6 @@
         </executions>
       </plugin>
     </plugins>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <artifactId>maven-failsafe-plugin</artifactId>
-          <configuration>
-            <forkCount>1</forkCount>
-            <systemPropertyVariables>
-              <derby.port>${derby.reserved.port}</derby.port>
-              <mysql.port>${mysql.reserved.port}</mysql.port>
-              <mysql.name>${mysql.database.name}</mysql.name>
-            </systemPropertyVariables>
-            <includes>
-              <include>**/*IT.java</include>
-            </includes>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
   </build>
 
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/contrib/storage-mongo/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/pom.xml b/contrib/storage-mongo/pom.xml
index 6f19612..cfd1f61 100644
--- a/contrib/storage-mongo/pom.xml
+++ b/contrib/storage-mongo/pom.xml
@@ -80,9 +80,15 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-	  <includes>
-	    <include>${mongo.TestSuite}</include>
-	  </includes>
+          <includes>
+            <include>${mongo.TestSuite}</include>
+          </includes>
+          <excludes>
+            <exclude>**/TestMongoFilterPushDown.java</exclude>
+            <exclude>**/TestMongoProjectPushDown.java</exclude>
+            <exclude>**/TestMongoQueries.java</exclude>
+            <exclude>**/TestMongoChunkAssignment.java</exclude>
+          </excludes>
           <systemProperties>
             <property>
               <name>logback.log.dir</name>

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
index ec97514..ffc6009 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 
 public class MongoTestBase extends PlanTestBase implements MongoTestConstants {
@@ -36,6 +37,8 @@ public class MongoTestBase extends PlanTestBase implements MongoTestConstants {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    // Make sure this test is only running as part of the suit
+    Assume.assumeTrue(MongoTestSuit.isRunningSuite());
     MongoTestSuit.initMongo();
     initMongoStoragePlugin();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
index c34afe3..122d52a 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
@@ -76,6 +76,12 @@ public class MongoTestSuit implements MongoTestConstants {
 
   private static volatile AtomicInteger initCount = new AtomicInteger(0);
 
+  private static volatile boolean runningSuite = false;
+
+  public static boolean isRunningSuite() {
+    return runningSuite;
+  }
+
   private static class DistributedMode {
     private static MongosSystemForTestFactory mongosTestFactory;
 
@@ -223,6 +229,7 @@ public class MongoTestSuit implements MongoTestConstants {
         TestTableGenerator.importData(DATATYPE_DB, DATATYPE_COLLECTION, DATATYPE_DATA);
       }
       initCount.incrementAndGet();
+      runningSuite = true;
     }
   }
 
@@ -253,6 +260,7 @@ public class MongoTestSuit implements MongoTestConstants {
           }
         }
         finally {
+          runningSuite = false;
           if (mongoClient != null) {
             mongoClient.close();
           }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 25ec1e6..173a3ee 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -176,6 +176,11 @@
       <version>2.8</version>
     </dependency>
     <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson</artifactId>
+      <version>2.8</version>
+    </dependency>
+    <dependency>
       <groupId>com.fasterxml.jackson.jaxrs</groupId>
       <artifactId>jackson-jaxrs-json-provider</artifactId>
       <version>${jackson.version}</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 4aaa537..4fa846f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec;
 
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
+import org.apache.drill.exec.server.options.OptionMetaData;
 import org.apache.drill.exec.server.options.OptionValidator;
 import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
 import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
@@ -115,6 +116,7 @@ public interface ExecConstants {
   String HTTP_ENABLE = "drill.exec.http.enabled";
   String HTTP_MAX_PROFILES = "drill.exec.http.max_profiles";
   String HTTP_PORT = "drill.exec.http.port";
+  String HTTP_PORT_HUNT = "drill.exec.http.porthunt";
   String HTTP_ENABLE_SSL = "drill.exec.http.ssl_enabled";
   String HTTP_CORS_ENABLED = "drill.exec.http.cors.enabled";
   String HTTP_CORS_ALLOWED_ORIGINS = "drill.exec.http.cors.allowedOrigins";
@@ -423,13 +425,13 @@ public interface ExecConstants {
    * such as changing system options.
    */
   String ADMIN_USERS_KEY = "security.admin.users";
-  StringValidator ADMIN_USERS_VALIDATOR = new StringValidator(ADMIN_USERS_KEY, true);
+  StringValidator ADMIN_USERS_VALIDATOR = new StringValidator(ADMIN_USERS_KEY);
 
   /**
    * Option whose value is a comma separated list of admin usergroups.
    */
   String ADMIN_USER_GROUPS_KEY = "security.admin.user_groups";
-  StringValidator ADMIN_USER_GROUPS_VALIDATOR = new StringValidator(ADMIN_USER_GROUPS_KEY, true);
+  StringValidator ADMIN_USER_GROUPS_VALIDATOR = new StringValidator(ADMIN_USER_GROUPS_KEY);
   /**
    * Option whose value is a string representing list of inbound impersonation policies.
    *
@@ -472,8 +474,7 @@ public interface ExecConstants {
    * for any query.
    */
   String ENABLE_QUERY_PROFILE_OPTION = "exec.query_profile.save";
-  BooleanValidator ENABLE_QUERY_PROFILE_VALIDATOR = new BooleanValidator(
-      ENABLE_QUERY_PROFILE_OPTION);
+  BooleanValidator ENABLE_QUERY_PROFILE_VALIDATOR = new BooleanValidator(ENABLE_QUERY_PROFILE_OPTION);
 
   /**
    * Profiles are normally written after the last client message to reduce latency.
@@ -482,8 +483,7 @@ public interface ExecConstants {
    * verification.
    */
   String QUERY_PROFILE_DEBUG_OPTION = "exec.query_profile.debug_mode";
-  BooleanValidator QUERY_PROFILE_DEBUG_VALIDATOR = new BooleanValidator(
-      QUERY_PROFILE_DEBUG_OPTION);
+  BooleanValidator QUERY_PROFILE_DEBUG_VALIDATOR = new BooleanValidator(QUERY_PROFILE_DEBUG_OPTION);
 
   String USE_DYNAMIC_UDFS_KEY = "exec.udf.use_dynamic";
   BooleanValidator USE_DYNAMIC_UDFS = new BooleanValidator(USE_DYNAMIC_UDFS_KEY);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
index 9b6e86e..5255c53 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
@@ -25,6 +25,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
 import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.server.options.OptionMetaData;
 import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.server.options.OptionValidator;
 import org.apache.drill.exec.server.options.OptionValue;
@@ -82,8 +83,8 @@ public class ClassCompilerSelector {
 
   public static final StringValidator JAVA_COMPILER_VALIDATOR = new StringValidator(JAVA_COMPILER_OPTION) {
     @Override
-    public void validate(final OptionValue v, final OptionSet manager) {
-      super.validate(v, manager);
+    public void validate(final OptionValue v, final OptionMetaData metaData, final OptionSet manager) {
+      super.validate(v, metaData, manager);
       try {
         CompilerPolicy.valueOf(v.string_val.toUpperCase());
       } catch (IllegalArgumentException e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 0dbeea5..435f35f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -40,7 +40,6 @@ import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.QueryProfileStoreContext;
-import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.QueryOptionManager;
 import org.apache.drill.exec.store.PartitionExplorer;
@@ -65,7 +64,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
 
   private final DrillbitContext drillbitContext;
   private final UserSession session;
-  private final OptionManager queryOptions;
+  private final QueryOptionManager queryOptions;
   private final PlannerSettings plannerSettings;
   private final ExecutionControls executionControls;
 
@@ -182,7 +181,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     return session.getCredentials().getUserName();
   }
 
-  public OptionManager getOptions() {
+  public QueryOptionManager getOptions() {
     return queryOptions;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 36285a5..74ef601 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -90,10 +90,10 @@ public class PlannerSettings implements Context{
 
   public static final DoubleValidator FILTER_MIN_SELECTIVITY_ESTIMATE_FACTOR =
           new MinRangeDoubleValidator("planner.filter.min_selectivity_estimate_factor",
-          0.0, 1.0, 0.0d, "planner.filter.max_selectivity_estimate_factor");
+          0.0, 1.0, "planner.filter.max_selectivity_estimate_factor");
   public static final DoubleValidator FILTER_MAX_SELECTIVITY_ESTIMATE_FACTOR =
           new MaxRangeDoubleValidator("planner.filter.max_selectivity_estimate_factor",
-          0.0, 1.0, 1.0d, "planner.filter.min_selectivity_estimate_factor");
+          0.0, 1.0, "planner.filter.min_selectivity_estimate_factor");
 
   public static final String TYPE_INFERENCE_KEY = "planner.enable_type_inference";
   public static final BooleanValidator TYPE_INFERENCE = new BooleanValidator(TYPE_INFERENCE_KEY);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index bdf12dd..9515b3b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -93,7 +93,6 @@ import org.apache.drill.exec.planner.physical.visitor.SwapHashJoinVisitor;
 import org.apache.drill.exec.planner.physical.visitor.TopProjectVisitor;
 import org.apache.drill.exec.planner.sql.parser.UnsupportedOperatorsVisitor;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
@@ -448,8 +447,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       log("Not enough memory for this plan", phyRelNode, logger, null);
       logger.debug("Re-planning without hash operations.");
 
-      queryOptions.setOption(OptionValue.createBoolean(OptionValue.OptionType.QUERY, PlannerSettings.HASHJOIN.getOptionName(), false, OptionValue.OptionScope.QUERY));
-      queryOptions.setOption(OptionValue.createBoolean(OptionValue.OptionType.QUERY, PlannerSettings.HASHAGG.getOptionName(), false, OptionValue.OptionScope.QUERY));
+      queryOptions.setLocalOption(PlannerSettings.HASHJOIN.getOptionName(), false);
+      queryOptions.setLocalOption(PlannerSettings.HASHAGG.getOptionName(), false);
 
       try {
         final RelNode relNode = transform(PlannerType.VOLCANO, PlannerPhase.PHYSICAL, drel, traits);

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
index 95115f6..0216163 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
@@ -23,6 +23,8 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.ValidationException;
 
 import org.apache.calcite.util.NlsString;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
@@ -30,8 +32,8 @@ import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.server.options.OptionManager;
 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.QueryOptionManager;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.sql.SqlLiteral;
@@ -64,19 +66,15 @@ public class SetOptionHandler extends AbstractSqlHandler {
     }
 
     final String scope = option.getScope();
-    final OptionValue.OptionType type;
     final OptionValue.OptionScope optionScope;
     if (scope == null) { // No scope mentioned assumed SESSION
-      type = OptionType.SESSION;
       optionScope = OptionScope.SESSION;
     } else {
       switch (scope.toLowerCase()) {
       case "session":
-        type = OptionType.SESSION;
         optionScope = OptionScope.SESSION;
         break;
       case "system":
-        type = OptionType.SYSTEM;
         optionScope = OptionScope.SYSTEM;
         break;
       default:
@@ -86,8 +84,8 @@ public class SetOptionHandler extends AbstractSqlHandler {
       }
     }
 
-    final OptionManager options = context.getOptions();
-    if (type == OptionType.SYSTEM) {
+    final QueryOptionManager options = context.getOptions();
+    if (optionScope == OptionScope.SYSTEM) {
       // If the user authentication is enabled, make sure the user who is trying to change the system option has
       // administrative privileges.
       if (context.isUserAuthenticationEnabled() &&
@@ -101,53 +99,55 @@ public class SetOptionHandler extends AbstractSqlHandler {
       }
     }
 
+    final String optionName = option.getName().toString();
+
     // Currently, we convert multi-part identifier to a string.
-    final String name = option.getName().toString();
+    final OptionManager chosenOptions = options.getOptionManager(optionScope);
+
     if (value != null) { // SET option
-      final OptionValue optionValue = createOptionValue(name, type, (SqlLiteral) value, optionScope);
-      options.setOption(optionValue);
+      final Object literalObj = sqlLiteralToObject((SqlLiteral) value);
+      chosenOptions.setLocalOption(optionName, literalObj);
     } else { // RESET option
-      if ("ALL".equalsIgnoreCase(name)) {
-        options.deleteAllOptions(type);
+      if ("ALL".equalsIgnoreCase(optionName)) {
+        chosenOptions.deleteAllLocalOptions();
       } else {
-        options.deleteOption(name, type);
+        chosenOptions.deleteLocalOption(optionName);
       }
     }
 
-    return DirectPlan.createDirectPlan(context, true, String.format("%s updated.", name));
+    return DirectPlan.createDirectPlan(context, true, String.format("%s updated.", optionName));
   }
 
-  private static OptionValue createOptionValue(final String name, final OptionValue.OptionType type,
-                                               final SqlLiteral literal, final OptionValue.OptionScope scope) {
+  private static Object sqlLiteralToObject(final SqlLiteral literal) {
     final Object object = literal.getValue();
     final SqlTypeName typeName = literal.getTypeName();
     switch (typeName) {
     case DECIMAL: {
       final BigDecimal bigDecimal = (BigDecimal) object;
       if (bigDecimal.scale() == 0) {
-        return OptionValue.createLong(type, name, bigDecimal.longValue(), scope);
+        return bigDecimal.longValue();
       } else {
-        return OptionValue.createDouble(type, name, bigDecimal.doubleValue(), scope);
+        return bigDecimal.doubleValue();
       }
     }
 
     case DOUBLE:
     case FLOAT:
-      return OptionValue.createDouble(type, name, ((BigDecimal) object).doubleValue(), scope);
+      return ((BigDecimal) object).doubleValue();
 
     case SMALLINT:
     case TINYINT:
     case BIGINT:
     case INTEGER:
-      return OptionValue.createLong(type, name, ((BigDecimal) object).longValue(), scope);
+      return ((BigDecimal) object).longValue();
 
     case VARBINARY:
     case VARCHAR:
     case CHAR:
-      return OptionValue.createString(type, name, ((NlsString) object).getValue(), scope);
+      return ((NlsString) object).getValue().toString();
 
     case BOOLEAN:
-      return OptionValue.createBoolean(type, name, (Boolean) object, scope);
+      return object;
 
     default:
       throw UserException.validationError()

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
index 9c94065..cf2baaf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
@@ -27,6 +27,7 @@ import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import org.apache.drill.exec.server.options.OptionMetaData;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
@@ -87,12 +88,12 @@ public class InboundImpersonationManager {
   public static class InboundImpersonationPolicyValidator extends StringValidator {
 
     public InboundImpersonationPolicyValidator(String name) {
-      super(name, true);
+      super(name);
     }
 
     @Override
-    public void validate(final OptionValue v, final OptionSet manager) {
-      super.validate(v, manager);
+    public void validate(final OptionValue v, final OptionMetaData metaData, final OptionSet manager) {
+      super.validate(v, metaData, manager);
 
       final List<ImpersonationPolicy> policies;
       try {

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
index c30d1b3..2e3ce3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
@@ -40,13 +40,9 @@ import org.apache.drill.exec.planner.sql.handlers.SqlHandlerUtil;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 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.OptionValue.OptionType;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 
 import com.google.common.collect.Maps;
-import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -60,7 +56,7 @@ public class UserSession implements AutoCloseable {
   private boolean supportComplexTypes = false;
   private UserCredentials credentials;
   private DrillProperties properties;
-  private OptionManager sessionOptions;
+  private SessionOptionManager sessionOptions;
   private final AtomicInteger queryCount;
   private final String sessionId;
 
@@ -156,7 +152,7 @@ public class UserSession implements AutoCloseable {
     return supportComplexTypes;
   }
 
-  public OptionManager getOptions() {
+  public SessionOptionManager getOptions() {
     return sessionOptions;
   }
 
@@ -249,9 +245,7 @@ public class UserSession implements AutoCloseable {
    * @param value option value
    */
   public void setSessionOption(String name, String value) {
-    OptionValue.Kind optionKind = ((SessionOptionManager) sessionOptions).getFallbackOptionManager().getValidator(name).getKind();
-    OptionValue optionValue = OptionValue.createOption(optionKind, OptionType.SESSION, name, value, OptionScope.SESSION);
-    sessionOptions.setOption(optionValue);
+    sessionOptions.setLocalOption(name, value);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index ae68f3e..9d8f125 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -31,6 +31,7 @@ import java.util.concurrent.SynchronousQueue;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.KerberosUtil;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
@@ -41,6 +42,7 @@ import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.TransportCheck;
 import org.apache.drill.exec.rpc.security.AuthenticatorProvider;
 import org.apache.drill.exec.rpc.security.AuthenticatorProviderImpl;
+import org.apache.drill.exec.server.options.OptionDefinition;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -64,6 +66,7 @@ public class BootStrapContext implements AutoCloseable {
   public static final String KERBEROS_NAME_MAPPING = SERVICE_LOGIN_PREFIX + ".auth_to_local";
 
   private final DrillConfig config;
+  private final CaseInsensitiveMap<OptionDefinition> definitions;
   private final AuthenticatorProvider authProvider;
   private final EventLoopGroup loop;
   private final EventLoopGroup loop2;
@@ -75,8 +78,10 @@ public class BootStrapContext implements AutoCloseable {
   private final ExecutorService scanDecodeExecutor;
   private final String hostName;
 
-  public BootStrapContext(DrillConfig config, ScanResult classpathScan) throws DrillbitStartupException {
+  public BootStrapContext(DrillConfig config, CaseInsensitiveMap<OptionDefinition> definitions,
+                          ScanResult classpathScan) throws DrillbitStartupException {
     this.config = config;
+    this.definitions = definitions;
     this.classpathScan = classpathScan;
     this.hostName = getCanonicalHostName();
     login(config);
@@ -190,6 +195,10 @@ public class BootStrapContext implements AutoCloseable {
     return config;
   }
 
+  public CaseInsensitiveMap<OptionDefinition> getDefinitions() {
+    return definitions;
+  }
+
   public EventLoopGroup getBitLoopGroup() {
     return loop;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 6f22b76..a44fba9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.StackTrace;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecConstants;
@@ -31,10 +32,10 @@ import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.zk.ZKClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.OptionDefinition;
 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.SystemOptionManager;
 import org.apache.drill.exec.server.rest.WebServer;
 import org.apache.drill.exec.service.ServiceEngine;
 import org.apache.drill.exec.store.StoragePluginRegistry;
@@ -82,19 +83,36 @@ public class Drillbit implements AutoCloseable {
 
   @VisibleForTesting
   public Drillbit(
+    final DrillConfig config,
+    final RemoteServiceSet serviceSet) throws Exception {
+    this(config, SystemOptionManager.createDefaultOptionDefinitions(), serviceSet, ClassPathScanner.fromPrescan(config));
+  }
+
+  @VisibleForTesting
+  public Drillbit(
       final DrillConfig config,
+      final CaseInsensitiveMap<OptionDefinition> definitions,
       final RemoteServiceSet serviceSet) throws Exception {
-    this(config, serviceSet, ClassPathScanner.fromPrescan(config));
+    this(config, definitions, serviceSet, ClassPathScanner.fromPrescan(config));
   }
 
   public Drillbit(
-      final DrillConfig config,
-      final RemoteServiceSet serviceSet,
-      final ScanResult classpathScan) throws Exception {
+    final DrillConfig config,
+    final RemoteServiceSet serviceSet,
+    final ScanResult classpathScan) throws Exception {
+    this(config, SystemOptionManager.createDefaultOptionDefinitions(), serviceSet, classpathScan);
+  }
+
+  @VisibleForTesting
+  public Drillbit(
+    final DrillConfig config,
+    final CaseInsensitiveMap<OptionDefinition> definitions,
+    final RemoteServiceSet serviceSet,
+    final ScanResult classpathScan) throws Exception {
     final Stopwatch w = Stopwatch.createStarted();
     logger.debug("Construction started.");
     final boolean allowPortHunting = serviceSet != null;
-    context = new BootStrapContext(config, classpathScan);
+    context = new BootStrapContext(config, definitions, classpathScan);
     manager = new WorkManager(context);
 
     webServer = new WebServer(context, manager);
@@ -198,7 +216,7 @@ public class Drillbit implements AutoCloseable {
       return;
     }
 
-    final OptionManager optionManager = getContext().getOptionManager();
+    final SystemOptionManager optionManager = getContext().getOptionManager();
 
     // parse out the properties, validate, and then set them
     final String systemProps[] = allSystemProps.split(",");
@@ -219,16 +237,16 @@ public class Drillbit implements AutoCloseable {
       }
 
       final OptionValue defaultValue = optionManager.getOption(optionName);
+
       if (defaultValue == null) {
         throwInvalidSystemOption(systemProp, "does not specify a valid option name");
       }
-      if (defaultValue.type != OptionType.SYSTEM) {
+
+      if (!defaultValue.accessibleScopes.inScopeOf(OptionScope.SYSTEM)) {
         throwInvalidSystemOption(systemProp, "does not specify a SYSTEM option ");
       }
 
-      final OptionValue optionValue = OptionValue.createOption(
-          defaultValue.kind, OptionType.SYSTEM, optionName, optionString, OptionScope.SYSTEM);
-      optionManager.setOption(optionValue);
+      optionManager.setLocalOption(defaultValue.kind, optionName, optionString);
     }
   }
 
@@ -287,22 +305,28 @@ public class Drillbit implements AutoCloseable {
   }
 
   public static Drillbit start(final StartupOptions options) throws DrillbitStartupException {
-    return start(DrillConfig.create(options.getConfigLocation()), null);
+    return start(DrillConfig.create(options.getConfigLocation()), SystemOptionManager.createDefaultOptionDefinitions(), null);
   }
 
   public static Drillbit start(final DrillConfig config) throws DrillbitStartupException {
-    return start(config, null);
+    return start(config, SystemOptionManager.createDefaultOptionDefinitions(), null);
+  }
+
+  public static Drillbit start(final DrillConfig config, final RemoteServiceSet remoteServiceSet) throws DrillbitStartupException {
+    return start(config, SystemOptionManager.createDefaultOptionDefinitions(), remoteServiceSet);
   }
 
   @SuppressWarnings("resource")
-  public static Drillbit start(final DrillConfig config, final RemoteServiceSet remoteServiceSet)
+  @VisibleForTesting
+  public static Drillbit start(final DrillConfig config, final CaseInsensitiveMap<OptionDefinition> validators,
+                               final RemoteServiceSet remoteServiceSet)
       throws DrillbitStartupException {
     logger.debug("Starting new Drillbit.");
     // TODO: allow passing as a parameter
     ScanResult classpathScan = ClassPathScanner.fromPrescan(config);
     Drillbit bit;
     try {
-      bit = new Drillbit(config, remoteServiceSet, classpathScan);
+      bit = new Drillbit(config, validators, remoteServiceSet, classpathScan);
     } catch (final Exception ex) {
       throw new DrillbitStartupException("Failure while initializing values in Drillbit.", ex);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index fb3ccf8..426b9d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -104,7 +104,7 @@ public class DrillbitContext implements AutoCloseable {
 
     this.reader = new PhysicalPlanReader(context.getConfig(), classpathScan, lpPersistence, endpoint, storagePlugins);
     this.operatorCreatorRegistry = new OperatorCreatorRegistry(classpathScan);
-    this.systemOptions = new SystemOptionManager(lpPersistence, provider,context.getConfig());
+    this.systemOptions = new SystemOptionManager(lpPersistence, provider, context.getConfig(), context.getDefinitions());
     this.functionRegistry = new FunctionImplementationRegistry(context.getConfig(), classpathScan, systemOptions);
     this.compiler = new CodeCompiler(context.getConfig(), systemOptions);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
index 9eba2a3..26f9108 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
@@ -17,44 +17,108 @@
  */
 package org.apache.drill.exec.server.options;
 
-import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
-import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
-import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
-import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
+import org.apache.drill.common.exceptions.UserException;
 
-public abstract class BaseOptionManager implements OptionSet {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
+import java.util.Iterator;
 
-  /**
-   * Gets the current option value given a validator.
-   *
-   * @param validator the validator
-   * @return option value
-   * @throws IllegalArgumentException - if the validator is not found
-   */
-  private OptionValue getOptionSafe(OptionValidator validator)  {
-    OptionValue value = getOption(validator.getOptionName());
-    return value == null ? validator.getDefault() : value;
+/**
+ * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
+ */
+public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
+
+  @Override
+  public OptionList getInternalOptionList() {
+    return getAllOptionList(true);
+  }
+
+  @Override
+  public OptionList getPublicOptionList() {
+    return getAllOptionList(false);
+  }
+
+  @Override
+  public void setLocalOption(final String name, final boolean value) {
+    setLocalOption(name, Boolean.valueOf(value));
+  }
+
+  @Override
+  public void setLocalOption(final String name, final long value) {
+    setLocalOption(name, Long.valueOf(value));
   }
 
   @Override
-  public boolean getOption(BooleanValidator validator) {
-    return getOptionSafe(validator).bool_val;
+  public void setLocalOption(final String name, final double value) {
+    setLocalOption(name, Double.valueOf(value));
   }
 
   @Override
-  public double getOption(DoubleValidator validator) {
-    return getOptionSafe(validator).float_val;
+  public void setLocalOption(final String name, final String value) {
+    setLocalOption(name, (Object) value);
   }
 
   @Override
-  public long getOption(LongValidator validator) {
-    return getOptionSafe(validator).num_val;
+  public void setLocalOption(final String name, final Object value) {
+    final OptionDefinition definition = getOptionDefinition(name);
+    final OptionValidator validator = definition.getValidator();
+    final OptionMetaData metaData = definition.getMetaData();
+    final OptionValue.AccessibleScopes type = definition.getMetaData().getType();
+    final OptionValue.OptionScope scope = getScope();
+    checkOptionPermissions(name, type, scope);
+    final OptionValue optionValue = OptionValue.create(type, name, value, scope);
+    validator.validate(optionValue, metaData, this);
+    setLocalOptionHelper(optionValue);
   }
 
   @Override
-  public String getOption(StringValidator validator) {
-    return getOptionSafe(validator).string_val;
+  public void setLocalOption(final OptionValue.Kind kind, final String name, final String valueStr) {
+    Object value;
+
+    switch (kind) {
+      case LONG:
+        value = Long.valueOf(valueStr);
+        break;
+      case DOUBLE:
+        value = Double.valueOf(valueStr);
+        break;
+      case STRING:
+        value = valueStr;
+        break;
+      case BOOLEAN:
+        value = Boolean.valueOf(valueStr);
+        break;
+      default:
+        throw new IllegalArgumentException(String.format("Unsupported kind %s", kind));
+    }
+
+    setLocalOption(name, value);
   }
 
+  private static void checkOptionPermissions(String name, OptionValue.AccessibleScopes type, OptionValue.OptionScope scope) {
+    if (!type.inScopeOf(scope)) {
+      throw UserException.permissionError()
+        .message(String.format("Cannot change option %s in scope %s", name, scope))
+        .build(logger);
+    }
+  }
+
+  abstract protected void setLocalOptionHelper(OptionValue optionValue);
+
+  abstract protected OptionValue.OptionScope getScope();
+
+  private OptionList getAllOptionList(boolean internal)
+  {
+    Iterator<OptionValue> values = this.iterator();
+    OptionList optionList = new OptionList();
+
+    while (values.hasNext()) {
+      OptionValue value = values.next();
+
+      if (getOptionDefinition(value.getName()).getMetaData().isInternal() == internal) {
+        optionList.add(value);
+      }
+    }
+
+    return optionList;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionSet.java
new file mode 100644
index 0000000..f664401
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionSet.java
@@ -0,0 +1,57 @@
+/**
+ * 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.options;
+
+/**
+ * A basic implementation of an {@link OptionSet}.
+ */
+public abstract class BaseOptionSet implements OptionSet {
+  /**
+   * Gets the current option value given a validator.
+   *
+   * @param validator the validator
+   * @return option value
+   * @throws IllegalArgumentException - if the validator is not found
+   */
+  private OptionValue getOptionSafe(OptionValidator validator) {
+    final String optionName = validator.getOptionName();
+    OptionValue value = getOption(optionName);
+    return value == null ? getDefault(optionName) : value;
+  }
+
+  @Override
+  public boolean getOption(TypeValidators.BooleanValidator validator) {
+    return getOptionSafe(validator).bool_val;
+  }
+
+  @Override
+  public double getOption(TypeValidators.DoubleValidator validator) {
+    return getOptionSafe(validator).float_val;
+  }
+
+  @Override
+  public long getOption(TypeValidators.LongValidator validator) {
+    return getOptionSafe(validator).num_val;
+  }
+
+  @Override
+  public String getOption(TypeValidators.StringValidator validator) {
+    return getOptionSafe(validator).string_val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java
index 2b3fdfc..f2d352c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java
@@ -21,7 +21,7 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 
 import org.apache.drill.common.config.DrillConfig;
-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 com.typesafe.config.ConfigValue;
@@ -58,17 +58,17 @@ public class DrillConfigIterator implements Iterable<OptionValue> {
       OptionValue optionValue = null;
       switch(cv.valueType()) {
       case BOOLEAN:
-        optionValue = OptionValue.createBoolean(OptionType.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
+        optionValue = OptionValue.create(AccessibleScopes.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
         break;
 
       case LIST:
       case OBJECT:
       case STRING:
-        optionValue = OptionValue.createString(OptionType.BOOT, name, cv.render(),OptionScope.BOOT);
+        optionValue = OptionValue.create(AccessibleScopes.BOOT, name, cv.render(),OptionScope.BOOT);
         break;
 
       case NUMBER:
-        optionValue = OptionValue.createLong(OptionType.BOOT, name, ((Number) cv.unwrapped()).longValue(),OptionScope.BOOT);
+        optionValue = OptionValue.create(OptionValue.AccessibleScopes.BOOT, name, ((Number) cv.unwrapped()).longValue(),OptionScope.BOOT);
         break;
 
       case NULL:

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
index 301fa34..032df60 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
@@ -20,18 +20,14 @@ package org.apache.drill.exec.server.options;
 import java.util.Iterator;
 
 import com.google.common.collect.Iterables;
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
 
 /**
- * An {@link OptionManager} which allows for falling back onto another {@link OptionManager}. This way method calls can
- * be delegated to the fallback manager in case the current manager does not handle the specified option. Also, all
- * options do not need to be stored at every contextual level. For example, if an option isn't changed from its default
- * within a session, then we can get the option from system options.
+ * An {@link OptionManager} which allows for falling back onto another {@link OptionManager} when retrieving options.
  * <p/>
  * {@link FragmentOptionManager} and {@link SessionOptionManager} use {@link SystemOptionManager} as the fall back
  * manager. {@link QueryOptionManager} uses {@link SessionOptionManager} as the fall back manager.
  */
-public abstract class FallbackOptionManager extends BaseOptionManager implements OptionManager {
+public abstract class FallbackOptionManager extends BaseOptionManager {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FallbackOptionManager.class);
 
   protected final OptionManager fallback;
@@ -75,65 +71,9 @@ public abstract class FallbackOptionManager extends BaseOptionManager implements
    */
   abstract OptionValue getLocalOption(String name);
 
-  /**
-   * Sets the option value for this manager without falling back.
-   *
-   * @param value the option value
-   * @return true iff the value was successfully set
-   */
-  abstract boolean setLocalOption(OptionValue value);
-
-  /**
-   * Deletes all options for this manager without falling back.
-   *
-   * If no options are set, calling this method should be no-op. See {@link OptionManager#deleteAllOptions}.
-   *
-   * @param type option type
-   * @return true iff the option type is supported
-   */
-  abstract boolean deleteAllLocalOptions(OptionType type);
-
-  /**
-   * Deletes the option with given name for this manager without falling back.
-   *
-   * This method will be called with an option name that is guaranteed to have an option validator. Also, if option
-   * with {@param name} does not exist within the manager, calling this method should be a no-op. See
-   * {@link OptionManager#deleteOption}.
-   *
-   * @param name option name
-   * @param type option type
-   * @return true iff the option type is supported
-   */
-  abstract boolean deleteLocalOption(String name, OptionType type);
-
-  @Override
-  public void setOption(OptionValue value) {
-    final OptionValidator validator =  getSystemOptionManager().getValidator(value.name);
-
-    validator.validate(value, this); // validate the option
-
-    // fallback if unable to set locally
-    if (!setLocalOption(value)) {
-      fallback.setOption(value);
-    }
-  }
-
   @Override
-  public void deleteOption(final String name, final OptionType type) {
-    getSystemOptionManager().getValidator(name); // ensure the option exists
-
-    // fallback if unable to delete locally
-    if (!deleteLocalOption(name, type)) {
-      fallback.deleteOption(name, type);
-    }
-  }
-
-  @Override
-  public void deleteAllOptions(final OptionType type) {
-    // fallback if unable to delete locally
-    if (!deleteAllLocalOptions(type)) {
-      fallback.deleteAllOptions(type);
-    }
+  public OptionDefinition getOptionDefinition(String name) {
+    return fallback.getOptionDefinition(name);
   }
 
   @Override
@@ -144,27 +84,4 @@ public abstract class FallbackOptionManager extends BaseOptionManager implements
     }
     return list;
   }
-
-  public OptionManager getFallback() {
-    return fallback;
-  }
-
-  /**
-   * {@link FragmentOptionManager} and {@link SessionOptionManager} use {@link SystemOptionManager} as the fall back
-   * manager so for both FragmentOptionManager and SessionOptionManager fallback is the SystemOptionManager so it is
-   * returned. But in case of {@link QueryOptionManager}, it uses {@link SessionOptionManager} as the fallback manager
-   * and since SessionOptionManager uses SystemOptionManager as fallback, SystemOptionManager can be fetched from the
-   * SessionOptionManager.
-   */
-  public SystemOptionManager getSystemOptionManager() {
-    final SystemOptionManager systemOptionManager;
-    if(fallback instanceof SessionOptionManager) {
-      final SessionOptionManager sessionOptionManager = (SessionOptionManager) fallback;
-      systemOptionManager = sessionOptionManager.getFallbackOptionManager();
-    }
-    else {
-      systemOptionManager = (SystemOptionManager) fallback;
-    }
-    return systemOptionManager;
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FragmentOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FragmentOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FragmentOptionManager.java
index 39f86d1..90f5623 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FragmentOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FragmentOptionManager.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.server.options;
 
 import com.google.common.collect.Maps;
 import org.apache.drill.common.map.CaseInsensitiveMap;
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
 
 import java.util.Map;
 
@@ -42,9 +41,27 @@ public class FragmentOptionManager extends InMemoryOptionManager {
   }
 
   @Override
-  boolean supportsOptionType(OptionType type) {
-    throw new UnsupportedOperationException("FragmentOptionManager does not support the given option value.");
+  public void deleteAllLocalOptions() {
+    throw new UnsupportedOperationException();
   }
 
+  @Override
+  public void deleteLocalOption(String name) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public OptionValue getDefault(String optionName) {
+    return fallback.getDefault(optionName);
+  }
 
+  @Override
+  protected OptionValue.OptionScope getScope() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setLocalOptionHelper(OptionValue value) {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/InMemoryOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/InMemoryOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/InMemoryOptionManager.java
index 7fc837e..cda5c48 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/InMemoryOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/InMemoryOptionManager.java
@@ -17,14 +17,12 @@
  */
 package org.apache.drill.exec.server.options;
 
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
-
 import java.util.Map;
 
 /**
- * {@link OptionManager} that hold options in memory rather than in a persistent store. Option stored in
+ * This is an {@link OptionManager} that holds options in memory rather than in a persistent store. Options stored in
  * {@link SessionOptionManager}, {@link QueryOptionManager}, and {@link FragmentOptionManager} are held in memory
- * (see {@link #options}) whereas {@link SystemOptionManager} stores options in a persistent store.
+ * (see {@link #options}) whereas the {@link SystemOptionManager} stores options in a persistent store.
  */
 public abstract class InMemoryOptionManager extends FallbackOptionManager {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InMemoryOptionManager.class);
@@ -42,13 +40,8 @@ public abstract class InMemoryOptionManager extends FallbackOptionManager {
   }
 
   @Override
-  boolean setLocalOption(final OptionValue value) {
-    if (supportsOptionType(value.type)) {
-      options.put(value.name, value);
-      return true;
-    } else {
-      return false;
-    }
+  public void setLocalOptionHelper(final OptionValue value) {
+    options.put(value.name, value);
   }
 
   @Override
@@ -57,31 +50,12 @@ public abstract class InMemoryOptionManager extends FallbackOptionManager {
   }
 
   @Override
-  boolean deleteAllLocalOptions(final OptionType type) {
-    if (supportsOptionType(type)) {
-      options.clear();
-      return true;
-    } else {
-      return false;
-    }
+  public void deleteAllLocalOptions() {
+    options.clear();
   }
 
   @Override
-  boolean deleteLocalOption(final String name, final OptionType type) {
-    if (supportsOptionType(type)) {
-      options.remove(name);
-      return true;
-    } else {
-      return false;
-    }
+  public void deleteLocalOption(final String name) {
+    options.remove(name);
   }
-
-  /**
-   * Check to see if implementations of this manager support the given option type.
-   *
-   * @param type option type
-   * @return true iff the type is supported
-   */
-  abstract boolean supportsOptionType(OptionType type);
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionDefinition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionDefinition.java
new file mode 100644
index 0000000..8b6234a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionDefinition.java
@@ -0,0 +1,47 @@
+/**
+ * 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.options;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This holds all the information about an option.
+ */
+public class OptionDefinition {
+  private final OptionValidator validator;
+  private final OptionMetaData metaData;
+
+  public OptionDefinition(OptionValidator validator) {
+    this.validator = Preconditions.checkNotNull(validator);
+    this.metaData = OptionMetaData.DEFAULT;
+  }
+
+  public OptionDefinition(OptionValidator validator, OptionMetaData metaData) {
+    this.validator = Preconditions.checkNotNull(validator);
+    this.metaData = Preconditions.checkNotNull(metaData);
+  }
+
+  public OptionValidator getValidator() {
+    return validator;
+  }
+
+  public OptionMetaData getMetaData() {
+    return metaData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
index cf11132..a98bbe9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
@@ -17,44 +17,104 @@
  */
 package org.apache.drill.exec.server.options;
 
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.common.exceptions.UserException;
+
+import javax.validation.constraints.NotNull;
 
 /**
  * Manager for Drill {@link OptionValue options}. Implementations must be case-insensitive to the name of an option.
+ *
+ * The options governed by an {@link OptionManager} fall into various categories. These categories are described below.
+ *
+ * <ul>
+ *   <li>
+ *     <b>Local:</b> Local options are options who have a value stored in this {@link OptionManager}. Whether an option is <b>Local</b> to an {@link OptionManager} or not should
+ *      be irrelevant to the user.
+ *   </li>
+ *   <li>
+ *     <b>Public:</b> Public options are options that are visible to end users in all the standard tables and rest endpoints.
+ *   </li>
+ *   <li>
+ *     <b>Internal:</b> Internal options are options that are only visible to end users if they check special tables and rest endpoints that are not documented. These options
+ *     are not intended to be modified by users and should only be modified by support during debugging. Internal options are also not gauranteed to be consistent accross
+ *     patch, minor, or major releases.
+ *   </li>
+ * </ul>
+ *
  */
 public interface OptionManager extends OptionSet, Iterable<OptionValue> {
 
   /**
-   * Sets an option value.
-   *
-   * @param value option value
-   * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
+   * Sets a boolean option on the {@link OptionManager}.
+   * @param name The name of the option.
+   * @param value The value of the option.
+   */
+  void setLocalOption(String name, boolean value);
+
+  /**
+   * Sets a long option on the {@link OptionManager}.
+   * @param name The name of the option.
+   * @param value The value of the option.
+   */
+  void setLocalOption(String name, long value);
+
+  /**
+   * Sets a double option on the {@link OptionManager}.
+   * @param name The name of the option.
+   * @param value The value of the option.
+   */
+  void setLocalOption(String name, double value);
+
+  /**
+   * Sets a String option on the {@link OptionManager}.
+   * @param name The name of the option.
+   * @param value The value of the option.
+   */
+  void setLocalOption(String name, String value);
+
+  /**
+   * Sets an option on the {@link OptionManager}.
+   * @param name The name of the option.
+   * @param value The value of the option.
+   */
+  void setLocalOption(String name, Object value);
+
+  /**
+   * Sets an option of the specified {@link OptionValue.Kind} on the {@link OptionManager}.
+   * @param kind The kind of the option.
+   * @param name The name of the option.
+   * @param value The value of the option.
    */
-  void setOption(OptionValue value);
+  void setLocalOption(OptionValue.Kind kind, String name, String value);
 
   /**
-   * Deletes the option. Unfortunately, the type is required given the fallback structure of option managers.
-   * See {@link FallbackOptionManager}.
+   * Deletes the option.
    *
-   * If the option name is valid (exists in {@link SystemOptionManager#VALIDATORS}),
+   * If the option name is valid (exists in the set of validators produced by {@link SystemOptionManager#createDefaultOptionDefinitions()}),
    * but the option was not set within this manager, calling this method should be a no-op.
    *
    * @param name option name
-   * @param type option type
    * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    */
-  void deleteOption(String name, OptionType type);
+  void deleteLocalOption(String name);
 
   /**
-   * Deletes all options. Unfortunately, the type is required given the fallback structure of option managers.
-   * See {@link FallbackOptionManager}.
+   * Deletes all options.
    *
    * If no options are set, calling this method should be no-op.
    *
-   * @param type option type
    * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    */
-  void deleteAllOptions(OptionType type);
+  void deleteAllLocalOptions();
+
+  /**
+   * Get the option definition corresponding to the given option name.
+   * @param name The name of the option to retrieve a validator for.
+   * @return The option validator corresponding to the given option name.
+   * @throws UserException - if the definition is not found
+   */
+  @NotNull
+  OptionDefinition getOptionDefinition(String name);
 
   /**
    * Gets the list of options managed this manager.
@@ -62,4 +122,20 @@ public interface OptionManager extends OptionSet, Iterable<OptionValue> {
    * @return the list of options
    */
   OptionList getOptionList();
+
+  /**
+   * Returns all the internal options contained in this option manager.
+   *
+   * @return All the internal options contained in this option manager.
+   */
+  @NotNull
+  OptionList getInternalOptionList();
+
+  /**
+   * Returns all the public options contained in this option manager.
+   *
+   * @return All the public options contained in this option manager.
+   */
+  @NotNull
+  OptionList getPublicOptionList();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java
new file mode 100644
index 0000000..f73c348
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java
@@ -0,0 +1,68 @@
+/**
+ * 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.options;
+
+/**
+ * Contains information about the scopes in which an option can be set, and an option's visibility.
+ */
+public class OptionMetaData {
+  public static final OptionMetaData DEFAULT = new OptionMetaData(OptionValue.AccessibleScopes.ALL, false, false);
+
+  private final OptionValue.AccessibleScopes type;
+  private final  boolean adminOnly;
+  private final boolean internal;
+
+  public OptionMetaData(OptionValue.AccessibleScopes type, boolean adminOnly, boolean internal) {
+    this.type = type;
+    this.adminOnly = adminOnly;
+    this.internal = internal;
+  }
+
+  public OptionValue.AccessibleScopes getType() {
+    return type;
+  }
+
+  public boolean isAdminOnly() {
+    return adminOnly;
+  }
+
+  public boolean isInternal() {
+    return internal;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()){
+      return false;
+    }
+
+    OptionMetaData metaData = (OptionMetaData) o;
+
+    return adminOnly == metaData.adminOnly;
+  }
+
+  @Override
+  public int hashCode() {
+    return (adminOnly ? 1 : 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionSet.java
index 662ec35..702d7df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionSet.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.server.options;
 /**
  * Immutable set of options accessible by name or validator.
  */
-
 public interface OptionSet {
 
   /**
@@ -38,6 +37,13 @@ public interface OptionSet {
   OptionValue getOption(String name);
 
   /**
+   * Gets the default value for the specified option.
+   * @param optionName The option to retrieve the default value for.
+   * @return The default value for the option.
+   */
+  OptionValue getDefault(String optionName);
+
+  /**
    * Gets the boolean value (from the option value) for the given boolean validator.
    *
    * @param validator the boolean validator

http://git-wip-us.apache.org/repos/asf/drill/blob/6adeb986/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
index 7122a19..1477339 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
@@ -28,16 +28,11 @@ public abstract class OptionValidator {
   // Stored here as well as in the option static class to allow insertion of option optionName into
   // the error messages produced by the validator
   private final String optionName;
-  private final boolean isAdminOption;
   public static final String OPTION_DEFAULTS_ROOT = "drill.exec.options.";
+
   /** By default, if admin option value is not specified, it would be set to false.*/
   public OptionValidator(String optionName) {
-    this(optionName, false);
-  }
-
-  public OptionValidator(String optionName, boolean isAdminOption) {
     this.optionName = optionName;
-    this.isAdminOption = isAdminOption;
   }
 
   /**
@@ -78,27 +73,13 @@ public abstract class OptionValidator {
   }
 
   /**
-   * @return true is option is system-level property that can be only specified by admin (not user).
-   */
-  public boolean isAdminOption() {
-    return isAdminOption;
-  }
-
-  /**
-   * Gets the default option value for this validator.
-   *
-   * @return default option value
-   */
-  public abstract OptionValue getDefault();
-
-  /**
    * Validates the option value.
    *
    * @param value the value to validate
    * @param manager the manager for accessing validation dependencies (options)
    * @throws UserException message to describe error with value, including range or list of expected values
    */
-  public abstract void validate(OptionValue value, OptionSet manager);
+  public abstract void validate(OptionValue value, OptionMetaData optionMetaData, OptionSet manager);
 
   /**
    * Gets the kind of this option value for this validator.
@@ -107,11 +88,7 @@ public abstract class OptionValidator {
    */
   public abstract Kind getKind();
 
-  /**
-   * Loads the default option value for this validator.
-   *
-   * @return  default option value
-   */
-  public abstract void loadDefault(DrillConfig bootConfig);
-
+  public String getConfigProperty() {
+    return OPTION_DEFAULTS_ROOT + getOptionName();
+  }
 }