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 2018/10/30 21:11:09 UTC

[1/2] impala git commit: IMPALA-7783: Skip test_default_timezone when testing a real cluster.

Repository: impala
Updated Branches:
  refs/heads/master 932bf2dd8 -> 35c0e4416


IMPALA-7783: Skip test_default_timezone when testing a real cluster.

test_shell_commandline.py::test_default_timezone assumes that the
cluster is running on the same platform as the test process, but
that's only guaranteed when the testing a local minicluster. When
run against a real cluster, the test executor can be a completely
different OS.

Change-Id: Ia4d4c503d2c77136cedd8f3fd830b6ce70d4457f
Reviewed-on: http://gerrit.cloudera.org:8080/11820
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4c923b29d884bef3592fdf2e3e1cbf04aca58dc2
Parents: 932bf2d
Author: David Knupp <dk...@cloudera.com>
Authored: Mon Oct 29 14:59:35 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Tue Oct 30 19:47:08 2018 +0000

----------------------------------------------------------------------
 tests/shell/test_shell_commandline.py | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4c923b29/tests/shell/test_shell_commandline.py
----------------------------------------------------------------------
diff --git a/tests/shell/test_shell_commandline.py b/tests/shell/test_shell_commandline.py
index fd18230..b69bece 100644
--- a/tests/shell/test_shell_commandline.py
+++ b/tests/shell/test_shell_commandline.py
@@ -730,11 +730,17 @@ class TestImpalaShell(ImpalaTestSuite):
     finally:
       os.remove(sql_path)
 
+  @pytest.mark.skipif(pytest.config.option.testing_remote_cluster,
+                      reason='Test assumes a minicluster.')
   def test_default_timezone(self):
     """Test that the default TIMEZONE query option is a valid timezone.
 
        It would be nice to check that the default timezone is the system's timezone,
        but doing this reliably on different Linux distributions is quite hard.
+
+       We skip this test on non-local clusters because the result_set from the
+       cluster is platform specific, but there's no guarantee the local machine
+       is the same OS, and the assert fails if there's a mismatch.
     """
     result_set = run_impala_shell_cmd('-q "set;"')
     tzname = find_query_option("TIMEZONE", result_set.stdout)


[2/2] impala git commit: IMPALA-7727: Fix TStatusCode to TErrorCode mapping

Posted by ta...@apache.org.
IMPALA-7727: Fix TStatusCode to TErrorCode mapping

- Uses a "GENERAL" TErrorCode type for all non-OK statuses.
- Detailed regression root cause description in the jira IMPALA-7727.
- Added a regression test.

Change-Id: Ie62527734aa73c1524c731773638590bdac9e789
Reviewed-on: http://gerrit.cloudera.org:8080/11778
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 35c0e4416d23b33d1c3a3e9594230a7ed0e11d59
Parents: 4c923b2
Author: Bharath Vissapragada <bh...@cloudera.com>
Authored: Wed Oct 24 17:24:02 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Tue Oct 30 19:55:12 2018 +0000

----------------------------------------------------------------------
 be/src/common/status.cc                          | 19 ++-----------------
 be/src/common/status.h                           |  3 ---
 be/src/service/child-query.cc                    | 12 +++++-------
 be/src/util/error-util.cc                        | 18 ++++++++++++++++++
 be/src/util/error-util.h                         |  4 ++++
 .../queries/QueryTest/compute-stats.test         |  8 ++++++++
 6 files changed, 37 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/be/src/common/status.cc
----------------------------------------------------------------------
diff --git a/be/src/common/status.cc b/be/src/common/status.cc
index 7adcce9..4ef4f9b 100644
--- a/be/src/common/status.cc
+++ b/be/src/common/status.cc
@@ -16,12 +16,11 @@
 // under the License.
 
 #include <boost/algorithm/string/join.hpp>
-
 #include <ostream>
 
 #include "common/status.h"
-#include "util/debug-util.h"
 
+#include "util/debug-util.h"
 #include "common/names.h"
 #include "gen-cpp/common.pb.h"
 #include "gen-cpp/ErrorCodes_types.h"
