You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2017/09/16 21:41:04 UTC

[5/5] incubator-impala git commit: Revert "IMPALA-5589: change "set" in impala-shell to show empty string for unset query options"

Revert "IMPALA-5589: change "set" in impala-shell to show empty string for unset query options"

Due to re-use of connections in the test infrastructure, this commit
is causing ASAN failures in the builds. That is being worked out
as part of IMPALA-5908, but, in the meanwhile, it's prudent
to revert.

This reverts commit 387bde0639ffd8ef580ccbf727152954e62bacbe.

Change-Id: I41bc8ab0f1df45bbd311030981a7c18989c2edc8
Reviewed-on: http://gerrit.cloudera.org:8080/8087
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/f0e79314
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/f0e79314
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/f0e79314

Branch: refs/heads/master
Commit: f0e79314fe9d9d3e920ad65c3ca6a4ef279e68fc
Parents: 5119ced
Author: Philip Zeyliger <ph...@cloudera.com>
Authored: Fri Sep 15 16:55:02 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sat Sep 16 04:06:53 2017 +0000

----------------------------------------------------------------------
 be/src/service/query-options-test.cc            |  12 ---
 be/src/service/query-options.cc                 |  10 +-
 be/src/service/query-options.h                  |   4 +-
 .../functional-query/queries/QueryTest/set.test |  24 ++---
 tests/hs2/hs2_test_suite.py                     |  10 +-
 tests/hs2/test_hs2.py                           | 104 ++++++++-----------
 tests/shell/test_shell_commandline.py           |   2 -
 7 files changed, 60 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/be/src/service/query-options-test.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 1bec770..397df5a 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -135,16 +135,4 @@ TEST(QueryOptions, ParseQueryOptions) {
   }
 }
 
