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:04 UTC

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

IMPALA-5690: Part 1: Rename ostream operators for thrift types

Thrift 0.9.3 implements "ostream& operator<<(ostream&, T)" for thrift
data types while impala did the same to enums and special types
including TNetworkAddress and TUniqueId. To prepare for the upgrade of
thrift 0.9.3, this patch renames these impala defined functions. In the
absence of operator<<, assertion macros like DCHECK_EQ can no longer be
used on non-enum thrift defined types.

Change-Id: I9c303997411237e988ef960157f781776f6fcb60
Reviewed-on: http://gerrit.cloudera.org:8080/9168
Reviewed-by: Tianyi Wang <tw...@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/e114778e
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/e114778e
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/e114778e

Branch: refs/heads/2.x
Commit: e114778ef32b6e5cfc230854cc50e79eedf2d0bf
Parents: f51dce0
Author: Tianyi Wang <tw...@cloudera.com>
Authored: Mon Feb 12 15:59:02 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Fri Apr 20 20:17:58 2018 +0000

----------------------------------------------------------------------
 be/src/benchmarks/scheduler-benchmark.cc        |  2 +-
 be/src/codegen/llvm-codegen.cc                  |  4 +-
 be/src/exec/exchange-node.cc                    |  2 +-
 be/src/exec/exec-node.cc                        |  8 +-
 be/src/exec/hdfs-scan-node-base.cc              | 13 +--
 be/src/exec/hdfs-scan-node.cc                   |  2 +-
 be/src/exec/parquet-column-readers.cc           |  2 +-
 be/src/exec/parquet-metadata-utils.cc           |  2 +-
 be/src/rpc/rpc-trace.cc                         |  4 +-
 be/src/rpc/thrift-client.cc                     | 16 ++--
 be/src/rpc/thrift-util.cc                       |  3 +-
 be/src/rpc/thrift-util.h                        |  2 +-
 be/src/runtime/client-cache.cc                  | 21 +++--
 be/src/runtime/coordinator-backend-state.cc     | 27 +++---
 be/src/runtime/coordinator.cc                   | 34 ++++----
 be/src/runtime/data-stream-mgr.cc               | 22 ++---
 be/src/runtime/data-stream-recvr.cc             |  7 +-
 be/src/runtime/data-stream-sender.cc            | 17 ++--
 be/src/runtime/fragment-instance-state.cc       |  6 +-
 be/src/runtime/krpc-data-stream-mgr.cc          |  8 +-
 be/src/runtime/krpc-data-stream-recvr.cc        |  7 +-
 be/src/runtime/krpc-data-stream-sender.cc       | 12 +--
 be/src/runtime/mem-tracker.cc                   |  4 +-
 be/src/runtime/query-exec-mgr.cc                |  6 +-
 be/src/runtime/query-state.cc                   |  2 +-
 be/src/runtime/runtime-filter-bank.cc           |  3 +-
 be/src/runtime/runtime-state.cc                 |  4 +-
 be/src/scheduling/admission-controller.cc       | 14 ++--
 be/src/scheduling/scheduler.cc                  | 14 ++--
 be/src/service/child-query.cc                   |  3 +-
 be/src/service/client-request-state.cc          | 18 ++--
 be/src/service/impala-beeswax-server.cc         |  2 +-
 be/src/service/impala-hs2-server.cc             |  4 +-
 be/src/service/impala-http-handler.cc           | 11 ++-
 be/src/service/impala-internal-service.cc       |  7 +-
 be/src/service/impala-server.cc                 | 48 ++++++-----
 be/src/service/query-options-test.cc            |  2 +-
 be/src/service/query-options.cc                 | 25 ++++--
 be/src/service/query-result-set.cc              |  4 +-
 be/src/statestore/statestore.cc                 |  4 +-
 be/src/util/collection-metrics.h                |  2 +-
 be/src/util/debug-util.cc                       | 88 ++++++--------------
 be/src/util/debug-util.h                        | 46 +++++-----
 be/src/util/histogram-metric.h                  |  4 +-
 be/src/util/metrics.h                           |  5 +-
 be/src/util/network-util.cc                     |  7 +-
 be/src/util/network-util.h                      |  3 -
 be/src/util/webserver.cc                        |  7 +-
 .../functional-query/queries/QueryTest/set.test |  6 +-
 tests/shell/test_shell_commandline.py           |  4 +-
 50 files changed, 278 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/benchmarks/scheduler-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/scheduler-benchmark.cc b/be/src/benchmarks/scheduler-benchmark.cc
index 7149dde..ef7fb9e 100644
--- a/be/src/benchmarks/scheduler-benchmark.cc
+++ b/be/src/benchmarks/scheduler-benchmark.cc
@@ -131,7 +131,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", PrintTReplicaPreference(replica_preference));
+      "Cluster Size, $0", PrintThriftEnum(replica_preference));
   Benchmark suite(suite_name, false /* micro_heuristics */);
   vector<TestCtx> test_ctx(CLUSTER_SIZES.size());
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 5d5ed15..7fe4ec1 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -1088,7 +1088,7 @@ Status LlvmCodeGen::FinalizeModule() {
   }
   string non_finalized_fns_str = ss.str();
   if (!non_finalized_fns_str.empty()) {
-    LOG(INFO) << "For query " << state_->query_id()
+    LOG(INFO) << "For query " << PrintId(state_->query_id())
               << " the following functions were not finalized and have been removed from "
                  "the module:\n"
               << non_finalized_fns_str;
@@ -1710,7 +1710,7 @@ void LlvmCodeGen::DiagnosticHandler::DiagnosticHandlerFn(
     info.print(diagnostic_printer);
     error_msg.flush();
     if (codegen->state_) {
-      LOG(INFO) << "Query " << codegen->state_->query_id() << " encountered a "
+      LOG(INFO) << "Query " << PrintId(codegen->state_->query_id()) << " encountered a "
           << codegen->diagnostic_handler_.error_str_;
     }
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/exchange-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.cc b/be/src/exec/exchange-node.cc
index 2dc662b..297a805 100644
--- a/be/src/exec/exchange-node.cc
+++ b/be/src/exec/exchange-node.cc
@@ -150,7 +150,7 @@ Status ExchangeNode::FillInputRowBatch(RuntimeState* state) {
   VLOG_FILE << "exch: has batch=" << (input_batch_ == NULL ? "false" : "true")
             << " #rows=" << (input_batch_ != NULL ? input_batch_->num_rows() : 0)
             << " is_cancelled=" << (ret_status.IsCancelled() ? "true" : "false")
-            << " instance_id=" << state->fragment_instance_id();
+            << " instance_id=" << PrintId(state->fragment_instance_id());
   return ret_status;
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 5f9f777..c94f38c 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -123,7 +123,7 @@ ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl
     limit_(tnode.limit),
     num_rows_returned_(0),
     runtime_profile_(RuntimeProfile::Create(pool_,
-        Substitute("$0 (id=$1)", PrintPlanNodeType(tnode.node_type), id_))),
+        Substitute("$0 (id=$1)", PrintThriftEnum(tnode.node_type), id_))),
     rows_returned_counter_(NULL),
     rows_returned_rate_(NULL),
     containing_subplan_(NULL),
@@ -209,8 +209,8 @@ void ExecNode::Close(RuntimeState* state) {
   if (expr_mem_tracker_ != nullptr) expr_mem_tracker_->Close();
   if (mem_tracker_ != nullptr) {
     if (mem_tracker()->consumption() != 0) {
-      LOG(WARNING) << "Query " << state->query_id() << " may have leaked memory." << endl
-          << state->instance_mem_tracker()->LogUsage(MemTracker::UNLIMITED_DEPTH);
+      LOG(WARNING) << "Query " << PrintId(state->query_id()) << " may have leaked memory."
+          << endl << state->instance_mem_tracker()->LogUsage(MemTracker::UNLIMITED_DEPTH);
       DCHECK_EQ(mem_tracker()->consumption(), 0)
           << "Leaked memory." << endl
           << state->instance_mem_tracker()->LogUsage(MemTracker::UNLIMITED_DEPTH);
@@ -232,7 +232,7 @@ Status ExecNode::ClaimBufferReservation(RuntimeState* state) {
   }
 
   RETURN_IF_ERROR(buffer_pool->RegisterClient(
-      Substitute("$0 id=$1 ptr=$2", PrintPlanNodeType(type_), id_, this),
+      Substitute("$0 id=$1 ptr=$2", PrintThriftEnum(type_), id_, this),
       state->query_state()->file_group(), state->instance_buffer_reservation(),
       mem_tracker(), resource_profile_.max_reservation, runtime_profile(),
       &buffer_pool_client_));

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 81642bf..b2c3779 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -799,23 +799,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 << file_format << "/" << "Unknown" << "(Skipped):" << file_cnt << " ";
+            ss << PrintThriftEnum(file_format) << "/" << "Unknown" << "(Skipped):"
+               << file_cnt << " ";
           } else {
-            ss << file_format << "/" << compressions_set.GetFirstType() << "(Skipped):"
+            ss << PrintThriftEnum(file_format) << "/"
+               << PrintThriftEnum(compressions_set.GetFirstType()) << "(Skipped):"
                << file_cnt << " ";
           }
         } else if (compressions_set.Size() == 1) {
-          ss << file_format << "/" << compressions_set.GetFirstType() << ":" << file_cnt
+          ss << PrintThriftEnum(file_format) << "/"
+             << PrintThriftEnum(compressions_set.GetFirstType()) << ":" << file_cnt
              << " ";
         } else {
-          ss << file_format << "/" << "(";
+          ss << PrintThriftEnum(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 << type;
+            ss << PrintThriftEnum(type);
             first = false;
           }
           ss << "):" << file_cnt << " ";

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index 710a8af..7c64338 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -158,7 +158,7 @@ Status HdfsScanNode::Init(const TPlanNode& tnode, RuntimeState* state) {
     max_materialized_row_batches_ = default_max_row_batches;
   }
   VLOG_QUERY << "Max row batch queue size for scan node '" << id_
-      << "' in fragment instance '" << state->fragment_instance_id()
+      << "' in fragment instance '" << PrintId(state->fragment_instance_id())
       << "': " << max_materialized_row_batches_;
   materialized_row_batches_.reset(new RowBatchQueue(max_materialized_row_batches_));
   return HdfsScanNodeBase::Init(tnode, state);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/parquet-column-readers.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-readers.cc b/be/src/exec/parquet-column-readers.cc
index 1406919..6d7c52f 100644
--- a/be/src/exec/parquet-column-readers.cc
+++ b/be/src/exec/parquet-column-readers.cc
@@ -1224,7 +1224,7 @@ bool BaseScalarColumnReader::NextLevels() {
 Status BaseScalarColumnReader::GetUnsupportedDecodingError() {
   return Status(Substitute(
       "File '$0' is corrupt: unexpected encoding: $1 for data page of column '$2'.",
-      filename(), PrintEncoding(page_encoding_), schema_element().name));
+      filename(), PrintThriftEnum(page_encoding_), schema_element().name));
 }
 
 bool BaseScalarColumnReader::NextPage() {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/exec/parquet-metadata-utils.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-metadata-utils.cc b/be/src/exec/parquet-metadata-utils.cc
index 3fc6b3e..3d05fe6 100644
--- a/be/src/exec/parquet-metadata-utils.cc
+++ b/be/src/exec/parquet-metadata-utils.cc
@@ -154,7 +154,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, PrintEncoding(encodings[i]), schema_element.name));
+          "'$2'.", filename, PrintThriftEnum(encodings[i]), schema_element.name));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/rpc/rpc-trace.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-trace.cc b/be/src/rpc/rpc-trace.cc
index 028f397..425312f 100644
--- a/be/src/rpc/rpc-trace.cc
+++ b/be/src/rpc/rpc-trace.cc
@@ -197,7 +197,7 @@ void* RpcEventHandler::getContext(const char* fn_name, void* server_context) {
   InvocationContext* ctxt_ptr =
       new InvocationContext(MonotonicMillis(), cnxn_ctx, it->second);
   VLOG_RPC << "RPC call: " << string(fn_name) << "(from "
-           << ctxt_ptr->cnxn_ctx->network_address << ")";
+           << TNetworkAddressToString(ctxt_ptr->cnxn_ctx->network_address) << ")";
   return reinterpret_cast<void*>(ctxt_ptr);
 }
 