@@ -164,24 +163,10 @@ Status::Status(const apache::hive::service::cli::thrift::TStatus& hs2_status)
   : msg_(
       hs2_status.statusCode
         == apache::hive::service::cli::thrift::TStatusCode::SUCCESS_STATUS ? NULL
-          : new ErrorMsg(
-              static_cast<TErrorCode::type>(hs2_status.statusCode),
+          : new ErrorMsg(HS2TStatusCodeToTErrorCode(hs2_status.statusCode),
               hs2_status.errorMessage)) {
 }
 
-Status& Status::operator=(
-    const apache::hive::service::cli::thrift::TStatus& hs2_status) {
-  delete msg_;
-  if (hs2_status.statusCode
-        == apache::hive::service::cli::thrift::TStatusCode::SUCCESS_STATUS) {
-    msg_ = NULL;
-  } else {
-    msg_ = new ErrorMsg(
-        static_cast<TErrorCode::type>(hs2_status.statusCode), hs2_status.errorMessage);
-  }
-  return *this;
-}
-
 Status Status::Expected(const ErrorMsg& error_msg) {
   return Status(error_msg, true);
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/be/src/common/status.h
----------------------------------------------------------------------
diff --git a/be/src/common/status.h b/be/src/common/status.h
index ff1f6c5..d60efa7 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -199,9 +199,6 @@ class NODISCARD Status {
   /// Retains the TErrorCode value and the message
   Status& operator=(const TStatus& status);
 
-  /// Retains the TErrorCode value and the message
-  Status& operator=(const apache::hive::service::cli::thrift::TStatus& hs2_status);
-
   bool ALWAYS_INLINE ok() const { return msg_ == NULL; }
 
   /// Return true if this is a user-initiated or internal cancellation.

http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/be/src/service/child-query.cc
----------------------------------------------------------------------
diff --git a/be/src/service/child-query.cc b/be/src/service/child-query.cc
index 2c5b316..58a17de 100644
--- a/be/src/service/child-query.cc
+++ b/be/src/service/child-query.cc
@@ -39,7 +39,6 @@ Status ChildQuery::ExecAndFetch() {
              << PrintId(session_id);
 
   // Create HS2 request and response structs.
-  Status status;
   TExecuteStatementResp exec_stmt_resp;
   TExecuteStatementReq exec_stmt_req;
   ImpalaServer::TUniqueIdToTHandleIdentifier(session_id, session_id,
@@ -66,24 +65,23 @@ Status ChildQuery::ExecAndFetch() {
     lock_guard<mutex> l(lock_);
     is_running_ = true;
   }
-  status = exec_stmt_resp.status;
-  RETURN_IF_ERROR(status);
+  RETURN_IF_ERROR(Status(exec_stmt_resp.status));
 
   TGetResultSetMetadataReq meta_req;
   meta_req.operationHandle = exec_stmt_resp.operationHandle;
   RETURN_IF_ERROR(IsCancelled());
   parent_server_->GetResultSetMetadata(meta_resp_, meta_req);
-  status = meta_resp_.status;
-  RETURN_IF_ERROR(status);
+  RETURN_IF_ERROR(Status(meta_resp_.status));
 
   // Fetch all results.
   TFetchResultsReq fetch_req;
   fetch_req.operationHandle = exec_stmt_resp.operationHandle;
   fetch_req.maxRows = 1024;
+  Status status;
   do {
     RETURN_IF_ERROR(IsCancelled());
     parent_server_->FetchResults(fetch_resp_, fetch_req);
-    status = fetch_resp_.status;
+    status = Status(fetch_resp_.status);
   } while (status.ok() && fetch_resp_.hasMoreRows);
   RETURN_IF_ERROR(IsCancelled());
 
@@ -99,7 +97,7 @@ Status ChildQuery::ExecAndFetch() {
 
   // Don't overwrite error from fetch. A failed fetch unregisters the query and we want to
   // preserve the original error status (e.g., CANCELLED).
-  if (status.ok()) status = close_resp.status;
+  if (status.ok()) status = Status(close_resp.status);
   return status;
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/be/src/util/error-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/error-util.cc b/be/src/util/error-util.cc
index 94d2ccd..445833f 100644
--- a/be/src/util/error-util.cc
+++ b/be/src/util/error-util.cc
@@ -24,6 +24,8 @@
 #include "common/logging.h"
 #include "common/names.h"
 
+using apache::hive::service::cli::thrift::TStatusCode;
+
 namespace impala {
 
 string GetStrErrMsg() {
@@ -216,4 +218,20 @@ string ErrorMsg::GetFullMessageDetails() const {
   return ss.str();
 }
 
+TErrorCode::type HS2TStatusCodeToTErrorCode(const TStatusCode::type& hs2Code) {
+  // There is no one-one mapping between HS2 error codes and TStatusCode types.
+  // So we return a "GENERAL" error type for ERROR_STATUS code. This lets the callers
+  // pick their own error message for substitution.
+  switch (hs2Code) {
+    case TStatusCode::SUCCESS_STATUS:
+      return TErrorCode::OK;
+    case TStatusCode::ERROR_STATUS:
+      return TErrorCode::GENERAL;
+    default:
+      DCHECK(false) << "Unexpected hs2Code: " << hs2Code;
+  }
+  LOG(ERROR) << "Unexpected hs2Code encountered: " << hs2Code;
+  return TErrorCode::UNUSED;
+}
+
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/be/src/util/error-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/error-util.h b/be/src/util/error-util.h
index 3975548..a93e6b7 100644
--- a/be/src/util/error-util.h
+++ b/be/src/util/error-util.h
@@ -27,6 +27,7 @@
 #include "gen-cpp/ErrorCodes_constants.h"
 #include "gen-cpp/ErrorCodes_types.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
+#include "gen-cpp/TCLIService_types.h"
 #include "gutil/strings/substitute.h"
 
 namespace impala {
@@ -170,6 +171,9 @@ size_t ErrorCount(const ErrorLogMap& errors);
 /// PrintErrorMap, but returns a string instead of using a stream.
 std::string PrintErrorMapToString(const ErrorLogMap& errors);
 
+/// Maps the HS2 TStatusCode types to the corresponding TErrorCode.
+TErrorCode::type HS2TStatusCodeToTErrorCode(
+    const apache::hive::service::cli::thrift::TStatusCode::type& hs2Code);
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/35c0e441/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
index ce147e4..8db8377 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
@@ -2197,3 +2197,11 @@ show column stats widetable_1000_cols
 ---- TYPES
 STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ====
+---- QUERY
+# Tests that the child query error status is propagated back to the parent
+# compute stats query. Forces an error by setting low mem limit.
+set mem_limit=1m;
+compute stats tpch_parquet.customer;
+---- CATCH
+Rejected query from pool default-pool: minimum memory reservation is greater than memory available to the query for buffer reservations
+====