You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by so...@apache.org on 2019/05/03 22:46:31 UTC

[drill] 05/08: DRILL-6974: SET option command modification

This is an automated email from the ASF dual-hosted git repository.

sorabh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit caa983105b0c6ea92770f0b86d7e28d457cb0bae
Author: Dmytriy Grinchenko <dm...@gmail.com>
AuthorDate: Wed Apr 17 12:23:34 2019 +0300

    DRILL-6974: SET option command modification
    
    - ALTER ... RESET ... and ALTER ... SET ... sub-parsers separated to 2
      different SqlCall classes with same parent SqlSetOption
    - parserImpls modified to handle new syntax of ALTER... SET...
      expresion:
      a) ALTER ... SET option.name - option.value - setting option value
      b) ALTER ... SET option.name - display option value
    - Handler for SqlSetOption separated to SetOptionHandler and
      ResetOptionhandler for better representation of handled statements
    - Base abstract class AbstractSqlSetHandler created to not repeat
      shared implementation of same functions
    - SetOptionHandler covered with unit tests for each statement
      form.
    
    Fix issues stated in the review
    closes #1763
---
 exec/java-exec/src/main/codegen/data/Parser.tdd    |   6 +-
 .../src/main/codegen/includes/parserImpls.ftl      |  63 +++++++++++
 .../drill/exec/planner/sql/DrillSqlWorker.java     |  22 ++--
 .../sql/handlers/AbstractSqlSetHandler.java        |  86 +++++++++++++++
 .../planner/sql/handlers/ResetOptionHandler.java   |  73 +++++++++++++
 .../planner/sql/handlers/SetOptionHandler.java     | 119 ++++++++-------------
 .../sql/parser/CompoundIdentifierConverter.java    |   4 +-
 .../planner/sql/parser/DrillSqlResetOption.java    | 103 ++++++++++++++++++
 .../exec/planner/sql/parser/DrillSqlSetOption.java |  75 +++++++++++++
 .../sql/handlers/ResetOptionHandlerTest.java       |  64 +++++++++++
 .../planner/sql/handlers/SetOptionHandlerTest.java |  90 ++++++++++++++++
 11 files changed, 623 insertions(+), 82 deletions(-)

diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 3580388..df97e24 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -59,7 +59,9 @@
     "SqlRefreshMetadata()",
     "SqlCreateFunction()",
     "SqlDropFunction()",
-    "SqlAnalyzeTable()"
+    "SqlAnalyzeTable()",
+    "DrillSqlSetOption(Span.of(), null)",
+    "DrillSqlResetOption(Span.of(), null)"
   ]
 
   # List of methods for parsing custom literals.
@@ -87,6 +89,8 @@
   # List of methods for parsing extensions to "ALTER <scope>" calls.
   # Each must accept arguments "(SqlParserPos pos, String scope)".
   alterStatementParserMethods: [
+    "DrillSqlSetOption",
+    "DrillSqlResetOption"
   ]
 
   # List of methods for parsing extensions to "DROP" calls.
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 25ee72b..f90859a 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -634,3 +634,66 @@ SqlNode SqlAnalyzeTable() :
         return new SqlAnalyzeTable(pos, tblName, estimate, fieldList, percent);
     }
 }
+
+
+/**
+ * Parses a SET statement without a leading "ALTER <SCOPE>":
+ *
+ * SET &lt;NAME&gt; [ = VALUE ]
+ * <p>
+ * Statement handles in: {@link SetAndResetOptionHandler}
+ */
+DrillSqlSetOption DrillSqlSetOption(Span s, String scope) :
+{
+    SqlParserPos pos;
+    SqlIdentifier name;
+    SqlNode val = null;
+}
+{
+    <SET> {
+        s.add(this);
+    }
+    name = CompoundIdentifier()
+    (
+        <EQ>
+        (
+            val = Literal()
+        |
+            val = SimpleIdentifier()
+        )
+    )?
+    {
+      pos = (val == null) ? s.end(name) : s.end(val);
+
+      return new DrillSqlSetOption(pos, scope, name, val);
+    }
+}
+
+/**
+ * Parses a RESET statement without a leading "ALTER <SCOPE>":
+ *
+ *  RESET { <NAME> | ALL }
+ * <p>
+ * Statement handles in: {@link SetAndResetOptionHandler}
+ */
+DrillSqlResetOption DrillSqlResetOption(Span s, String scope) :
+{
+    SqlIdentifier name;
+}
+{
+    <RESET> {
+        s.add(this);
+    }
+    (
+        name = CompoundIdentifier()
+    |
+        <ALL> {
+            name = new SqlIdentifier(token.image.toUpperCase(Locale.ROOT), getPos());
+        }
+    )
+    {
+        return new DrillSqlResetOption(s.end(name), scope, name);
+    }
+}
+
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 09fbbdc..5ce9c19 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -41,11 +41,14 @@ import org.apache.drill.exec.planner.sql.handlers.DescribeSchemaHandler;
 import org.apache.drill.exec.planner.sql.handlers.DescribeTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
 import org.apache.drill.exec.planner.sql.handlers.RefreshMetadataHandler;
