You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by sa...@apache.org on 2018/04/23 17:39:03 UTC

[15/20] impala git commit: IMPALA-5690: Part 1: Rename ostream operators for thrift types

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/query-options.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 91378ab..a0b7227 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -68,14 +68,29 @@ void impala::OverlayQueryOptions(const TQueryOptions& src, const QueryOptionsMas
 #undef QUERY_OPT_FN
 }
 
+// Choose different print function based on the type.
+// TODO: In thrift 0.11.0 operator << is implemented for enums and this indirection can be
+// removed.
+template<typename T, typename std::enable_if_t<std::is_enum<T>::value>* = nullptr>
+string PrintQueryOptionValue(const T& option) {
+  return PrintThriftEnum(option);
+}
+
+template<typename T, typename std::enable_if_t<std::is_integral<T>::value>* = nullptr>
+string PrintQueryOptionValue(const T& option)  {
+  return std::to_string(option);
+}
+
+const string& PrintQueryOptionValue(const std::string& option)  {
+  return option;
+}
+
 void impala::TQueryOptionsToMap(const TQueryOptions& query_options,
     map<string, string>* configuration) {
 #define QUERY_OPT_FN(NAME, ENUM, LEVEL)\
   {\
     if (query_options.__isset.NAME) { \
-      stringstream val;\
-      val << query_options.NAME;\
-      (*configuration)[#ENUM] = val.str();\
+      (*configuration)[#ENUM] = PrintQueryOptionValue(query_options.NAME); \
     } else { \
       (*configuration)[#ENUM] = ""; \
     }\
@@ -384,7 +399,7 @@ Status impala::SetQueryOption(const string& key, const string& value,
         if (size < RuntimeFilterBank::MIN_BLOOM_FILTER_SIZE ||
             size > RuntimeFilterBank::MAX_BLOOM_FILTER_SIZE) {
           return Status(Substitute("$0 is not a valid Bloom filter size for $1. "
-                  "Valid sizes are in [$2, $3].", value, PrintTImpalaQueryOptions(
+                  "Valid sizes are in [$2, $3].", value, PrintThriftEnum(
                       static_cast<TImpalaQueryOptions::type>(option)),
                   RuntimeFilterBank::MIN_BLOOM_FILTER_SIZE,
                   RuntimeFilterBank::MAX_BLOOM_FILTER_SIZE));
@@ -396,7 +411,7 @@ Status impala::SetQueryOption(const string& key, const string& value,
             && FLAGS_min_buffer_size <= RuntimeFilterBank::MAX_BLOOM_FILTER_SIZE) {
           return Status(Substitute("$0 should not be less than $1 which is the minimum "
               "buffer size that can be allocated by the buffer pool",
-              PrintTImpalaQueryOptions(static_cast<TImpalaQueryOptions::type>(option)),
+              PrintThriftEnum(static_cast<TImpalaQueryOptions::type>(option)),
               FLAGS_min_buffer_size));
         }
         if (option == TImpalaQueryOptions::RUNTIME_BLOOM_FILTER_SIZE) {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/query-result-set.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-result-set.cc b/be/src/service/query-result-set.cc
index 8d00af5..aacd849 100644
--- a/be/src/service/query-result-set.cc
+++ b/be/src/service/query-result-set.cc
@@ -182,8 +182,8 @@ Status AsciiQueryResultSet::AddOneRow(const TResultRow& row) {
   out_stream.precision(ASCII_PRECISION);
   for (int i = 0; i < num_col; ++i) {
     // ODBC-187 - ODBC can only take "\t" as the delimiter
-    out_stream << (i > 0 ? "\t" : "");
-    out_stream << row.colVals[i];
+    if (i > 0) out_stream << '\t';
+    PrintTColumnValue(out_stream, row.colVals[i]);
   }
   result_set_->push_back(out_stream.str());
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/statestore/statestore.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index 02363fe..5c1952d 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -460,7 +460,7 @@ void Statestore::SubscribersHandler(const Webserver::ArgumentMap& args,
     Value subscriber_id(subscriber.second->id().c_str(), document->GetAllocator());
     sub_json.AddMember("id", subscriber_id, document->GetAllocator());
 
-    Value address(lexical_cast<string>(subscriber.second->network_address()).c_str(),
+    Value address(TNetworkAddressToString(subscriber.second->network_address()).c_str(),
         document->GetAllocator());
     sub_json.AddMember("address", address, document->GetAllocator());
 
@@ -875,7 +875,7 @@ void Statestore::DoSubscriberUpdate(UpdateKind update_kind, int thread_id,
         // TODO: Consider if a metric to track the number of failures would be useful.
         LOG(INFO) << "Subscriber '" << subscriber->id() << "' has failed, disconnected "
                   << "or re-registered (last known registration ID: "
-                  << update.registration_id << ")";
+                  << PrintId(update.registration_id) << ")";
         UnregisterSubscriber(subscriber.get());
       } else {
         LOG(INFO) << "Failure was already detected for subscriber '" << subscriber->id()

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/collection-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/collection-metrics.h b/be/src/util/collection-metrics.h
index 1081c33..79ae072 100644
--- a/be/src/util/collection-metrics.h
+++ b/be/src/util/collection-metrics.h
@@ -160,7 +160,7 @@ class StatsMetric : public Metric {
     boost::lock_guard<boost::mutex> l(lock_);
     rapidjson::Value container(rapidjson::kObjectType);
     AddStandardFields(document, &container);
-    rapidjson::Value units(PrintTUnit(unit_).c_str(), document->GetAllocator());
+    rapidjson::Value units(PrintThriftEnum(unit_).c_str(), document->GetAllocator());
     container.AddMember("units", units, document->GetAllocator());
 
     if (StatsSelection & StatsType::COUNT) {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/debug-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index 1cb61e4..edf1749 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -50,62 +50,33 @@ DECLARE_string(hostname);
 
 namespace impala {
 
-#define THRIFT_ENUM_OUTPUT_FN_IMPL(E, MAP) \
-  ostream& operator<<(ostream& os, const E::type& e) {\
-    map<int, const char*>::const_iterator i;\
-    i = MAP.find(e);\
-    if (i != MAP.end()) {\
-      os << i->second;\
-    }\
-    return os;\
+#define PRINT_THRIFT_ENUM_IMPL(T) \
+  string PrintThriftEnum(const T::type& value) { \
+    map<int, const char*>::const_iterator it = _##T##_VALUES_TO_NAMES.find(value); \
+    return it == _##T##_VALUES_TO_NAMES.end() ? std::to_string(value) : it->second; \
   }
 
-// Macro to stamp out operator<< for thrift enums.  Why doesn't thrift do this?
-#define THRIFT_ENUM_OUTPUT_FN(E) THRIFT_ENUM_OUTPUT_FN_IMPL(E , _##E##_VALUES_TO_NAMES)
-
-// Macro to implement Print function that returns string for thrift enums. Make sure you
-// define a corresponding THRIFT_ENUM_OUTPUT_FN.
-#define THRIFT_ENUM_PRINT_FN(E) \
-  string Print##E(const E::type& e) {\
-    stringstream ss;\
-    ss << e;\
-    return ss.str();\
-  }
-
-THRIFT_ENUM_OUTPUT_FN(TFunctionBinaryType);
-THRIFT_ENUM_OUTPUT_FN(TCatalogObjectType);
-THRIFT_ENUM_OUTPUT_FN(TDdlType);
-THRIFT_ENUM_OUTPUT_FN(TCatalogOpType);
-THRIFT_ENUM_OUTPUT_FN(THdfsFileFormat);
-THRIFT_ENUM_OUTPUT_FN(THdfsCompression);
-THRIFT_ENUM_OUTPUT_FN(TReplicaPreference);
-THRIFT_ENUM_OUTPUT_FN(TSessionType);
-THRIFT_ENUM_OUTPUT_FN(TStmtType);
-THRIFT_ENUM_OUTPUT_FN(QueryState);
-THRIFT_ENUM_OUTPUT_FN(Encoding);
-THRIFT_ENUM_OUTPUT_FN(CompressionCodec);
-THRIFT_ENUM_OUTPUT_FN(Type);
-THRIFT_ENUM_OUTPUT_FN(TMetricKind);
-THRIFT_ENUM_OUTPUT_FN(TUnit);
-THRIFT_ENUM_OUTPUT_FN(TImpalaQueryOptions);
-
-THRIFT_ENUM_PRINT_FN(TCatalogObjectType);
-THRIFT_ENUM_PRINT_FN(TDdlType);
-THRIFT_ENUM_PRINT_FN(TCatalogOpType);
-THRIFT_ENUM_PRINT_FN(TReplicaPreference);
-THRIFT_ENUM_PRINT_FN(TSessionType);
-THRIFT_ENUM_PRINT_FN(TStmtType);
-THRIFT_ENUM_PRINT_FN(QueryState);
-THRIFT_ENUM_PRINT_FN(Encoding);
-THRIFT_ENUM_PRINT_FN(TMetricKind);
-THRIFT_ENUM_PRINT_FN(TUnit);
-THRIFT_ENUM_PRINT_FN(TImpalaQueryOptions);
-
-
-ostream& operator<<(ostream& os, const TUniqueId& id) {
-  os << PrintId(id);
-  return os;
-}
+PRINT_THRIFT_ENUM_IMPL(QueryState)
+PRINT_THRIFT_ENUM_IMPL(Encoding)
+PRINT_THRIFT_ENUM_IMPL(TCatalogObjectType)
+PRINT_THRIFT_ENUM_IMPL(TCatalogOpType)
+PRINT_THRIFT_ENUM_IMPL(TDdlType)
+PRINT_THRIFT_ENUM_IMPL(TExplainLevel)
+PRINT_THRIFT_ENUM_IMPL(THdfsCompression)
+PRINT_THRIFT_ENUM_IMPL(THdfsFileFormat)
+PRINT_THRIFT_ENUM_IMPL(THdfsSeqCompressionMode)
+PRINT_THRIFT_ENUM_IMPL(TImpalaQueryOptions)
+PRINT_THRIFT_ENUM_IMPL(TJoinDistributionMode)
+PRINT_THRIFT_ENUM_IMPL(TMetricKind)
+PRINT_THRIFT_ENUM_IMPL(TParquetArrayResolution)
+PRINT_THRIFT_ENUM_IMPL(TParquetFallbackSchemaResolution)
+PRINT_THRIFT_ENUM_IMPL(TPlanNodeType)
+PRINT_THRIFT_ENUM_IMPL(TPrefetchMode)
+PRINT_THRIFT_ENUM_IMPL(TReplicaPreference)
+PRINT_THRIFT_ENUM_IMPL(TRuntimeFilterMode)
+PRINT_THRIFT_ENUM_IMPL(TSessionType)
+PRINT_THRIFT_ENUM_IMPL(TStmtType)
+PRINT_THRIFT_ENUM_IMPL(TUnit)
 
 string PrintId(const TUniqueId& id, const string& separator) {
   stringstream out;
@@ -158,15 +129,6 @@ bool ParseId(const string& s, TUniqueId* id) {
   return valid;
 }
 
-string PrintPlanNodeType(const TPlanNodeType::type& type) {
-  map<int, const char*>::const_iterator i;
-  i = _TPlanNodeType_VALUES_TO_NAMES.find(type);
-  if (i != _TPlanNodeType_VALUES_TO_NAMES.end()) {
-    return i->second;
-  }
-  return "Invalid plan node type";
-}
-
 string PrintTuple(const Tuple* t, const TupleDescriptor& d) {
   if (t == NULL) return "null";
   stringstream out;

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/debug-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index 27d6cee..c5a5697 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -44,34 +44,34 @@ class Tuple;
 class TupleRow;
 class RowBatch;
 
-std::ostream& operator<<(std::ostream& os, const TFunctionBinaryType::type& op);
-std::ostream& operator<<(std::ostream& os, const TUniqueId& id);
-std::ostream& operator<<(std::ostream& os, const THdfsFileFormat::type& type);
-std::ostream& operator<<(std::ostream& os, const THdfsCompression::type& type);
-std::ostream& operator<<(std::ostream& os, const TStmtType::type& type);
-std::ostream& operator<<(std::ostream& os, const TUnit::type& type);
-std::ostream& operator<<(std::ostream& os, const TMetricKind::type& type);
-std::ostream& operator<<(std::ostream& os, const beeswax::QueryState::type& type);
-std::ostream& operator<<(std::ostream& os, const parquet::Encoding::type& type);
-std::ostream& operator<<(std::ostream& os, const parquet::CompressionCodec::type& type);
-std::ostream& operator<<(std::ostream& os, const parquet::Type::type& type);
+// TODO: remove these functions and use operator << after upgrading to Thrift 0.11.0 or
+// higher.
+std::string PrintThriftEnum(const beeswax::QueryState::type& value);
+std::string PrintThriftEnum(const parquet::Encoding::type& value);
+std::string PrintThriftEnum(const TCatalogObjectType::type& value);
+std::string PrintThriftEnum(const TCatalogOpType::type& value);
+std::string PrintThriftEnum(const TDdlType::type& value);
+std::string PrintThriftEnum(const TExplainLevel::type& value);
+std::string PrintThriftEnum(const THdfsCompression::type& value);
+std::string PrintThriftEnum(const THdfsFileFormat::type& value);
+std::string PrintThriftEnum(const THdfsSeqCompressionMode::type& value);
+std::string PrintThriftEnum(const TImpalaQueryOptions::type& value);
+std::string PrintThriftEnum(const TJoinDistributionMode::type& value);
+std::string PrintThriftEnum(const TMetricKind::type& value);
+std::string PrintThriftEnum(const TParquetArrayResolution::type& value);
+std::string PrintThriftEnum(const TParquetFallbackSchemaResolution::type& value);
+std::string PrintThriftEnum(const TPlanNodeType::type& value);
+std::string PrintThriftEnum(const TPrefetchMode::type& value);
+std::string PrintThriftEnum(const TReplicaPreference::type& value);
+std::string PrintThriftEnum(const TRuntimeFilterMode::type& value);
+std::string PrintThriftEnum(const TSessionType::type& value);
+std::string PrintThriftEnum(const TStmtType::type& value);
+std::string PrintThriftEnum(const TUnit::type& value);
 
 std::string PrintTuple(const Tuple* t, const TupleDescriptor& d);
 std::string PrintRow(TupleRow* row, const RowDescriptor& d);
 std::string PrintBatch(RowBatch* batch);
 std::string PrintId(const TUniqueId& id, const std::string& separator = ":");
-std::string PrintPlanNodeType(const TPlanNodeType::type& type);
-std::string PrintTCatalogObjectType(const TCatalogObjectType::type& type);
-std::string PrintTDdlType(const TDdlType::type& type);
-std::string PrintTCatalogOpType(const TCatalogOpType::type& type);
-std::string PrintTReplicaPreference(const TReplicaPreference::type& type);
-std::string PrintTSessionType(const TSessionType::type& type);
-std::string PrintTStmtType(const TStmtType::type& type);
-std::string PrintQueryState(const beeswax::QueryState::type& type);
-std::string PrintEncoding(const parquet::Encoding::type& type);
-std::string PrintTMetricKind(const TMetricKind::type& type);
-std::string PrintTUnit(const TUnit::type& type);
-std::string PrintTImpalaQueryOptions(const TImpalaQueryOptions::type& type);
 
 /// Returns the fully qualified path, e.g. "database.table.array_col.item.field"
 std::string PrintPath(const TableDescriptor& tbl_desc, const SchemaPath& path);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/histogram-metric.h
----------------------------------------------------------------------
diff --git a/be/src/util/histogram-metric.h b/be/src/util/histogram-metric.h
index d4e09e4..43d4eaf 100644
--- a/be/src/util/histogram-metric.h
+++ b/be/src/util/histogram-metric.h
@@ -62,10 +62,10 @@ class HistogramMetric : public Metric {
       container.AddMember("min", histogram_->MinValue(), document->GetAllocator());
       container.AddMember("count", histogram_->TotalCount(), document->GetAllocator());
     }
-    rapidjson::Value type_value(PrintTMetricKind(TMetricKind::HISTOGRAM).c_str(),
+    rapidjson::Value type_value(PrintThriftEnum(TMetricKind::HISTOGRAM).c_str(),
         document->GetAllocator());
     container.AddMember("kind", type_value, document->GetAllocator());
-    rapidjson::Value units(PrintTUnit(unit()).c_str(), document->GetAllocator());
+    rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
     container.AddMember("units", units, document->GetAllocator());
 
     *value = container;

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index b513c1e..99115c4 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -152,10 +152,9 @@ class ScalarMetric: public Metric {
     ToJsonValue(GetValue(), TUnit::NONE, document, &metric_value);
     container.AddMember("value", metric_value, document->GetAllocator());
 
-    rapidjson::Value type_value(PrintTMetricKind(kind()).c_str(),
-        document->GetAllocator());
+    rapidjson::Value type_value(PrintThriftEnum(kind()).c_str(), document->GetAllocator());
     container.AddMember("kind", type_value, document->GetAllocator());
-    rapidjson::Value units(PrintTUnit(unit()).c_str(), document->GetAllocator());
+    rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
     container.AddMember("units", units, document->GetAllocator());
     *val = container;
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/network-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/network-util.cc b/be/src/util/network-util.cc
index 7a10965..49c96b1 100644
--- a/be/src/util/network-util.cc
+++ b/be/src/util/network-util.cc
@@ -174,15 +174,10 @@ bool IsWildcardAddress(const string& ipaddress) {
 
 string TNetworkAddressToString(const TNetworkAddress& address) {
   stringstream ss;
-  ss << address;
+  ss << address.hostname << ":" << dec << address.port;
   return ss.str();
 }
 
-ostream& operator<<(ostream& out, const TNetworkAddress& hostport) {
-  out << hostport.hostname << ":" << dec << hostport.port;
-  return out;
-}
-
 /// Pick a random port in the range of ephemeral ports
 /// https://tools.ietf.org/html/rfc6335
 int FindUnusedEphemeralPort(vector<int>* used_ports) {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/network-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/network-util.h b/be/src/util/network-util.h
index 5b108dc..ef270ee 100644
--- a/be/src/util/network-util.h
+++ b/be/src/util/network-util.h
@@ -73,9 +73,6 @@ std::string TNetworkAddressToString(const TNetworkAddress& address);
 Status TNetworkAddressToSockaddr(const TNetworkAddress& address,
     kudu::Sockaddr* sockaddr);
 
-/// Prints a hostport as ipaddress:port
-std::ostream& operator<<(std::ostream& out, const TNetworkAddress& hostport);
-
 /// Returns a ephemeral port that is currently unused. Returns -1 on an error or if
 /// a free ephemeral port can't be found after 100 tries. If 'used_ports' is non-NULL,
 /// does not select those ports and adds the selected port to 'used_ports'.

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/util/webserver.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index ea0a6e9..a77c6da 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -220,10 +220,10 @@ string Webserver::Url() {
 }
 
 Status Webserver::Start() {
-  LOG(INFO) << "Starting webserver on " << http_address_;
+  LOG(INFO) << "Starting webserver on " << TNetworkAddressToString(http_address_);
 
   stringstream listening_spec;
-  listening_spec << http_address_;
+  listening_spec << TNetworkAddressToString(http_address_);
 
   if (IsSecure()) {
     LOG(INFO) << "Webserver: Enabling HTTPS support";
@@ -320,7 +320,8 @@ Status Webserver::Start() {
 
   if (context_ == nullptr) {
     stringstream error_msg;
-    error_msg << "Webserver: Could not start on address " << http_address_;
+    error_msg << "Webserver: Could not start on address "
+              << TNetworkAddressToString(http_address_);
     return Status(error_msg.str());
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/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 32ad938..c1db9f5 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -17,7 +17,7 @@ set all;
 'DISABLE_CACHED_READS','0','DEPRECATED'
 'DISABLE_CODEGEN','0','REGULAR'
 'DISABLE_OUTERMOST_TOPN','0','DEVELOPMENT'
-'EXPLAIN_LEVEL','1','REGULAR'
+'EXPLAIN_LEVEL','STANDARD','REGULAR'
 'HBASE_CACHE_BLOCKS','0','ADVANCED'
 'HBASE_CACHING','0','ADVANCED'
 'MAX_ERRORS','100','ADVANCED'
@@ -50,7 +50,7 @@ set all;
 'DISABLE_CACHED_READS','0','DEPRECATED'
 'DISABLE_CODEGEN','0','REGULAR'
 'DISABLE_OUTERMOST_TOPN','0','DEVELOPMENT'
-'EXPLAIN_LEVEL','3','REGULAR'
+'EXPLAIN_LEVEL','VERBOSE','REGULAR'
 'HBASE_CACHE_BLOCKS','0','ADVANCED'
 'HBASE_CACHING','0','ADVANCED'
 'MAX_ERRORS','100','ADVANCED'
@@ -83,7 +83,7 @@ set all;
 'DISABLE_CACHED_READS','0','DEPRECATED'
 'DISABLE_CODEGEN','0','REGULAR'
 'DISABLE_OUTERMOST_TOPN','0','DEVELOPMENT'
-'EXPLAIN_LEVEL','0','REGULAR'
+'EXPLAIN_LEVEL','MINIMAL','REGULAR'
 'HBASE_CACHE_BLOCKS','0','ADVANCED'
 'HBASE_CACHING','0','ADVANCED'
 'MAX_ERRORS','100','ADVANCED'

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/tests/shell/test_shell_commandline.py
----------------------------------------------------------------------
diff --git a/tests/shell/test_shell_commandline.py b/tests/shell/test_shell_commandline.py
index 10513b6..97f4a30 100644
--- a/tests/shell/test_shell_commandline.py
+++ b/tests/shell/test_shell_commandline.py
@@ -248,8 +248,8 @@ class TestImpalaShell(ImpalaTestSuite):
     args = '-q "set"'
     result_set = run_impala_shell_cmd(args)
     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 explain_level is STANDARD
+    assert 'EXPLAIN_LEVEL: [STANDARD]' 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