You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2022/06/23 09:32:35 UTC

[impala] 02/02: IMPALA-11233: Unset all query option

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

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

commit f5fc08573352d0a1943296209791a4db17268086
Author: gaoxq <ga...@gmail.com>
AuthorDate: Wed Apr 20 16:41:40 2022 +0800

    IMPALA-11233: Unset all query option
    
    When using jdbc connection pool, a connection set some query options,
    after query finished, connection is closed and put back to the connection
    pool. When connection used again, the last query option also come into
    effect. We need a feature that a set statement can reset all query option
    without recreating a new connection.
    
    Support UNSET statements in SQL dialect. UNSET ALL can unset all query
    option.
    
    Testing:
      - add unset all query option in test_hs2.py
    
    Change-Id: Iabf23622daab733ddab20dd3ca73af6c9bd5c250
    Reviewed-on: http://gerrit.cloudera.org:8080/18430
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/client-request-state.cc             | 13 ++++++++--
 be/src/service/query-options.cc                    | 21 ++++++++++++++++
 be/src/service/query-options.h                     |  4 +++
 common/thrift/Frontend.thrift                      | 11 ++++++--
 fe/src/main/cup/sql-parser.cup                     | 21 +++++++++++-----
 .../java/org/apache/impala/analysis/SetStmt.java   | 17 ++++++++-----
 .../apache/impala/analysis/AnalyzeStmtsTest.java   |  1 +
 .../org/apache/impala/analysis/ParserTest.java     |  2 +-
 shell/impala_shell.py                              |  4 +++
 tests/custom_cluster/test_set_and_unset.py         | 29 ++++++++++++++++++++++
 tests/hs2/test_hs2.py                              | 16 ++++++++++++
 tests/shell/test_shell_interactive.py              | 10 ++++++++
 12 files changed, 132 insertions(+), 17 deletions(-)

diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 48c231dd4..ea1c2298a 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -281,10 +281,19 @@ Status ClientRequestState::Exec() {
           VLOG_QUERY << "ClientRequestState::Exec() SET: idle_session_timeout="
                      << PrettyPrinter::Print(session_->session_timeout, TUnit::TIME_S);
         }
+      } else if (exec_request_->set_query_option_request.__isset.query_option_type
+          && exec_request_->set_query_option_request.query_option_type
+              == TQueryOptionType::UNSET_ALL) {
+        // "UNSET ALL"
+        RETURN_IF_ERROR(ResetAllQueryOptions(
+            &session_->set_query_options, &session_->set_query_options_mask));
+        SetResultSet({}, {}, {});
       } else {
         // "SET" or "SET ALL"
-        bool is_set_all = exec_request_->set_query_option_request.__isset.is_set_all &&
-            exec_request_->set_query_option_request.is_set_all;
+        bool is_set_all =
+            exec_request_->set_query_option_request.__isset.query_option_type
+            && exec_request_->set_query_option_request.query_option_type
+                == TQueryOptionType::SET_ALL;
         PopulateResultForSet(is_set_all);
       }
       break;
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 39ecb9954..dc0195c42 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1304,3 +1304,24 @@ void impala::PopulateQueryOptionLevels(QueryOptionLevels* query_option_levels)
 #undef QUERY_OPT_FN
 #undef REMOVED_QUERY_OPT_FN
 }