+import org.apache.drill.exec.planner.sql.handlers.ResetOptionHandler;
 import org.apache.drill.exec.planner.sql.handlers.SchemaHandler;
 import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
+import org.apache.drill.exec.planner.sql.parser.DrillSqlResetOption;
+import org.apache.drill.exec.planner.sql.parser.DrillSqlSetOption;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.planner.sql.parser.SqlSchema;
 import org.apache.drill.exec.testing.ControlsInjector;
@@ -150,15 +153,22 @@ public class DrillSqlWorker {
     final AbstractSqlHandler handler;
     final SqlHandlerConfig config = new SqlHandlerConfig(context, parser);
 
-    switch(sqlNode.getKind()) {
+    switch (sqlNode.getKind()) {
       case EXPLAIN:
         handler = new ExplainHandler(config, textPlan);
         context.setSQLStatementType(SqlStatementType.EXPLAIN);
         break;
       case SET_OPTION:
-        handler = new SetOptionHandler(context);
-        context.setSQLStatementType(SqlStatementType.SETOPTION);
-        break;
+        if (sqlNode instanceof DrillSqlSetOption) {
+          handler = new SetOptionHandler(context);
+          context.setSQLStatementType(SqlStatementType.SETOPTION);
+          break;
+        }
+        if (sqlNode instanceof DrillSqlResetOption) {
+          handler = new ResetOptionHandler(context);
+          context.setSQLStatementType(SqlStatementType.SETOPTION);
+          break;
+        }
       case DESCRIBE_TABLE:
         if (sqlNode instanceof DrillSqlDescribeTable) {
           handler = new DescribeTableHandler(config);
@@ -184,8 +194,8 @@ public class DrillSqlWorker {
       case DROP_VIEW:
       case OTHER_DDL:
       case OTHER:
-        if(sqlNode instanceof SqlCreateTable) {
-          handler = ((DrillSqlCall)sqlNode).getSqlHandler(config, textPlan);
+        if (sqlNode instanceof SqlCreateTable) {
+          handler = ((DrillSqlCall) sqlNode).getSqlHandler(config, textPlan);
           context.setSQLStatementType(SqlStatementType.CTAS);
           break;
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AbstractSqlSetHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AbstractSqlSetHandler.java
new file mode 100644
index 0000000..71d7636
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AbstractSqlSetHandler.java
@@ -0,0 +1,86 @@
+/*
+ * 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.planner.sql.handlers;
+
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.QueryOptionManager;
+import org.apache.drill.exec.util.ImpersonationUtil;
+
+/**
+ * Base handler for SQL_SET kind statements.
+ */
+abstract class AbstractSqlSetHandler extends AbstractSqlHandler {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSqlHandler.class);
+
+  final QueryContext context;
+
+  AbstractSqlSetHandler(QueryContext context) {
+    this.context = context;
+  }
+
+  /**
+   * Extracts query {@link OptionValue.OptionScope} from the {@link SqlSetOption}.
+   * @param statement Statement object
+   * @param options Options object
+   * @return parsed query scope
+   */
+  OptionValue.OptionScope getScope(SqlSetOption statement, QueryOptionManager options) {
+    String scope = statement.getScope();
+
+    if (scope == null) {
+      return OptionValue.OptionScope.SESSION;
+    }
+
+    switch (scope.toLowerCase()) {
+      case "session":
+        if (options.getBoolean(ExecConstants.SKIP_ALTER_SESSION_QUERY_PROFILE)) {
+          logger.debug("Will not write profile for ALTER SESSION SET ... ");
+          context.skipWritingProfile(true);
+        }
+        return OptionValue.OptionScope.SESSION;
+      case "system":
+        return OptionValue.OptionScope.SYSTEM;
+      default:
+        throw UserException.validationError()
+            .message("Invalid OPTION scope %s. Scope must be SESSION or SYSTEM.", scope)
+            .build(logger);
+    }
+  }
+
+  /**
+   * Admin privileges checker.
+   * @param options Options object
+   */
+  void checkAdminPrivileges(QueryOptionManager options) {
+    if (context.isUserAuthenticationEnabled()
+        && !ImpersonationUtil.hasAdminPrivileges(
+            context.getQueryUserName(),
+            ExecConstants.ADMIN_USERS_VALIDATOR.getAdminUsers(options),
+            ExecConstants.ADMIN_USER_GROUPS_VALIDATOR.getAdminUserGroups(options))) {
+
+      throw UserException
+          .permissionError()
+          .message("Not authorized to change SYSTEM options.")
+          .build(logger);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandler.java
new file mode 100644
index 0000000..8f99192
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandler.java
@@ -0,0 +1,73 @@
+/*
+ * 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.planner.sql.handlers;
+
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.parser.DrillSqlResetOption;
+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.QueryOptionManager;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSetOption;
+
+/**
+ * Converts a {@link SqlNode} representing: "ALTER .. RESET option | ALL" statement to a {@link PhysicalPlan}.
+ * See {@link DrillSqlResetOption}.
+ * <p>
+ * These statements have side effects i.e. the options within the system context or the session context are modified.
+ * The resulting {@link DirectPlan} returns to the client a string that is the name of the option that was updated
+ * or a value of the property
+ */
+public class ResetOptionHandler extends AbstractSqlSetHandler {
+
+  /**
+   * Class constructor.
+   * @param context Context of the Query
+   */
+  public ResetOptionHandler(QueryContext context) {
+    super(context);
+  }
+
+  /**
+   * Handles {@link DrillSqlResetOption} query
+   */
+  @Override
+  public final PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException {
+    QueryOptionManager options = context.getOptions();
+    SqlSetOption statement = unwrap(sqlNode, SqlSetOption.class);
+    OptionScope optionScope = getScope(statement, context.getOptions());
+
+    if (optionScope == OptionValue.OptionScope.SYSTEM) {
+      checkAdminPrivileges(options);
+    }
+
+    OptionManager optionManager = options.getOptionManager(optionScope);
+    String optionName = statement.getName().toString();
+
+    if ("ALL".equalsIgnoreCase(optionName)) {
+      optionManager.deleteAllLocalOptions();
+    } else {
+      optionManager.deleteLocalOption(optionName);
+    }
+    return DirectPlan.createDirectPlan(context, true, String.format("%s updated.", optionName));
+  }
+}
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 8a6a30c..41a1b78 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
@@ -20,108 +20,68 @@ package org.apache.drill.exec.planner.sql.handlers;
 import java.math.BigDecimal;
 
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.ValidationException;
-
 import org.apache.calcite.util.NlsString;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.parser.DrillSqlSetOption;
 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.QueryOptionManager;
-import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSetOption;
 
 /**
- * Converts a {@link SqlNode} representing "ALTER .. SET option = value" and "ALTER ... RESET ..." statements to a
- * {@link PhysicalPlan}. See {@link SqlSetOption}. These statements have side effects i.e. the options within the
- * system context or the session context are modified. The resulting {@link DirectPlan} returns to the client a string
- * that is the name of the option that was updated.
+ * Converts a {@link SqlNode} representing: "ALTER .. SET option = value" or "ALTER ... SET option"
+ * statement to a {@link PhysicalPlan}. See {@link DrillSqlSetOption}
+ * <p>
+ * These statements have side effects i.e. the options within the system context or the session context are modified.
+ * The resulting {@link DirectPlan} returns to the client a string that is the name of the option that was updated
+ * or a value of the property
  */
-public class SetOptionHandler extends AbstractSqlHandler {
+public class SetOptionHandler extends AbstractSqlSetHandler {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionHandler.class);
 
-  private final QueryContext context;
-
   public SetOptionHandler(QueryContext context) {
-    this.context = context;
+    super(context);
   }
 
+  /**
+   * Handles {@link DrillSqlSetOption} query
+   */
   @Override
-  public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, ForemanSetupException {
-    final SqlSetOption option = unwrap(sqlNode, SqlSetOption.class);
-    final SqlNode value = option.getValue();
-    if (value != null && !(value instanceof SqlLiteral)) {
-      throw UserException.validationError()
-          .message("Drill does not support assigning non-literal values in SET statements.")
-          .build(logger);
-    }
+  public final PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException {
+    // sqlNode could contain DrillSqlResetOption or DrillSqlSetOption, depends on parsed statement
+    SqlSetOption statement = unwrap(sqlNode, SqlSetOption.class);
+    OptionScope optionScope = getScope(statement, context.getOptions());
+    OptionManager optionManager = context.getOptions().getOptionManager(optionScope);
+
+    String optionName = statement.getName().toString();
+    SqlNode optionValue = statement.getValue();
+
+    if (optionValue == null) {
+      String value = String.valueOf(optionManager.getOption(optionName).getValue());
 
-    final QueryOptionManager options = context.getOptions();
-    final String scope = option.getScope();
-    final OptionValue.OptionScope optionScope;
-    if (scope == null) { // No scope mentioned assumed SESSION
-      optionScope = OptionScope.SESSION;
+      return DirectPlan.createDirectPlan(context, new SetOptionViewResult(optionName, value));
     } else {
-      switch (scope.toLowerCase()) {
-      case "session":
-        optionScope = OptionScope.SESSION;
-        // Skip writing profiles for "ALTER SESSION SET" queries
-        if (options.getBoolean(ExecConstants.SKIP_ALTER_SESSION_QUERY_PROFILE)) {
-          logger.debug("Will not write profile for ALTER SESSION SET ... ");
-          context.skipWritingProfile(true);
-        }
-        break;
-      case "system":
-        optionScope = OptionScope.SYSTEM;
-        break;
-      default:
+      if (optionScope == OptionValue.OptionScope.SYSTEM) {
+        checkAdminPrivileges(context.getOptions());
+      }
+      if (!(optionValue instanceof SqlLiteral)) {
         throw UserException.validationError()
-            .message("Invalid OPTION scope %s. Scope must be SESSION or SYSTEM.", scope)
+            .message("Drill does not support assigning non-literal values in SET statements.")
             .build(logger);
       }
-    }
+      optionManager.setLocalOption(optionName, sqlLiteralToObject((SqlLiteral) optionValue));
 
-    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() &&
-          !ImpersonationUtil.hasAdminPrivileges(
-            context.getQueryUserName(),
-            ExecConstants.ADMIN_USERS_VALIDATOR.getAdminUsers(options),
-            ExecConstants.ADMIN_USER_GROUPS_VALIDATOR.getAdminUserGroups(options))) {
-        throw UserException.permissionError()
-            .message("Not authorized to change SYSTEM options.")
-            .build(logger);
-      }
+      return DirectPlan.createDirectPlan(context, true, String.format("%s updated.", optionName));
     }
-
-    final String optionName = option.getName().toString();
-
-    // Currently, we convert multi-part identifier to a string.
-    final OptionManager chosenOptions = options.getOptionManager(optionScope);
-
-    if (value != null) { // SET option
-      final Object literalObj = sqlLiteralToObject((SqlLiteral) value);
-      chosenOptions.setLocalOption(optionName, literalObj);
-    } else { // RESET option
-      if ("ALL".equalsIgnoreCase(optionName)) {
-        chosenOptions.deleteAllLocalOptions();
-      } else {
-        chosenOptions.deleteLocalOption(optionName);
-      }
-    }
-
-    return DirectPlan.createDirectPlan(context, true, String.format("%s updated.", optionName));
   }
 
-  private static Object sqlLiteralToObject(final SqlLiteral literal) {
+  private static Object sqlLiteralToObject(SqlLiteral literal) {
     final Object object = literal.getValue();
     final SqlTypeName typeName = literal.getTypeName();
     switch (typeName) {
@@ -147,7 +107,7 @@ public class SetOptionHandler extends AbstractSqlHandler {
     case VARBINARY:
     case VARCHAR:
     case CHAR:
-      return ((NlsString) object).getValue().toString();
+      return ((NlsString) object).getValue();
 
     case BOOLEAN:
       return object;
@@ -158,4 +118,17 @@ public class SetOptionHandler extends AbstractSqlHandler {
         .build(logger);
     }
   }
+
+  /**
+   * Representation of "SET property.name" query result.
+   */
+  public static class SetOptionViewResult {
+    public String name;
+    public String value;
+
+    SetOptionViewResult(String name, String value) {
+      this.name = name;
+      this.value = value;
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index 119b27d..ac0d163 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -27,7 +27,6 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOrderBy;
 import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.SqlSetOption;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.util.SqlVisitor;
 
@@ -77,7 +76,8 @@ public class CompoundIdentifierConverter extends SqlShuttle {
         .put(SqlOrderBy.class, arrayOf(D, E, D, D))
         .put(SqlDropTable.class, arrayOf(D, D))
         .put(SqlRefreshMetadata.class, arrayOf(D, D, E))
-        .put(SqlSetOption.class, arrayOf(D, D, D))
+        .put(DrillSqlSetOption.class, arrayOf(D, D, D))
+        .put(DrillSqlResetOption.class, arrayOf(D, D))
         .put(SqlCreateFunction.class, arrayOf(D))
         .put(SqlDropFunction.class, arrayOf(D))
         .put(SqlSchema.Create.class, arrayOf(D, D, D, D, D, D))
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlResetOption.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlResetOption.java
new file mode 100644
index 0000000..18c199c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlResetOption.java
@@ -0,0 +1,103 @@
+/*
+ * 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.planner.sql.parser;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.drill.exec.planner.sql.handlers.ResetOptionHandler;
+
+/**
+ * Sql parse tree node to represent statement: {@code RESET { <NAME> | ALL } }.
+ * Statement handled in: {@link ResetOptionHandler}
+ */
+public final class DrillSqlResetOption extends SqlSetOption {
+
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("RESET_OPTION", SqlKind.SET_OPTION) {
+    @Override
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      SqlNode scopeNode = operands[0];
+      return new DrillSqlResetOption(pos, scopeNode == null ? null : scopeNode.toString(), (SqlIdentifier) operands[1]);
+    }
+  };
+
+  public DrillSqlResetOption(SqlParserPos pos, String scope, SqlIdentifier name) {
+    super(pos, scope, name, null);
+  }
+
+  @Override
+  public SqlKind getKind() {
+    return SqlKind.SET_OPTION;
+  }
+
+  @Override
+  public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override
+  public List<SqlNode> getOperandList() {
+    List<SqlNode> operandList = new ArrayList<>();
+
+    SqlIdentifier scopeIdentifier = (this.getScope() == null) ? null : new SqlIdentifier(this.getScope(),
+      SqlParserPos.ZERO);
+
+    operandList.add(scopeIdentifier);
+    operandList.add(this.getName());
+    return ImmutableNullableList.copyOf(operandList);
+  }
+
+  @Override
+  public void setOperand(int i, SqlNode operand) {
+    switch (i) {
+      case 0:
+        if (operand != null) {
+          this.setScope(((SqlIdentifier) operand).getSimple());
+        } else {
+          this.setScope(null);
+        }
+        break;
+      case 1:
+        this.setName((SqlIdentifier) operand);
+        break;
+      default:
+        throw new AssertionError(i);
+    }
+  }
+
+  @Override
+  protected void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword("RESET");
+
+    SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.SIMPLE);
+
+    this.getName().unparse(writer, leftPrec, rightPrec);
+    writer.endList(frame);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlSetOption.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlSetOption.java
new file mode 100644
index 0000000..8cf5f77
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlSetOption.java
@@ -0,0 +1,75 @@
+/*
+ * 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.planner.sql.parser;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
+
+/**
+ * Sql parse tree node to represent statement: {@code SET <NAME> [ = VALUE ]}.
+ * Statement handled in: {@link SetOptionHandler}
+ */
+public final class DrillSqlSetOption extends SqlSetOption {
+
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SET_OPTION", SqlKind.SET_OPTION) {
+    @Override
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      SqlNode scopeNode = operands[0];
+      String scope = scopeNode == null ? null : scopeNode.toString();
+      return new DrillSqlSetOption(pos, scope, (SqlIdentifier) operands[1], operands[2]);
+    }
+  };
+
+    public DrillSqlSetOption(SqlParserPos pos, String scope, SqlIdentifier name, SqlNode value) {
+    super(pos, scope, name, value);
+  }
+
+  @Override
+  public SqlKind getKind() {
+    return SqlKind.SET_OPTION;
+  }
+
+  @Override
+  public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override
+  protected void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword("SET");
+
+    SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.SIMPLE);
+    this.getName().unparse(writer, leftPrec, rightPrec);
+
+    if (this.getValue() != null) {
+      writer.sep("=");
+      this.getValue().unparse(writer, leftPrec, rightPrec);
+    }
+
+    writer.endList(frame);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandlerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandlerTest.java
new file mode 100644
index 0000000..cc3d4a2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/ResetOptionHandlerTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.planner.sql.handlers;
+
+import org.apache.drill.categories.SqlTest;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SqlTest.class)
+public class ResetOptionHandlerTest extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher).maxParallelization(1));
+  }
+
+  @Test
+  public void testReset() throws Exception {
+    int defaultValue = Integer.valueOf(client.queryBuilder()
+        .sql("SELECT val from sys.options where name = '%s' limit 1", ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE)
+        .singletonString());
+
+    int testValue = defaultValue + 55;
+
+    try {
+      client.alterSession(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE, testValue);
+      client.testBuilder()
+          .sqlQuery("select name, val from sys.options where name = '%s'", ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE)
+          .unOrdered()
+          .baselineColumns("name", "val")
+          .baselineValues(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE, String.valueOf(testValue))
+          .go();
+
+      client.resetSession(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE);
+      client.testBuilder()
+          .sqlQuery("select name, val from sys.options where name = '%s'", ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE)
+          .unOrdered()
+          .baselineColumns("name", "val")
+          .baselineValues(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE, String.valueOf(defaultValue))
+          .go();
+    } finally {
+      client.resetSession(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandlerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandlerTest.java
new file mode 100644
index 0000000..def1240
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandlerTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.planner.sql.handlers;
+
+import org.apache.drill.categories.SqlTest;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.compile.ClassCompilerSelector;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SqlTest.class)
+public class SetOptionHandlerTest extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher).maxParallelization(1));
+  }
+
+  @Test
+  public void testSimpleSetQuery() throws Exception {
+    String defaultValue = client.queryBuilder()
+        .sql("SELECT val from sys.options where name = '%s' limit 1",
+            ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION)
+        .singletonString();
+
+    boolean newValue = !Boolean.valueOf(defaultValue);
+    try {
+      client.alterSession(ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION, newValue);
+
+      String changedValue = client.queryBuilder()
+          .sql("SELECT val from sys.options where name = '%s' limit 1",
+              ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION)
+          .singletonString();
+
+      Assert.assertEquals(String.valueOf(newValue), changedValue);
+    } finally {
+      client.resetSession(ClassCompilerSelector.JAVA_COMPILER_DEBUG_OPTION);
+    }
+  }
+
+  @Test
+  public void testViewSetQuery() throws Exception {
+    client.testBuilder()  // BIT
+        .sqlQuery("SET `%s`", ExecConstants.ENABLE_ITERATOR_VALIDATION_OPTION)
+        .unOrdered()
+        .sqlBaselineQuery("SELECT name, val as value FROM sys.options where name = '%s' limit 1",
+            ExecConstants.ENABLE_ITERATOR_VALIDATION_OPTION)
+        .go();
+
+    client.testBuilder()  // BIGINT
+        .sqlQuery("SET `%s`", ExecConstants.OUTPUT_BATCH_SIZE)
+        .unOrdered()
+        .sqlBaselineQuery("SELECT name, val as value FROM sys.options where name = '%s' limit 1",
+            ExecConstants.OUTPUT_BATCH_SIZE)
+        .go();
+
+    client.testBuilder()  // FLOAT
+        .sqlQuery("SET `%s`", ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR)
+        .unOrdered()
+        .sqlBaselineQuery("SELECT name, val as value FROM sys.options where name = '%s' limit 1",
+            ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR)
+        .go();
+
+    client.testBuilder()  // VARCHAR
+        .sqlQuery("SET `%s`", ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL)
+        .unOrdered()
+        .sqlBaselineQuery("SELECT name, val as value FROM sys.options where name = '%s' limit 1",
+            ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL)
+        .go();
+  }
+}