@@ -207,7 +207,7 @@ void RpcEventHandler::postWrite(void* ctx, const char* fn_name, uint32_t bytes)
   const string& call_name = string(fn_name);
   // TODO: bytes is always 0, how come?
   VLOG_RPC << "RPC call: " << server_name_ << ":" << call_name << " from "
-           << rpc_ctx->cnxn_ctx->network_address << " took "
+           << TNetworkAddressToString(rpc_ctx->cnxn_ctx->network_address) << " took "
            << PrettyPrinter::Print(elapsed_time * 1000L * 1000L, TUnit::TIME_NS);
   MethodDescriptor* descriptor = rpc_ctx->method_descriptor;
   delete rpc_ctx;

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/rpc/thrift-client.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-client.cc b/be/src/rpc/thrift-client.cc
index 1f8d99e..e0852eb 100644
--- a/be/src/rpc/thrift-client.cc
+++ b/be/src/rpc/thrift-client.cc
@@ -67,14 +67,14 @@ Status ThriftClientImpl::Open() {
     try {
       transport_->close();
     } catch (const TException& e) {
-      VLOG(1) << "Error closing socket to: " << address_ << ", ignoring (" << e.what()
-                << ")";
+      VLOG(1) << "Error closing socket to: " << TNetworkAddressToString(address_)
+              << ", ignoring (" << e.what() << ")";
     }
     // In certain cases in which the remote host is overloaded, this failure can
     // happen quite frequently. Let's print this error message without the stack
     // trace as there aren't many callers of this function.
     const string& err_msg = Substitute("Couldn't open transport for $0 ($1)",
-        lexical_cast<string>(address_), e.what());
+        TNetworkAddressToString(address_), e.what());
     VLOG(1) << err_msg;
     return Status::Expected(err_msg);
   }