+
+Status impala::ResetAllQueryOptions(
+    TQueryOptions* query_options, QueryOptionsMask* set_query_options_mask) {
+  static const TQueryOptions defaults = DefaultQueryOptions();
+#define QUERY_OPT_FN(NAME, ENUM, LEVEL)                  \
+  if (query_options->NAME != defaults.NAME) {            \
+    query_options->__isset.NAME = defaults.__isset.NAME; \
+    query_options->NAME = defaults.NAME;                 \
+    int option = GetQueryOptionForKey(#NAME);            \
+    DCHECK_GE(option, 0);                                \
+    if (set_query_options_mask != nullptr) {             \
+      DCHECK_LT(option, set_query_options_mask->size()); \
+      set_query_options_mask->reset(option);             \
+    }                                                    \
+  }
+#define REMOVED_QUERY_OPT_FN(NAME, ENUM)
+  QUERY_OPTS_TABLE
+#undef QUERY_OPT_FN
+#undef REMOVED_QUERY_OPT_FN
+  return Status::OK();
+}
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 48444a4f9..b459f8b81 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -333,6 +333,10 @@ Status ParseQueryOptions(const std::string& options, TQueryOptions* query_option
 /// entries.
 void PopulateQueryOptionLevels(QueryOptionLevels* query_option_levels);
 
+/// Reset all query options to its default value if they are not equal to default value.
+/// The bit corresponding to query option 'key' in set_query_options_mask is unset.
+Status ResetAllQueryOptions(
+    TQueryOptions* query_options, QueryOptionsMask* set_query_options_mask);
 }
 
 #endif
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 0afb07498..c6a79171f 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -462,13 +462,20 @@ struct TCatalogOpRequest {
   19: optional TDescribeHistoryParams describe_history_params
 }
 
+// Query options type
+enum TQueryOptionType {
+  SET_ONE = 0
+  SET_ALL = 1
+  UNSET_ALL = 2
+}
+
 // Parameters for the SET query option command
 struct TSetQueryOptionRequest {
   // Set for "SET key=value", unset for "SET" and "SET ALL" statements.
   1: optional string key
   2: optional string value
-  // Set true for "SET ALL"
-  3: optional bool is_set_all
+  // query option type
+  3: optional TQueryOptionType query_option_type
 }
 
 struct TShutdownParams {
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 6e16c290e..ca8c69175 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -57,6 +57,7 @@ import org.apache.impala.thrift.THdfsFileFormat;
 import org.apache.impala.thrift.TOwnerType;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TQueryOptions;
+import org.apache.impala.thrift.TQueryOptionType;
 import org.apache.impala.thrift.TShowStatsOp;
 import org.apache.impala.thrift.TTablePropertyType;
 import org.apache.impala.thrift.TPrincipalType;
@@ -359,6 +360,7 @@ nonterminal List<SetOperand> values_operand_list;
 // USE stmt
 nonterminal UseStmt use_stmt;
 nonterminal SetStmt set_stmt;
+nonterminal SetStmt unset_stmt;
 nonterminal ShowTablesStmt show_tables_stmt;
 nonterminal ShowDbsStmt show_dbs_stmt;
 nonterminal ShowStatsStmt show_stats_stmt, show_partitions_stmt,
@@ -712,6 +714,8 @@ stmt ::=
   {: RESULT = reset_metadata; :}
   | set_stmt:set
   {: RESULT = set; :}
+  | unset_stmt:unset
+  {: RESULT = unset; :}
   | show_roles_stmt:show_roles
   {: RESULT = show_roles; :}
   | show_grant_principal_stmt:show_grant_principal
@@ -2919,19 +2923,24 @@ select_clause ::=
 
 set_stmt ::=
   KW_SET ident_or_default:key EQUAL numeric_literal:l
-  {: RESULT = new SetStmt(key, l.getStringValue(), false); :}
+  {: RESULT = new SetStmt(key, l.getStringValue(), TQueryOptionType.SET_ONE); :}
   | KW_SET ident_or_default:key EQUAL STRING_LITERAL:l
-  {: RESULT = new SetStmt(key, l, false); :}
+  {: RESULT = new SetStmt(key, l, TQueryOptionType.SET_ONE); :}
   | KW_SET ident_or_default:key EQUAL SUBTRACT numeric_literal:l
   {:
     l.swapSign();
-    RESULT = new SetStmt(key, l.getStringValue(), false); :}
+    RESULT = new SetStmt(key, l.getStringValue(), TQueryOptionType.SET_ONE); :}
   | KW_SET ident_or_default:key EQUAL word:value
-  {: RESULT = new SetStmt(key, value, false); :}
+  {: RESULT = new SetStmt(key, value, TQueryOptionType.SET_ONE); :}
   | KW_SET KW_ALL
-  {: RESULT = new SetStmt(null, null, true); :}
+  {: RESULT = new SetStmt(null, null, TQueryOptionType.SET_ALL); :}
   | KW_SET
-  {: RESULT = new SetStmt(null, null, false); :}
+  {: RESULT = new SetStmt(null, null, TQueryOptionType.SET_ONE); :}
+  ;
+
+unset_stmt ::=
+  KW_UNSET KW_ALL
+  {: RESULT = new SetStmt(null, null, TQueryOptionType.UNSET_ALL); :}
   ;
 
 // Top-level function call, e.g. ": shutdown()", used for admin commands, etc.
diff --git a/fe/src/main/java/org/apache/impala/analysis/SetStmt.java b/fe/src/main/java/org/apache/impala/analysis/SetStmt.java
index 1fa1faf64..0d270b877 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SetStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SetStmt.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.analysis;
 
+import org.apache.impala.thrift.TQueryOptionType;
 import org.apache.impala.thrift.TSetQueryOptionRequest;
 
 import com.google.common.base.Preconditions;
@@ -27,7 +28,7 @@ import com.google.common.base.Preconditions;
 public class SetStmt extends StatementBase {
   private final String key_;
   private final String value_;
-  private final boolean isSetAll_;
+  private final TQueryOptionType queryOptionType_;
 
   // This key is deprecated in Impala 2.0; COMPRESSION_CODEC_KEY replaces this
   private static final String DEPRECATED_PARQUET_CODEC_KEY = "PARQUET_COMPRESSION_CODEC";
@@ -41,19 +42,23 @@ public class SetStmt extends StatementBase {
     return key;
   }
 
-  public SetStmt(String key, String value, boolean isSetAll) {
+  public SetStmt(String key, String value, TQueryOptionType queryOptionType) {
     Preconditions.checkArgument((key == null) == (value == null));
     Preconditions.checkArgument(key == null || !key.isEmpty());
-    Preconditions.checkArgument(!isSetAll || (key == null && value == null) );
+    Preconditions.checkArgument(
+        queryOptionType != TQueryOptionType.SET_ALL || (key == null && value == null));
     key_ = key;
     value_ = value;
-    isSetAll_ = isSetAll;
+    queryOptionType_ = queryOptionType;
   }
 
   @Override
   public String toSql(ToSqlOptions options) {
+    if (queryOptionType_ == TQueryOptionType.UNSET_ALL) {
+      return "UNSET ALL";
+    }
     if (key_ == null) {
-      if (isSetAll_) return "SET ALL";
+      if (queryOptionType_ == TQueryOptionType.SET_ALL) return "SET ALL";
       return "SET";
     }
     Preconditions.checkNotNull(value_);
@@ -71,7 +76,7 @@ public class SetStmt extends StatementBase {
       request.setKey(resolveThriftKey(key_));
       request.setValue(value_);
     }
-    if (isSetAll_) request.setIs_set_all(true);
+    request.setQuery_option_type(queryOptionType_);
     return request;
   }
 }
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
index 43c925057..a2280d6b6 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
@@ -4637,6 +4637,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
   public void TestSetQueryOption() {
     AnalyzesOk("set foo=true");
     AnalyzesOk("set");
+    AnalyzesOk("unset all");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index d70018638..c8d0419a3 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3505,7 +3505,7 @@ public class ParserTest extends FrontendTestBase {
         "Encountered: IDENTIFIER\n" +
         "Expected: ALTER, COMMENT, COMPUTE, COPY, CREATE, DELETE, DESCRIBE, DROP, " +
             "EXPLAIN, GRANT, INSERT, INVALIDATE, LOAD, REFRESH, REVOKE, SELECT, SET, " +
-            "SHOW, TRUNCATE, UPDATE, UPSERT, USE, VALUES, WITH\n");
+            "SHOW, TRUNCATE, UNSET, UPDATE, UPSERT, USE, VALUES, WITH\n");
 
     // missing select list
     ParserError("select from t",
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index 503ef222c..6e79e0a81 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -894,6 +894,10 @@ class ImpalaShell(cmd.Cmd, object):
       del self.set_query_options[option]
     elif self._handle_unset_shell_options(option):
       print('Unsetting shell option %s' % option)
+    elif option == 'ALL':
+      print('Unsetting all option')
+      for key in list(self.set_query_options.keys()):
+        del self.set_query_options[key]
     else:
       print("No option called %s is set" % option)
 
diff --git a/tests/custom_cluster/test_set_and_unset.py b/tests/custom_cluster/test_set_and_unset.py
index dc5e41496..3d3c6ab5b 100644
--- a/tests/custom_cluster/test_set_and_unset.py
+++ b/tests/custom_cluster/test_set_and_unset.py
@@ -84,6 +84,35 @@ class TestSetAndUnset(CustomClusterTestSuite, HS2TestSuite):
     get_profile_req.sessionHandle = self.session_handle
     assert "BATCH_SIZE=999" in self.hs2_client.GetRuntimeProfile(get_profile_req).profile
 
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      default_query_options=[('debug_action', 'custom')])
+  @needs_session(TCLIService.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6)
+  def test_unset_all(self):
+    """
+    Starts Impala cluster with a custom query option, and checks unset option
+    works correctly.
+
+    The Beeswax API and the HiveServer2 implementations are slightly different,
+    so the same test is run in both contexts.
+    """
+    # Beeswax API:
+    result = self.execute_query_expect_success(self.client, "set all")
+    assert "DEBUG_ACTION\tcustom\tDEVELOPMENT" in result.data, "baseline"
+    self.execute_query_expect_success(self.client, "set debug_action=hey")
+    assert "DEBUG_ACTION\they\tDEVELOPMENT" in \
+        self.execute_query_expect_success(self.client, "set all").data, "session override"
+    self.execute_query_expect_success(self.client, 'unset all')
+    assert "DEBUG_ACTION\tcustom\tDEVELOPMENT" in \
+        self.execute_query_expect_success(self.client, "set all").data, "unset all"
+
+    # HS2:
+    assert ("DEBUG_ACTION", "custom") in self.get_set_results(), "baseline"
+    self.execute_statement("set debug_action='hey'")
+    assert ("DEBUG_ACTION", "hey") in self.get_set_results(), "session override"
+    self.execute_statement("unset all")
+    assert ("DEBUG_ACTION", "custom") in self.get_set_results(), "unset all"
+
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
       impalad_args="--idle_session_timeout=321")
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index 6b15bb3a8..9adcfd2f2 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -145,6 +145,22 @@ class TestHS2(HS2TestSuite):
     # Removed options are returned by "SET ALL" for the benefit of impala-shell.
     assert levels["MAX_IO_BUFFERS"] == "REMOVED"
 
+  @needs_session()
+  def test_session_option_levels_via_unset_all(self):
+    self.execute_statement("SET COMPRESSION_CODEC=gzip")
+    self.execute_statement("SET SYNC_DDL=1")
+    vals, levels = self.get_session_options("SET")
+    assert vals["COMPRESSION_CODEC"] == "GZIP"
+    assert vals["SYNC_DDL"] == "1"
+
+    # Unset all query options
+    self.execute_statement("UNSET ALL")
+    vals2, levels = self.get_session_options("SET")
+
+    # Reset to default value
+    assert vals2["COMPRESSION_CODEC"] == ""
+    assert vals2["SYNC_DDL"] == "0"
+
   @SkipIfDockerizedCluster.internal_hostname
   def test_open_session_http_addr(self):
     """Check that OpenSession returns the coordinator's http address."""
diff --git a/tests/shell/test_shell_interactive.py b/tests/shell/test_shell_interactive.py
index 14e185420..f3dca01d0 100755
--- a/tests/shell/test_shell_interactive.py
+++ b/tests/shell/test_shell_interactive.py
@@ -700,6 +700,16 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
     # Verify that query options under [impala] override those under [impala.query_options]
     assert "\tDEFAULT_FILE_FORMAT: avro" in result.stdout
 
+    # unset all query options
+    cmds = "unset all;set all;"
+    result = run_impala_shell_interactive(vector, cmds, shell_args=args)
+    assert "\tMT_DOP: " in result.stdout
+    assert "\tMAX_ERRORS: [100]" in result.stdout
+    assert "\tEXPLAIN_LEVEL: [STANDARD]" in result.stdout
+    assert "INVALID_QUERY_OPTION is not supported for the impalad being connected to, "\
+           "ignoring." in result.stdout
+    assert "\tDEFAULT_FILE_FORMAT: [TEXT]" in result.stdout
+
   def test_commandline_flag_disable_live_progress(self, vector):
     """Test the command line flag disable_live_progress with live_progress."""
     if vector.get_value('strict_hs2_protocol'):