-TEST(QueryOptions, MapOptionalDefaultlessToEmptyString) {
-  TQueryOptions options;
-  map<string, string> map;
-
-  TQueryOptionsToMap(options, &map);
-  // No default set
-  EXPECT_EQ(map["COMPRESSION_CODEC"], "");
-  EXPECT_EQ(map["MT_DOP"], "");
-  // Has defaults
-  EXPECT_EQ(map["EXPLAIN_LEVEL"], "1");
-}
-
 IMPALA_TEST_MAIN();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/be/src/service/query-options.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index e950d31..21327f3 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -67,13 +67,9 @@ void impala::TQueryOptionsToMap(const TQueryOptions& query_options,
     map<string, string>* configuration) {
 #define QUERY_OPT_FN(NAME, ENUM)\
   {\
-    if (query_options.__isset.NAME) { \
-      stringstream val;\
-      val << query_options.NAME;\
-      (*configuration)[#ENUM] = val.str();\
-    } else { \
-      (*configuration)[#ENUM] = ""; \
-    }\
+    stringstream val;\
+    val << query_options.NAME;\
+    (*configuration)[#ENUM] = val.str();\
   }
   QUERY_OPTS_TABLE
 #undef QUERY_OPT_FN

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/be/src/service/query-options.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 3dada7d..bb8c301 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -99,9 +99,7 @@ class TQueryOptions;
   ;
 
 
-/// Converts a TQueryOptions struct into a map of key, value pairs.  Options that
-/// aren't set and lack defaults in common/thrift/ImpalaInternalService.thrift are
-/// mapped to the empty string.
+/// Converts a TQueryOptions struct into a map of key, value pairs.
 void TQueryOptionsToMap(const TQueryOptions& query_options,
     std::map<std::string, std::string>* configuration);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/testdata/workloads/functional-query/queries/QueryTest/set.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/set.test b/testdata/workloads/functional-query/queries/QueryTest/set.test
index 45c8343..a7004e0 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -20,13 +20,13 @@ set
 'MEM_LIMIT','0'
 'NUM_NODES','0'
 'NUM_SCANNER_THREADS','0'
-'COMPRESSION_CODEC',''
+'COMPRESSION_CODEC','NONE'
 'PARQUET_FILE_SIZE','0'
 'REQUEST_POOL',''
-'RESERVATION_REQUEST_TIMEOUT',''
+'RESERVATION_REQUEST_TIMEOUT','0'
 'RM_INITIAL_MEM','0'
 'SYNC_DDL','0'
-'V_CPU_CORES',''
+'V_CPU_CORES','0'
 ---- TYPES
 STRING, STRING
 ====
@@ -52,13 +52,13 @@ set;
 'MEM_LIMIT','0'
 'NUM_NODES','0'
 'NUM_SCANNER_THREADS','0'
-'COMPRESSION_CODEC',''
+'COMPRESSION_CODEC','NONE'
 'PARQUET_FILE_SIZE','0'
 'REQUEST_POOL',''
-'RESERVATION_REQUEST_TIMEOUT',''
+'RESERVATION_REQUEST_TIMEOUT','0'
 'RM_INITIAL_MEM','0'
 'SYNC_DDL','0'
-'V_CPU_CORES',''
+'V_CPU_CORES','0'
 ---- TYPES
 STRING, STRING
 ====
@@ -84,13 +84,13 @@ set;
 'MEM_LIMIT','0'
 'NUM_NODES','0'
 'NUM_SCANNER_THREADS','0'
-'COMPRESSION_CODEC',''
+'COMPRESSION_CODEC','NONE'
 'PARQUET_FILE_SIZE','0'
 'REQUEST_POOL',''
-'RESERVATION_REQUEST_TIMEOUT',''
+'RESERVATION_REQUEST_TIMEOUT','0'
 'RM_INITIAL_MEM','0'
 'SYNC_DDL','0'
-'V_CPU_CORES',''
+'V_CPU_CORES','0'
 ---- TYPES
 STRING, STRING
 ====
@@ -117,13 +117,13 @@ set;
 'MEM_LIMIT','0'
 'NUM_NODES','0'
 'NUM_SCANNER_THREADS','0'
-'COMPRESSION_CODEC',''
+'COMPRESSION_CODEC','NONE'
 'PARQUET_FILE_SIZE','1610612736'
 'REQUEST_POOL',''
-'RESERVATION_REQUEST_TIMEOUT',''
+'RESERVATION_REQUEST_TIMEOUT','0'
 'RM_INITIAL_MEM','0'
 'SYNC_DDL','0'
-'V_CPU_CORES',''
+'V_CPU_CORES','0'
 ---- TYPES
 STRING, STRING
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/tests/hs2/hs2_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/hs2/hs2_test_suite.py b/tests/hs2/hs2_test_suite.py
index 1b2f89f..2c2cd51 100644
--- a/tests/hs2/hs2_test_suite.py
+++ b/tests/hs2/hs2_test_suite.py
@@ -124,6 +124,7 @@ class HS2TestSuite(ImpalaTestSuite):
     fetch_results_req.maxRows = size
     fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
     HS2TestSuite.check_response(fetch_results_resp)
+    num_rows = size
     if expected_num_rows is not None:
       assert self.get_num_rows(fetch_results_resp.results) == expected_num_rows
     return fetch_results_resp
@@ -227,12 +228,3 @@ class HS2TestSuite(ImpalaTestSuite):
       sleep(interval)
     assert False, 'Did not reach expected operation state %s in time, actual state was ' \
         '%s' % (expected_state, get_operation_status_resp.operationState)
-
-  def execute_statement(self, statement):
-    """Executes statement and returns response, which is checked."""
-    execute_statement_req = TCLIService.TExecuteStatementReq()
-    execute_statement_req.sessionHandle = self.session_handle
-    execute_statement_req.statement = statement
-    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
-    HS2TestSuite.check_response(execute_statement_resp)
-    return execute_statement_resp

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/tests/hs2/test_hs2.py
----------------------------------------------------------------------
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index 2f984b1..f42b29a 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -45,48 +45,6 @@ class TestHS2(HS2TestSuite):
     for k, v in open_session_req.configuration.items():
       assert open_session_resp.configuration[k] == v
 
-  @needs_session()
-  def test_session_options_via_set(self):
-    """
-    Tests that session options are returned by a SET
-    query and can be updated by a "SET k=v" query.
-    """
-    def get_session_options():
-      """Returns dictionary of query options."""
-      execute_statement_resp = self.execute_statement("SET")
-
-      fetch_results_req = TCLIService.TFetchResultsReq()
-      fetch_results_req.operationHandle = execute_statement_resp.operationHandle
-      fetch_results_req.maxRows = 1000
-      fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
-      TestHS2.check_response(fetch_results_resp)
-
-      # Close the query
-      close_operation_req = TCLIService.TCloseOperationReq()
-      close_operation_req.operationHandle = execute_statement_resp.operationHandle
-      TestHS2.check_response(self.hs2_client.CloseOperation(close_operation_req))
-
-      # Results are returned in a columnar way:
-      cols = fetch_results_resp.results.columns
-      assert len(cols) == 2
-      vals = dict(zip(cols[0].stringVal.values, cols[1].stringVal.values))
-      return vals
-
-    vals = get_session_options()
-
-    # No default; should be empty string.
-    assert vals["COMPRESSION_CODEC"] == ""
-    # Has default of 0
-    assert vals["SYNC_DDL"] == "0"
-
-    # Set an option using SET
-    self.execute_statement("SET COMPRESSION_CODEC=gzip")
-
-    vals2 = get_session_options()
-    assert vals2["COMPRESSION_CODEC"] == "GZIP"
-    # Should be unchanged
-    assert vals2["SYNC_DDL"] == "0"
-
   def test_open_session_http_addr(self):
     """Check that OpenSession returns the coordinator's http address."""
     open_session_req = TCLIService.TOpenSessionReq()
@@ -214,8 +172,11 @@ class TestHS2(HS2TestSuite):
   @needs_session()
   def test_get_operation_status(self):
     """Tests that GetOperationStatus returns a valid result for a running query"""
-    statement = "SELECT COUNT(*) FROM functional.alltypes"
-    execute_statement_resp = self.execute_statement(statement)
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT COUNT(*) FROM functional.alltypes"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     get_operation_status_resp = \
         self.get_operation_status(execute_statement_resp.operationHandle)
@@ -252,8 +213,11 @@ class TestHS2(HS2TestSuite):
   def test_get_operation_status_error(self):
     """Tests that GetOperationStatus returns a valid result for a query that encountered
     an error"""
-    statement = "SELECT * FROM functional.alltypeserror"
-    execute_statement_resp = self.execute_statement(statement)
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT * FROM functional.alltypeserror"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     get_operation_status_resp = self.wait_for_operation_state( \
         execute_statement_resp.operationHandle, TCLIService.TOperationState.ERROR_STATE)
@@ -368,15 +332,17 @@ class TestHS2(HS2TestSuite):
     assert "Sql Statement: GET_SCHEMAS" in profile_page
     assert "Query Type: DDL" in profile_page
 
-
   @needs_session(conf_overlay={"idle_session_timeout": "5"})
   def test_get_operation_status_session_timeout(self):
     """Regression test for IMPALA-4488: GetOperationStatus() would not keep a session
     alive"""
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
     # Choose a long-running query so that it can't finish before the session timeout.
-    statement = """select * from functional.alltypes a
+    execute_statement_req.statement = """select * from functional.alltypes a
     join functional.alltypes b join functional.alltypes c"""
-    execute_statement_resp = self.execute_statement(statement)
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     now = time.time()
     # Loop until the session would be timed-out if IMPALA-4488 had not been fixed.
@@ -388,7 +354,11 @@ class TestHS2(HS2TestSuite):
       time.sleep(0.1)
 
   def get_log(self, query_stmt):
-    execute_statement_resp = self.execute_statement(query_stmt)
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = query_stmt
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     # Fetch results to make sure errors are generated. Errors are only guaranteed to be
     # seen by the coordinator after FetchResults() returns eos.
@@ -419,8 +389,11 @@ class TestHS2(HS2TestSuite):
 
   @needs_session()
   def test_get_exec_summary(self):
-    statement = "SELECT COUNT(1) FROM functional.alltypes"
-    execute_statement_resp = self.execute_statement(statement)
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT COUNT(1) FROM functional.alltypes"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     exec_summary_req = ImpalaHiveServer2Service.TGetExecSummaryReq()
     exec_summary_req.operationHandle = execute_statement_resp.operationHandle
@@ -442,15 +415,18 @@ class TestHS2(HS2TestSuite):
 
   @needs_session()
   def test_get_profile(self):
-    statement = "SELECT COUNT(2) FROM functional.alltypes"
-    execute_statement_resp = self.execute_statement(statement)
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT COUNT(2) FROM functional.alltypes"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
+    TestHS2.check_response(execute_statement_resp)
 
     get_profile_req = ImpalaHiveServer2Service.TGetRuntimeProfileReq()
     get_profile_req.operationHandle = execute_statement_resp.operationHandle
     get_profile_req.sessionHandle = self.session_handle
     get_profile_resp = self.hs2_client.GetRuntimeProfile(get_profile_req)
     TestHS2.check_response(get_profile_resp)
-    assert statement in get_profile_resp.profile
+    assert execute_statement_req.statement in get_profile_resp.profile
     # If ExecuteStatement() has completed but the results haven't been fetched yet, the
     # query must have at least reached RUNNING.
     assert "Query State: RUNNING" in get_profile_resp.profile or \
@@ -463,7 +439,7 @@ class TestHS2(HS2TestSuite):
 
     get_profile_resp = self.hs2_client.GetRuntimeProfile(get_profile_req)
     TestHS2.check_response(get_profile_resp)
-    assert statement in get_profile_resp.profile
+    assert execute_statement_req.statement in get_profile_resp.profile
     # After fetching the results, we must be in state FINISHED.
     assert "Query State: FINISHED" in get_profile_resp.profile, get_profile_resp.profile
 
@@ -473,20 +449,26 @@ class TestHS2(HS2TestSuite):
 
     get_profile_resp = self.hs2_client.GetRuntimeProfile(get_profile_req)
     TestHS2.check_response(get_profile_resp)
-    assert statement in get_profile_resp.profile
+    assert execute_statement_req.statement in get_profile_resp.profile
     assert "Query State: FINISHED" in get_profile_resp.profile, get_profile_resp.profile
 
   @needs_session(conf_overlay={"use:database": "functional"})
   def test_change_default_database(self):
-    statement = "SELECT 1 FROM alltypes LIMIT 1"
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT 1 FROM alltypes LIMIT 1"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
     # Will fail if there's no table called 'alltypes' in the database
-    self.execute_statement(statement)
+    TestHS2.check_response(execute_statement_resp)
 
   @needs_session(conf_overlay={"use:database": "FUNCTIONAL"})
   def test_change_default_database_case_insensitive(self):
-    statement = "SELECT 1 FROM alltypes LIMIT 1"
+    execute_statement_req = TCLIService.TExecuteStatementReq()
+    execute_statement_req.sessionHandle = self.session_handle
+    execute_statement_req.statement = "SELECT 1 FROM alltypes LIMIT 1"
+    execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
     # Will fail if there's no table called 'alltypes' in the database
-    self.execute_statement(statement)
+    TestHS2.check_response(execute_statement_resp)
 
   @needs_session(conf_overlay={"use:database": "doesnt-exist"})
   def test_bad_default_database(self):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e79314/tests/shell/test_shell_commandline.py
----------------------------------------------------------------------
diff --git a/tests/shell/test_shell_commandline.py b/tests/shell/test_shell_commandline.py
index 0602e77..488de49 100644
--- a/tests/shell/test_shell_commandline.py
+++ b/tests/shell/test_shell_commandline.py
@@ -248,8 +248,6 @@ class TestImpalaShell(ImpalaTestSuite):
     assert 'MEM_LIMIT: [0]' in result_set.stdout
     # test to check that explain_level is 1
     assert 'EXPLAIN_LEVEL: [1]' in result_set.stdout
-    # test to check that configs without defaults show up as []
-    assert 'COMPRESSION_CODEC: []' in result_set.stdout
     # test values displayed after setting value
     args = '-q "set mem_limit=1g;set"'
     result_set = run_impala_shell_cmd(args)