@@ -91,7 +91,7 @@ Status ThriftClientImpl::OpenWithRetry(uint32_t num_tries, uint64_t wait_ms) {
     Status status = Open();
     if (status.ok()) return status;
 
-    LOG(INFO) << "Unable to connect to " << address_;
+    LOG(INFO) << "Unable to connect to " << TNetworkAddressToString(address_);
     if (num_tries == 0) {
       LOG(INFO) << "(Attempt " << try_count << ", will retry indefinitely)";
     } else {
@@ -109,15 +109,15 @@ void ThriftClientImpl::Close() {
   try {
     if (transport_.get() != NULL && transport_->isOpen()) transport_->close();
   } catch (const TException& e) {
-    LOG(INFO) << "Error closing connection to: " << address_ << ", ignoring (" << e.what()
-              << ")";
+    LOG(INFO) << "Error closing connection to: " << TNetworkAddressToString(address_)
+              << ", ignoring (" << e.what() << ")";
     // Forcibly close the socket (since the transport may have failed to get that far
     // during close())
     try {
       if (socket_.get() != NULL) socket_->close();
     } catch (const TException& e) {
-      LOG(INFO) << "Error closing socket to: " << address_ << ", ignoring (" << e.what()
-                << ")";
+      LOG(INFO) << "Error closing socket to: " << TNetworkAddressToString(address_)
+                << ", ignoring (" << e.what() << ")";
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/rpc/thrift-util.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-util.cc b/be/src/rpc/thrift-util.cc
index b8326f5..c0ba537 100644
--- a/be/src/rpc/thrift-util.cc
+++ b/be/src/rpc/thrift-util.cc
@@ -156,7 +156,7 @@ Status WaitForServer(const string& host, int port, int num_retries,
   return Status("Server did not come up");
 }
 
-std::ostream& operator<<(std::ostream& out, const TColumnValue& colval) {
+void PrintTColumnValue(std::ostream& out, const TColumnValue& colval) {
   if (colval.__isset.bool_val) {
     out << ((colval.bool_val) ? "true" : "false");
   } else if (colval.__isset.double_val) {
@@ -176,7 +176,6 @@ std::ostream& operator<<(std::ostream& out, const TColumnValue& colval) {
   } else {
     out << "NULL";
   }
-  return out;
 }
 
 bool TNetworkAddressComparator(const TNetworkAddress& a, const TNetworkAddress& b) {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/rpc/thrift-util.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-util.h b/be/src/rpc/thrift-util.h
index 24b0b6f..ed95a71 100644
--- a/be/src/rpc/thrift-util.h
+++ b/be/src/rpc/thrift-util.h
@@ -139,7 +139,7 @@ Status WaitForServer(const std::string& host, int port, int num_retries,
    int retry_interval_ms);
 
 /// Print a TColumnValue. If null, print "NULL".
-std::ostream& operator<<(std::ostream& out, const TColumnValue& colval);
+void PrintTColumnValue(std::ostream& out, const TColumnValue& colval);
 
 /// Compares two TNetworkAddresses alphanumerically by their host:port
 /// string representation

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/client-cache.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/client-cache.cc b/be/src/runtime/client-cache.cc
index af530f7..d066d6b 100644
--- a/be/src/runtime/client-cache.cc
+++ b/be/src/runtime/client-cache.cc
@@ -43,7 +43,7 @@ Status ClientCacheHelper::GetClient(const TNetworkAddress& address,
   shared_ptr<PerHostCache> host_cache;
   {
     lock_guard<mutex> lock(cache_lock_);
-    VLOG(2) << "GetClient(" << address << ")";
+    VLOG(2) << "GetClient(" << TNetworkAddressToString(address) << ")";
     shared_ptr<PerHostCache>* ptr = &per_host_caches_[address];
     if (ptr->get() == NULL) ptr->reset(new PerHostCache());
     host_cache = *ptr;
@@ -53,7 +53,8 @@ Status ClientCacheHelper::GetClient(const TNetworkAddress& address,
     lock_guard<mutex> lock(host_cache->lock);
     if (!host_cache->clients.empty()) {
       *client_key = host_cache->clients.front();
-      VLOG(2) << "GetClient(): returning cached client for " << address;
+      VLOG(2) << "GetClient(): returning cached client for " <<
+          TNetworkAddressToString(address);
       host_cache->clients.pop_front();
       if (metrics_enabled_) clients_in_use_metric_->Increment(1);
       return Status::OK();
@@ -78,7 +79,8 @@ Status ClientCacheHelper::ReopenClient(ClientFactory factory_method,
     DCHECK(client != client_map_.end());
     client_impl = client->second;
   }
-  VLOG(1) << "ReopenClient(): re-creating client for " << client_impl->address();
+  VLOG(1) << "ReopenClient(): re-creating client for " <<
+      TNetworkAddressToString(client_impl->address());
 
   client_impl->Close();
 
@@ -109,7 +111,8 @@ Status ClientCacheHelper::ReopenClient(ClientFactory factory_method,
 Status ClientCacheHelper::CreateClient(const TNetworkAddress& address,
     ClientFactory factory_method, ClientKey* client_key) {
   shared_ptr<ThriftClientImpl> client_impl(factory_method(address, client_key));
-  VLOG(2) << "CreateClient(): creating new client for " << client_impl->address();
+  VLOG(2) << "CreateClient(): creating new client for " <<
+      TNetworkAddressToString(client_impl->address());
 
   if (!client_impl->init_status().ok()) {
     *client_key = nullptr;
@@ -145,7 +148,8 @@ void ClientCacheHelper::ReleaseClient(ClientKey* client_key) {
     DCHECK(client != client_map_.end());
     client_impl = client->second;
   }
-  VLOG(2) << "Releasing client for " << client_impl->address() << " back to cache";
+  VLOG(2) << "Releasing client for " << TNetworkAddressToString(client_impl->address())
+      << " back to cache";
   {
     lock_guard<mutex> lock(cache_lock_);
     PerHostCacheMap::iterator cache = per_host_caches_.find(client_impl->address());
@@ -167,7 +171,8 @@ void ClientCacheHelper::DestroyClient(ClientKey* client_key) {
     DCHECK(client != client_map_.end());
     client_impl = client->second;
   }
-  VLOG(1) << "Broken Connection, destroy client for " << client_impl->address();
+  VLOG(1) << "Broken Connection, destroy client for " <<
+      TNetworkAddressToString(client_impl->address());
 
   client_impl->Close();
   if (metrics_enabled_) total_clients_metric_->Increment(-1);
@@ -188,7 +193,7 @@ void ClientCacheHelper::CloseConnections(const TNetworkAddress& address) {
 
   {
     VLOG(2) << "Invalidating all " << cache->clients.size() << " clients for: "
-            << address;
+            << TNetworkAddressToString(address);
     lock_guard<mutex> entry_lock(cache->lock);
     lock_guard<mutex> map_lock(client_map_lock_);
     for (ClientKey client_key: cache->clients) {
@@ -208,7 +213,7 @@ string ClientCacheHelper::DebugString() {
   for (const PerHostCacheMap::value_type& cache: per_host_caches_) {
     lock_guard<mutex> host_cache_lock(cache.second->lock);
     if (!first) out << " ";
-    out << cache.first << ":" << cache.second->clients.size();
+    out << TNetworkAddressToString(cache.first) << ":" << cache.second->clients.size();
     first = false;
   }
   out << "])";

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/coordinator-backend-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index e8db00e..c701f8e 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -58,7 +58,7 @@ void Coordinator::BackendState::Init(
   int prev_fragment_idx = -1;
   for (const FInstanceExecParams* instance_params:
        backend_exec_params_->instance_params) {
-    DCHECK_EQ(host_, instance_params->host);  // all hosts must be the same
+    DCHECK(host_ == instance_params->host);  // all hosts must be the same
     int fragment_idx = instance_params->fragment().idx;
     DCHECK_LT(fragment_idx, fragment_stats.size());
     if (prev_fragment_idx != -1 && fragment_idx != prev_fragment_idx) {
@@ -157,7 +157,8 @@ void Coordinator::BackendState::Exec(
   rpc_params.__set_query_ctx(query_ctx);
   SetRpcParams(debug_options, filter_routing_table, &rpc_params);
   VLOG_FILE << "making rpc: ExecQueryFInstances"
-      << " host=" << impalad_address() << " query_id=" << PrintId(query_id_);
+      << " host=" << TNetworkAddressToString(impalad_address()) << " query_id="
+      << PrintId(query_id_);
 
   // guard against concurrent UpdateBackendExecStatus() that may arrive after RPC returns
   lock_guard<mutex> l(lock_);
@@ -223,8 +224,9 @@ void Coordinator::BackendState::LogFirstInProgress(
   for (Coordinator::BackendState* backend_state : backend_states) {
     lock_guard<mutex> l(backend_state->lock_);
     if (!backend_state->IsDone()) {
-      VLOG_QUERY << "query_id=" << backend_state->query_id_
-                 << ": first in-progress backend: " << backend_state->impalad_address();
+      VLOG_QUERY << "query_id=" << PrintId(backend_state->query_id_)
+                 << ": first in-progress backend: "
+                 << TNetworkAddressToString(backend_state->impalad_address());
       break;
     }
   }
@@ -249,7 +251,7 @@ bool Coordinator::BackendState::ApplyExecStatusReport(
     int instance_idx = GetInstanceIdx(instance_exec_status.fragment_instance_id);
     DCHECK_EQ(instance_stats_map_.count(instance_idx), 1);
     InstanceStats* instance_stats = instance_stats_map_[instance_idx];
-    DCHECK_EQ(instance_stats->exec_params_.instance_id,
+    DCHECK(instance_stats->exec_params_.instance_id ==
         instance_exec_status.fragment_instance_id);
     // Ignore duplicate or out-of-order messages.
     if (instance_stats->done_) continue;
@@ -304,7 +306,8 @@ bool Coordinator::BackendState::ApplyExecStatusReport(
     // Append the log messages from each update with the global state of the query
     // execution
     MergeErrorMaps(backend_exec_status.error_log, &error_log_);
-    VLOG_FILE << "host=" << host_ << " error log: " << PrintErrorMapToString(error_log_);
+    VLOG_FILE << "host=" << TNetworkAddressToString(host_) << " error log: " <<
+        PrintErrorMapToString(error_log_);
   }
 
   // TODO: keep backend-wide stopwatch?
@@ -349,8 +352,8 @@ bool Coordinator::BackendState::Cancel() {
   params.protocol_version = ImpalaInternalServiceVersion::V1;
   params.__set_query_id(query_id_);
   TCancelQueryFInstancesResult dummy;
-  VLOG_QUERY << "sending CancelQueryFInstances rpc for query_id="
-             << query_id_ << " backend=" << TNetworkAddressToString(impalad_address());
+  VLOG_QUERY << "sending CancelQueryFInstances rpc for query_id=" << PrintId(query_id_) <<
+      " backend=" << TNetworkAddressToString(impalad_address());
 
   Status rpc_status;
   Status client_status;
@@ -370,14 +373,14 @@ bool Coordinator::BackendState::Cancel() {
   }
   if (!client_status.ok()) {
     status_.MergeStatus(client_status);
-    VLOG_QUERY << "CancelQueryFInstances query_id= " << query_id_
+    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id_)
                << " failed to connect to " << TNetworkAddressToString(impalad_address())
                << " :" << client_status.msg().msg();
     return true;
   }
   if (!rpc_status.ok()) {
     status_.MergeStatus(rpc_status);
-    VLOG_QUERY << "CancelQueryFInstances query_id= " << query_id_
+    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id_)
                << " rpc to " << TNetworkAddressToString(impalad_address())
                << " failed: " << rpc_status.msg().msg();
     return true;
@@ -386,7 +389,7 @@ bool Coordinator::BackendState::Cancel() {
 }
 
 void Coordinator::BackendState::PublishFilter(const TPublishFilterParams& rpc_params) {
-  DCHECK_EQ(rpc_params.dst_query_id, query_id_);
+  DCHECK(rpc_params.dst_query_id == query_id_);
   {
     // If the backend is already done, it's not waiting for this filter, so we skip
     // sending it in this case.
@@ -412,7 +415,7 @@ Coordinator::BackendState::InstanceStats::InstanceStats(
   : exec_params_(exec_params),
     profile_(nullptr) {
   const string& profile_name = Substitute("Instance $0 (host=$1)",
-      PrintId(exec_params.instance_id), lexical_cast<string>(exec_params.host));
+      PrintId(exec_params.instance_id), TNetworkAddressToString(exec_params.host));
   profile_ = RuntimeProfile::Create(obj_pool, profile_name);
   fragment_stats->root_profile()->AddChild(profile_);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 5a3de5d..d6a70e7 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -90,7 +90,7 @@ Status Coordinator::Exec() {
   const TQueryExecRequest& request = schedule_.request();
   DCHECK(request.plan_exec_info.size() > 0);
 
-  VLOG_QUERY << "Exec() query_id=" << query_id()
+  VLOG_QUERY << "Exec() query_id=" << PrintId(query_id())
              << " stmt=" << request.query_ctx.client_request.stmt;
   stmt_type_ = request.stmt_type;
 
@@ -347,7 +347,7 @@ void Coordinator::StartBackendExec() {
   DebugOptions debug_options(schedule_.query_options());
 
   VLOG_QUERY << "starting execution on " << num_backends << " backends for query_id="
-             << query_id();
+             << PrintId(query_id());
   query_events_->MarkEvent(Substitute("Ready to start on $0 backends", num_backends));
 
   for (BackendState* backend_state: backend_states_) {
@@ -360,7 +360,7 @@ void Coordinator::StartBackendExec() {
 
   exec_complete_barrier_->Wait();
   VLOG_QUERY << "started execution on " << num_backends << " backends for query_id="
-             << query_id();
+             << PrintId(query_id());
   query_events_->MarkEvent(
       Substitute("All $0 execution backends ($1 fragment instances) started",
         num_backends, schedule_.GetNumFragmentInstances()));
@@ -472,10 +472,11 @@ Status Coordinator::UpdateStatus(const Status& status, const string& backend_hos
 
   if (is_fragment_failure) {
     // Log the id of the fragment that first failed so we can track it down more easily.
-    VLOG_QUERY << "query_id=" << query_id() << " failed because fragment_instance_id="
-               << instance_id << " on host=" << backend_hostname << " failed.";
+    VLOG_QUERY << "query_id=" << PrintId(query_id())
+               << " failed because fragment_instance_id=" << PrintId(instance_id)
+               << " on host=" << backend_hostname << " failed.";
   } else {
-    VLOG_QUERY << "query_id=" << query_id() << " failed due to error on host="
+    VLOG_QUERY << "query_id=" << PrintId(query_id()) << " failed due to error on host="
                << backend_hostname;
   }
   return query_status_;
@@ -488,7 +489,7 @@ Status Coordinator::FinalizeHdfsInsert() {
   DCHECK(has_called_wait_);
   DCHECK(finalize_params() != nullptr);
 
-  VLOG_QUERY << "Finalizing query: " << query_id();
+  VLOG_QUERY << "Finalizing query: " << PrintId(query_id());
   SCOPED_TIMER(finalization_timer_);
   Status return_status = GetStatus();
   if (return_status.ok()) {
@@ -520,14 +521,15 @@ Status Coordinator::WaitForBackendCompletion() {
   unique_lock<mutex> l(lock_);
   while (num_remaining_backends_ > 0 && query_status_.ok()) {
     VLOG_QUERY << "Coordinator waiting for backends to finish, "
-               << num_remaining_backends_ << " remaining. query_id=" << query_id();
+               << num_remaining_backends_ << " remaining. query_id="
+               << PrintId(query_id());
     backend_completion_cv_.Wait(l);
   }
   if (query_status_.ok()) {
-    VLOG_QUERY << "All backends finished successfully. query_id=" << query_id();
+    VLOG_QUERY << "All backends finished successfully. query_id=" << PrintId(query_id());
   } else {
     VLOG_QUERY << "All backends finished due to one or more errors. query_id="
-               << query_id() << ". " << query_status_.GetDetail();
+               << PrintId(query_id()) << ". " << query_status_.GetDetail();
   }
 
   return query_status_;
@@ -572,7 +574,7 @@ Status Coordinator::Wait() {
 }
 
 Status Coordinator::GetNext(QueryResultSet* results, int max_rows, bool* eos) {
-  VLOG_ROW << "GetNext() query_id=" << query_id();
+  VLOG_ROW << "GetNext() query_id=" << PrintId(query_id());
   DCHECK(has_called_wait_);
   SCOPED_TIMER(query_profile_->total_time_counter());
 
@@ -626,7 +628,7 @@ void Coordinator::Cancel(const Status* cause) {
 }
 
 void Coordinator::CancelInternal() {
-  VLOG_QUERY << "Cancel() query_id=" << query_id();
+  VLOG_QUERY << "Cancel() query_id=" << PrintId(query_id());
   // TODO: remove when restructuring cancellation, which should happen automatically
   // as soon as the coordinator knows that the query is finished
   DCHECK(!query_status_.ok());
@@ -687,9 +689,9 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
     DCHECK_GT(num_remaining_backends_, 0);
     if (VLOG_QUERY_IS_ON && num_remaining_backends_ > 1) {
       VLOG_QUERY << "Backend completed: "
-          << " host=" << backend_state->impalad_address()
+          << " host=" << TNetworkAddressToString(backend_state->impalad_address())
           << " remaining=" << num_remaining_backends_ - 1
-          << " query_id=" << query_id();
+          << " query_id=" << PrintId(query_id());
       BackendState::LogFirstInProgress(backend_states_);
     }
     if (--num_remaining_backends_ == 0 || !status.ok()) {
@@ -729,7 +731,7 @@ void Coordinator::ComputeQuerySummary() {
 
   stringstream info;
   for (BackendState* backend_state: backend_states_) {
-    info << backend_state->impalad_address() << "("
+    info << TNetworkAddressToString(backend_state->impalad_address()) << "("
          << PrettyPrinter::Print(backend_state->GetPeakConsumption(), TUnit::BYTES)
          << ") ";
   }
@@ -892,7 +894,7 @@ void Coordinator::FilterState::ApplyUpdate(const TUpdateFilterParams& params,
       if (!coord->filter_mem_tracker_->TryConsume(heap_space)) {
         VLOG_QUERY << "Not enough memory to allocate filter: "
                    << PrettyPrinter::Print(heap_space, TUnit::BYTES)
-                   << " (query_id=" << coord->query_id() << ")";
+                   << " (query_id=" << PrintId(coord->query_id()) << ")";
         // Disable, as one missing update means a correct filter cannot be produced.
         Disable(coord->filter_mem_tracker_);
       } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/data-stream-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-mgr.cc b/be/src/runtime/data-stream-mgr.cc
index ed1e29e..8f55f0a 100644
--- a/be/src/runtime/data-stream-mgr.cc
+++ b/be/src/runtime/data-stream-mgr.cc
@@ -82,7 +82,7 @@ shared_ptr<DataStreamRecvrBase> DataStreamMgr::CreateRecvr(const RowDescriptor*
   DCHECK(profile != nullptr);
   DCHECK(parent_tracker != nullptr);
   VLOG_FILE << "creating receiver for fragment_instance_id="
-            << fragment_instance_id << ", node=" << dest_node_id;
+            << PrintId(fragment_instance_id) << ", node=" << dest_node_id;
   shared_ptr<DataStreamRecvr> recvr(new DataStreamRecvr(this, parent_tracker, row_desc,
       fragment_instance_id, dest_node_id, num_senders, is_merging, buffer_size, profile));
   size_t hash_value = GetHashValue(fragment_instance_id, dest_node_id);
@@ -127,9 +127,9 @@ shared_ptr<DataStreamRecvr> DataStreamMgr::FindRecvrOrWait(
   const string& time_taken = PrettyPrinter::Print(sw.ElapsedTime(), TUnit::TIME_NS);
   if (timed_out) {
     LOG(INFO) << "Datastream sender timed-out waiting for recvr for fragment_instance_id="
-              << fragment_instance_id << " (time-out was: " << time_taken << "). "
-              << "Increase --datastream_sender_timeout_ms if you see this message "
-              << "frequently.";
+              << PrintId(fragment_instance_id) << " (time-out was: " << time_taken <<
+              "). Increase --datastream_sender_timeout_ms if you see this message "
+              "frequently.";
   } else {
     VLOG_RPC << "Datastream sender waited for " << time_taken
              << ", and did not time-out.";
@@ -148,7 +148,7 @@ shared_ptr<DataStreamRecvr> DataStreamMgr::FindRecvrOrWait(
 
 shared_ptr<DataStreamRecvr> DataStreamMgr::FindRecvr(
     const TUniqueId& fragment_instance_id, PlanNodeId node_id, bool acquire_lock) {
-  VLOG_ROW << "looking up fragment_instance_id=" << fragment_instance_id
+  VLOG_ROW << "looking up fragment_instance_id=" << PrintId(fragment_instance_id)
            << ", node=" << node_id;
   size_t hash_value = GetHashValue(fragment_instance_id, node_id);
   if (acquire_lock) lock_.lock();
@@ -169,7 +169,7 @@ shared_ptr<DataStreamRecvr> DataStreamMgr::FindRecvr(
 
 Status DataStreamMgr::AddData(const TUniqueId& fragment_instance_id,
     PlanNodeId dest_node_id, const TRowBatch& thrift_batch, int sender_id) {
-  VLOG_ROW << "AddData(): fragment_instance_id=" << fragment_instance_id
+  VLOG_ROW << "AddData(): fragment_instance_id=" << PrintId(fragment_instance_id)
            << " node=" << dest_node_id
            << " size=" << RowBatch::GetDeserializedSize(thrift_batch);
   bool already_unregistered;
@@ -197,7 +197,7 @@ Status DataStreamMgr::AddData(const TUniqueId& fragment_instance_id,
 
 Status DataStreamMgr::CloseSender(const TUniqueId& fragment_instance_id,
     PlanNodeId dest_node_id, int sender_id) {
-  VLOG_FILE << "CloseSender(): fragment_instance_id=" << fragment_instance_id
+  VLOG_FILE << "CloseSender(): fragment_instance_id=" << PrintId(fragment_instance_id)
             << ", node=" << dest_node_id;
   Status status;
   bool already_unregistered;
@@ -243,7 +243,7 @@ Status DataStreamMgr::CloseSender(const TUniqueId& fragment_instance_id,
 
 Status DataStreamMgr::DeregisterRecvr(
     const TUniqueId& fragment_instance_id, PlanNodeId node_id) {
-  VLOG_QUERY << "DeregisterRecvr(): fragment_instance_id=" << fragment_instance_id
+  VLOG_QUERY << "DeregisterRecvr(): fragment_instance_id=" << PrintId(fragment_instance_id)
              << ", node=" << node_id;
   size_t hash_value = GetHashValue(fragment_instance_id, node_id);
   lock_guard<mutex> l(lock_);
@@ -268,7 +268,7 @@ Status DataStreamMgr::DeregisterRecvr(
   }
 
   stringstream err;
-  err << "unknown row receiver id: fragment_instance_id=" << fragment_instance_id
+  err << "unknown row receiver id: fragment_instance_id=" << PrintId(fragment_instance_id)
       << " node_id=" << node_id;
   LOG(ERROR) << err.str();
   return Status(err.str());
@@ -276,7 +276,7 @@ Status DataStreamMgr::DeregisterRecvr(
 
 void DataStreamMgr::Cancel(const TUniqueId& fragment_instance_id) {
   VLOG_QUERY << "cancelling all streams for fragment_instance_id="
-             << fragment_instance_id;
+             << PrintId(fragment_instance_id);
   lock_guard<mutex> l(lock_);
   FragmentRecvrSet::iterator i =
       fragment_recvr_set_.lower_bound(make_pair(fragment_instance_id, 0));
@@ -285,7 +285,7 @@ void DataStreamMgr::Cancel(const TUniqueId& fragment_instance_id) {
     if (recvr.get() == NULL) {
       // keep going but at least log it
       stringstream err;
-      err << "Cancel(): missing in stream_map: fragment_instance_id=" << i->first
+      err << "Cancel(): missing in stream_map: fragment_instance_id=" << PrintId(i->first)
           << " node=" << i->second;
       LOG(ERROR) << err.str();
     } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/data-stream-recvr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-recvr.cc b/be/src/runtime/data-stream-recvr.cc
index cdea4a0..8d9047f 100644
--- a/be/src/runtime/data-stream-recvr.cc
+++ b/be/src/runtime/data-stream-recvr.cc
@@ -112,7 +112,8 @@ Status DataStreamRecvr::SenderQueue::GetBatch(RowBatch** next_batch) {
   unique_lock<mutex> l(lock_);
   // wait until something shows up or we know we're done
   while (!is_cancelled_ && batch_queue_.empty() && num_remaining_senders_ > 0) {
-    VLOG_ROW << "wait arrival fragment_instance_id=" << recvr_->fragment_instance_id()
+    VLOG_ROW << "wait arrival fragment_instance_id="
+             << PrintId(recvr_->fragment_instance_id())
              << " node=" << recvr_->dest_node_id();
     // Don't count time spent waiting on the sender as active time.
     CANCEL_SAFE_SCOPED_TIMER(recvr_->data_arrival_timer_, &is_cancelled_);
@@ -221,7 +222,7 @@ void DataStreamRecvr::SenderQueue::DecrementSenders() {
   DCHECK_GT(num_remaining_senders_, 0);
   num_remaining_senders_ = max(0, num_remaining_senders_ - 1);
   VLOG_FILE << "decremented senders: fragment_instance_id="
-            << recvr_->fragment_instance_id()
+            << PrintId(recvr_->fragment_instance_id())
             << " node_id=" << recvr_->dest_node_id()
             << " #senders=" << num_remaining_senders_;
   if (num_remaining_senders_ == 0) data_arrival_cv_.NotifyOne();
@@ -233,7 +234,7 @@ void DataStreamRecvr::SenderQueue::Cancel() {
     if (is_cancelled_) return;
     is_cancelled_ = true;
     VLOG_QUERY << "cancelled stream: fragment_instance_id_="
-               << recvr_->fragment_instance_id()
+               << PrintId(recvr_->fragment_instance_id())
                << " node_id=" << recvr_->dest_node_id();
   }
   // Wake up all threads waiting to produce/consume batches.  They will all

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/data-stream-sender.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-sender.cc b/be/src/runtime/data-stream-sender.cc
index f68788e..7d0766d 100644
--- a/be/src/runtime/data-stream-sender.cc
+++ b/be/src/runtime/data-stream-sender.cc
@@ -165,8 +165,9 @@ Status DataStreamSender::Channel::Init(RuntimeState* state) {
 }
 
 Status DataStreamSender::Channel::SendBatch(TRowBatch* batch) {
-  VLOG_ROW << "Channel::SendBatch() fragment_instance_id=" << fragment_instance_id_
-           << " dest_node=" << dest_node_id_ << " #rows=" << batch->num_rows;
+  VLOG_ROW << "Channel::SendBatch() fragment_instance_id="
+           << PrintId(fragment_instance_id_) << " dest_node=" << dest_node_id_
+           << " #rows=" << batch->num_rows;
   // return if the previous batch saw an error
   RETURN_IF_ERROR(GetSendStatus());
   {
@@ -193,8 +194,8 @@ void DataStreamSender::Channel::TransmitData(int thread_id, const TRowBatch* bat
 
 void DataStreamSender::Channel::TransmitDataHelper(const TRowBatch* batch) {
   DCHECK(batch != NULL);
-  VLOG_ROW << "Channel::TransmitData() fragment_instance_id=" << fragment_instance_id_
-           << " dest_node=" << dest_node_id_
+  VLOG_ROW << "Channel::TransmitData() fragment_instance_id="
+           << PrintId(fragment_instance_id_) << " dest_node=" << dest_node_id_
            << " #rows=" << batch->num_rows;
   TTransmitDataParams params;
   params.protocol_version = ImpalaInternalServiceVersion::V1;
@@ -276,15 +277,15 @@ Status DataStreamSender::Channel::GetSendStatus() {
   WaitForRpc();
   if (!rpc_status_.ok()) {
     LOG(ERROR) << "channel send to " << TNetworkAddressToString(address_) << " failed "
-               << "(fragment_instance_id=" << fragment_instance_id_ << "): "
+               << "(fragment_instance_id=" << PrintId(fragment_instance_id_) << "): "
                << rpc_status_.GetDetail();
   }
   return rpc_status_;
 }
 
 Status DataStreamSender::Channel::FlushAndSendEos(RuntimeState* state) {
-  VLOG_RPC << "Channel::FlushAndSendEos() fragment_instance_id=" << fragment_instance_id_
-           << " dest_node=" << dest_node_id_
+  VLOG_RPC << "Channel::FlushAndSendEos() fragment_instance_id="
+           << PrintId(fragment_instance_id_) << " dest_node=" << dest_node_id_
            << " #rows= " << batch_->num_rows();
 
   // We can return an error here and not go on to send the EOS RPC because the error that
@@ -314,7 +315,7 @@ Status DataStreamSender::Channel::FlushAndSendEos(RuntimeState* state) {
   rpc_status_ = DoTransmitDataRpc(&client, params, &res);
   if (!rpc_status_.ok()) {
     LOG(ERROR) << "Failed to send EOS to " << TNetworkAddressToString(address_)
-               << " (fragment_instance_id=" << fragment_instance_id_ << "): "
+               << " (fragment_instance_id=" << PrintId(fragment_instance_id_) << "): "
                << rpc_status_.GetDetail();
     return rpc_status_;
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/fragment-instance-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/fragment-instance-state.cc b/be/src/runtime/fragment-instance-state.cc
index 1a0d452..a6ae1ff 100644
--- a/be/src/runtime/fragment-instance-state.cc
+++ b/be/src/runtime/fragment-instance-state.cc
@@ -361,7 +361,7 @@ void FragmentInstanceState::ReportProfileThread() {
     SendReport(false, Status::OK());
   }
 
-  VLOG_FILE << "exiting reporting thread: instance_id=" << instance_id();
+  VLOG_FILE << "exiting reporting thread: instance_id=" << PrintId(instance_id());
 }
 
 void FragmentInstanceState::SendReport(bool done, const Status& status) {
@@ -370,7 +370,7 @@ void FragmentInstanceState::SendReport(bool done, const Status& status) {
 
   if (VLOG_FILE_IS_ON) {
     VLOG_FILE << "Reporting " << (done ? "final " : "") << "profile for instance "
-        << runtime_state_->fragment_instance_id();
+        << PrintId(runtime_state_->fragment_instance_id());
     stringstream ss;
     profile()->PrettyPrint(&ss);
     VLOG_FILE << ss.str();
@@ -554,5 +554,5 @@ void FragmentInstanceState::PrintVolumeIds() {
   profile()->AddInfoString(HdfsScanNodeBase::HDFS_SPLIT_STATS_DESC, str.str());
   VLOG_FILE
       << "Hdfs split stats (<volume id>:<# splits>/<split lengths>) for query="
-      << query_id() << ":\n" << str.str();
+      << PrintId(query_id()) << ":\n" << str.str();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/krpc-data-stream-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/krpc-data-stream-mgr.cc b/be/src/runtime/krpc-data-stream-mgr.cc
index cd8d90b..2aca6c8 100644
--- a/be/src/runtime/krpc-data-stream-mgr.cc
+++ b/be/src/runtime/krpc-data-stream-mgr.cc
@@ -104,7 +104,7 @@ shared_ptr<DataStreamRecvrBase> KrpcDataStreamMgr::CreateRecvr(
   DCHECK(profile != nullptr);
   DCHECK(parent_tracker != nullptr);
   DCHECK(client != nullptr);
-  VLOG_FILE << "creating receiver for fragment_instance_id="<< finst_id
+  VLOG_FILE << "creating receiver for fragment_instance_id="<< PrintId(finst_id)
             << ", node=" << dest_node_id;
   shared_ptr<KrpcDataStreamRecvr> recvr(new KrpcDataStreamRecvr(
       this, parent_tracker, row_desc, finst_id, dest_node_id, num_senders, is_merging,
@@ -149,7 +149,7 @@ shared_ptr<DataStreamRecvrBase> KrpcDataStreamMgr::CreateRecvr(
 
 shared_ptr<KrpcDataStreamRecvr> KrpcDataStreamMgr::FindRecvr(
     const TUniqueId& finst_id, PlanNodeId dest_node_id, bool* already_unregistered) {
-  VLOG_ROW << "looking up fragment_instance_id=" << finst_id
+  VLOG_ROW << "looking up fragment_instance_id=" << PrintId(finst_id)
            << ", node=" << dest_node_id;
   *already_unregistered = false;
   uint32_t hash_value = GetHashValue(finst_id, dest_node_id);
@@ -290,7 +290,7 @@ void KrpcDataStreamMgr::CloseSender(const EndDataStreamRequestPB* request,
 
 Status KrpcDataStreamMgr::DeregisterRecvr(
     const TUniqueId& finst_id, PlanNodeId dest_node_id) {
-  VLOG_QUERY << "DeregisterRecvr(): fragment_instance_id=" << finst_id
+  VLOG_QUERY << "DeregisterRecvr(): fragment_instance_id=" << PrintId(finst_id)
              << ", node=" << dest_node_id;
   uint32_t hash_value = GetHashValue(finst_id, dest_node_id);
   lock_guard<mutex> l(lock_);
@@ -321,7 +321,7 @@ Status KrpcDataStreamMgr::DeregisterRecvr(
 }
 
 void KrpcDataStreamMgr::Cancel(const TUniqueId& finst_id) {
-  VLOG_QUERY << "cancelling all streams for fragment_instance_id=" << finst_id;
+  VLOG_QUERY << "cancelling all streams for fragment_instance_id=" << PrintId(finst_id);
   lock_guard<mutex> l(lock_);
   FragmentRecvrSet::iterator iter =
       fragment_recvr_set_.lower_bound(make_pair(finst_id, 0));

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/krpc-data-stream-recvr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/krpc-data-stream-recvr.cc b/be/src/runtime/krpc-data-stream-recvr.cc
index 6e47bd6..be51f32 100644
--- a/be/src/runtime/krpc-data-stream-recvr.cc
+++ b/be/src/runtime/krpc-data-stream-recvr.cc
@@ -232,7 +232,8 @@ Status KrpcDataStreamRecvr::SenderQueue::GetBatch(RowBatch** next_batch) {
       // is pending insertion so this thread is guaranteed to wake up at some point.
       DCHECK(deferred_rpcs_.empty() ||
           (num_deserialize_tasks_pending_ + num_pending_enqueue_) > 0);
-      VLOG_ROW << "wait arrival fragment_instance_id=" << recvr_->fragment_instance_id()
+      VLOG_ROW << "wait arrival fragment_instance_id="
+               << PrintId(recvr_->fragment_instance_id())
                << " node=" << recvr_->dest_node_id();
       // Don't count time spent waiting on the sender as active time.
       CANCEL_SAFE_SCOPED_TIMER(recvr_->data_wait_timer_, &is_cancelled_);
@@ -534,7 +535,7 @@ void KrpcDataStreamRecvr::SenderQueue::DecrementSenders() {
   DCHECK_GT(num_remaining_senders_, 0);
   num_remaining_senders_ = max(0, num_remaining_senders_ - 1);
   VLOG_FILE << "decremented senders: fragment_instance_id="
-            << recvr_->fragment_instance_id()
+            << PrintId(recvr_->fragment_instance_id())
             << " node_id=" << recvr_->dest_node_id()
             << " #senders=" << num_remaining_senders_;
   if (num_remaining_senders_ == 0) data_arrival_cv_.notify_one();
@@ -555,7 +556,7 @@ void KrpcDataStreamRecvr::SenderQueue::Cancel() {
     }
   }
   VLOG_QUERY << "cancelled stream: fragment_instance_id="
-             << recvr_->fragment_instance_id()
+             << PrintId(recvr_->fragment_instance_id())
              << " node_id=" << recvr_->dest_node_id();
   // Wake up all threads waiting to produce/consume batches. They will all
   // notice that the stream is cancelled and handle it.

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/krpc-data-stream-sender.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/krpc-data-stream-sender.cc b/be/src/runtime/krpc-data-stream-sender.cc
index 0f11dec..cd30f06 100644
--- a/be/src/runtime/krpc-data-stream-sender.cc
+++ b/be/src/runtime/krpc-data-stream-sender.cc
@@ -333,7 +333,7 @@ Status KrpcDataStreamSender::Channel::WaitForRpc(std::unique_lock<SpinLock>* loc
   DCHECK(!rpc_in_flight_);
   if (UNLIKELY(!rpc_status_.ok())) {
     LOG(ERROR) << "channel send to " << TNetworkAddressToString(address_) << " failed: "
-               << "(fragment_instance_id=" << fragment_instance_id_ << "): "
+               << "(fragment_instance_id=" << PrintId(fragment_instance_id_) << "): "
                << rpc_status_.GetDetail();
     return rpc_status_;
   }
@@ -449,8 +449,8 @@ Status KrpcDataStreamSender::Channel::DoTransmitDataRpc() {
 
 Status KrpcDataStreamSender::Channel::TransmitData(
     const OutboundRowBatch* outbound_batch) {
-  VLOG_ROW << "Channel::TransmitData() fragment_instance_id=" << fragment_instance_id_
-           << " dest_node=" << dest_node_id_
+  VLOG_ROW << "Channel::TransmitData() fragment_instance_id="
+           << PrintId(fragment_instance_id_) << " dest_node=" << dest_node_id_
            << " #rows=" << outbound_batch->header()->num_rows();
   std::unique_lock<SpinLock> l(lock_);
   RETURN_IF_ERROR(WaitForRpc(&l));
@@ -529,8 +529,8 @@ Status KrpcDataStreamSender::Channel::DoEndDataStreamRpc() {
 }
 
 Status KrpcDataStreamSender::Channel::FlushAndSendEos(RuntimeState* state) {
-  VLOG_RPC << "Channel::FlushAndSendEos() fragment_instance_id=" << fragment_instance_id_
-           << " dest_node=" << dest_node_id_
+  VLOG_RPC << "Channel::FlushAndSendEos() fragment_instance_id="
+           << PrintId(fragment_instance_id_) << " dest_node=" << dest_node_id_
            << " #rows= " << batch_->num_rows();
 
   // We can return an error here and not go on to send the EOS RPC because the error that
@@ -544,7 +544,7 @@ Status KrpcDataStreamSender::Channel::FlushAndSendEos(RuntimeState* state) {
     DCHECK(rpc_status_.ok());
     if (UNLIKELY(remote_recvr_closed_)) return Status::OK();
     VLOG_RPC << "calling EndDataStream() to terminate channel. fragment_instance_id="
-             << fragment_instance_id_;
+             << PrintId(fragment_instance_id_);
     rpc_in_flight_ = true;
     COUNTER_ADD(parent_->eos_sent_counter_, 1);
     RETURN_IF_ERROR(DoEndDataStreamRpc());

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/mem-tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.cc b/be/src/runtime/mem-tracker.cc
index e5aa290..96c02a2 100644
--- a/be/src/runtime/mem-tracker.cc
+++ b/be/src/runtime/mem-tracker.cc
@@ -196,7 +196,7 @@ MemTracker* MemTracker::CreateQueryMemTracker(const TUniqueId& id,
       ExecEnv::GetInstance()->pool_mem_trackers()->GetRequestPoolMemTracker(
           pool_name, true);
   MemTracker* tracker = obj_pool->Add(new MemTracker(
-      byte_limit, Substitute("Query($0)", lexical_cast<string>(id)), pool_tracker));
+      byte_limit, Substitute("Query($0)", PrintId(id)), pool_tracker));
   tracker->is_query_mem_tracker_ = true;
   tracker->query_id_ = id;
   return tracker;
@@ -370,7 +370,7 @@ Status MemTracker::MemLimitExceeded(RuntimeState* state, const std::string& deta
        << " without exceeding limit." << endl;
   }
   ss << "Error occurred on backend " << GetBackendString();
-  if (state != nullptr) ss << " by fragment " << state->fragment_instance_id();
+  if (state != nullptr) ss << " by fragment " << PrintId(state->fragment_instance_id());
   ss << endl;
   ExecEnv* exec_env = ExecEnv::GetInstance();
   MemTracker* process_tracker = exec_env->process_mem_tracker();

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/query-exec-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/query-exec-mgr.cc b/be/src/runtime/query-exec-mgr.cc
index 967dc4b..2d66f57 100644
--- a/be/src/runtime/query-exec-mgr.cc
+++ b/be/src/runtime/query-exec-mgr.cc
@@ -44,7 +44,7 @@ DEFINE_int32(log_mem_usage_interval, 0, "If non-zero, impalad will output memory
 Status QueryExecMgr::StartQuery(const TExecQueryFInstancesParams& params) {
   TUniqueId query_id = params.query_ctx.query_id;
   VLOG_QUERY << "StartQueryFInstances() query_id=" << PrintId(query_id)
-             << " coord=" << params.query_ctx.coord_address;
+             << " coord=" << TNetworkAddressToString(params.query_ctx.coord_address);
 
   bool dummy;
   QueryState* qs = GetOrCreateQueryState(params.query_ctx, &dummy);
@@ -92,7 +92,7 @@ QueryState* QueryExecMgr::GetQueryState(const TUniqueId& query_id) {
     refcnt = qs->refcnt_.Add(1);
   }
   DCHECK(qs != nullptr && refcnt > 0);
-  VLOG_QUERY << "QueryState: query_id=" << query_id << " refcnt=" << refcnt;
+  VLOG_QUERY << "QueryState: query_id=" << PrintId(query_id) << " refcnt=" << refcnt;
   return qs;
 }
 
@@ -167,7 +167,7 @@ void QueryExecMgr::ReleaseQueryState(QueryState* qs) {
     // someone else might have gc'd the entry
     if (it == map_ref->end()) return;
     qs_from_map = it->second;
-    DCHECK_EQ(qs_from_map->query_ctx().query_id, query_id);
+    DCHECK(qs_from_map->query_ctx().query_id == query_id);
     int32_t cnt = qs_from_map->refcnt_.Load();
     DCHECK_GE(cnt, 0);
     // someone else might have increased the refcnt in the meantime

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/query-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/query-state.cc b/be/src/runtime/query-state.cc
index 04a4283..22616ed 100644
--- a/be/src/runtime/query-state.cc
+++ b/be/src/runtime/query-state.cc
@@ -413,7 +413,7 @@ void QueryState::ExecFInstance(FragmentInstanceState* fis) {
 }
 
 void QueryState::Cancel() {
-  VLOG_QUERY << "Cancel: query_id=" << query_id();
+  VLOG_QUERY << "Cancel: query_id=" << PrintId(query_id());
   (void) instances_prepared_promise_.Get();
   if (!is_cancelled_.CompareAndSwap(0, 1)) return;
   for (auto entry: fis_map_) entry.second->Cancel();

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/runtime-filter-bank.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter-bank.cc b/be/src/runtime/runtime-filter-bank.cc
index 4e23a42..64638a6 100644
--- a/be/src/runtime/runtime-filter-bank.cc
+++ b/be/src/runtime/runtime-filter-bank.cc
@@ -263,7 +263,8 @@ void RuntimeFilterBank::Close() {
   obj_pool_.Clear();
   mem_pool_.FreeAll();
   if (buffer_pool_client_.is_registered()) {
-    VLOG_FILE << "RuntimeFilterBank (Fragment Id: " << state_->fragment_instance_id()
+    VLOG_FILE << "RuntimeFilterBank (Fragment Id: "
+              << PrintId(state_->fragment_instance_id())
               << ") returning reservation " << total_bloom_filter_mem_required_;
     state_->query_state()->initial_reservations()->Return(
         &buffer_pool_client_, total_bloom_filter_mem_required_);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index 4b39ec8..29ea737 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -167,7 +167,7 @@ bool RuntimeState::LogError(const ErrorMsg& message, int vlog_level) {
   // All errors go to the log, unreported_error_count_ is counted independently of the
   // size of the error_log to account for errors that were already reported to the
   // coordinator
-  VLOG(vlog_level) << "Error from query " << query_id() << ": " << message.msg();
+  VLOG(vlog_level) << "Error from query " << PrintId(query_id()) << ": " << message.msg();
   if (ErrorCount(error_log_) < query_options().max_errors) {
     AppendError(&error_log_, message);
     return true;
@@ -239,7 +239,7 @@ void RuntimeState::ReleaseResources() {
 
   // No more memory should be tracked for this instance at this point.
   if (instance_mem_tracker_->consumption() != 0) {
-    LOG(WARNING) << "Query " << query_id() << " may have leaked memory." << endl
+    LOG(WARNING) << "Query " << PrintId(query_id()) << " may have leaked memory." << endl
                  << instance_mem_tracker_->LogUsage(MemTracker::UNLIMITED_DEPTH);
   }
   instance_mem_tracker_->Close();

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/scheduling/admission-controller.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/admission-controller.cc b/be/src/scheduling/admission-controller.cc
index 640a6af..7cdcd02 100644
--- a/be/src/scheduling/admission-controller.cc
+++ b/be/src/scheduling/admission-controller.cc
@@ -505,7 +505,7 @@ Status AdmissionController::AdmitQuery(QuerySchedule* schedule) {
     pool_config_map_[pool_name] = pool_cfg;
     PoolStats* stats = GetPoolStats(pool_name);
     stats->UpdateConfigMetrics(pool_cfg);
-    VLOG_QUERY << "Schedule for id=" << schedule->query_id() << " in pool_name="
+    VLOG_QUERY << "Schedule for id=" << PrintId(schedule->query_id()) << " in pool_name="
                << pool_name << " cluster_mem_needed="
                << PrintBytes(schedule->GetClusterMemoryEstimate())
                << " PoolConfig: max_requests=" << max_requests << " max_queued="
@@ -526,7 +526,7 @@ Status AdmissionController::AdmitQuery(QuerySchedule* schedule) {
 
     if (CanAdmitRequest(*schedule, pool_cfg, false, &not_admitted_reason)) {
       DCHECK_EQ(stats->local_stats().num_queued, 0);
-      VLOG_QUERY << "Admitted query id=" << schedule->query_id();
+      VLOG_QUERY << "Admitted query id=" << PrintId(schedule->query_id());
       stats->Admit(*schedule);
       UpdateHostMemAdmitted(*schedule, schedule->GetPerHostMemoryEstimate());
       schedule->set_is_admitted(true);
@@ -537,7 +537,7 @@ Status AdmissionController::AdmitQuery(QuerySchedule* schedule) {
     }
 
     // We cannot immediately admit but do not need to reject, so queue the request
-    VLOG_QUERY << "Queuing, query id=" << schedule->query_id();
+    VLOG_QUERY << "Queuing, query id=" << PrintId(schedule->query_id());
     stats->Queue(*schedule);
     queue->Enqueue(&queue_node);
   }
@@ -600,7 +600,7 @@ Status AdmissionController::AdmitQuery(QuerySchedule* schedule) {
     schedule->set_is_admitted(true);
     schedule->summary_profile()->AddInfoString(PROFILE_INFO_KEY_ADMISSION_RESULT,
         PROFILE_INFO_VAL_ADMIT_QUEUED);
-    VLOG_QUERY << "Admitted queued query id=" << schedule->query_id();
+    VLOG_QUERY << "Admitted queued query id=" << PrintId(schedule->query_id());
     VLOG_RPC << "Final: " << stats->DebugString();
     return Status::OK();
   }
@@ -615,7 +615,7 @@ void AdmissionController::ReleaseQuery(const QuerySchedule& schedule) {
     stats->Release(schedule);
     UpdateHostMemAdmitted(schedule, -schedule.GetPerHostMemoryEstimate());
     pools_for_updates_.insert(pool_name);
-    VLOG_RPC << "Released query id=" << schedule.query_id() << " "
+    VLOG_RPC << "Released query id=" << PrintId(schedule.query_id()) << " "
              << stats->DebugString();
   }
   dequeue_cv_.NotifyOne();
@@ -875,11 +875,11 @@ void AdmissionController::DequeueLoop() {
         // TODO: Requests further in the queue may be blocked unnecessarily. Consider a
         // better policy once we have better test scenarios.
         if (!CanAdmitRequest(schedule, pool_config, true, &not_admitted_reason)) {
-          VLOG_RPC << "Could not dequeue query id=" << schedule.query_id()
+          VLOG_RPC << "Could not dequeue query id=" << PrintId(schedule.query_id())
                    << " reason: " << not_admitted_reason;
           break;
         }
-        VLOG_RPC << "Dequeuing query=" << schedule.query_id();
+        VLOG_RPC << "Dequeuing query=" << PrintId(schedule.query_id());
         queue.Dequeue();
         stats->Dequeue(schedule, false);
         stats->Admit(schedule);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/scheduling/scheduler.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index 60302d3..a26d175 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -172,7 +172,7 @@ void Scheduler::UpdateMembership(
       // adds the IP address to local_backend_descriptor_. If it is empty, then either
       // that code has been changed, or someone else is sending malformed packets.
       VLOG(1) << "Ignoring subscription request with empty IP address from subscriber: "
-              << be_desc.address;
+              << TNetworkAddressToString(be_desc.address);
       continue;
     }
     if (item.key == local_backend_id_
@@ -181,9 +181,8 @@ void Scheduler::UpdateMembership(
       // will try to re-register (i.e. overwrite their subscription), but there is
       // likely a configuration problem.
       LOG_EVERY_N(WARNING, 30) << "Duplicate subscriber registration from address: "
-                               << be_desc.address
-                               << " (we are: " << local_backend_descriptor_.address
-                               << ")";
+           << TNetworkAddressToString(be_desc.address) << " (we are: "
+           << TNetworkAddressToString(local_backend_descriptor_.address) << ")";
       continue;
     }
     if (be_desc.is_executor) {
@@ -216,7 +215,7 @@ const TBackendDescriptor& Scheduler::LookUpBackendDesc(
   const TBackendDescriptor* desc = executor_config.LookUpBackendDesc(host);
   if (desc == nullptr) {
     // Local host may not be in executor_config if it's a dedicated coordinator.
-    DCHECK_EQ(host, local_backend_descriptor_.address);
+    DCHECK(host == local_backend_descriptor_.address);
     DCHECK(!local_backend_descriptor_.is_executor);
     desc = &local_backend_descriptor_;
   }
@@ -731,7 +730,7 @@ void Scheduler::ComputeBackendExecParams(QuerySchedule* schedule) {
 
   stringstream min_reservation_ss;
   for (const auto& e: per_backend_params) {
-    min_reservation_ss << e.first << "("
+    min_reservation_ss << TNetworkAddressToString(e.first) << "("
          << PrettyPrinter::Print(e.second.min_reservation_bytes, TUnit::BYTES)
          << ") ";
   }
@@ -907,7 +906,8 @@ void Scheduler::AssignmentCtx::RecordScanRangeAssignment(
   scan_range_params_list->push_back(scan_range_params);
 
   if (VLOG_FILE_IS_ON) {
-    VLOG_FILE << "Scheduler assignment to executor: " << executor.address << "("
+    VLOG_FILE << "Scheduler assignment to executor: "
+              << TNetworkAddressToString(executor.address) << "("
               << (remote_read ? "remote" : "local") << " selection)";
   }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/child-query.cc
----------------------------------------------------------------------
diff --git a/be/src/service/child-query.cc b/be/src/service/child-query.cc
index 520a834..94609bf 100644
--- a/be/src/service/child-query.cc
+++ b/be/src/service/child-query.cc
@@ -134,7 +134,8 @@ void ChildQuery::Cancel() {
   Status status = ImpalaServer::THandleIdentifierToTUniqueId(hs2_handle_.operationId,
       &session_id, &secret_unused);
   if (status.ok()) {
-    VLOG_QUERY << "Cancelling and closing child query with operation id: " << session_id;
+    VLOG_QUERY << "Cancelling and closing child query with operation id: " <<
+        PrintId(session_id);
   } else {
     VLOG_QUERY << "Cancelling and closing child query. Failed to get query id: " <<
         status;

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/client-request-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 6d38521..12b9b78 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -91,7 +91,7 @@ ClientRequestState::ClientRequestState(
 
   profile_->set_name("Query (id=" + PrintId(query_id()) + ")");
   summary_profile_->AddInfoString("Session ID", PrintId(session_id()));
-  summary_profile_->AddInfoString("Session Type", PrintTSessionType(session_type()));
+  summary_profile_->AddInfoString("Session Type", PrintThriftEnum(session_type()));
   if (session_type() == TSessionType::HIVESERVER2) {
     summary_profile_->AddInfoString("HiveServer2 Protocol Version",
         Substitute("V$0", 1 + session->hs2_version));
@@ -102,14 +102,14 @@ ClientRequestState::ClientRequestState(
       TimePrecision::Nanosecond));
   summary_profile_->AddInfoString("End Time", "");
   summary_profile_->AddInfoString("Query Type", "N/A");
-  summary_profile_->AddInfoString("Query State", PrintQueryState(BeeswaxQueryState()));
+  summary_profile_->AddInfoString("Query State", PrintThriftEnum(BeeswaxQueryState()));
   summary_profile_->AddInfoString("Query Status", "OK");
   summary_profile_->AddInfoString("Impala Version", GetVersionString(/* compact */ true));
   summary_profile_->AddInfoString("User", effective_user());
   summary_profile_->AddInfoString("Connected User", connected_user());
   summary_profile_->AddInfoString("Delegated User", do_as_user());
   summary_profile_->AddInfoString("Network Address",
-      lexical_cast<string>(session_->network_address));
+      TNetworkAddressToString(session_->network_address));
   summary_profile_->AddInfoString("Default Db", default_db());
   summary_profile_->AddInfoStringRedacted(
       "Sql Statement", query_ctx_.client_request.stmt);
@@ -140,7 +140,7 @@ Status ClientRequestState::Exec(TExecRequest* exec_request) {
   exec_request_ = *exec_request;
 
   profile_->AddChild(server_profile_);
-  summary_profile_->AddInfoString("Query Type", PrintTStmtType(stmt_type()));
+  summary_profile_->AddInfoString("Query Type", PrintThriftEnum(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)",
@@ -489,7 +489,7 @@ Status ClientRequestState::ExecQueryOrDmlRequest(
 
 Status ClientRequestState::ExecDdlRequest() {
   string op_type = catalog_op_type() == TCatalogOpType::DDL ?
-      PrintTDdlType(ddl_type()) : PrintTCatalogOpType(catalog_op_type());
+      PrintThriftEnum(ddl_type()) : PrintThriftEnum(catalog_op_type());
   summary_profile_->AddInfoString("DDL Type", op_type);
 
   if (catalog_op_type() != TCatalogOpType::DDL &&
@@ -575,7 +575,7 @@ void ClientRequestState::Done() {
     uint64_t latest_kudu_ts =
         coord_->dml_exec_state()->GetKuduLatestObservedTimestamp();
     if (latest_kudu_ts > 0) {
-      VLOG_RPC << "Updating session (id=" << session_id()  << ") with latest "
+      VLOG_RPC << "Updating session (id=" << PrintId(session_id())  << ") with latest "
                << "observed Kudu timestamp: " << latest_kudu_ts;
       lock_guard<mutex> session_lock(session_->lock);
       session_->kudu_latest_observed_ts = std::max<uint64_t>(
@@ -599,7 +599,7 @@ void ClientRequestState::Done() {
 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", PrintTStmtType(TStmtType::DDL));
+  summary_profile_->AddInfoString("Query Type", PrintThriftEnum(TStmtType::DDL));
   RETURN_IF_ERROR(frontend_->ExecHiveServer2MetadataOp(exec_request,
       &metadata_op_result));
   result_metadata_ = metadata_op_result.schema;
@@ -926,7 +926,7 @@ Status ClientRequestState::UpdateCatalog() {
     catalog_update.header.__set_requesting_user(effective_user());
     if (!coord()->dml_exec_state()->PrepareCatalogUpdate(&catalog_update)) {
       VLOG_QUERY << "No partitions altered, not updating metastore (query id: "
-                 << query_id() << ")";
+                 << PrintId(query_id()) << ")";
     } else {
       // TODO: We track partitions written to, not created, which means
       // that we do more work than is necessary, because written-to
@@ -1110,7 +1110,7 @@ void ClientRequestState::ClearResultCache() {
 void ClientRequestState::UpdateOperationState(
     TOperationState::type operation_state) {
   operation_state_ = operation_state;
-  summary_profile_->AddInfoString("Query State", PrintQueryState(BeeswaxQueryState()));
+  summary_profile_->AddInfoString("Query State", PrintThriftEnum(BeeswaxQueryState()));
 }
 
 beeswax::QueryState::type ClientRequestState::BeeswaxQueryState() const {

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/impala-beeswax-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index 4875adb..1096677 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -286,7 +286,7 @@ void ImpalaServer::get_log(string& log, const LogContextId& context) {
   shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
   if (request_state.get() == nullptr) {
     stringstream str;
-    str << "unknown query id: " << query_id;
+    str << "unknown query id: " << PrintId(query_id);
     LOG(ERROR) << str.str();
     return;
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/impala-hs2-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 765fccf..36c7169 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -338,8 +338,8 @@ void ImpalaServer::OpenSession(TOpenSessionResp& return_val,
   TQueryOptionsToMap(state->QueryOptions(), &return_val.configuration);
 
   // OpenSession() should return the coordinator's HTTP server address.
-  const string& http_addr = lexical_cast<string>(
-      MakeNetworkAddress(FLAGS_hostname, FLAGS_webserver_port));
+  const string& http_addr = TNetworkAddressToString(MakeNetworkAddress(
+      FLAGS_hostname, FLAGS_webserver_port));
   return_val.configuration.insert(make_pair("http_addr", http_addr));
 
   // Put the session state in session_state_map_

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/impala-http-handler.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 9b8d597..7be4370 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -198,7 +198,7 @@ void ImpalaHttpHandler::CloseSessionHandler(const Webserver::ArgumentMap& args,
     return;
   }
   stringstream ss;
-  ss << "Session " << unique_id << " closed successfully";
+  ss << "Session " << PrintId(unique_id) << " closed successfully";
   Value message(ss.str().c_str(), document->GetAllocator());
   document->AddMember("contents", message, document->GetAllocator());
 }
@@ -250,7 +250,7 @@ void ImpalaHttpHandler::InflightQueryIdsHandler(const Webserver::ArgumentMap& ar
   stringstream ss;
   server_->client_request_state_map_.DoFuncForAllEntries(
       [&](const std::shared_ptr<ClientRequestState>& request_state) {
-          ss << request_state->query_id() << "\n";
+          ss << PrintId(request_state->query_id()) << "\n";
       });
   document->AddMember(Webserver::ENABLE_RAW_JSON_KEY, true, document->GetAllocator());
   Value query_ids(ss.str().c_str(), document->GetAllocator());
@@ -419,7 +419,7 @@ void ImpalaHttpHandler::QueryStateHandler(const Webserver::ArgumentMap& args,
     for (const ImpalaServer::QueryLocations::value_type& location:
          server_->query_locations_) {
       Value location_json(kObjectType);
-      Value location_name(lexical_cast<string>(location.first).c_str(),
+      Value location_name(TNetworkAddressToString(location.first).c_str(),
           document->GetAllocator());
       location_json.AddMember("location", location_name, document->GetAllocator());
       location_json.AddMember("count", static_cast<uint64_t>(location.second.size()),
@@ -440,8 +440,7 @@ void ImpalaHttpHandler::SessionsHandler(const Webserver::ArgumentMap& args,
            server_->session_state_map_) {
     shared_ptr<ImpalaServer::SessionState> state = session.second;
     Value session_json(kObjectType);
-    Value type(PrintTSessionType(state->session_type).c_str(),
-        document->GetAllocator());
+    Value type(PrintThriftEnum(state->session_type).c_str(), document->GetAllocator());
     session_json.AddMember("type", type, document->GetAllocator());
 
     session_json.AddMember("inflight_queries",
@@ -459,7 +458,7 @@ void ImpalaHttpHandler::SessionsHandler(const Webserver::ArgumentMap& args,
     Value session_id(PrintId(session.first).c_str(), document->GetAllocator());
     session_json.AddMember("session_id", session_id, document->GetAllocator());
 
-    Value network_address(lexical_cast<string>(state->network_address).c_str(),
+    Value network_address(TNetworkAddressToString(state->network_address).c_str(),
         document->GetAllocator());
     session_json.AddMember("network_address", network_address, document->GetAllocator());
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/impala-internal-service.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-internal-service.cc b/be/src/service/impala-internal-service.cc
index 5be8765..53a62da 100644
--- a/be/src/service/impala-internal-service.cc
+++ b/be/src/service/impala-internal-service.cc
@@ -41,7 +41,8 @@ ImpalaInternalService::ImpalaInternalService() {
 
 void ImpalaInternalService::ExecQueryFInstances(TExecQueryFInstancesResult& return_val,
     const TExecQueryFInstancesParams& params) {
-  VLOG_QUERY << "ExecQueryFInstances():" << " query_id=" << params.query_ctx.query_id;
+  VLOG_QUERY << "ExecQueryFInstances():" << " query_id=" <<
+      PrintId(params.query_ctx.query_id);
   FAULT_INJECTION_RPC_DELAY(RPC_EXECQUERYFINSTANCES);
   DCHECK(params.__isset.coord_state_idx);
   DCHECK(params.__isset.query_ctx);
@@ -53,14 +54,14 @@ void ImpalaInternalService::ExecQueryFInstances(TExecQueryFInstancesResult& retu
 template <typename T> void SetUnknownIdError(
     const string& id_type, const TUniqueId& id, T* status_container) {
   Status status(ErrorMsg(TErrorCode::INTERNAL_ERROR,
-      Substitute("Unknown $0 id: $1", id_type, lexical_cast<string>(id))));
+      Substitute("Unknown $0 id: $1", id_type, PrintId(id))));
   status.SetTStatus(status_container);
 }
 
 void ImpalaInternalService::CancelQueryFInstances(
     TCancelQueryFInstancesResult& return_val,
     const TCancelQueryFInstancesParams& params) {
-  VLOG_QUERY << "CancelQueryFInstances(): query_id=" << params.query_id;
+  VLOG_QUERY << "CancelQueryFInstances(): query_id=" << PrintId(params.query_id);
   FAULT_INJECTION_RPC_DELAY(RPC_CANCELQUERYFINSTANCES);
   DCHECK(params.__isset.query_id);
   QueryState::ScopedRef qs(params.query_id);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 829ac79..811abab 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -492,16 +492,16 @@ Status ImpalaServer::LogAuditRecord(const ClientRequestState& request_state,
   if (request.stmt_type == TStmtType::DDL) {
     if (request.catalog_op_request.op_type == TCatalogOpType::DDL) {
       writer.String(
-          PrintTDdlType(request.catalog_op_request.ddl_params.ddl_type).c_str());
+          PrintThriftEnum(request.catalog_op_request.ddl_params.ddl_type).c_str());
     } else {
-      writer.String(PrintTCatalogOpType(request.catalog_op_request.op_type).c_str());
+      writer.String(PrintThriftEnum(request.catalog_op_request.op_type).c_str());
     }
   } else {
-    writer.String(PrintTStmtType(request.stmt_type).c_str());
+    writer.String(PrintThriftEnum(request.stmt_type).c_str());
   }
   writer.String("network_address");
-  writer.String(
-      lexical_cast<string>(request_state.session()->network_address).c_str());
+  writer.String(TNetworkAddressToString(
+      request_state.session()->network_address).c_str());
   writer.String("sql_statement");
   string stmt = replace_all_copy(request_state.sql_stmt(), "\n", " ");
   Redact(&stmt);
@@ -513,7 +513,7 @@ Status ImpalaServer::LogAuditRecord(const ClientRequestState& request_state,
     writer.String("name");
     writer.String(event.name.c_str());
     writer.String("object_type");
-    writer.String(PrintTCatalogObjectType(event.object_type).c_str());
+    writer.String(PrintThriftEnum(event.object_type).c_str());
     writer.String("privilege");
     writer.String(event.privilege.c_str());
     writer.EndObject();
@@ -758,7 +758,7 @@ void ImpalaServer::ArchiveQuery(const ClientRequestState& query) {
   // FLAGS_log_query_to_file will have been set to false
   if (FLAGS_log_query_to_file) {
     stringstream ss;
-    ss << UnixMillis() << " " << query.query_id() << " " << encoded_profile_str;
+    ss << UnixMillis() << " " << PrintId(query.query_id()) << " " << encoded_profile_str;
     status = profile_logger_->AppendEntry(ss.str());
     if (!status.ok()) {
       LOG_EVERY_N(WARNING, 1000) << "Could not write to profile log file file ("
@@ -797,7 +797,7 @@ void ImpalaServer::AddPoolConfiguration(TQueryCtx* ctx,
   Status status = exec_env_->request_pool_service()->ResolveRequestPool(*ctx,
       &resolved_pool);
   if (!status.ok()) {
-    VLOG_RPC << "Not adding pool query options for query=" << ctx->query_id
+    VLOG_RPC << "Not adding pool query options for query=" << PrintId(ctx->query_id)
              << " ResolveRequestPool status: " << status.GetDetail();
     return;
   }
@@ -806,7 +806,7 @@ void ImpalaServer::AddPoolConfiguration(TQueryCtx* ctx,
   TPoolConfig config;
   status = exec_env_->request_pool_service()->GetPoolConfig(resolved_pool, &config);
   if (!status.ok()) {
-    VLOG_RPC << "Not adding pool query options for query=" << ctx->query_id
+    VLOG_RPC << "Not adding pool query options for query=" << PrintId(ctx->query_id)
              << " GetConfigPool status: " << status.GetDetail();
     return;
   }
@@ -1012,7 +1012,7 @@ Status ImpalaServer::SetQueryInflight(shared_ptr<SessionState> session_state,
 
 Status ImpalaServer::UnregisterQuery(const TUniqueId& query_id, bool check_inflight,
     const Status* cause) {
-  VLOG_QUERY << "UnregisterQuery(): query_id=" << query_id;
+  VLOG_QUERY << "UnregisterQuery(): query_id=" << PrintId(query_id);
 
   RETURN_IF_ERROR(CancelInternal(query_id, check_inflight, cause));
 
@@ -1206,7 +1206,7 @@ void ImpalaServer::ReportExecStatus(
 
 void ImpalaServer::TransmitData(
     TTransmitDataResult& return_val, const TTransmitDataParams& params) {
-  VLOG_ROW << "TransmitData(): instance_id=" << params.dest_fragment_instance_id
+  VLOG_ROW << "TransmitData(): instance_id=" << PrintId(params.dest_fragment_instance_id)
            << " node_id=" << params.dest_node_id
            << " #rows=" << params.row_batch.num_rows
            << " sender_id=" << params.sender_id
@@ -1306,14 +1306,14 @@ void ImpalaServer::CancelFromThreadPool(uint32_t thread_id,
     Status status = UnregisterQuery(cancellation_work.query_id(), true,
         &cancellation_work.cause());
     if (!status.ok()) {
-      VLOG_QUERY << "Query de-registration (" << cancellation_work.query_id()
+      VLOG_QUERY << "Query de-registration (" << PrintId(cancellation_work.query_id())
                  << ") failed";
     }
   } else {
     Status status = CancelInternal(cancellation_work.query_id(), true,
         &cancellation_work.cause());
     if (!status.ok()) {
-      VLOG_QUERY << "Query cancellation (" << cancellation_work.query_id()
+      VLOG_QUERY << "Query cancellation (" << PrintId(cancellation_work.query_id())
                  << ") did not succeed: " << status.GetDetail();
     }
   }
@@ -1628,7 +1628,7 @@ void ImpalaServer::MembershipCallback(
         stringstream cause_msg;
         cause_msg << "Cancelled due to unreachable impalad(s): ";
         for (int i = 0; i < cancellation_entry->second.size(); ++i) {
-          cause_msg << cancellation_entry->second[i];
+          cause_msg << TNetworkAddressToString(cancellation_entry->second[i]);
           if (i + 1 != cancellation_entry->second.size()) cause_msg << ", ";
         }
         string cause_str = cause_msg.str();
@@ -1793,13 +1793,15 @@ void ImpalaServer::ConnectionEnd(
     connection_to_sessions_map_.erase(it);
   }
 
-  LOG(INFO) << "Connection from client " << connection_context.network_address
-            << " closed, closing " << sessions_to_close.size() << " associated session(s)";
+  LOG(INFO) << "Connection from client "
+            << TNetworkAddressToString(connection_context.network_address)
+            << " closed, closing " << sessions_to_close.size()
+            << " associated session(s)";
 
   for (const TUniqueId& session_id: sessions_to_close) {
     Status status = CloseSessionInternal(session_id, true);
     if (!status.ok()) {
-      LOG(WARNING) << "Error closing session " << session_id << ": "
+      LOG(WARNING) << "Error closing session " << PrintId(session_id) << ": "
                    << status.GetDetail();
     }
   }
@@ -1853,7 +1855,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
         int64_t last_accessed_ms = session_state.second->last_accessed_ms;
         int64_t session_timeout_ms = session_state.second->session_timeout * 1000;
         if (now - last_accessed_ms <= session_timeout_ms) continue;
-        LOG(INFO) << "Expiring session: " << session_state.first << ", user:"
+        LOG(INFO) << "Expiring session: " << PrintId(session_state.first) << ", user:"
                   << session_state.second->connected_user << ", last active: "
                   << ToStringFromUnixMillis(last_accessed_ms);
         session_state.second->expired = true;
@@ -1908,7 +1910,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
         // If the query time limit expired, we must cancel the query.
         if (expiration_event->kind == ExpirationKind::EXEC_TIME_LIMIT) {
           int32_t exec_time_limit_s = query_state->query_options().exec_time_limit_s;
-          VLOG_QUERY << "Expiring query " << expiration_event->query_id
+          VLOG_QUERY << "Expiring query " << PrintId(expiration_event->query_id)
                      << " due to execution time limit of " << exec_time_limit_s << "s.";
           const string& err_msg = Substitute(
               "Query $0 expired due to execution time limit of $1",
@@ -1951,7 +1953,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
           // Otherwise time to expire this query
           VLOG_QUERY
               << "Expiring query due to client inactivity: "
-              << expiration_event->query_id << ", last activity was at: "
+              << PrintId(expiration_event->query_id) << ", last activity was at: "
               << ToStringFromUnixMillis(query_state->last_active_ms());
           const string& err_msg = Substitute(
               "Query $0 expired due to client inactivity (timeout is $1)",
@@ -2026,7 +2028,7 @@ Status ImpalaServer::Start(int32_t thrift_be_port, int32_t beeswax_port,
 
   if (!FLAGS_is_coordinator) {
     LOG(INFO) << "Initialized executor Impala server on "
-              << ExecEnv::GetInstance()->backend_address();
+              << TNetworkAddressToString(ExecEnv::GetInstance()->backend_address());
   } else {
     // Initialize the client servers.
     boost::shared_ptr<ImpalaServer> handler = shared_from_this();
@@ -2084,7 +2086,7 @@ Status ImpalaServer::Start(int32_t thrift_be_port, int32_t beeswax_port,
     }
   }
   LOG(INFO) << "Initialized coordinator/executor Impala server on "
-      << ExecEnv::GetInstance()->backend_address();
+      << TNetworkAddressToString(ExecEnv::GetInstance()->backend_address());
 
   // Start the RPC services.
   RETURN_IF_ERROR(exec_env_->StartKrpcService());
@@ -2141,7 +2143,7 @@ void ImpalaServer::UpdateFilter(TUpdateFilterResult& result,
   shared_ptr<ClientRequestState> client_request_state =
       GetClientRequestState(params.query_id);
   if (client_request_state.get() == nullptr) {
-    LOG(INFO) << "Could not find client request state: " << params.query_id;
+    LOG(INFO) << "Could not find client request state: " << PrintId(params.query_id);
     return;
   }
   client_request_state->coord()->UpdateFilter(params);

http://git-wip-us.apache.org/repos/asf/impala/blob/e114778e/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 c684b4a..ed8d986 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -366,7 +366,7 @@ TEST(QueryOptions, MapOptionalDefaultlessToEmptyString) {
   EXPECT_EQ(map["COMPRESSION_CODEC"], "");
   EXPECT_EQ(map["MT_DOP"], "");
   // Has defaults
-  EXPECT_EQ(map["EXPLAIN_LEVEL"], "1");
+  EXPECT_EQ(map["EXPLAIN_LEVEL"], "STANDARD");
 }
 
 /// Overlay a with b. batch_size is set in both places.