You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2022/10/20 17:24:45 UTC

[impala] 01/02: IMPALA-11645: Remove PrintThriftEnum functions in debug-utils.cc

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

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

commit 50944811351af142baa75ca0742da58f92407f6d
Author: Daniel Becker <da...@cloudera.com>
AuthorDate: Mon Oct 10 16:49:07 2022 +0200

    IMPALA-11645: Remove PrintThriftEnum functions in debug-utils.cc
    
    Before IMPALA-5690 we implemented operator<< for Thrift enums in Impala
    code. These functions printed the names of the enums.
    
    Then we upgraded to Thrift 0.9.3, but that release included THRIFT-2067,
    which implemented operator<< for Thrift enums, but printed the number
    value of enums instead of their names. To preserve the old behaviour in
    Impala, we renamed our own implementations of operator<< to
    PrintThriftEnum, a function that we defined for each Thrift enum we
    used, and which returned a string with the names - not the numbers - of
    the enums.
    
    After upgrading Thrift to a version that included THRIFT-3921 (any
    version starting from 0.11.0), these PrintThriftEnum functions are no
    longer necessary as the operator<< provided by Thrift now prints the
    names of enums, which is the behaviour we want.
    
    This change removes all PrintThriftEnum functions. At the call sites
    where the result of the function call was inserted into a stream, now the
    enum is inserted directly (invoking the operator<< provided by Thrift).
    At the call sites where a string object is expected, we replaced the
    call to PrintThriftEnum with a call to a new function template,
    PrintValue(), which converts any object for which operator<< is defined
    to a string.
    
    Change-Id: Ie36f6f0f032f3f5c7596929a1eccb80ff16fcd83
    Reviewed-on: http://gerrit.cloudera.org:8080/19118
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/scheduler-benchmark.cc      |  2 +-
 be/src/exec/exec-node.cc                      |  4 +--
 be/src/exec/hdfs-scan-node-base.cc            | 18 ++++++-------
 be/src/exec/kudu/kudu-scanner.cc              |  2 +-
 be/src/exec/parquet/parquet-column-readers.cc |  4 +--
 be/src/exec/parquet/parquet-metadata-utils.cc |  2 +-
 be/src/exec/partitioned-hash-join-builder.cc  |  2 +-
 be/src/runtime/coordinator.cc                 |  4 +--
 be/src/runtime/fragment-instance-state.cc     |  4 +--
 be/src/service/client-request-state.cc        | 21 +++++++--------
 be/src/service/impala-http-handler.cc         |  2 +-
 be/src/service/query-options.cc               | 12 ++++-----
 be/src/util/collection-metrics.cc             |  2 +-
 be/src/util/debug-util.cc                     | 37 -------------------------
 be/src/util/debug-util.h                      | 39 +++++----------------------
 be/src/util/histogram-metric.cc               |  4 +--
 be/src/util/metrics.cc                        |  6 ++---
 17 files changed, 50 insertions(+), 115 deletions(-)

diff --git a/be/src/benchmarks/scheduler-benchmark.cc b/be/src/benchmarks/scheduler-benchmark.cc
index 3fc60791d..345b257ec 100644
--- a/be/src/benchmarks/scheduler-benchmark.cc
+++ b/be/src/benchmarks/scheduler-benchmark.cc
@@ -133,7 +133,7 @@ void BenchmarkFunction(int num_iterations, void* data) {
 /// blocks. Scheduling will be done according to the parameter 'replica_preference'.
 void RunClusterSizeBenchmark(TReplicaPreference::type replica_preference) {
   string suite_name = strings::Substitute(
-      "Cluster Size, $0", PrintThriftEnum(replica_preference));
+      "Cluster Size, $0", PrintValue(replica_preference));
   Benchmark suite(suite_name, false /* micro_heuristics */);
   vector<TestCtx> test_ctx(CLUSTER_SIZES.size());
 
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 5f2ff77e3..7d4bd3070 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -244,7 +244,7 @@ ExecNode::ExecNode(ObjectPool* pool, const PlanNode& pnode, const DescriptorTbl&
     resource_profile_(pnode.tnode_->resource_profile),
     limit_(pnode.tnode_->limit),
     runtime_profile_(RuntimeProfile::Create(
-        pool_, Substitute("$0 (id=$1)", PrintThriftEnum(type_), id_))),
+        pool_, Substitute("$0 (id=$1)", PrintValue(type_), id_))),
     rows_returned_counter_(nullptr),
     rows_returned_rate_(nullptr),
     containing_subplan_(nullptr),
@@ -277,7 +277,7 @@ Status ExecNode::Prepare(RuntimeState* state) {
     RETURN_IF_ERROR(children_[i]->Prepare(state));
   }
   reservation_manager_.Init(
-      Substitute("$0 id=$1 ptr=$2", PrintThriftEnum(type_), id_, this), runtime_profile_,
+      Substitute("$0 id=$1 ptr=$2", PrintValue(type_), id_, this), runtime_profile_,
       state->instance_buffer_reservation(), mem_tracker_.get(), resource_profile_,
       debug_options_);
   if (!IsInSubplan()) {
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 02c306cb2..b5bab2c13 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -272,7 +272,7 @@ Status HdfsScanPlanNode::ProcessScanRangesAndInitSharedState(FragmentState* stat
         // TODO: this should be a DCHECK but we sometimes hit it. It's likely IMPALA-1702.
         LOG(ERROR) << "Bad table descriptor! table_id=" << hdfs_table_->id()
                    << " partition_id=" << split.partition_id() << "\n"
-                   << PrintThrift(state->fragment())
+                   << state->fragment()
                    << state->fragment_ctx().DebugString();
         return Status("Query encountered invalid metadata, likely due to IMPALA-1702."
                       " Try rerunning the query.");
@@ -754,7 +754,7 @@ void HdfsScanNodeBase::SkipScanRange(io::ScanRange* scan_range) {
   HdfsPartitionDescriptor* partition = hdfs_table_->GetPartition(partition_id);
   DCHECK(partition != nullptr) << "table_id=" << hdfs_table_->id()
                                << " partition_id=" << partition_id << "\n"
-                               << PrintThrift(runtime_state_->instance_ctx());
+                               << runtime_state_->instance_ctx();
   const HdfsFileDesc* desc = GetFileDesc(partition_id, *scan_range->file_string());
   if (metadata->is_sequence_header) {
     // File ranges haven't been issued yet, skip entire file.
@@ -1155,26 +1155,26 @@ void HdfsScanNodeBase::StopAndFinalizeCounters() {
           if (file_format == THdfsFileFormat::PARQUET) {
             // If a scan range stored as parquet is skipped, its compression type
             // cannot be figured out without reading the data.
-            ss << PrintThriftEnum(file_format) << "/" << "Unknown" << "(Skipped):"
+            ss << file_format << "/" << "Unknown" << "(Skipped):"
                << file_cnt << " ";
           } else {
-            ss << PrintThriftEnum(file_format) << "/"
-               << PrintThriftEnum(compressions_set.GetFirstType()) << "(Skipped):"
+            ss << file_format << "/"
+               << compressions_set.GetFirstType() << "(Skipped):"
                << file_cnt << " ";
           }
         } else if (compressions_set.Size() == 1) {
-          ss << PrintThriftEnum(file_format) << "/"
-             << PrintThriftEnum(compressions_set.GetFirstType()) << ":" << file_cnt
+          ss << file_format << "/"
+             << compressions_set.GetFirstType() << ":" << file_cnt
              << " ";
         } else {
-          ss << PrintThriftEnum(file_format) << "/" << "(";
+          ss << file_format << "/" << "(";
           bool first = true;
           for (auto& elem : _THdfsCompression_VALUES_TO_NAMES) {
             THdfsCompression::type type = static_cast<THdfsCompression::type>(
                 elem.first);
             if (!compressions_set.HasType(type)) continue;
             if (!first) ss << ",";
-            ss << PrintThriftEnum(type);
+            ss << type;
             first = false;
           }
           ss << "):" << file_cnt << " ";
diff --git a/be/src/exec/kudu/kudu-scanner.cc b/be/src/exec/kudu/kudu-scanner.cc
index 7f510bac5..4a404bcaf 100644
--- a/be/src/exec/kudu/kudu-scanner.cc
+++ b/be/src/exec/kudu/kudu-scanner.cc
@@ -199,7 +199,7 @@ Status KuduScanner::OpenNextScanToken(const string& scan_token, bool* eos) {
   RETURN_IF_ERROR(StringToKuduReadMode(FLAGS_kudu_read_mode, &mode));
   if (state_->query_options().kudu_read_mode != TKuduReadMode::DEFAULT) {
     RETURN_IF_ERROR(StringToKuduReadMode(
-        PrintThriftEnum(state_->query_options().kudu_read_mode), &mode));
+        PrintValue(state_->query_options().kudu_read_mode), &mode));
   }
   KUDU_RETURN_IF_ERROR(
       scanner_->SetReadMode(mode), BuildErrorString("Could not set scanner ReadMode"));
diff --git a/be/src/exec/parquet/parquet-column-readers.cc b/be/src/exec/parquet/parquet-column-readers.cc
index b2f82590f..20cfe38b7 100644
--- a/be/src/exec/parquet/parquet-column-readers.cc
+++ b/be/src/exec/parquet/parquet-column-readers.cc
@@ -253,7 +253,7 @@ class ScalarColumnReader : public BaseScalarColumnReader {
 
   void __attribute__((noinline)) SetBoolDecodeError() {
     parent_->parse_status_ = Status(TErrorCode::PARQUET_CORRUPT_BOOL_VALUE, filename(),
-        PrintThriftEnum(page_encoding_), col_chunk_reader_.stream()->file_offset());
+        PrintValue(page_encoding_), col_chunk_reader_.stream()->file_offset());
   }
 
   ParquetTimestampDecoder& GetTimestampDecoder() {
@@ -1522,7 +1522,7 @@ bool BaseScalarColumnReader::JumpToNextPage() {
 Status BaseScalarColumnReader::GetUnsupportedDecodingError() {
   return Status(Substitute(
       "File '$0' is corrupt: unexpected encoding: $1 for data page of column '$2'.",
-      filename(), PrintThriftEnum(page_encoding_), schema_element().name));
+      filename(), PrintValue(page_encoding_), schema_element().name));
 }
 
 Status BaseScalarColumnReader::LogCorruptNumValuesInMetadataError() {
diff --git a/be/src/exec/parquet/parquet-metadata-utils.cc b/be/src/exec/parquet/parquet-metadata-utils.cc
index 6b8c15edf..c23dd2ae7 100644
--- a/be/src/exec/parquet/parquet-metadata-utils.cc
+++ b/be/src/exec/parquet/parquet-metadata-utils.cc
@@ -313,7 +313,7 @@ Status ParquetMetadataUtils::ValidateRowGroupColumn(
   for (int i = 0; i < encodings.size(); ++i) {
     if (!IsEncodingSupported(encodings[i])) {
       return Status(Substitute("File '$0' uses an unsupported encoding: $1 for column "
-          "'$2'.", filename, PrintThriftEnum(encodings[i]), schema_element.name));
+          "'$2'.", filename, PrintValue(encodings[i]), schema_element.name));
     }
   }
 
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index 75c52ec0f..1204e3ed2 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -1364,7 +1364,7 @@ void PhjBuilderConfig::Codegen(FragmentState* state) {
 
 string PhjBuilder::DebugString() const {
   stringstream ss;
-  ss << " PhjBuilder op=" << PrintThriftEnum(join_op_)
+  ss << " PhjBuilder op=" << join_op_
      << " is_separate_build=" << is_separate_build_
      << " num_probe_threads=" << num_probe_threads_
      << " state=" << PrintState(state_)
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 0d474484e..3d2fae021 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -649,7 +649,7 @@ string Coordinator::FilterDebugString() {
       for (int i = 0; i < 3; ++i) row.push_back("");
     } else if (state.is_min_max_filter()) {
       // Add the filter type for minmax filters.
-      row.push_back(PrintThriftEnum(state.desc().type));
+      row.push_back(PrintValue(state.desc().type));
       row.push_back("");
 
       // Also add the min/max value for the accumulated filter as follows.
@@ -685,7 +685,7 @@ string Coordinator::FilterDebugString() {
       }
       row.push_back("");
     } else if (state.is_in_list_filter()) {
-      row.push_back(PrintThriftEnum(state.desc().type));
+      row.push_back(PrintValue(state.desc().type));
       // Skip 3 fields belong to Bloom/MinMax filters.
       for (int i = 0; i < 3; ++i) row.push_back("");
       const InListFilterPB& in_list_filterPB =
diff --git a/be/src/runtime/fragment-instance-state.cc b/be/src/runtime/fragment-instance-state.cc
index df546c186..876383cac 100644
--- a/be/src/runtime/fragment-instance-state.cc
+++ b/be/src/runtime/fragment-instance-state.cc
@@ -339,8 +339,8 @@ void FragmentInstanceState::GetStatusReport(FragmentInstanceExecStatusPB* instan
       if (rows_counter != nullptr) {
         summary_data->set_rows_returned(rows_counter->value());
         // row count stats for a join node
-        string hash_type = PrintThriftEnum(TPlanNodeType::HASH_JOIN_NODE);
-        string nested_loop_type = PrintThriftEnum(TPlanNodeType::NESTED_LOOP_JOIN_NODE);
+        string hash_type = PrintValue(TPlanNodeType::HASH_JOIN_NODE);
+        string nested_loop_type = PrintValue(TPlanNodeType::NESTED_LOOP_JOIN_NODE);
         if (node->name().rfind(hash_type, 0) == 0
             || node->name().rfind(nested_loop_type, 0) == 0) {
           per_join_rows_produced[node->metadata().plan_node_id] = rows_counter->value();
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 1c8fa9e6e..f03a454f1 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -143,7 +143,7 @@ ClientRequestState::ClientRequestState(const TQueryCtx& query_ctx, Frontend* fro
 
   profile_->set_name("Query (id=" + PrintId(query_id()) + ")");
   summary_profile_->AddInfoString("Session ID", PrintId(session_id()));
-  summary_profile_->AddInfoString("Session Type", PrintThriftEnum(session_type()));
+  summary_profile_->AddInfoString("Session Type", PrintValue(session_type()));
   if (session_type() == TSessionType::HIVESERVER2 ||
       session_type() == TSessionType::EXTERNAL_FRONTEND) {
     summary_profile_->AddInfoString("HiveServer2 Protocol Version",
@@ -155,7 +155,7 @@ ClientRequestState::ClientRequestState(const TQueryCtx& query_ctx, Frontend* fro
       TimePrecision::Nanosecond));
   summary_profile_->AddInfoString("End Time", "");
   summary_profile_->AddInfoString("Query Type", "N/A");
-  summary_profile_->AddInfoString("Query State", PrintThriftEnum(BeeswaxQueryState()));
+  summary_profile_->AddInfoString("Query State", PrintValue(BeeswaxQueryState()));
   summary_profile_->AddInfoString(
       "Impala Query State", ExecStateToString(exec_state()));
   summary_profile_->AddInfoString("Query Status", "OK");
@@ -223,7 +223,7 @@ Status ClientRequestState::Exec() {
   MarkActive();
 
   profile_->AddChild(server_profile_);
-  summary_profile_->AddInfoString("Query Type", PrintThriftEnum(stmt_type()));
+  summary_profile_->AddInfoString("Query Type", PrintValue(stmt_type()));
   summary_profile_->AddInfoString("Query Options (set by configuration)",
       DebugQueryOptions(query_ctx_.client_request.query_options));
   summary_profile_->AddInfoString("Query Options (set by configuration and planner)",
@@ -754,7 +754,7 @@ bool ClientRequestState::ShouldRunExecDdlAsync() {
 
 Status ClientRequestState::ExecDdlRequest() {
   string op_type = catalog_op_type() == TCatalogOpType::DDL ?
-      PrintThriftEnum(ddl_type()) : PrintThriftEnum(catalog_op_type());
+      PrintValue(ddl_type()) : PrintValue(catalog_op_type());
   bool async_ddl = ShouldRunExecDdlAsync();
   bool async_ddl_enabled = exec_request_->query_options.enable_async_ddl_execution;
   string exec_mode = (async_ddl && async_ddl_enabled) ? "asynchronous" : "synchronous";
@@ -1008,7 +1008,7 @@ Status ClientRequestState::Finalize(bool check_inflight, const Status* cause) {
 Status ClientRequestState::Exec(const TMetadataOpRequest& exec_request) {
   TResultSet metadata_op_result;
   // Like the other Exec(), fill out as much profile information as we're able to.
-  summary_profile_->AddInfoString("Query Type", PrintThriftEnum(TStmtType::DDL));
+  summary_profile_->AddInfoString("Query Type", PrintValue(TStmtType::DDL));
   RETURN_IF_ERROR(frontend_->ExecHiveServer2MetadataOp(exec_request,
       &metadata_op_result));
   result_metadata_ = metadata_op_result.schema;
@@ -1667,7 +1667,7 @@ void ClientRequestState::ClearResultCache() {
 
 void ClientRequestState::UpdateExecState(ExecState exec_state) {
   exec_state_.Store(exec_state);
-  summary_profile_->AddInfoString("Query State", PrintThriftEnum(BeeswaxQueryState()));
+  summary_profile_->AddInfoString("Query State", PrintValue(BeeswaxQueryState()));
   summary_profile_->AddInfoString("Impala Query State", ExecStateToString(exec_state));
 }
 
@@ -1905,13 +1905,12 @@ Status ClientRequestState::LogAuditRecord(const Status& query_status) {
   writer.String("statement_type");
   if (request.stmt_type == TStmtType::DDL) {
     if (request.catalog_op_request.op_type == TCatalogOpType::DDL) {
-      writer.String(
-          PrintThriftEnum(request.catalog_op_request.ddl_params.ddl_type).c_str());
+      writer.String(PrintValue(request.catalog_op_request.ddl_params.ddl_type).c_str());
     } else {
-      writer.String(PrintThriftEnum(request.catalog_op_request.op_type).c_str());
+      writer.String(PrintValue(request.catalog_op_request.op_type).c_str());
     }
   } else {
-    writer.String(PrintThriftEnum(request.stmt_type).c_str());
+    writer.String(PrintValue(request.stmt_type).c_str());
   }
   writer.String("network_address");
   writer.String(TNetworkAddressToString(
@@ -1928,7 +1927,7 @@ Status ClientRequestState::LogAuditRecord(const Status& query_status) {
     writer.String("name");
     writer.String(event.name.c_str());
     writer.String("object_type");
-    writer.String(PrintThriftEnum(event.object_type).c_str());
+    writer.String(PrintValue(event.object_type).c_str());
     writer.String("privilege");
     writer.String(event.privilege.c_str());
     writer.EndObject();
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index caffd7b9b..1c320dfad 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -568,7 +568,7 @@ void ImpalaHttpHandler::SessionsHandler(const Webserver::WebRequest& req,
            server_->session_state_map_) {
     shared_ptr<ImpalaServer::SessionState> state = session.second;
     Value session_json(kObjectType);
-    Value type(PrintThriftEnum(state->session_type).c_str(), document->GetAllocator());
+    Value type(PrintValue(state->session_type).c_str(), document->GetAllocator());
     session_json.AddMember("type", type, document->GetAllocator());
 
     session_json.AddMember("inflight_queries",
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index d8245a8c2..fc31006e9 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -73,11 +73,9 @@ void impala::OverlayQueryOptions(const TQueryOptions& src, const QueryOptionsMas
 }
 
 // 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);
+  return PrintValue(option);
 }
 
 template <typename T, typename std::enable_if_t<std::is_arithmetic<T>::value>* = nullptr>
@@ -91,9 +89,9 @@ const string& PrintQueryOptionValue(const std::string& option)  {
 
 const string PrintQueryOptionValue(const impala::TCompressionCodec& compression_codec) {
   if (compression_codec.codec != THdfsCompression::ZSTD) {
-    return Substitute("$0", PrintThriftEnum(compression_codec.codec));
+    return Substitute("$0", PrintValue(compression_codec.codec));
   } else {
-    return Substitute("$0:$1", PrintThriftEnum(compression_codec.codec),
+    return Substitute("$0:$1", PrintValue(compression_codec.codec),
         compression_codec.compression_level);
   }
 }
@@ -383,7 +381,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, PrintThriftEnum(
+                  "Valid sizes are in [$2, $3].", value, PrintValue(
                       static_cast<TImpalaQueryOptions::type>(option)),
                   RuntimeFilterBank::MIN_BLOOM_FILTER_SIZE,
                   RuntimeFilterBank::MAX_BLOOM_FILTER_SIZE));
@@ -395,7 +393,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",
-              PrintThriftEnum(static_cast<TImpalaQueryOptions::type>(option)),
+              PrintValue(static_cast<TImpalaQueryOptions::type>(option)),
               FLAGS_min_buffer_size));
         }
         if (option == TImpalaQueryOptions::RUNTIME_BLOOM_FILTER_SIZE) {
diff --git a/be/src/util/collection-metrics.cc b/be/src/util/collection-metrics.cc
index 5bd9f2b21..8df2577d4 100644
--- a/be/src/util/collection-metrics.cc
+++ b/be/src/util/collection-metrics.cc
@@ -128,7 +128,7 @@ void StatsMetric<T, StatsSelection>::ToJson(
   lock_guard<mutex> l(lock_);
   rapidjson::Value container(rapidjson::kObjectType);
   AddStandardFields(document, &container);
-  rapidjson::Value units(PrintThriftEnum(unit_).c_str(), document->GetAllocator());
+  rapidjson::Value units(PrintValue(unit_).c_str(), document->GetAllocator());
   container.AddMember("units", units, document->GetAllocator());
 
   if (StatsSelection & StatsType::COUNT) {
diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index 59d61c8b0..be4a87d1d 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -69,43 +69,6 @@ DECLARE_string(hostname);
 
 namespace impala {
 
-#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; \
-  }
-
-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(TJoinOp)
-PRINT_THRIFT_ENUM_IMPL(TKuduReadMode)
-PRINT_THRIFT_ENUM_IMPL(TMetricKind)
-PRINT_THRIFT_ENUM_IMPL(TParquetArrayResolution)
-PRINT_THRIFT_ENUM_IMPL(TSchemaResolutionStrategy)
-PRINT_THRIFT_ENUM_IMPL(TPlanNodeType)
-PRINT_THRIFT_ENUM_IMPL(TPrefetchMode)
-PRINT_THRIFT_ENUM_IMPL(TReplicaPreference)
-PRINT_THRIFT_ENUM_IMPL(TRuntimeFilterMode)
-PRINT_THRIFT_ENUM_IMPL(TRuntimeFilterType)
-PRINT_THRIFT_ENUM_IMPL(TSessionType)
-PRINT_THRIFT_ENUM_IMPL(TStmtType)
-PRINT_THRIFT_ENUM_IMPL(TUnit)
-PRINT_THRIFT_ENUM_IMPL(TParquetTimestampType)
-PRINT_THRIFT_ENUM_IMPL(TTransactionalType)
-PRINT_THRIFT_ENUM_IMPL(TMinmaxFilteringLevel)
-PRINT_THRIFT_ENUM_IMPL(TKuduReplicaSelection)
-PRINT_THRIFT_ENUM_IMPL(TMinmaxFilterFastCodePathMode)
-PRINT_THRIFT_ENUM_IMPL(TParquetBloomFilterWrite)
-
 string PrintId(const TUniqueId& id, const string& separator) {
   stringstream out;
   // Outputting the separator string resets the stream width.
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index e25ee92ad..51b73d979 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -51,38 +51,13 @@ class TupleRow;
 // Forward declaration to avoid including descriptors.h.
 typedef std::vector<int> SchemaPath;
 
-// 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 TJoinOp::type& value);
-std::string PrintThriftEnum(const TKuduReadMode::type& value);
-std::string PrintThriftEnum(const TMetricKind::type& value);
-std::string PrintThriftEnum(const TParquetArrayResolution::type& value);
-std::string PrintThriftEnum(const TSchemaResolutionStrategy::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 TRuntimeFilterType::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 PrintThriftEnum(const TParquetTimestampType::type& value);
-std::string PrintThriftEnum(const TTransactionalType::type& value);
-std::string PrintThriftEnum(const TMinmaxFilteringLevel::type& value);
-std::string PrintThriftEnum(const TKuduReplicaSelection::type& value);
-std::string PrintThriftEnum(const TMinmaxFilterFastCodePathMode::type& value);
-std::string PrintThriftEnum(const TParquetBloomFilterWrite::type& value);
+// Converts a value for which operator<< is defined to a std::string.
+template<class T>
+std::string PrintValue(const T& value) {
+  std::stringstream s;
+  s << value;
+  return s.str();
+}
 
 std::string PrintTuple(const Tuple* t, const TupleDescriptor& d);
 std::string PrintRow(TupleRow* row, const RowDescriptor& d);
diff --git a/be/src/util/histogram-metric.cc b/be/src/util/histogram-metric.cc
index 21c7a8f5d..2de1a3272 100644
--- a/be/src/util/histogram-metric.cc
+++ b/be/src/util/histogram-metric.cc
@@ -64,9 +64,9 @@ void HistogramMetric::ToJson(rapidjson::Document* document, rapidjson::Value* va
     container.AddMember("count", histogram_->TotalCount(), document->GetAllocator());
   }
   rapidjson::Value type_value(
-      PrintThriftEnum(TMetricKind::HISTOGRAM).c_str(), document->GetAllocator());
+      PrintValue(TMetricKind::HISTOGRAM).c_str(), document->GetAllocator());
   container.AddMember("kind", type_value, document->GetAllocator());
-  rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
+  rapidjson::Value units(PrintValue(unit()).c_str(), document->GetAllocator());
   container.AddMember("units", units, document->GetAllocator());
 
   *value = container;
diff --git a/be/src/util/metrics.cc b/be/src/util/metrics.cc
index d967a8a14..b1c22085d 100644
--- a/be/src/util/metrics.cc
+++ b/be/src/util/metrics.cc
@@ -67,9 +67,9 @@ void ScalarMetric<T, metric_kind_t>::ToJson(Document* document, Value* val) {
   ToJsonValue(GetValue(), TUnit::NONE, document, &metric_value);
   container.AddMember("value", metric_value, document->GetAllocator());
 
-  Value type_value(PrintThriftEnum(kind()).c_str(), document->GetAllocator());
+  Value type_value(PrintValue(kind()).c_str(), document->GetAllocator());
   container.AddMember("kind", type_value, document->GetAllocator());
-  Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
+  Value units(PrintValue(unit()).c_str(), document->GetAllocator());
   container.AddMember("units", units, document->GetAllocator());
   *val = container;
 }
@@ -85,7 +85,7 @@ void ScalarMetric<T, metric_kind_t>::ToLegacyJson(Document* document) {
 template <typename T, TMetricKind::type metric_kind_t>
 TMetricKind::type ScalarMetric<T, metric_kind_t>::ToPrometheus(
     string name, stringstream* val, stringstream* metric_kind) {
-  string metric_type = PrintThriftEnum(kind()).c_str();
+  string metric_type = PrintValue(kind()).c_str();
   // prometheus doesn't support 'property', so ignore it
   if (!metric_type.compare("property")) {
     return TMetricKind::PROPERTY;