You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2020/02/19 06:19:05 UTC

[impala] 02/02: IMPALA-9373: Trial run of include-what-you-use

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

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

commit 04fd9ae268d89b07e2a692a916bf2ddcfb2e351b
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Mon Feb 10 16:25:19 2020 -0800

    IMPALA-9373: Trial run of include-what-you-use
    
    Implemented recommendations from IWYU in a subset of
    files, mostly in util. Did a few cleanups related to
    systematic problems that I noticed as a result.
    
    I noticed that uid-util.h was pulling in boost UUID headers
    to a lot of compilation units, so refactored that a little
    bit, including pulling out the hash functions into
    unique-id-hash.h and moving some inline functions into
    client-request-state-map.cc.
    
    Systematically replaced the general boost mutex header with the
    internal pthread-based one. This is equivalent for us, since
    we assume that boost::mutex is implemented by pthread_mutex_t,
    e.g. for the implementation of ConditionVariable.
    
    Switch include guards to pragma once just as general cleanup.
    
    Prefix string with std:: consistently in headers so that they
    don't depend on "using" declarations pulled in from random
    headers.
    
    Look at includes of C++ stream headers, including iostream and
    stringstream, and replaced them with iosfwd or removed them
    if possible.
    
    Compile time:
    Measured a full ASAN build of the impalad binary on an 8 core
    machine with cccache enabled, but cleared. It used very slightly
    less CPU, probably because we are still pulling in most of the
    same system headers.
    
    Before:
    real    9m27.502s
    user    64m39.775s
    sys     2m49.002s
    
    After:
    real    9m26.561s
    user    64m28.948s
    sys     2m48.252s
    
    So for the moment, the only significant wins are on incremental
    builds, where touching header files should not require as many
    recompilations. Compile times should start to drop meaningfully
    once we thin out more unnecessary includes - currently it seems
    like most compile units end up with large chunks of boost/std
    code included via transitive header dependencies.
    
    Change-Id: I3450e0ffcb8b183e18ac59c8b33b9ecbd3f60e20
    Reviewed-on: http://gerrit.cloudera.org:8080/15202
    Reviewed-by: Joe McDonnell <jo...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/lock-benchmark.cc                |  6 +--
 be/src/benchmarks/process-wide-locks-benchmark.cc  |  1 +
 be/src/catalog/catalog-server.h                    |  7 +--
 be/src/codegen/llvm-codegen.cc                     |  2 +-
 be/src/common/logging.cc                           | 14 ++---
 be/src/common/names.h                              |  2 +-
 be/src/common/object-pool.h                        |  8 +--
 be/src/exec/blocking-plan-root-sink.cc             |  2 +-
 be/src/exec/data-sink.h                            |  6 +--
 be/src/exec/exec-node.cc                           |  1 +
 be/src/exec/exec-node.h                            | 10 ++--
 be/src/exec/hash-table.h                           |  7 +--
 be/src/exec/hbase-table-scanner.h                  | 12 ++---
 be/src/exec/hdfs-plugin-text-scanner.h             |  6 +--
 be/src/exec/hdfs-scan-node.h                       |  2 +-
 be/src/exec/kudu-scanner.h                         |  2 +-
 be/src/exec/parquet/parquet-column-chunk-reader.h  |  6 +--
 be/src/exec/parquet/parquet-column-stats.inline.h  |  4 +-
 be/src/exec/parquet/parquet-level-decoder.h        |  4 +-
 be/src/exec/parquet/parquet-metadata-utils.h       |  4 +-
 be/src/exec/parquet/parquet-page-reader.h          |  4 +-
 be/src/exec/plan-root-sink.cc                      |  2 +-
 be/src/exec/read-write-util.h                      |  7 +--
 be/src/exec/scanner-context.h                      |  8 +--
 be/src/exec/write-stream.h                         |  7 +--
 be/src/exec/write-stream.inline.h                  |  2 +-
 be/src/experiments/data-provider.h                 | 14 ++---
 be/src/exprs/operators-ir.cc                       |  3 +-
 be/src/rpc/auth-provider.h                         |  7 +--
 be/src/rpc/impala-service-pool.cc                  |  4 +-
 be/src/rpc/thrift-client.h                         |  9 +---
 be/src/rpc/thrift-server.cc                        |  2 +-
 be/src/rpc/thrift-server.h                         |  7 +--
 be/src/runtime/bufferpool/buffer-pool-internal.h   |  9 ++--
 be/src/runtime/bufferpool/buffer-pool.cc           |  1 -
 be/src/runtime/client-cache.h                      | 11 ++--
 be/src/runtime/collection-value-builder.h          |  2 +-
 be/src/runtime/coordinator-backend-state.h         |  7 +--
 be/src/runtime/coordinator-filter-state.h          |  4 +-
 be/src/runtime/coordinator.cc                      |  9 ++--
 be/src/runtime/coordinator.h                       |  7 +--
 be/src/runtime/date-parse-util.h                   |  2 +-
 be/src/runtime/decimal-test.cc                     |  4 +-
 be/src/runtime/decimal-value.inline.h              |  8 +--
 be/src/runtime/descriptors.h                       |  9 +---
 be/src/runtime/dml-exec-state.h                    | 10 ++--
 be/src/runtime/fragment-instance-state.cc          | 17 +++---
 be/src/runtime/fragment-instance-state.h           |  8 +--
 be/src/runtime/hdfs-fs-cache.h                     | 10 ++--
 be/src/runtime/initial-reservations.cc             |  2 +-
 be/src/runtime/io/disk-io-mgr-stress.cc            |  2 +-
 be/src/runtime/io/disk-io-mgr.h                    |  9 ++--
 be/src/runtime/io/handle-cache.h                   |  7 +--
 .../io/local-file-system-with-fault-injection.h    |  8 +--
 be/src/runtime/io/request-ranges.h                 |  7 +--
 be/src/runtime/krpc-data-stream-mgr.h              | 13 +++--
 be/src/runtime/krpc-data-stream-recvr.cc           |  2 +-
 be/src/runtime/krpc-data-stream-recvr.h            |  7 +--
 be/src/runtime/krpc-data-stream-sender.h           |  2 +-
 be/src/runtime/lib-cache.h                         |  8 +--
 be/src/runtime/mem-tracker.h                       |  8 +--
 be/src/runtime/query-exec-mgr.cc                   | 19 ++++---
 be/src/runtime/query-exec-mgr.h                    | 16 ++----
 be/src/runtime/query-state.cc                      |  2 +
 be/src/runtime/query-state.h                       | 26 +++++----
 be/src/runtime/raw-value.h                         |  9 +---
 be/src/runtime/row-batch.h                         |  2 +-
 be/src/runtime/runtime-filter-bank.cc              |  1 +
 be/src/runtime/runtime-filter.h                    |  9 +---
 be/src/runtime/scanner-mem-limiter.h               |  2 +-
 be/src/runtime/sorted-run-merger.h                 |  9 +---
 be/src/runtime/string-search.h                     |  9 +---
 be/src/runtime/thread-resource-mgr.h               |  7 +--
 be/src/runtime/tmp-file-mgr.h                      |  8 +--
 be/src/scheduling/executor-group.h                 |  1 +
 be/src/scheduling/query-schedule.h                 |  4 +-
 be/src/scheduling/scheduler.h                      |  7 +--
 be/src/service/CMakeLists.txt                      |  1 +
 be/src/service/child-query.h                       |  7 +--
 be/src/service/client-request-state-map.cc         | 61 ++++++++++++++++++++++
 be/src/service/client-request-state-map.h          | 34 ++----------
 be/src/service/client-request-state.cc             |  5 ++
 be/src/service/client-request-state.h              |  5 +-
 be/src/service/data-stream-service.cc              |  3 +-
 be/src/service/impala-hs2-server.cc                |  1 +
 be/src/service/impala-http-handler.cc              |  2 +-
 be/src/service/impala-http-handler.h               |  8 +--
 be/src/service/impala-server.cc                    |  1 +
 be/src/service/impala-server.h                     | 17 +++---
 be/src/service/impalad-main.cc                     | 23 ++++----
 be/src/statestore/failure-detector.h               | 11 ++--
 be/src/statestore/statestore-subscriber.cc         |  2 +-
 be/src/statestore/statestore-subscriber.h          | 13 ++---
 be/src/statestore/statestore.h                     |  2 +-
 be/src/testutil/mini-kdc-wrapper.h                 |  8 +--
 be/src/thirdparty/mustache/mustache.cc             |  1 +
 be/src/thirdparty/mustache/mustache.h              |  4 +-
 be/src/transport/TSaslServerTransport.h            |  7 +--
 be/src/udf/uda-test-harness.h                      |  9 +---
 be/src/udf/udf-internal.h                          |  7 +--
 be/src/udf/udf-test-harness.h                      |  7 +--
 be/src/util/CMakeLists.txt                         |  1 +
 be/src/util/auth-util.cc                           | 11 ++--
 be/src/util/auth-util.h                            | 10 ++--
 be/src/util/bit-stream-utils.h                     |  9 ++--
 be/src/util/bit-util.cc                            |  3 ++
 be/src/util/bit-util.h                             | 10 ++--
 be/src/util/blocking-queue-test.cc                 |  5 +-
 be/src/util/blocking-queue.h                       | 13 ++---
 be/src/util/bloom-filter-ir.cc                     |  3 +-
 be/src/util/bloom-filter.cc                        | 13 ++++-
 be/src/util/bloom-filter.h                         | 15 +++---
 be/src/util/buffer-builder.h                       |  8 +--
 be/src/util/cgroup-util.cc                         |  6 ++-
 be/src/util/codec.cc                               |  9 +++-
 be/src/util/codec.h                                | 13 ++---
 be/src/util/coding-util.cc                         |  8 ++-
 be/src/util/coding-util.h                          |  9 ++--
 be/src/util/collection-metrics.cc                  |  8 +++
 be/src/util/collection-metrics.h                   | 18 +++++--
 be/src/util/common-metrics.cc                      |  5 +-
 be/src/util/common-metrics.h                       | 10 ++--
 be/src/util/compress.cc                            |  9 +++-
 be/src/util/compress.h                             |  7 +--
 be/src/util/condition-variable.h                   | 11 ++--
 be/src/util/cpu-info.cc                            | 10 ++--
 be/src/util/cpu-info.h                             |  7 +--
 be/src/util/cyclic-barrier.cc                      |  3 ++
 be/src/util/cyclic-barrier.h                       |  6 ++-
 be/src/util/debug-util.cc                          | 25 ++++++---
 be/src/util/debug-util.h                           | 27 +++++-----
 be/src/util/decimal-util.cc                        |  1 -
 be/src/util/decimal-util.h                         | 13 +++--
 be/src/util/decompress.cc                          |  7 +++
 be/src/util/decompress.h                           |  8 +--
 be/src/util/default-path-handlers.cc               |  1 +
 be/src/util/disk-info.cc                           | 22 ++++----
 be/src/util/disk-info.h                            |  8 ++-
 be/src/util/dynamic-util.cc                        |  1 -
 be/src/util/event-metrics.cc                       | 11 +++-
 be/src/util/event-metrics.h                        | 27 ++++++----
 be/src/util/filesystem-util.cc                     | 27 +++++++---
 be/src/util/flat_buffer.h                          |  6 +--
 be/src/util/hdfs-util.cc                           |  5 +-
 be/src/util/hdfs-util.h                            |  8 ++-
 be/src/util/internal-queue-test.cc                 |  5 +-
 be/src/util/mem-info.h                             |  7 +--
 be/src/util/memory-metrics.h                       |  7 +--
 be/src/util/memusage-path-handlers.cc              |  1 +
 be/src/util/metrics.h                              |  2 +-
 be/src/util/parse-util.h                           |  8 +--
 be/src/util/periodic-counter-updater.h             |  9 +---
 be/src/util/process-state-info.h                   |  8 +--
 be/src/util/progress-updater.h                     |  9 +---
 be/src/util/runtime-profile-counters.h             | 10 ++--
 be/src/util/runtime-profile.h                      |  9 +---
 be/src/util/sharded-query-map-util.h               |  7 +--
 be/src/util/simple-logger.h                        |  6 +--
 be/src/util/stopwatch.h                            |  8 +--
 be/src/util/streaming-sampler.h                    |  7 +--
 be/src/util/string-parser-test.cc                  |  6 +--
 be/src/util/system-state-info.h                    |  1 -
 be/src/util/thread-pool-test.cc                    |  4 +-
 be/src/util/thread-pool.h                          |  8 +--
 be/src/util/uid-util-test.cc                       |  2 +
 be/src/util/{decimal-util.cc => uid-util.cc}       | 15 +++---
 be/src/util/uid-util.h                             | 37 +------------
 .../util/{cyclic-barrier.cc => unique-id-hash.h}   | 36 ++++++++-----
 be/src/util/webserver.h                            |  6 +--
 169 files changed, 653 insertions(+), 747 deletions(-)

diff --git a/be/src/benchmarks/lock-benchmark.cc b/be/src/benchmarks/lock-benchmark.cc
index fbb948f..b29d1c7 100644
--- a/be/src/benchmarks/lock-benchmark.cc
+++ b/be/src/benchmarks/lock-benchmark.cc
@@ -15,13 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdlib.h>
 #include <stdio.h>
+#include <stdlib.h>
 #include <iostream>
-#include <vector>
 #include <sstream>
+#include <vector>
 #include <boost/thread.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include "util/benchmark.h"
 #include "util/cpu-info.h"
 #include "util/spinlock.h"
diff --git a/be/src/benchmarks/process-wide-locks-benchmark.cc b/be/src/benchmarks/process-wide-locks-benchmark.cc
index 373f5b7..42b1f20 100644
--- a/be/src/benchmarks/process-wide-locks-benchmark.cc
+++ b/be/src/benchmarks/process-wide-locks-benchmark.cc
@@ -38,6 +38,7 @@
 #include "util/stopwatch.h"
 #include "util/thread.h"
 #include "util/uid-util.h"
+#include "util/unique-id-hash.h"
 
 #include "common/init.h"
 #include "common/names.h"
diff --git a/be/src/catalog/catalog-server.h b/be/src/catalog/catalog-server.h
index 2cc6b6a..65168d1 100644
--- a/be/src/catalog/catalog-server.h
+++ b/be/src/catalog/catalog-server.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_CATALOG_CATALOG_SERVER_H
-#define IMPALA_CATALOG_CATALOG_SERVER_H
+#pragma once
 
 #include <string>
 #include <vector>
 #include <boost/shared_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_set.hpp>
 
 #include "gen-cpp/CatalogService.h"
@@ -236,5 +235,3 @@ class CatalogServer {
 };
 
 }
-
-#endif
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index c518f0b..705fa48 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -23,7 +23,7 @@
 #include <unordered_set>
 
 #include <boost/algorithm/string.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <gutil/strings/substitute.h>
 
 #include <llvm/ADT/Triple.h>
diff --git a/be/src/common/logging.cc b/be/src/common/logging.cc
index d39d5ab..46407ae 100644
--- a/be/src/common/logging.cc
+++ b/be/src/common/logging.cc
@@ -17,19 +17,19 @@
 
 #include "common/logging.h"
 
-#include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
-#include <boost/uuid/uuid.hpp>
-#include <boost/uuid/uuid_generators.hpp>
-#include <boost/uuid/uuid_io.hpp>
+#include <stdio.h>
 #include <cerrno>
 #include <ctime>
 #include <fstream>
-#include <gutil/strings/substitute.h>
 #include <iostream>
 #include <map>
 #include <sstream>
-#include <stdio.h>
+#include <boost/thread/locks.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+#include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
 #include "service/impala-server.h"
diff --git a/be/src/common/names.h b/be/src/common/names.h
index 54eca6f..9e47726 100644
--- a/be/src/common/names.h
+++ b/be/src/common/names.h
@@ -136,7 +136,7 @@ using boost::thread;
 using boost::thread_group;
 #endif
 
-#ifdef BOOST_THREAD_MUTEX_HPP
+#ifdef BOOST_THREAD_PTHREAD_MUTEX_HPP
 using boost::mutex;
 using boost::timed_mutex;
 using boost::try_mutex;
diff --git a/be/src/common/object-pool.h b/be/src/common/object-pool.h
index 6375d34..fa6d725 100644
--- a/be/src/common/object-pool.h
+++ b/be/src/common/object-pool.h
@@ -15,13 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_COMMON_OBJECT_POOL_H
-#define IMPALA_COMMON_OBJECT_POOL_H
+#pragma once
 
 #include <vector>
 #include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "gutil/macros.h"
 #include "util/spinlock.h"
@@ -67,5 +65,3 @@ class ObjectPool {
 };
 
 }
-
-#endif
diff --git a/be/src/exec/blocking-plan-root-sink.cc b/be/src/exec/blocking-plan-root-sink.cc
index fcff3fd..8e04997 100644
--- a/be/src/exec/blocking-plan-root-sink.cc
+++ b/be/src/exec/blocking-plan-root-sink.cc
@@ -24,7 +24,7 @@
 #include "util/pretty-printer.h"
 
 #include <memory>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 using namespace std;
 using boost::unique_lock;
diff --git a/be/src/exec/data-sink.h b/be/src/exec/data-sink.h
index 1f06625..c8e296c 100644
--- a/be/src/exec/data-sink.h
+++ b/be/src/exec/data-sink.h
@@ -98,8 +98,8 @@ class DataSink {
   /// If this is the sink at the root of a fragment, 'sink_id' must be a unique ID for
   /// the sink for use in runtime profiles and other purposes. Otherwise this is a join
   /// build sink owned by an ExecNode and 'sink_id' must be -1.
-  DataSink(TDataSinkId sink_id, const DataSinkConfig& sink_config, const string& name,
-      RuntimeState* state);
+  DataSink(TDataSinkId sink_id, const DataSinkConfig& sink_config,
+      const std::string& name, RuntimeState* state);
   virtual ~DataSink();
 
   /// Setup. Call before Send(), Open(), or Close() during the prepare phase of the query
@@ -151,7 +151,7 @@ class DataSink {
   const RowDescriptor* row_desc_;
 
   /// The name to be used in profiles etc. Passed by derived classes in the ctor.
-  const string name_;
+  const std::string name_;
 
   /// The runtime profile for this DataSink. Initialized in ctor. Not owned.
   RuntimeProfile* profile_ = nullptr;
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 16f1a50..3b93172 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -65,6 +65,7 @@
 #include "runtime/runtime-state.h"
 #include "util/debug-util.h"
 #include "util/runtime-profile-counters.h"
+#include "util/uid-util.h"
 
 #include "common/names.h"
 
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index 2e7a1cb..e57648e 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -15,12 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_EXEC_EXEC_NODE_H
-#define IMPALA_EXEC_EXEC_NODE_H
-
+#include <iosfwd>
 #include <memory>
-#include <sstream>
 #include <vector>
 
 #include "common/status.h"
@@ -518,6 +516,4 @@ inline bool ExecNode::EvalPredicate(ScalarExprEvaluator* eval, TupleRow* row) {
   if (v.is_null || !v.val) return false;
   return true;
 }
-
-}
-#endif
+} // namespace impala
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index 2d33a65..2bfd68c 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_EXEC_HASH_TABLE_H
-#define IMPALA_EXEC_HASH_TABLE_H
+#pragma once
 
+#include <cstdint>
 #include <memory>
 #include <vector>
-#include <boost/cstdint.hpp>
 #include <boost/scoped_array.hpp>
 #include <boost/scoped_ptr.hpp>
 
@@ -1122,5 +1121,3 @@ class HashTable {
 };
 
 }
-
-#endif
diff --git a/be/src/exec/hbase-table-scanner.h b/be/src/exec/hbase-table-scanner.h
index 0d9abe2..3811a24 100644
--- a/be/src/exec/hbase-table-scanner.h
+++ b/be/src/exec/hbase-table-scanner.h
@@ -15,16 +15,15 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_EXEC_HBASE_TABLE_SCANNER_H
-#define IMPALA_EXEC_HBASE_TABLE_SCANNER_H
+#pragma once
 
-#include <boost/scoped_ptr.hpp>
 #include <jni.h>
+#include <iosfwd>
 #include <string>
-#include <sstream>
 #include <vector>
-#include "gen-cpp/PlanNodes_types.h"
+#include <boost/scoped_ptr.hpp>
 #include "exec/scan-node.h"
+#include "gen-cpp/PlanNodes_types.h"
 #include "runtime/hbase-table-factory.h"
 #include "runtime/hbase-table.h"
 
@@ -318,7 +317,4 @@ class HBaseTableScanner {
   /// formatted data.
   inline void WriteTupleSlot(const SlotDescriptor* slot_desc, Tuple* tuple, void* data);
 };
-
 }  // namespace impala
-
-#endif
diff --git a/be/src/exec/hdfs-plugin-text-scanner.h b/be/src/exec/hdfs-plugin-text-scanner.h
index 4b00165..e86abc0 100644
--- a/be/src/exec/hdfs-plugin-text-scanner.h
+++ b/be/src/exec/hdfs-plugin-text-scanner.h
@@ -15,10 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_EXEC_HDFS_PLUGIN_TEXT_SCANNER_H
-#define IMPALA_EXEC_HDFS_PLUGIN_TEXT_SCANNER_H
+#pragma once
 
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 
 #include "common/status.h"
 #include "exec/scan-node.h"
@@ -90,4 +89,3 @@ class HdfsPluginTextScanner {
   static Status LoadPluginLibrary(const std::string& plugin_name, LoadedPlugin* plugin);
 };
 }
-#endif
diff --git a/be/src/exec/hdfs-scan-node.h b/be/src/exec/hdfs-scan-node.h
index d073262..1ee7e8d 100644
--- a/be/src/exec/hdfs-scan-node.h
+++ b/be/src/exec/hdfs-scan-node.h
@@ -23,7 +23,7 @@
 #include <vector>
 
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/atomic.h"
 #include "exec/filter-context.h"
diff --git a/be/src/exec/kudu-scanner.h b/be/src/exec/kudu-scanner.h
index 011a40b..4c3e5cc 100644
--- a/be/src/exec/kudu-scanner.h
+++ b/be/src/exec/kudu-scanner.h
@@ -94,7 +94,7 @@ class KuduScanner {
   }
 
   /// Builds the error string by adding the PlanNode id and KuduTable to the message.
-  string BuildErrorString(const char* msg);
+  std::string BuildErrorString(const char* msg);
 
   KuduScanNodeBase* scan_node_;
   RuntimeState* state_;
diff --git a/be/src/exec/parquet/parquet-column-chunk-reader.h b/be/src/exec/parquet/parquet-column-chunk-reader.h
index 2d685bb..3dca706 100644
--- a/be/src/exec/parquet/parquet-column-chunk-reader.h
+++ b/be/src/exec/parquet/parquet-column-chunk-reader.h
@@ -64,8 +64,8 @@ class ParquetColumnChunkReader {
   }
 
   /// Moved to implementation to be able to forward declare class in scoped_ptr.
-  ParquetColumnChunkReader(HdfsParquetScanner* parent, string schema_name, int slot_id,
-      ValueMemoryType value_mem_type);
+  ParquetColumnChunkReader(HdfsParquetScanner* parent, std::string schema_name,
+      int slot_id, ValueMemoryType value_mem_type);
   ~ParquetColumnChunkReader();
 
   /// Resets the reader for each row group in the file and creates the scan
@@ -127,7 +127,7 @@ class ParquetColumnChunkReader {
 
  private:
   HdfsParquetScanner* parent_;
-  string schema_name_;
+  std::string schema_name_;
 
   ParquetPageReader page_reader_;
 
diff --git a/be/src/exec/parquet/parquet-column-stats.inline.h b/be/src/exec/parquet/parquet-column-stats.inline.h
index 82368e9..38cef7c 100644
--- a/be/src/exec/parquet/parquet-column-stats.inline.h
+++ b/be/src/exec/parquet/parquet-column-stats.inline.h
@@ -147,7 +147,7 @@ namespace {
 /// any validation.
 /// Used as a helper function for decoding values that need additional custom validation.
 template <typename T, parquet::Type::type ParquetType>
-inline bool DecodePlainValueNoValidation(const string& buffer, T* result) {
+inline bool DecodePlainValueNoValidation(const std::string& buffer, T* result) {
   int size = buffer.size();
   const uint8_t* data = reinterpret_cast<const uint8_t*>(buffer.data());
   return ParquetPlainEncoder::Decode<T, ParquetType>(
@@ -192,7 +192,7 @@ inline bool ColumnStats<DateValue>::DecodePlainValue(
 /// parquet::Statistics stores string values directly and does not use plain encoding.
 template <>
 inline void ColumnStats<StringValue>::EncodePlainValue(
-    const StringValue& v, int64_t bytes_needed, string* out) {
+    const StringValue& v, int64_t bytes_needed, std::string* out) {
   out->assign(v.ptr, v.len);
 }
 
diff --git a/be/src/exec/parquet/parquet-level-decoder.h b/be/src/exec/parquet/parquet-level-decoder.h
index 58bda02..f8d26a9 100644
--- a/be/src/exec/parquet/parquet-level-decoder.h
+++ b/be/src/exec/parquet/parquet-level-decoder.h
@@ -52,7 +52,7 @@ class ParquetLevelDecoder {
   /// Initialize the LevelDecoder. Reads and advances the provided data buffer if the
   /// encoding requires reading metadata from the page header. 'cache_size' will be
   /// rounded up to a multiple of 32 internally.
-  Status Init(const string& filename, const parquet::Encoding::type* encoding,
+  Status Init(const std::string& filename, const parquet::Encoding::type* encoding,
       MemPool* cache_pool, int cache_size, int max_level, uint8_t** data, int* data_size);
 
   /// Returns the next level or INVALID_LEVEL if there was an error. Not as efficient
@@ -140,7 +140,7 @@ class ParquetLevelDecoder {
   int cache_size_ = 0;
 
   /// Name of the parquet file. Used for reporting level decoding errors.
-  string filename_;
+  std::string filename_;
 
   /// Error code to use when reporting level decoding errors.
   TErrorCode::type decoding_error_code_;
diff --git a/be/src/exec/parquet/parquet-metadata-utils.h b/be/src/exec/parquet/parquet-metadata-utils.h
index 4639abf..afea3f5 100644
--- a/be/src/exec/parquet/parquet-metadata-utils.h
+++ b/be/src/exec/parquet/parquet-metadata-utils.h
@@ -37,7 +37,7 @@ class ParquetMetadataUtils {
 
   /// Validate column offsets by checking if the dictionary page comes before the data
   /// pages and checking if the column offsets lie within the file.
-  static Status ValidateColumnOffsets(const string& filename, int64_t file_length,
+  static Status ValidateColumnOffsets(const std::string& filename, int64_t file_length,
       const parquet::RowGroup& row_group);
 
   /// Check that a file offset is in the file. Return an error status with a detailed
@@ -215,7 +215,7 @@ class ParquetSchemaResolver {
   /// found. The name comparison is case-insensitive because that's how Impala treats
   /// db/table/column/field names. If there are several matches with different casing,
   /// then the index of the first match is returned.
-  int FindChildWithName(SchemaNode* node, const string& name) const;
+  int FindChildWithName(SchemaNode* node, const std::string& name) const;
 
   /// The ResolvePathHelper() logic for arrays.
   Status ResolveArray(ArrayEncoding array_encoding, const SchemaPath& path, int idx,
diff --git a/be/src/exec/parquet/parquet-page-reader.h b/be/src/exec/parquet/parquet-page-reader.h
index b98e94a..fca29e6 100644
--- a/be/src/exec/parquet/parquet-page-reader.h
+++ b/be/src/exec/parquet/parquet-page-reader.h
@@ -31,7 +31,7 @@ class ParquetPageReader {
   ScannerContext::Stream* stream() const { return stream_; }
   uint64_t PageHeadersRead() const { return page_headers_read_; }
 
-  ParquetPageReader(HdfsParquetScanner* parent, string schema_name)
+  ParquetPageReader(HdfsParquetScanner* parent, std::string schema_name)
       : parent_(parent),
         schema_name_(schema_name)
   {
@@ -72,7 +72,7 @@ class ParquetPageReader {
   Status AdvanceStream(int64_t bytes);
 
   HdfsParquetScanner* parent_;
-  string schema_name_;
+  std::string schema_name_;
 
   /// Header for current data page.
   parquet::PageHeader current_page_header_;
diff --git a/be/src/exec/plan-root-sink.cc b/be/src/exec/plan-root-sink.cc
index bce6324..fc506ef 100644
--- a/be/src/exec/plan-root-sink.cc
+++ b/be/src/exec/plan-root-sink.cc
@@ -27,7 +27,7 @@
 #include "util/pretty-printer.h"
 
 #include <memory>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 using namespace std;
 using boost::unique_lock;
diff --git a/be/src/exec/read-write-util.h b/be/src/exec/read-write-util.h
index ecbebaf..d9e9e8e 100644
--- a/be/src/exec/read-write-util.h
+++ b/be/src/exec/read-write-util.h
@@ -15,11 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_EXEC_READ_WRITE_UTIL_H
-#define IMPALA_EXEC_READ_WRITE_UTIL_H
-
-#include <boost/cstdint.hpp>
+#include <cstdint>
 #include <sstream>
 #include "common/logging.h"
 #include "common/status.h"
@@ -327,4 +325,3 @@ inline int ReadWriteUtil::DecodeVIntSize(int8_t byte) {
 }
 
 }
-#endif
diff --git a/be/src/exec/scanner-context.h b/be/src/exec/scanner-context.h
index 9c93c38..f4575ee 100644
--- a/be/src/exec/scanner-context.h
+++ b/be/src/exec/scanner-context.h
@@ -15,13 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_EXEC_SCANNER_CONTEXT_H
-#define IMPALA_EXEC_SCANNER_CONTEXT_H
-
+#include <cstdint>
 #include <deque>
 
-#include <boost/cstdint.hpp>
 #include <boost/scoped_ptr.hpp>
 
 #include "common/compiler-util.h"
@@ -439,5 +437,3 @@ class ScannerContext {
 };
 
 }
-
-#endif
diff --git a/be/src/exec/write-stream.h b/be/src/exec/write-stream.h
index d592a09..17609f0 100644
--- a/be/src/exec/write-stream.h
+++ b/be/src/exec/write-stream.h
@@ -15,11 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_EXEC_WRITE_STREAM_H
-#define IMPALA_EXEC_WRITE_STREAM_H
-
-#include <boost/cstdint.hpp>
+#include <cstdint>
 
 #include "common/status.h"
 
@@ -65,4 +63,3 @@ class WriteStream {
 };
 
 } // namespace impala
-#endif
diff --git a/be/src/exec/write-stream.inline.h b/be/src/exec/write-stream.inline.h
index 80c3459..f3b09a2 100644
--- a/be/src/exec/write-stream.inline.h
+++ b/be/src/exec/write-stream.inline.h
@@ -90,7 +90,7 @@ inline int WriteStream::WriteBoolean(bool b) {
   return WriteBytes(1, &val);
 }
 
-inline string WriteStream::String() {
+inline std::string WriteStream::String() {
   return buffer_.str();
 }
 
diff --git a/be/src/experiments/data-provider.h b/be/src/experiments/data-provider.h
index fe0b2a9..7a183c9 100644
--- a/be/src/experiments/data-provider.h
+++ b/be/src/experiments/data-provider.h
@@ -15,20 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_EXPERIMENT_DATAPROVIDER_H
-#define IMPALA_EXPERIMENT_DATAPROVIDER_H
+#pragma once
 
 #include <math.h>
-#include <iostream>
+#include <cstdint>
+#include <iosfwd>
 #include <limits>
-#include <boost/cstdint.hpp>
-#include <boost/scoped_ptr.hpp>
-#include <boost/random/uniform_int.hpp>
+#include <boost/generator_iterator.hpp>
 #include <boost/random/linear_congruential.hpp>
 #include <boost/random/uniform_int.hpp>
 #include <boost/random/uniform_real.hpp>
 #include <boost/random/variate_generator.hpp>
-#include <boost/generator_iterator.hpp>
+#include <boost/scoped_ptr.hpp>
 
 #include "runtime/mem-pool.h"
 #include "runtime/types.h"
@@ -247,5 +245,3 @@ template<> inline float DataProvider::ColDesc::Generate<float>(double d, int i)
 template<> inline double DataProvider::ColDesc::Generate<double>(double d, int i) const {
   return Generate<double>(d, i, min.d, max.d);
 }
-
-#endif
diff --git a/be/src/exprs/operators-ir.cc b/be/src/exprs/operators-ir.cc
index 6bd2942..17f2992 100644
--- a/be/src/exprs/operators-ir.cc
+++ b/be/src/exprs/operators-ir.cc
@@ -17,10 +17,9 @@
 
 #include "exprs/operators.h"
 
+#include <cstdint>
 #include <functional>
 
-#include <boost/cstdint.hpp>
-
 #include "exprs/anyval-util.h"
 #include "gutil/strings/substitute.h"
 #include "runtime/string-value.inline.h"
diff --git a/be/src/rpc/auth-provider.h b/be/src/rpc/auth-provider.h
index 72b2763..b3026fc 100644
--- a/be/src/rpc/auth-provider.h
+++ b/be/src/rpc/auth-provider.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RPC_AUTH_PROVIDER_H
-#define IMPALA_RPC_AUTH_PROVIDER_H
+#pragma once
 
 #include <string>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <sasl/sasl.h>
 
 #include "common/status.h"
@@ -214,5 +213,3 @@ class NoAuthProvider : public AuthProvider {
 Status InitAuth(const std::string& appname);
 
 }
-
-#endif
diff --git a/be/src/rpc/impala-service-pool.cc b/be/src/rpc/impala-service-pool.cc
index a4f28cb..304dfe9 100644
--- a/be/src/rpc/impala-service-pool.cc
+++ b/be/src/rpc/impala-service-pool.cc
@@ -17,11 +17,11 @@
 
 #include "rpc/impala-service-pool.h"
 
-#include <boost/thread/mutex.hpp>
-#include <glog/logging.h>
 #include <memory>
 #include <string>
 #include <vector>
+#include <boost/thread/pthread/mutex.hpp>
+#include <glog/logging.h>
 
 #include "exec/kudu-util.h"
 #include "gutil/strings/numbers.h"
diff --git a/be/src/rpc/thrift-client.h b/be/src/rpc/thrift-client.h
index 8194016..89d5460 100644
--- a/be/src/rpc/thrift-client.h
+++ b/be/src/rpc/thrift-client.h
@@ -15,12 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_RPC_THRIFT_CLIENT_H
-#define IMPALA_RPC_THRIFT_CLIENT_H
-
-#include <ostream>
-#include <sstream>
 #include <boost/shared_ptr.hpp>
 #include <common/status.h>
 #include <thrift/Thrift.h>
@@ -28,7 +24,6 @@
 #include <thrift/transport/TSSLSocket.h>
 #include <thrift/transport/TBufferTransports.h>
 #include <thrift/protocol/TBinaryProtocol.h>
-#include <sstream>
 #include <gflags/gflags.h>
 
 #include "transport/TSaslClientTransport.h"
@@ -161,6 +156,4 @@ ThriftClient<InterfaceType>::ThriftClient(const std::string& ipaddress, int port
   protocol_.reset(new apache::thrift::protocol::TBinaryProtocol(transport_));
   iface_.reset(new InterfaceType(protocol_));
 }
-
 }
-#endif
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index d4c8c86..5dab197 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -18,7 +18,7 @@
 #include <boost/algorithm/string.hpp>
 #include <boost/filesystem.hpp>
 #include <boost/thread.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/uuid/uuid_io.hpp>
 
 #include <openssl/err.h>
diff --git a/be/src/rpc/thrift-server.h b/be/src/rpc/thrift-server.h
index 8c43a37..adf1b63 100644
--- a/be/src/rpc/thrift-server.h
+++ b/be/src/rpc/thrift-server.h
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RPC_THRIFT_SERVER_H
-#define IMPALA_RPC_THRIFT_SERVER_H
+#pragma once
 
 #include <boost/scoped_ptr.hpp>
 #include <boost/shared_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/uuid/uuid_generators.hpp>
 #include <thrift/TProcessor.h>
@@ -488,5 +487,3 @@ struct SSLProtoVersions {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/bufferpool/buffer-pool-internal.h b/be/src/runtime/bufferpool/buffer-pool-internal.h
index 4cd8cf9..a0aa906 100644
--- a/be/src/runtime/bufferpool/buffer-pool-internal.h
+++ b/be/src/runtime/bufferpool/buffer-pool-internal.h
@@ -72,13 +72,12 @@
 /// invariant can be increased. Operations block waiting for enough writes to complete
 /// to satisfy the invariant.
 
-#ifndef IMPALA_RUNTIME_BUFFER_POOL_INTERNAL_H
-#define IMPALA_RUNTIME_BUFFER_POOL_INTERNAL_H
+#pragma once
 
+#include <iosfwd>
 #include <memory>
-#include <sstream>
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "runtime/bufferpool/buffer-pool-counters.h"
 #include "runtime/bufferpool/buffer-pool.h"
@@ -391,5 +390,3 @@ class BufferPool::Client {
   PageList in_flight_write_pages_;
 };
 }
-
-#endif
diff --git a/be/src/runtime/bufferpool/buffer-pool.cc b/be/src/runtime/bufferpool/buffer-pool.cc
index 4b70f7a..e36dfb8 100644
--- a/be/src/runtime/bufferpool/buffer-pool.cc
+++ b/be/src/runtime/bufferpool/buffer-pool.cc
@@ -29,7 +29,6 @@
 #include "util/metrics.h"
 #include "util/runtime-profile-counters.h"
 #include "util/time.h"
-#include "util/uid-util.h"
 
 DEFINE_int32(concurrent_scratch_ios_per_device, 2,
     "Set this to influence the number of concurrent write I/Os issues to write data to "
diff --git a/be/src/runtime/client-cache.h b/be/src/runtime/client-cache.h
index b41c9b1..191b578 100644
--- a/be/src/runtime/client-cache.h
+++ b/be/src/runtime/client-cache.h
@@ -15,15 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_CLIENT_CACHE_H
-#define IMPALA_RUNTIME_CLIENT_CACHE_H
+#pragma once
 
-#include <vector>
 #include <list>
 #include <string>
-#include <boost/unordered_map.hpp>
-#include <boost/thread/mutex.hpp>
+#include <vector>
 #include <boost/bind.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+#include <boost/unordered_map.hpp>
 #include <gutil/strings/substitute.h>
 
 #include "catalog/catalog-service-client-wrapper.h"
@@ -492,5 +491,3 @@ class ClientCache {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/collection-value-builder.h b/be/src/runtime/collection-value-builder.h
index c98816b..ba9ddd6 100644
--- a/be/src/runtime/collection-value-builder.h
+++ b/be/src/runtime/collection-value-builder.h
@@ -67,7 +67,7 @@ class CollectionValueBuilder {
         if (UNLIKELY(new_buf == NULL)) {
           *tuple_mem = NULL;
           *num_tuples = 0;
-          string path = tuple_desc_.table_desc() == NULL ? "" :
+          std::string path = tuple_desc_.table_desc() == NULL ? "" :
               PrintPath(*tuple_desc_.table_desc(), tuple_desc_.tuple_path());
           return pool_->mem_tracker()->MemLimitExceeded(state_,
               ErrorMsg(TErrorCode::COLLECTION_ALLOC_FAILED, new_buffer_size,
diff --git a/be/src/runtime/coordinator-backend-state.h b/be/src/runtime/coordinator-backend-state.h
index f1de695..033c006 100644
--- a/be/src/runtime/coordinator-backend-state.h
+++ b/be/src/runtime/coordinator-backend-state.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_COORDINATOR_BACKEND_STATE_H
-#define IMPALA_RUNTIME_COORDINATOR_BACKEND_STATE_H
+#pragma once
 
 #include <vector>
 #include <unordered_set>
@@ -28,7 +27,7 @@
 #include <boost/accumulators/statistics/min.hpp>
 #include <boost/accumulators/statistics/stats.hpp>
 #include <boost/accumulators/statistics/variance.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "gen-cpp/control_service.proxy.h"
 #include "kudu/rpc/rpc_controller.h"
@@ -548,5 +547,3 @@ class Coordinator::BackendResourceState {
   FRIEND_TEST(CoordinatorBackendStateTest, StateMachine);
 };
 }
-
-#endif
diff --git a/be/src/runtime/coordinator-filter-state.h b/be/src/runtime/coordinator-filter-state.h
index cb018b8..9856022 100644
--- a/be/src/runtime/coordinator-filter-state.h
+++ b/be/src/runtime/coordinator-filter-state.h
@@ -69,7 +69,7 @@ class Coordinator::FilterState {
   }
 
   BloomFilterPB& bloom_filter() { return bloom_filter_; }
-  string& bloom_filter_directory() { return bloom_filter_directory_; }
+  std::string& bloom_filter_directory() { return bloom_filter_directory_; }
   MinMaxFilterPB& min_max_filter() { return min_max_filter_; }
   std::vector<FilterTarget>* targets() { return &targets_; }
   const std::vector<FilterTarget>& targets() const { return targets_; }
@@ -137,7 +137,7 @@ class Coordinator::FilterState {
   BloomFilterPB bloom_filter_;
   /// When the filter is a Bloom filter, we use this string to store the contents of the
   /// aggregated Bloom filter.
-  string bloom_filter_directory_;
+  std::string bloom_filter_directory_;
   MinMaxFilterPB min_max_filter_;
 
   /// Time at which first local filter arrived.
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 07cfadc..ca24d9b 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -34,17 +34,17 @@
 #include "gen-cpp/ImpalaInternalService_constants.h"
 #include "kudu/rpc/rpc_context.h"
 #include "kudu/rpc/rpc_sidecar.h"
+#include "runtime/coordinator-backend-state.h"
+#include "runtime/coordinator-filter-state.h"
+#include "runtime/debug-options.h"
 #include "runtime/exec-env.h"
 #include "runtime/fragment-instance-state.h"
 #include "runtime/hdfs-fs-cache.h"
 #include "runtime/query-exec-mgr.h"
-#include "runtime/coordinator-filter-state.h"
-#include "runtime/coordinator-backend-state.h"
-#include "runtime/debug-options.h"
 #include "runtime/query-state.h"
 #include "scheduling/admission-controller.h"
-#include "scheduling/scheduler.h"
 #include "scheduling/query-schedule.h"
+#include "scheduling/scheduler.h"
 #include "service/client-request-state.h"
 #include "util/bloom-filter.h"
 #include "util/hdfs-bulk-ops.h"
@@ -53,6 +53,7 @@
 #include "util/min-max-filter.h"
 #include "util/pretty-printer.h"
 #include "util/table-printer.h"
+#include "util/uid-util.h"
 
 #include "common/names.h"
 
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index 019f3fb..5d57dbf 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_COORDINATOR_H
-#define IMPALA_RUNTIME_COORDINATOR_H
+#pragma once
 
 #include <memory>
 #include <string>
 #include <vector>
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <rapidjson/document.h>
 
@@ -587,5 +586,3 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/date-parse-util.h b/be/src/runtime/date-parse-util.h
index a6174a7..731e8e7 100644
--- a/be/src/runtime/date-parse-util.h
+++ b/be/src/runtime/date-parse-util.h
@@ -62,7 +62,7 @@ class DateParser {
   /// Format the date values using the given format context.
   /// dt_ctx -- date format context
   /// date -- the date value
-  static string Format(const datetime_parse_util::DateTimeFormatContext& dt_ctx,
+  static std::string Format(const datetime_parse_util::DateTimeFormatContext& dt_ctx,
       const DateValue& date);
 
  private:
diff --git a/be/src/runtime/decimal-test.cc b/be/src/runtime/decimal-test.cc
index 87d3652..4d4bad3 100644
--- a/be/src/runtime/decimal-test.cc
+++ b/be/src/runtime/decimal-test.cc
@@ -15,12 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdlib.h>
 #include <stdio.h>
+#include <stdlib.h>
+#include <cstdint>
 #include <iostream>
 #include <limits>
 #include <sstream>
-#include <boost/cstdint.hpp>
 #include <boost/lexical_cast.hpp>
 #include "runtime/decimal-value.inline.h"
 #include "runtime/raw-value.h"
diff --git a/be/src/runtime/decimal-value.inline.h b/be/src/runtime/decimal-value.inline.h
index b2d5fc0..ea1bee2 100644
--- a/be/src/runtime/decimal-value.inline.h
+++ b/be/src/runtime/decimal-value.inline.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_DECIMAL_VALUE_INLINE_H
-#define IMPALA_RUNTIME_DECIMAL_VALUE_INLINE_H
+#pragma once
 
 #include "runtime/decimal-value.h"
 
@@ -24,8 +23,6 @@
 #include <functional>
 #include <iomanip>
 #include <limits>
-#include <ostream>
-#include <sstream>
 
 #include "common/logging.h"
 #include "util/arithmetic-util.h"
@@ -792,7 +789,4 @@ inline Decimal8Value ToDecimal8(const Decimal16Value& v, bool* overflow) {
 inline Decimal16Value ToDecimal16(const Decimal16Value& v, bool* overflow) {
   return v;
 }
-
 }
-
-#endif
diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h
index fd70f3b..a54e4c6 100644
--- a/be/src/runtime/descriptors.h
+++ b/be/src/runtime/descriptors.h
@@ -15,14 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_RUNTIME_DESCRIPTORS_H
-#define IMPALA_RUNTIME_DESCRIPTORS_H
-
+#include <iosfwd>
 #include <unordered_map>
 #include <vector>
 #include <boost/scoped_ptr.hpp>
-#include <ostream>
 
 #include "codegen/impala-ir.h"
 #include "common/global-types.h"
@@ -631,7 +629,4 @@ class RowDescriptor {
   /// Provide quick way to check if there are variable length slots.
   bool has_varlen_slots_;
 };
-
 }
-
-#endif
diff --git a/be/src/runtime/dml-exec-state.h b/be/src/runtime/dml-exec-state.h
index e601be4..c01cf8d 100644
--- a/be/src/runtime/dml-exec-state.h
+++ b/be/src/runtime/dml-exec-state.h
@@ -15,14 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_RUNTIME_DML_EXEC_STATE_H
-#define IMPALA_RUNTIME_DML_EXEC_STATE_H
-
-#include <string>
 #include <map>
+#include <string>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
-#include <boost/thread/mutex.hpp>
 
 #include "common/hdfs.h"
 #include "common/status.h"
@@ -145,5 +143,3 @@ class DmlExecState {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/fragment-instance-state.cc b/be/src/runtime/fragment-instance-state.cc
index 48e4825..099e98b 100644
--- a/be/src/runtime/fragment-instance-state.cc
+++ b/be/src/runtime/fragment-instance-state.cc
@@ -25,30 +25,31 @@
 #include <boost/thread/lock_guard.hpp>
 #include <boost/date_time/posix_time/posix_time_types.hpp>
 
-#include "common/names.h"
 #include "codegen/llvm-codegen.h"
-#include "exec/plan-root-sink.h"
+#include "exec/exchange-node.h"
 #include "exec/exec-node.h"
 #include "exec/hdfs-scan-node-base.h"
-#include "exec/exchange-node.h"
+#include "exec/plan-root-sink.h"
 #include "exec/scan-node.h"
+#include "gen-cpp/ImpalaInternalService_types.h"
 #include "kudu/rpc/rpc_context.h"
-#include "runtime/exec-env.h"
 #include "runtime/backend-client.h"
 #include "runtime/client-cache.h"
+#include "runtime/exec-env.h"
 #include "runtime/krpc-data-stream-mgr.h"
-#include "runtime/query-state.h"
-#include "runtime/query-state.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/query-state.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-filter-bank.h"
 #include "runtime/runtime-state.h"
 #include "runtime/thread-resource-mgr.h"
 #include "scheduling/query-schedule.h"
-#include "util/debug-util.h"
 #include "util/container-util.h"
+#include "util/debug-util.h"
 #include "util/periodic-counter-updater.h"
-#include "gen-cpp/ImpalaInternalService_types.h"
+#include "util/uid-util.h"
+
+#include "common/names.h"
 
 using kudu::rpc::RpcContext;
 using namespace impala;
diff --git a/be/src/runtime/fragment-instance-state.h b/be/src/runtime/fragment-instance-state.h
index c151457..395a519 100644
--- a/be/src/runtime/fragment-instance-state.h
+++ b/be/src/runtime/fragment-instance-state.h
@@ -15,13 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_FRAGMENT_INSTANCE_STATE_H
-#define IMPALA_RUNTIME_FRAGMENT_INSTANCE_STATE_H
+#pragma once
 
 #include <string>
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/atomic.h"
 #include "common/status.h"
@@ -302,5 +300,3 @@ class FragmentInstanceState {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/hdfs-fs-cache.h b/be/src/runtime/hdfs-fs-cache.h
index 30b43be..8592dc3 100644
--- a/be/src/runtime/hdfs-fs-cache.h
+++ b/be/src/runtime/hdfs-fs-cache.h
@@ -15,14 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_HDFS_FS_CACHE_H
-#define IMPALA_RUNTIME_HDFS_FS_CACHE_H
+#pragma once
 
 #include <string>
 #include <boost/scoped_ptr.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
-#include <boost/thread/mutex.hpp>
 #include "common/hdfs.h"
 
 #include "common/status.h"
@@ -60,7 +58,7 @@ class HdfsFsCache {
 
   /// Get NameNode info from path, set error message if path is not valid.
   /// Exposed as a static method for testing purpose.
-  static string GetNameNodeFromPath(const string& path, string* err);
+  static std::string GetNameNodeFromPath(const std::string& path, std::string* err);
 
   /// S3A access key retrieved by running command in Init().
   /// If either s3a_secret_key_ or this are empty, the default value is taken from the
@@ -85,5 +83,3 @@ class HdfsFsCache {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/initial-reservations.cc b/be/src/runtime/initial-reservations.cc
index c5d225f..02728fa 100644
--- a/be/src/runtime/initial-reservations.cc
+++ b/be/src/runtime/initial-reservations.cc
@@ -19,7 +19,7 @@
 
 #include <limits>
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <gflags/gflags.h>
 
 #include "common/logging.h"
diff --git a/be/src/runtime/io/disk-io-mgr-stress.cc b/be/src/runtime/io/disk-io-mgr-stress.cc
index cb4ba0b..3f0bf6c 100644
--- a/be/src/runtime/io/disk-io-mgr-stress.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress.cc
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "runtime/io/disk-io-mgr-stress.h"
 
diff --git a/be/src/runtime/io/disk-io-mgr.h b/be/src/runtime/io/disk-io-mgr.h
index c08e17d..9e5648a 100644
--- a/be/src/runtime/io/disk-io-mgr.h
+++ b/be/src/runtime/io/disk-io-mgr.h
@@ -15,12 +15,15 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_IO_DISK_IO_MGR_H
+#pragma once
+
+// This macro is used by some plugin text scanners to detect the version of Impala
+// they are built against.
 #define IMPALA_RUNTIME_IO_DISK_IO_MGR_H
 
 #include <vector>
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/atomic.h"
 #include "common/hdfs.h"
@@ -447,5 +450,3 @@ class DiskIoMgr : public CacheLineAligned {
 };
 }
 }
-
-#endif
diff --git a/be/src/runtime/io/handle-cache.h b/be/src/runtime/io/handle-cache.h
index cc873d7..899ef5d 100644
--- a/be/src/runtime/io/handle-cache.h
+++ b/be/src/runtime/io/handle-cache.h
@@ -15,15 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_DISK_IO_MGR_HANDLE_CACHE_H
-#define IMPALA_RUNTIME_DISK_IO_MGR_HANDLE_CACHE_H
+#pragma once
 
 #include <array>
 #include <list>
 #include <map>
 #include <memory>
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/hdfs.h"
 #include "common/status.h"
@@ -223,5 +222,3 @@ class FileHandleCache {
 };
 }
 }
-
-#endif
diff --git a/be/src/runtime/io/local-file-system-with-fault-injection.h b/be/src/runtime/io/local-file-system-with-fault-injection.h
index d638bc6..4df93be 100644
--- a/be/src/runtime/io/local-file-system-with-fault-injection.h
+++ b/be/src/runtime/io/local-file-system-with-fault-injection.h
@@ -31,7 +31,7 @@ namespace io {
 class LocalFileSystemWithFaultInjection : public LocalFileSystem {
 public:
   // Public interface to set the fault injection
-  void SetWriteFaultInjection(const string& function_name, int err_no);
+  void SetWriteFaultInjection(const std::string& function_name, int err_no);
 
   virtual ~LocalFileSystemWithFaultInjection() {}
 
@@ -50,9 +50,9 @@ private:
   // Used for defining fault injection. This structure represents a function name meant
   // to fail alongside with the desired error code that will be used to populate errno.
   struct WriteFaultInjectionItem {
-    WriteFaultInjectionItem(const string& function_name, int e)
+    WriteFaultInjectionItem(const std::string& function_name, int e)
         : function(function_name), err_no(e) {}
-    string function;
+    std::string function;
     int err_no;
   };
 
@@ -64,7 +64,7 @@ private:
   // Compares 'function_name' to fault_injection_to_write_->first. If they match
   // then sets errno to fault_injection_to_write_->second and returns true. Returns false
   // otherwise.
-  bool DebugFaultInjection(const string& function_name);
+  bool DebugFaultInjection(const std::string& function_name);
 };
 
 }
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index e6dc48f..203836a 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -15,14 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_IO_REQUEST_RANGES_H
-#define IMPALA_RUNTIME_IO_REQUEST_RANGES_H
+#pragma once
 
 #include <cstdint>
 #include <deque>
 #include <functional>
 
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <gtest/gtest_prod.h> // for FRIEND_TEST
 
 #include "common/atomic.h"
@@ -629,5 +628,3 @@ inline bool BufferDescriptor::is_client_buffer() const {
 }
 }
 }
-
-#endif
diff --git a/be/src/runtime/krpc-data-stream-mgr.h b/be/src/runtime/krpc-data-stream-mgr.h
index eb371c4..21bf615 100644
--- a/be/src/runtime/krpc-data-stream-mgr.h
+++ b/be/src/runtime/krpc-data-stream-mgr.h
@@ -15,26 +15,26 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_KRPC_DATA_STREAM_MGR_H
-#define IMPALA_RUNTIME_KRPC_DATA_STREAM_MGR_H
+#pragma once
 
 #include <list>
 #include <queue>
 #include <set>
-#include <boost/thread/mutex.hpp>
 #include <boost/thread/condition_variable.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/unordered_set.hpp>
 
-#include "common/status.h"
 #include "common/object-pool.h"
-#include "runtime/descriptors.h"  // for PlanNodeId
+#include "common/status.h"
+#include "gen-cpp/Types_types.h" // for TUniqueId
+#include "runtime/descriptors.h" // for PlanNodeId
 #include "runtime/row-batch.h"
 #include "util/metrics-fwd.h"
 #include "util/promise.h"
 #include "util/runtime-profile.h"
 #include "util/thread-pool.h"
-#include "gen-cpp/Types_types.h"  // for TUniqueId
+#include "util/unique-id-hash.h"
 
 #include "gutil/macros.h"
 
@@ -486,4 +486,3 @@ class KrpcDataStreamMgr : public CacheLineAligned {
 };
 
 } // namespace impala
-#endif // IMPALA_RUNTIME_KRPC_DATA_STREAM_MGR_H
diff --git a/be/src/runtime/krpc-data-stream-recvr.cc b/be/src/runtime/krpc-data-stream-recvr.cc
index b098a5e..6368ce3 100644
--- a/be/src/runtime/krpc-data-stream-recvr.cc
+++ b/be/src/runtime/krpc-data-stream-recvr.cc
@@ -21,7 +21,7 @@
 #include <queue>
 
 #include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "exec/kudu-util.h"
 #include "kudu/rpc/rpc_context.h"
diff --git a/be/src/runtime/krpc-data-stream-recvr.h b/be/src/runtime/krpc-data-stream-recvr.h
index f9fcc22..d98a8cd 100644
--- a/be/src/runtime/krpc-data-stream-recvr.h
+++ b/be/src/runtime/krpc-data-stream-recvr.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_KRPC_DATA_STREAM_RECVR_H
-#define IMPALA_RUNTIME_KRPC_DATA_STREAM_RECVR_H
+#pragma once
 
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/object-pool.h"
 #include "common/status.h"
@@ -301,5 +300,3 @@ class KrpcDataStreamRecvr {
 };
 
 } // namespace impala
-
-#endif // IMPALA_RUNTIME_KRPC_DATA_STREAM_RECVR_H
diff --git a/be/src/runtime/krpc-data-stream-sender.h b/be/src/runtime/krpc-data-stream-sender.h
index 0e5d6d6..01bda16 100644
--- a/be/src/runtime/krpc-data-stream-sender.h
+++ b/be/src/runtime/krpc-data-stream-sender.h
@@ -157,7 +157,7 @@ class KrpcDataStreamSender : public DataSink {
   Status CodegenHashRow(LlvmCodeGen* codegen, llvm::Function** fn);
 
   /// Returns the name of the partitioning type of this data stream sender.
-  string PartitionTypeName() const;
+  std::string PartitionTypeName() const;
 
   /// Sender instance id, unique within a fragment.
   const int sender_id_;
diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h
index b8a2981..7630fe1 100644
--- a/be/src/runtime/lib-cache.h
+++ b/be/src/runtime/lib-cache.h
@@ -15,15 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_LIB_CACHE_H
-#define IMPALA_RUNTIME_LIB_CACHE_H
+#pragma once
 
 #include <string>
 #include <boost/scoped_ptr.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/unordered_set.hpp>
-#include <boost/thread/mutex.hpp>
 #include "common/atomic.h"
 #include "common/object-pool.h"
 #include "common/status.h"
@@ -236,5 +234,3 @@ class LibCacheEntryHandle {
 };
 
 }
-
-#endif
diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h
index a72a7f3..7ae3978 100644
--- a/be/src/runtime/mem-tracker.h
+++ b/be/src/runtime/mem-tracker.h
@@ -15,16 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_MEM_TRACKER_H
-#define IMPALA_RUNTIME_MEM_TRACKER_H
+#pragma once
 
 #include <stdint.h>
 #include <map>
 #include <memory>
 #include <queue>
 #include <vector>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 
 #include "common/logging.h"
@@ -527,5 +525,3 @@ class PoolMemTrackerRegistry {
   SpinLock pool_to_mem_trackers_lock_;
 };
 }
-
-#endif
diff --git a/be/src/runtime/query-exec-mgr.cc b/be/src/runtime/query-exec-mgr.cc
index 5c282d3..823bd59 100644
--- a/be/src/runtime/query-exec-mgr.cc
+++ b/be/src/runtime/query-exec-mgr.cc
@@ -18,20 +18,27 @@
 
 #include "runtime/query-exec-mgr.h"
 
+#include <memory>
+#include <ostream>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include <gflags/gflags.h>
 #include <gperftools/malloc_extension.h>
 #include <gutil/strings/substitute.h>
-#include <boost/thread/locks.hpp>
-#include <boost/thread/lock_guard.hpp>
 
+#include "common/compiler-util.h"
 #include "common/logging.h"
-#include "runtime/query-state.h"
-#include "runtime/fragment-instance-state.h"
+#include "gen-cpp/Types_types.h"
+#include "gen-cpp/control_service.pb.h"
 #include "runtime/exec-env.h"
-#include "runtime/mem-tracker.h"
+#include "runtime/query-state.h"
+#include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
-#include "util/uid-util.h"
+#include "util/network-util.h"
 #include "util/thread.h"
 
 #include "common/names.h"
diff --git a/be/src/runtime/query-exec-mgr.h b/be/src/runtime/query-exec-mgr.h
index 68c2e4c..cc15cc2 100644
--- a/be/src/runtime/query-exec-mgr.h
+++ b/be/src/runtime/query-exec-mgr.h
@@ -15,25 +15,19 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_QUERY_EXEC_MGR_H
-#define IMPALA_RUNTIME_QUERY_EXEC_MGR_H
-
-#include <boost/thread/mutex.hpp>
-#include <unordered_map>
+#pragma once
 
 #include "common/status.h"
-#include "gen-cpp/Types_types.h"
+#include "util/aligned-new.h"
 #include "util/sharded-query-map-util.h"
 
 namespace impala {
 
+class ExecQueryFInstancesRequestPB;
 class QueryState;
-class Thread;
-class TExecPlanFragmentParams;
+class TExecPlanFragmentInfo;
 class TQueryCtx;
 class TUniqueId;
-class FragmentInstanceState;
 
 /// A daemon-wide registry and manager of QueryStates. This is the central
 /// entry point for gaining refcounted access to a QueryState. It also initiates
@@ -81,5 +75,3 @@ class QueryExecMgr : public CacheLineAligned {
   void ExecuteQueryHelper(QueryState* qs);
 };
 }
-
-#endif
diff --git a/be/src/runtime/query-state.cc b/be/src/runtime/query-state.cc
index bf89cbd..bae2098 100644
--- a/be/src/runtime/query-state.cc
+++ b/be/src/runtime/query-state.cc
@@ -43,11 +43,13 @@
 #include "runtime/scanner-mem-limiter.h"
 #include "service/control-service.h"
 #include "service/data-stream-service.h"
+#include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
 #include "util/system-state-info.h"
 #include "util/thread.h"
+#include "util/uid-util.h"
 
 #include "gen-cpp/control_service.pb.h"
 #include "gen-cpp/control_service.proxy.h"
diff --git a/be/src/runtime/query-state.h b/be/src/runtime/query-state.h
index a00dba2..43c204b 100644
--- a/be/src/runtime/query-state.h
+++ b/be/src/runtime/query-state.h
@@ -15,23 +15,27 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_QUERY_STATE_H
-#define IMPALA_RUNTIME_QUERY_STATE_H
+#pragma once
 
+#include <cstdint>
 #include <memory>
-#include <mutex>
+#include <string>
 #include <unordered_map>
 
 #include "common/atomic.h"
+#include "common/compiler-util.h"
+#include "common/logging.h"
 #include "common/object-pool.h"
+#include "common/status.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gen-cpp/Types_types.h"
-#include "gen-cpp/data_stream_service.pb.h"
+#include "gen-cpp/control_service.pb.h"
+#include "gutil/macros.h"
 #include "gutil/threading/thread_collision_warner.h" // for DFAKE_*
 #include "runtime/tmp-file-mgr.h"
-#include "util/container-util.h"
 #include "util/counting-barrier.h"
-#include "util/uid-util.h"
+#include "util/spinlock.h"
+#include "util/unique-id-hash.h"
 
 namespace kudu {
 namespace rpc {
@@ -42,15 +46,17 @@ class RpcContext;
 namespace impala {
 
 class ControlServiceProxy;
-class RuntimeFilterBank;
+class DescriptorTbl;
 class FragmentInstanceState;
 class InitialReservations;
 class MemTracker;
-class ReportExecStatusRequestPB;
+class PublishFilterParamsPB;
 class ReservationTracker;
+class RuntimeFilterBank;
+class RuntimeProfile;
 class RuntimeState;
 class ScannerMemLimiter;
-class ThriftSerializer;
+class TRuntimeProfileForest;
 
 /// Central class for all backend execution state (example: the FragmentInstanceStates
 /// of the individual fragment instances) created for a particular query.
@@ -466,5 +472,3 @@ class QueryState {
   const char* BackendExecStateToString(const BackendExecState& state);
 };
 }
-
-#endif
diff --git a/be/src/runtime/raw-value.h b/be/src/runtime/raw-value.h
index f94d122..b036ace 100644
--- a/be/src/runtime/raw-value.h
+++ b/be/src/runtime/raw-value.h
@@ -15,11 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_RUNTIME_RAW_VALUE_H
-#define IMPALA_RUNTIME_RAW_VALUE_H
-
-#include <sstream>
+#include <iosfwd>
 #include <string>
 
 #include "codegen/impala-ir.h"
@@ -124,7 +122,4 @@ class RawValue {
   // Returns positive zero for floating point types.
   static inline const void* PositiveFloatingZero(const ColumnType& type);
 };
-
 }
-
-#endif
diff --git a/be/src/runtime/row-batch.h b/be/src/runtime/row-batch.h
index dd737d7..93cf2cd 100644
--- a/be/src/runtime/row-batch.h
+++ b/be/src/runtime/row-batch.h
@@ -87,7 +87,7 @@ class OutboundRowBatch {
   vector<int32_t> tuple_offsets_;
 
   /// Contains the actual data of all the tuples. The data could be compressed.
-  string tuple_data_;
+  std::string tuple_data_;
 };
 
 /// A RowBatch encapsulates a batch of rows, each composed of a number of tuples.
diff --git a/be/src/runtime/runtime-filter-bank.cc b/be/src/runtime/runtime-filter-bank.cc
index d4b378f..6f98360 100644
--- a/be/src/runtime/runtime-filter-bank.cc
+++ b/be/src/runtime/runtime-filter-bank.cc
@@ -42,6 +42,7 @@
 #include "util/bloom-filter.h"
 #include "util/min-max-filter.h"
 #include "util/pretty-printer.h"
+#include "util/uid-util.h"
 
 #include "common/names.h"
 
diff --git a/be/src/runtime/runtime-filter.h b/be/src/runtime/runtime-filter.h
index aa57a35..db1296a 100644
--- a/be/src/runtime/runtime-filter.h
+++ b/be/src/runtime/runtime-filter.h
@@ -15,11 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_RUNTIME_RUNTIME_FILTER_H
-#define IMPALA_RUNTIME_RUNTIME_FILTER_H
-
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "runtime/raw-value.h"
 #include "runtime/runtime-filter-bank.h"
@@ -151,7 +149,4 @@ class RuntimeFilter {
   /// 'arrival_mutex_'
   mutable ConditionVariable arrival_cv_;
 };
-
 }
-
-#endif
diff --git a/be/src/runtime/scanner-mem-limiter.h b/be/src/runtime/scanner-mem-limiter.h
index 78768fa..246b4be 100644
--- a/be/src/runtime/scanner-mem-limiter.h
+++ b/be/src/runtime/scanner-mem-limiter.h
@@ -21,7 +21,7 @@
 #include <unordered_map>
 #include <vector>
 
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 
 namespace impala {
 class MemTracker;
diff --git a/be/src/runtime/sorted-run-merger.h b/be/src/runtime/sorted-run-merger.h
index cfa07cb..1bf4485 100644
--- a/be/src/runtime/sorted-run-merger.h
+++ b/be/src/runtime/sorted-run-merger.h
@@ -15,12 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_RUNTIME_SORTED_RUN_MERGER_H_
-#define IMPALA_RUNTIME_SORTED_RUN_MERGER_H_
+#pragma once
 
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/object-pool.h"
 #include "util/tuple-row-compare.h"
@@ -111,7 +109,4 @@ class SortedRunMerger {
   /// Times calls to get the next batch of rows from the input run.
   RuntimeProfile::Counter* get_next_batch_timer_;
 };
-
 }
-
-#endif
diff --git a/be/src/runtime/string-search.h b/be/src/runtime/string-search.h
index 5bb5d02..910fae1 100644
--- a/be/src/runtime/string-search.h
+++ b/be/src/runtime/string-search.h
@@ -15,15 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_STRING_SEARCH_H
-#define IMPALA_RUNTIME_STRING_SEARCH_H
+#pragma once
 
 #include <algorithm>
+#include <cstdint>
 #include <cstring>
 #include <vector>
 
-#include <boost/cstdint.hpp>
-
 #include "common/logging.h"
 #include "runtime/string-value.h"
 
@@ -192,7 +190,4 @@ class StringSearch {
   int skip_;
   int rskip_;
 };
-
 }
-
-#endif
diff --git a/be/src/runtime/thread-resource-mgr.h b/be/src/runtime/thread-resource-mgr.h
index 7e63ae1..d8dc6fc 100644
--- a/be/src/runtime/thread-resource-mgr.h
+++ b/be/src/runtime/thread-resource-mgr.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_THREAD_RESOURCE_MGR_H
-#define IMPALA_RUNTIME_THREAD_RESOURCE_MGR_H
+#pragma once
 
 #include <stdlib.h>
 
 #include <boost/function.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include <list>
 
@@ -236,5 +235,3 @@ class ThreadResourcePool {
   int next_callback_idx_ = 0;
 };
 } // namespace impala
-
-#endif
diff --git a/be/src/runtime/tmp-file-mgr.h b/be/src/runtime/tmp-file-mgr.h
index 70ec1d4..0630b04 100644
--- a/be/src/runtime/tmp-file-mgr.h
+++ b/be/src/runtime/tmp-file-mgr.h
@@ -15,15 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RUNTIME_TMP_FILE_MGR_H
-#define IMPALA_RUNTIME_TMP_FILE_MGR_H
+#pragma once
 
 #include <functional>
 #include <memory>
 #include <utility>
 
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/object-pool.h"
 #include "common/status.h"
@@ -451,7 +450,4 @@ class TmpFileMgr {
   /// Metrics to track the scratch space HWM.
   AtomicHighWaterMarkGauge* scratch_bytes_used_metric_;
 };
-
 }
-
-#endif
diff --git a/be/src/scheduling/executor-group.h b/be/src/scheduling/executor-group.h
index 6e2fee7..f616505 100644
--- a/be/src/scheduling/executor-group.h
+++ b/be/src/scheduling/executor-group.h
@@ -25,6 +25,7 @@
 #include "scheduling/hash-ring.h"
 #include "util/container-util.h"
 #include "util/network-util.h"
+#include "util/unique-id-hash.h"
 
 namespace impala {
 
diff --git a/be/src/scheduling/query-schedule.h b/be/src/scheduling/query-schedule.h
index 474157d..9cbbfa9 100644
--- a/be/src/scheduling/query-schedule.h
+++ b/be/src/scheduling/query-schedule.h
@@ -312,7 +312,7 @@ class QuerySchedule {
   /// GetClusterMemoryToAdmit().
   void UpdateMemoryRequirements(const TPoolConfig& pool_cfg);
 
-  const string& executor_group() const { return executor_group_; }
+  const std::string& executor_group() const { return executor_group_; }
 
   void set_executor_group(string executor_group);
 
@@ -380,7 +380,7 @@ class QuerySchedule {
 
   /// The name of the executor group that this schedule was computed for. Set by the
   /// Scheduler and only valid after scheduling completes successfully.
-  string executor_group_;
+  std::string executor_group_;
 
   /// Populate fragment_exec_params_ from request_.plan_exec_info.
   /// Sets is_coord_fragment and exchange_input_fragments.
diff --git a/be/src/scheduling/scheduler.h b/be/src/scheduling/scheduler.h
index f5f5c2e..76c30a5 100644
--- a/be/src/scheduling/scheduler.h
+++ b/be/src/scheduling/scheduler.h
@@ -15,14 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef SCHEDULING_SCHEDULER_H
-#define SCHEDULING_SCHEDULER_H
+#pragma once
 
 #include <list>
 #include <string>
 #include <vector>
 #include <boost/heap/binomial_heap.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <gtest/gtest_prod.h> // for FRIEND_TEST
 
@@ -436,5 +435,3 @@ class Scheduler {
 };
 
 }
-
-#endif
diff --git a/be/src/service/CMakeLists.txt b/be/src/service/CMakeLists.txt
index 20fd237..939df22 100644
--- a/be/src/service/CMakeLists.txt
+++ b/be/src/service/CMakeLists.txt
@@ -28,6 +28,7 @@ set_source_files_properties(${DATA_STREAM_SERVICE_PROTO_SRCS} PROPERTIES GENERAT
 add_library(Service
   child-query.cc
   client-request-state.cc
+  client-request-state-map.cc
   ${CONTROL_SERVICE_PROTO_SRCS}
   control-service.cc
   ${DATA_STREAM_SERVICE_PROTO_SRCS}
diff --git a/be/src/service/child-query.h b/be/src/service/child-query.h
index b75b5a8..3447af0 100644
--- a/be/src/service/child-query.h
+++ b/be/src/service/child-query.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_SERVICE_CHILD_QUERY_H
-#define IMPALA_SERVICE_CHILD_QUERY_H
+#pragma once
 
 #include <string>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/status.h"
 #include "impala-server.h"
@@ -217,5 +216,3 @@ class ChildQueryExecutor {
   Status child_queries_status_;
 };
 }
-
-#endif
diff --git a/be/src/service/client-request-state-map.cc b/be/src/service/client-request-state-map.cc
new file mode 100644
index 0000000..468e14a
--- /dev/null
+++ b/be/src/service/client-request-state-map.cc
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "service/client-request-state-map.h"
+
+#include "gutil/strings/substitute.h"
+#include "util/container-util.h"
+#include "util/uid-util.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+Status ClientRequestStateMap::AddClientRequestState(
+    const TUniqueId& query_id, shared_ptr<ClientRequestState> request_state) {
+  ScopedShardedMapRef<shared_ptr<ClientRequestState>> map_ref(query_id, this);
+  DCHECK(map_ref.get() != nullptr);
+
+  auto entry = map_ref->find(query_id);
+  if (entry != map_ref->end()) {
+    // There shouldn't be an active query with that same id.
+    // (query_id is globally unique)
+    return Status(ErrorMsg(TErrorCode::INTERNAL_ERROR,
+        strings::Substitute("query id $0 already exists", PrintId(query_id))));
+  }
+  map_ref->insert(make_pair(query_id, request_state));
+  return Status::OK();
+}
+
+Status ClientRequestStateMap::DeleteClientRequestState(
+    const TUniqueId& query_id, shared_ptr<ClientRequestState>* request_state) {
+  ScopedShardedMapRef<shared_ptr<ClientRequestState>> map_ref(query_id, this);
+  DCHECK(map_ref.get() != nullptr);
+  auto entry = map_ref->find(query_id);
+  if (entry == map_ref->end()) {
+    string error_msg =
+        strings::Substitute("Invalid or unknown query handle $0", PrintId(query_id));
+    VLOG(1) << error_msg;
+    return Status::Expected(error_msg);
+  } else if (request_state != nullptr) {
+    *request_state = entry->second;
+  }
+  map_ref->erase(entry);
+  return Status::OK();
+}
+
+} // namespace impala
diff --git a/be/src/service/client-request-state-map.h b/be/src/service/client-request-state-map.h
index 20c5ccc..e131b6c 100644
--- a/be/src/service/client-request-state-map.h
+++ b/be/src/service/client-request-state-map.h
@@ -17,8 +17,9 @@
 
 #pragma once
 
-#include "gutil/strings/substitute.h"
+#include "common/status.h"
 #include "util/sharded-query-map-util.h"
+#include "util/unique-id-hash.h"
 
 namespace impala {
 
@@ -35,40 +36,13 @@ class ClientRequestStateMap
   /// Adds the given (query_id, request_state) pair to the map. Returns an error Status
   /// if the query id already exists in the map.
   Status AddClientRequestState(
-      const TUniqueId& query_id, std::shared_ptr<ClientRequestState> request_state) {
-    ScopedShardedMapRef<std::shared_ptr<ClientRequestState>> map_ref(query_id, this);
-    DCHECK(map_ref.get() != nullptr);
-
-    auto entry = map_ref->find(query_id);
-    if (entry != map_ref->end()) {
-      // There shouldn't be an active query with that same id.
-      // (query_id is globally unique)
-      return Status(ErrorMsg(TErrorCode::INTERNAL_ERROR,
-          strings::Substitute("query id $0 already exists", PrintId(query_id))));
-    }
-    map_ref->insert(make_pair(query_id, request_state));
-    return Status::OK();
-  }
+      const TUniqueId& query_id, std::shared_ptr<ClientRequestState> request_state);
 
   /// Deletes the specified (query_id, request_state) pair from the map and sets the given
   /// request_state pointer to the ClientRequestState associated with the given query_id.
   /// If request_state == nullptr, it is not set. Returns an error Status if the query_id
   /// cannot be found in the map.
   Status DeleteClientRequestState(const TUniqueId& query_id,
-      std::shared_ptr<ClientRequestState>* request_state = nullptr) {
-    ScopedShardedMapRef<std::shared_ptr<ClientRequestState>> map_ref(query_id, this);
-    DCHECK(map_ref.get() != nullptr);
-    auto entry = map_ref->find(query_id);
-    if (entry == map_ref->end()) {
-      string error_msg =
-          strings::Substitute("Invalid or unknown query handle $0", PrintId(query_id));
-      VLOG(1) << error_msg;
-      return Status::Expected(error_msg);
-    } else if (request_state != nullptr) {
-      *request_state = entry->second;
-    }
-    map_ref->erase(entry);
-    return Status::OK();
-  }
+      std::shared_ptr<ClientRequestState>* request_state = nullptr);
 };
 }
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 6197d1c..887746f 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -42,6 +42,7 @@
 #include "service/impala-server.h"
 #include "service/query-options.h"
 #include "service/query-result-set.h"
+#include "util/auth-util.h"
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/lineage-util.h"
@@ -1434,6 +1435,10 @@ Status ClientRequestState::InitExecRequest(const TQueryCtx& query_ctx) {
       exec_env_->frontend()->GetExecRequest(query_ctx, &exec_request_));
 }
 
+const string& ClientRequestState::effective_user() const {
+  return GetEffectiveUser(query_ctx_.session);
+}
+
 void ClientRequestState::UpdateEndTime() {
   // Update the query's end time only if it isn't set previously.
   if (end_time_us_.CompareAndSwap(0, UnixMicros())) {
diff --git a/be/src/service/client-request-state.h b/be/src/service/client-request-state.h
index 1ef3b1d..8c8afc1 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -26,7 +26,6 @@
 #include "service/child-query.h"
 #include "service/impala-server.h"
 #include "service/query-result-set.h"
-#include "util/auth-util.h"
 #include "util/condition-variable.h"
 #include "util/runtime-profile.h"
 #include "gen-cpp/Frontend_types.h"
@@ -208,9 +207,7 @@ class ClientRequestState {
   ImpalaServer::SessionState* session() const { return session_.get(); }
 
   /// Queries are run and authorized on behalf of the effective_user.
-  const std::string& effective_user() const {
-      return GetEffectiveUser(query_ctx_.session);
-  }
+  const std::string& effective_user() const;
   const std::string& connected_user() const { return query_ctx_.session.connected_user; }
   bool user_has_profile_access() const { return user_has_profile_access_; }
   const std::string& do_as_user() const { return query_ctx_.session.delegated_user; }
diff --git a/be/src/service/data-stream-service.cc b/be/src/service/data-stream-service.cc
index 8785225..76ef7ba 100644
--- a/be/src/service/data-stream-service.cc
+++ b/be/src/service/data-stream-service.cc
@@ -26,14 +26,15 @@
 #include "kudu/util/monotime.h"
 #include "rpc/rpc-mgr.h"
 #include "rpc/rpc-mgr.inline.h"
-#include "runtime/krpc-data-stream-mgr.h"
 #include "runtime/exec-env.h"
+#include "runtime/krpc-data-stream-mgr.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/query-state.h"
 #include "runtime/row-batch.h"
 #include "service/impala-server.h"
 #include "util/memory-metrics.h"
 #include "util/parse-util.h"
+#include "util/uid-util.h"
 
 #include "gen-cpp/data_stream_service.pb.h"
 #include "gen-cpp/data_stream_service.proxy.h"
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index f8e5dc0..37011f2 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -54,6 +54,7 @@
 #include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 #include "util/string-parser.h"
+#include "util/uid-util.h"
 
 #include "common/names.h"
 
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 48d94ce..3c5e2c0 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -20,7 +20,7 @@
 #include <algorithm>
 #include <sstream>
 #include <boost/lexical_cast.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <rapidjson/prettywriter.h>
 #include <rapidjson/stringbuffer.h>
 
diff --git a/be/src/service/impala-http-handler.h b/be/src/service/impala-http-handler.h
index b821012..b45bec9 100644
--- a/be/src/service/impala-http-handler.h
+++ b/be/src/service/impala-http-handler.h
@@ -15,10 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_SERVICE_IMPALA_HTTP_HANDLER_H
-#define IMPALA_SERVICE_IMPALA_HTTP_HANDLER_H
+#pragma once
 
-#include <sstream>
+#include <iosfwd>
 #include <rapidjson/document.h>
 #include "kudu/util/web_callback_registry.h"
 #include "util/webserver.h"
@@ -249,7 +248,4 @@ class ImpalaHttpHandler {
   void ResetResourcePoolStatsHandler(
       const Webserver::WebRequest& req, rapidjson::Document* document);
 };
-
 }
-
-#endif
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 76184f8..a554cdf 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -86,6 +86,7 @@
 #include "util/redactor.h"
 #include "util/runtime-profile-counters.h"
 #include "util/runtime-profile.h"
+#include "util/simple-logger.h"
 #include "util/string-parser.h"
 #include "util/summary-util.h"
 #include "util/test-info.h"
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 0746784..5fa6f26 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -15,25 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_SERVICE_IMPALA_SERVER_H
-#define IMPALA_SERVICE_IMPALA_SERVER_H
+#pragma once
 
 #include <atomic>
+#include <unordered_map>
 #include <boost/random/random_device.hpp>
-#include <boost/thread/mutex.hpp>
-#include <boost/shared_ptr.hpp>
 #include <boost/scoped_ptr.hpp>
+#include <boost/shared_ptr.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/unordered_set.hpp>
 #include <boost/uuid/uuid.hpp>
 #include <boost/uuid/uuid_generators.hpp>
 #include <boost/uuid/uuid_io.hpp>
-#include <unordered_map>
 
 #include "common/status.h"
 #include "gen-cpp/Frontend_types.h"
 #include "gen-cpp/ImpalaHiveServer2Service.h"
 #include "gen-cpp/ImpalaInternalService.h"
 #include "gen-cpp/ImpalaService.h"
+#include "gen-cpp/control_service.pb.h"
 #include "kudu/util/random.h"
 #include "rpc/thrift-server.h"
 #include "runtime/timestamp-value.h"
@@ -73,6 +73,7 @@ class TGetExecSummaryResp;
 class TGetExecSummaryReq;
 class ClientRequestState;
 class QuerySchedule;
+class SimpleLogger;
 
 /// An ImpalaServer contains both frontend and backend functionality;
 /// it implements ImpalaService (Beeswax), ImpalaHiveServer2Service (HiveServer2)
@@ -499,11 +500,11 @@ class ImpalaServer : public ImpalaServiceIf,
   QueryOptionLevels query_option_levels_;
 
   /// The prefix of audit event log filename.
-  static const string AUDIT_EVENT_LOG_FILE_PREFIX;
+  static const std::string AUDIT_EVENT_LOG_FILE_PREFIX;
 
   /// The default executor group name for executors that do not explicitly belong to a
   /// specific executor group.
-  static const string DEFAULT_EXECUTOR_GROUP_NAME;
+  static const std::string DEFAULT_EXECUTOR_GROUP_NAME;
 
   /// Per-session state.  This object is reference counted using shared_ptrs.  There
   /// is one ref count in the SessionStateMap for as long as the session is active.
@@ -1422,5 +1423,3 @@ class ImpalaServer : public ImpalaServiceIf,
   AtomicInt64 shutdown_deadline_{0};
 };
 }
-
-#endif
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index e76af56..22db7fe 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -23,27 +23,28 @@
 #include <jni.h>
 
 #include "catalog/catalog-util.h"
-#include "common/logging.h"
+#include "codegen/llvm-codegen.h"
 #include "common/init.h"
+#include "common/logging.h"
+#include "common/status.h"
 #include "exec/hbase-table-scanner.h"
 #include "exec/hbase-table-writer.h"
 #include "exprs/hive-udf-call.h"
 #include "exprs/timezone_db.h"
-#include "runtime/hbase-table.h"
-#include "codegen/llvm-codegen.h"
-#include "common/status.h"
+#include "gen-cpp/ImpalaInternalService.h"
+#include "gen-cpp/ImpalaService.h"
+#include "rpc/rpc-trace.h"
+#include "rpc/thrift-server.h"
+#include "rpc/thrift-util.h"
 #include "runtime/coordinator.h"
 #include "runtime/exec-env.h"
+#include "runtime/hbase-table.h"
+#include "service/fe-support.h"
+#include "service/impala-server.h"
 #include "util/common-metrics.h"
 #include "util/jni-util.h"
+#include "util/metrics.h"
 #include "util/network-util.h"
-#include "rpc/thrift-util.h"
-#include "rpc/thrift-server.h"
-#include "rpc/rpc-trace.h"
-#include "service/impala-server.h"
-#include "service/fe-support.h"
-#include "gen-cpp/ImpalaService.h"
-#include "gen-cpp/ImpalaInternalService.h"
 #include "util/thread.h"
 
 #include "common/names.h"
diff --git a/be/src/statestore/failure-detector.h b/be/src/statestore/failure-detector.h
index 5962d06..bfc7a52 100644
--- a/be/src/statestore/failure-detector.h
+++ b/be/src/statestore/failure-detector.h
@@ -15,14 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef STATESTORE_FAILURE_DETECTOR_H
-#define STATESTORE_FAILURE_DETECTOR_H
-
-#include <boost/thread/thread_time.hpp>
 #include <string>
 #include <boost/date_time/posix_time/posix_time_types.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+#include <boost/thread/thread_time.hpp>
 
 namespace impala {
 
@@ -143,7 +141,4 @@ class MissedHeartbeatFailureDetector : public FailureDetector {
   /// Number of consecutive heartbeats missed by peer.
   std::map<std::string, int32_t> missed_heartbeat_counts_;
 };
-
 }
-
-#endif // IMPALA_SPARROW_FAILURE_DETECTOR_H
diff --git a/be/src/statestore/statestore-subscriber.cc b/be/src/statestore/statestore-subscriber.cc
index 6f4973d..0d3070e 100644
--- a/be/src/statestore/statestore-subscriber.cc
+++ b/be/src/statestore/statestore-subscriber.cc
@@ -23,7 +23,7 @@
 #include <boost/algorithm/string/join.hpp>
 #include <boost/date_time/posix_time/posix_time.hpp>
 #include <boost/thread/lock_options.hpp>
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
diff --git a/be/src/statestore/statestore-subscriber.h b/be/src/statestore/statestore-subscriber.h
index 4e3aefe..051b761 100644
--- a/be/src/statestore/statestore-subscriber.h
+++ b/be/src/statestore/statestore-subscriber.h
@@ -15,16 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef STATESTORE_STATESTORE_SUBSCRIBER_H
-#define STATESTORE_STATESTORE_SUBSCRIBER_H
+#pragma once
 
 #include <string>
 
 #include <boost/scoped_ptr.hpp>
 #include <boost/shared_ptr.hpp>
-#include <boost/thread/mutex.hpp>
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 
 #include "gen-cpp/StatestoreService.h"
 #include "gen-cpp/StatestoreSubscriber.h"
@@ -235,7 +233,7 @@ class StatestoreSubscriber {
     bool populate_min_subscriber_topic_version = false;
 
     /// Only subscribe to keys with the provided prefix.
-    string filter_prefix;
+    std::string filter_prefix;
 
     /// The last version of the topic this subscriber processed.
     /// -1 if no updates have been processed yet.
@@ -315,7 +313,4 @@ class StatestoreSubscriber {
     return MonotonicMillis() - last_registration_ms_.Load();
   }
 };
-
 }
-
-#endif
diff --git a/be/src/statestore/statestore.h b/be/src/statestore/statestore.h
index 727d369..9a26e07 100644
--- a/be/src/statestore/statestore.h
+++ b/be/src/statestore/statestore.h
@@ -24,8 +24,8 @@
 #include <string>
 #include <vector>
 
-#include <boost/thread/shared_mutex.hpp>
 #include <boost/scoped_ptr.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/uuid/uuid_generators.hpp>
 
diff --git a/be/src/testutil/mini-kdc-wrapper.h b/be/src/testutil/mini-kdc-wrapper.h
index c233dda..ce5bf49 100644
--- a/be/src/testutil/mini-kdc-wrapper.h
+++ b/be/src/testutil/mini-kdc-wrapper.h
@@ -48,19 +48,19 @@ class MiniKdcWrapper {
   Status TearDownMiniKDC();
 
   /// Kinit a user to the mini KDC.
-  Status Kinit(const string& username);
+  Status Kinit(const std::string& username);
 
   /// Creates a new user with the given username.
   /// The password is the same as the username.
-  Status CreateUserPrincipal(const string& username);
+  Status CreateUserPrincipal(const std::string& username);
 
   /// Creates a keytab file under the 'unique_test_dir_' path which is configured to
   /// authenticate the service principal 'spn'. The path to the file is returned as a
   /// string in 'kt_path'.
-  Status CreateServiceKeytab(const string& spn, string* kt_path);
+  Status CreateServiceKeytab(const std::string& spn, std::string* kt_path);
 
   /// Returns the environment variable ""KRB5CCNAME" configured in the setup of mini-kdc.
-  const string GetKrb5CCname() const {
+  const std::string GetKrb5CCname() const {
     return kdc_->GetEnvVars()["KRB5CCNAME"];
   }
 
diff --git a/be/src/thirdparty/mustache/mustache.cc b/be/src/thirdparty/mustache/mustache.cc
index e976677..5398ce6 100644
--- a/be/src/thirdparty/mustache/mustache.cc
+++ b/be/src/thirdparty/mustache/mustache.cc
@@ -18,6 +18,7 @@
 
 #include <iostream>
 #include <fstream>
+#include <sstream>
 #include <vector>
 #include <stack>
 
diff --git a/be/src/thirdparty/mustache/mustache.h b/be/src/thirdparty/mustache/mustache.h
index 9a2d9fb..e1374c8 100644
--- a/be/src/thirdparty/mustache/mustache.h
+++ b/be/src/thirdparty/mustache/mustache.h
@@ -10,8 +10,10 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#pragma once
+
 #include "rapidjson/document.h"
-#include <sstream>
+#include <iosfwd>
 
 // Routines for rendering Mustache (http://mustache.github.io) templates with RapidJson
 // (https://code.google.com/p/rapidjson/) documents.
diff --git a/be/src/transport/TSaslServerTransport.h b/be/src/transport/TSaslServerTransport.h
index b01fcf3..f55b119 100644
--- a/be/src/transport/TSaslServerTransport.h
+++ b/be/src/transport/TSaslServerTransport.h
@@ -18,14 +18,13 @@
  * under the License.
  */
 
-#ifndef IMPALA_TRANSPORT_TSASLSERVERTRANSPORT_H
-#define IMPALA_TRANSPORT_TSASLSERVERTRANSPORT_H
+#pragma once
 
 #include <string>
 #include <pthread.h>
 
 #include <boost/shared_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <thrift/transport/TTransport.h>
 #include "transport/TSasl.h"
 #include "transport/TSaslTransport.h"
@@ -199,5 +198,3 @@ class TSaslServerTransport : public TSaslTransport {
 };
 
 }}} // apache::thrift::transport
-
-#endif // #ifndef IMPALA_TRANSPORT_TSSLSERVERTRANSPORT_H
diff --git a/be/src/udf/uda-test-harness.h b/be/src/udf/uda-test-harness.h
index cb33ef9..2cddbe9 100644
--- a/be/src/udf/uda-test-harness.h
+++ b/be/src/udf/uda-test-harness.h
@@ -15,15 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UDA_TEST_HARNESS_H
-#define IMPALA_UDA_TEST_HARNESS_H
+#pragma once
 
 // THIS FILE IS USED BY THE STANDALONE IMPALA UDF DEVELOPMENT KIT.
 // IT MUST BE BUILDABLE WITH C++98 AND WITHOUT ANY INTERNAL IMPALA HEADERS.
 
 #include <string>
-#include <sstream>
 #include <vector>
 
 #include <boost/scoped_ptr.hpp>
@@ -290,9 +287,5 @@ class UdaTestHarness4 : public UdaTestHarnessBase<RESULT, INTERMEDIATE> {
   const std::vector<INPUT3>* input3_;
   const std::vector<INPUT4>* input4_;
 };
-
 }
-
 #include "udf/uda-test-harness-impl.h"
-
-#endif
diff --git a/be/src/udf/udf-internal.h b/be/src/udf/udf-internal.h
index 41f8562..d970d91 100644
--- a/be/src/udf/udf-internal.h
+++ b/be/src/udf/udf-internal.h
@@ -15,15 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UDF_UDF_INTERNAL_H
-#define IMPALA_UDF_UDF_INTERNAL_H
+#pragma once
 
 #include <string.h>
+#include <cstdint>
 #include <map>
 #include <string>
 #include <utility>
 #include <vector>
-#include <boost/cstdint.hpp>
 
 /// Be very careful when adding Impala includes in this file. We don't want to pull
 /// in unnecessary dependencies for the development libs.
@@ -320,5 +319,3 @@ static_assert(
     offsetof(CollectionVal, num_tuples) == offsetof(StringVal, len), "Wrong offset.");
 static_assert(offsetof(CollectionVal, ptr) == offsetof(StringVal, ptr), "Wrong offset.");
 } // namespace impala_udf
-
-#endif
diff --git a/be/src/udf/udf-test-harness.h b/be/src/udf/udf-test-harness.h
index fe10b48..982ea62 100644
--- a/be/src/udf/udf-test-harness.h
+++ b/be/src/udf/udf-test-harness.h
@@ -15,9 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UDF_TEST_HARNESS_H
-#define IMPALA_UDF_TEST_HARNESS_H
+#pragma once
 
 // THIS FILE IS USED BY THE STANDALONE IMPALA UDF DEVELOPMENT KIT.
 // IT MUST BE BUILDABLE WITH C++98 AND WITHOUT ANY INTERNAL IMPALA HEADERS.
@@ -327,7 +325,4 @@ class UdfTestHarness {
     }
   }
 };
-
 }
-
-#endif
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 1f28ffe..ba4e94f 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -93,6 +93,7 @@ add_library(Util
   thread.cc
   time.cc
   tuple-row-compare.cc
+  uid-util.cc
   url-parser.cc
   ${SQUEASEL_SRC_DIR}/squeasel.c
   webserver.cc
diff --git a/be/src/util/auth-util.cc b/be/src/util/auth-util.cc
index 3085156..238b9de 100644
--- a/be/src/util/auth-util.cc
+++ b/be/src/util/auth-util.cc
@@ -17,13 +17,16 @@
 
 #include "util/auth-util.h"
 
-#include <boost/algorithm/string.hpp>
+#include <ostream>
 
-#include "service/client-request-state.h"
-#include "util/network-util.h"
+#include <boost/algorithm/string/classification.hpp>
+
+#include "common/logging.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "kudu/security/init.h"
-#include "exec/kudu-util.h"
+#include "kudu/util/status.h"
+#include "util/kudu-status-util.h"
+#include "util/network-util.h"
 
 using namespace std;
 using boost::algorithm::is_any_of;
diff --git a/be/src/util/auth-util.h b/be/src/util/auth-util.h
index f73e962..f7dcd86 100644
--- a/be/src/util/auth-util.h
+++ b/be/src/util/auth-util.h
@@ -15,11 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_AUTH_UTIL_H
-#define IMPALA_UTIL_AUTH_UTIL_H
+#pragma once
 
 #include <string>
+#include <gflags/gflags_declare.h>
+
+#include "common/status.h"
+#include "gutil/strings/substitute.h"
 #include "service/impala-server.h"
 
 DECLARE_string(principal);
@@ -68,6 +70,4 @@ Status ParseKerberosPrincipal(const std::string& principal, std::string* service
 inline bool IsKerberosEnabled() {
   return !FLAGS_principal.empty();
 }
-
 } // namespace impala
-#endif
diff --git a/be/src/util/bit-stream-utils.h b/be/src/util/bit-stream-utils.h
index b94386a..d0755ca 100644
--- a/be/src/util/bit-stream-utils.h
+++ b/be/src/util/bit-stream-utils.h
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_BIT_STREAM_UTILS_H
-#define IMPALA_UTIL_BIT_STREAM_UTILS_H
-
-#include <boost/cstdint.hpp>
 #include <string.h>
+#include <cstdint>
+
 #include "common/compiler-util.h"
 #include "common/logging.h"
 #include "util/bit-packing.h"
@@ -201,5 +200,3 @@ class BatchedBitReader {
   const uint8_t* buffer_end_ = nullptr;
 };
 }
-
-#endif
diff --git a/be/src/util/bit-util.cc b/be/src/util/bit-util.cc
index c8a7318..9937434 100644
--- a/be/src/util/bit-util.cc
+++ b/be/src/util/bit-util.cc
@@ -16,7 +16,10 @@
 // under the License.
 
 #include "util/bit-util.h"
+
+#include <emmintrin.h>
 #include <immintrin.h>
+#include <ostream>
 
 namespace {
 // ByteSwapScalarLoop is only used in bit-util.cc, so put it in this anonymous
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 8e02611..45f2f19 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -15,9 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_BIT_UTIL_H
-#define IMPALA_BIT_UTIL_H
+#pragma once
 
 #if defined(__APPLE__)
 #include <machine/endian.h>
@@ -26,13 +24,15 @@
 #endif
 
 #include <climits>
+#include <cstdint>
 #include <limits>
-#include <typeinfo>
 #include <type_traits>
 
 #include "common/compiler-util.h"
+#include "common/logging.h"
 #include "gutil/bits.h"
 #include "runtime/multi-precision.h"
+#include "util/arithmetic-util.h"
 #include "util/cpu-info.h"
 #include "util/sse-util.h"
 
@@ -427,5 +427,3 @@ class SimdByteSwap {
   static void ByteSwapSimd(const void* src, const int len, void* dst);
 };
 }
-
-#endif
diff --git a/be/src/util/blocking-queue-test.cc b/be/src/util/blocking-queue-test.cc
index c807054..4a78e7e 100644
--- a/be/src/util/blocking-queue-test.cc
+++ b/be/src/util/blocking-queue-test.cc
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
+#include <unistd.h>
 #include <boost/thread.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <glog/logging.h>
-#include <unistd.h>
 
 #include "testutil/gtest-util.h"
 #include "util/blocking-queue.h"
diff --git a/be/src/util/blocking-queue.h b/be/src/util/blocking-queue.h
index 811e029..d2bc0dd 100644
--- a/be/src/util/blocking-queue.h
+++ b/be/src/util/blocking-queue.h
@@ -15,15 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_BLOCKING_QUEUE_H
-#define IMPALA_UTIL_BLOCKING_QUEUE_H
-
-#include <boost/thread/mutex.hpp>
-#include <boost/scoped_ptr.hpp>
+#include <unistd.h>
 #include <deque>
 #include <memory>
-#include <unistd.h>
+#include <boost/scoped_ptr.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/atomic.h"
 #include "common/compiler-util.h"
@@ -306,7 +304,4 @@ class BlockingQueue : public CacheLineAligned {
   /// Soft limit on total bytes in queue. -1 if no limit.
   const int64_t max_bytes_;
 };
-
 }
-
-#endif
diff --git a/be/src/util/bloom-filter-ir.cc b/be/src/util/bloom-filter-ir.cc
index 9a2e299..0042c44 100644
--- a/be/src/util/bloom-filter-ir.cc
+++ b/be/src/util/bloom-filter-ir.cc
@@ -16,8 +16,7 @@
 // under the License.
 
 #include "util/bloom-filter.h"
-
-#include "codegen/impala-ir.h"
+#include "util/hash-util.h"
 
 using namespace impala;
 
diff --git a/be/src/util/bloom-filter.cc b/be/src/util/bloom-filter.cc
index f9493a4..10f0987 100644
--- a/be/src/util/bloom-filter.cc
+++ b/be/src/util/bloom-filter.cc
@@ -17,10 +17,21 @@
 
 #include "util/bloom-filter.h"
 
+#include <emmintrin.h>
+#include <math.h>
+#include <string.h>
+
+#include <cmath>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+
+#include "gen-cpp/data_stream_service.pb.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
 #include "runtime/exec-env.h"
-#include "runtime/runtime-state.h"
 
 using namespace std;
 
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index 168b7ed..d1f6258 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -15,20 +15,21 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_BLOOM_H
-#define IMPALA_UTIL_BLOOM_H
+#pragma once
 
-#include <math.h>
 #include <stdint.h>
 
+#include <algorithm>
+#include <cstring>
 #include <limits>
 
 #include <immintrin.h>
 
 #include "common/compiler-util.h"
-#include "gen-cpp/ImpalaInternalService_types.h"
-#include "gen-cpp/data_stream_service.pb.h"
+#include "common/logging.h"
+#include "common/status.h"
 #include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/bufferpool/buffer-pool.h"
 #include "util/cpu-info.h"
 #include "util/hash-util.h"
@@ -41,6 +42,7 @@ class RpcController;
 
 namespace impala {
 class BloomFilter;
+class BloomFilterPB;
 } // namespace impala
 
 // Need this forward declaration since we make bloom_filter_test_util::BfUnion() a friend
@@ -293,7 +295,4 @@ inline bool ALWAYS_INLINE BloomFilter::Find(const uint32_t hash) const noexcept
     return BucketFind(bucket_idx, hash);
   }
 }
-
 } // namespace impala
-
-#endif // IMPALA_UTIL_BLOOM_H
diff --git a/be/src/util/buffer-builder.h b/be/src/util/buffer-builder.h
index a421e20..7cde7f2 100644
--- a/be/src/util/buffer-builder.h
+++ b/be/src/util/buffer-builder.h
@@ -15,12 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_BUFFER_BUILDER_H
-#define IMPALA_UTIL_BUFFER_BUILDER_H
+#pragma once
 
 #include <stdlib.h>
-#include <boost/cstdint.hpp>
+#include <cstdint>
 
 namespace impala {
 
@@ -53,5 +51,3 @@ class BufferBuilder {
   int size_;
 };
 }
-
-#endif
diff --git a/be/src/util/cgroup-util.cc b/be/src/util/cgroup-util.cc
index 412ec55..d9bd359 100644
--- a/be/src/util/cgroup-util.cc
+++ b/be/src/util/cgroup-util.cc
@@ -21,9 +21,13 @@
 #include <fstream>
 #include <iostream>
 #include <utility>
+#include <vector>
 
-#include <boost/algorithm/string.hpp>
+#include <boost/algorithm/string/classification.hpp>
+#include <boost/algorithm/string/constants.hpp>
+#include <boost/algorithm/string/split.hpp>
 
+#include "common/logging.h"
 #include "gutil/strings/escaping.h"
 #include "gutil/strings/substitute.h"
 #include "util/error-util.h"
diff --git a/be/src/util/codec.cc b/be/src/util/codec.cc
index 18508b6..3e91643 100644
--- a/be/src/util/codec.cc
+++ b/be/src/util/codec.cc
@@ -17,8 +17,15 @@
 
 #include "util/codec.h"
 
-#include <gutil/strings/substitute.h>
+#include <ostream>
+#include <utility>
 
+#include <zstd.h>
+
+#include "common/compiler-util.h"
+#include "common/logging.h"
+#include "gutil/strings/substitute.h"
+#include "util/bit-util.h"
 #include "util/compress.h"
 #include "util/decompress.h"
 
diff --git a/be/src/util/codec.h b/be/src/util/codec.h
index c17bc7d..2b4afe7 100644
--- a/be/src/util/codec.h
+++ b/be/src/util/codec.h
@@ -15,14 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_CODEC_H
-#define IMPALA_UTIL_CODEC_H
-
-#include "common/status.h"
+#include <cstdint>
+#include <map>
+#include <string>
 
 #include <boost/scoped_ptr.hpp>
 
+#include "common/status.h"
+#include "gen-cpp/CatalogObjects_types.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/mem-pool.h"
 
 namespace impala {
@@ -203,6 +206,4 @@ class Codec {
   /// This is set to true for codecs that implement ProcessBlockStreaming().
   bool supports_streaming_;
 };
-
 }
-#endif
diff --git a/be/src/util/coding-util.cc b/be/src/util/coding-util.cc
index 6a8ea2e..6ebf0d5 100644
--- a/be/src/util/coding-util.cc
+++ b/be/src/util/coding-util.cc
@@ -17,9 +17,13 @@
 
 #include "util/coding-util.h"
 
-#include <exception>
+#include <cctype>
+#include <limits>
 #include <sstream>
-#include <boost/algorithm/string.hpp>
+
+#include <boost/algorithm/string/classification.hpp>
+#include <boost/function.hpp>
+#include <sasl/sasl.h>
 
 #include "common/compiler-util.h"
 #include "common/logging.h"
diff --git a/be/src/util/coding-util.h b/be/src/util/coding-util.h
index 723fb92..bb2b039 100644
--- a/be/src/util/coding-util.h
+++ b/be/src/util/coding-util.h
@@ -15,12 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef UTIL_CODING_UTIL_H
-#define UTIL_CODING_UTIL_H
+#pragma once
 
+#include <cstdint>
+#include <iosfwd>
 #include <string>
 #include <vector>
-#include <boost/cstdint.hpp>
 
 namespace impala {
 
@@ -75,7 +75,4 @@ bool Base64Decode(const char* in, int64_t in_len, int64_t out_max, char* out,
 /// inspects each character in turn, and copies them all to *out; use
 /// judiciously.
 void EscapeForHtml(const std::string& in, std::stringstream* out);
-
 }
-
-#endif
diff --git a/be/src/util/collection-metrics.cc b/be/src/util/collection-metrics.cc
index 1160ccf..5bd9f2b 100644
--- a/be/src/util/collection-metrics.cc
+++ b/be/src/util/collection-metrics.cc
@@ -17,6 +17,14 @@
 
 #include "util/collection-metrics.h"
 
+#include <ostream>
+
+#include <boost/accumulators/statistics/count.hpp>
+#include <boost/accumulators/statistics/max.hpp>
+#include <boost/accumulators/statistics/mean.hpp>
+#include <boost/accumulators/statistics/min.hpp>
+#include <boost/accumulators/statistics/variance.hpp>
+
 #include "util/json-util.h"
 #include "util/pretty-printer.h"
 
diff --git a/be/src/util/collection-metrics.h b/be/src/util/collection-metrics.h
index d4e883a..e93b5c9 100644
--- a/be/src/util/collection-metrics.h
+++ b/be/src/util/collection-metrics.h
@@ -17,18 +17,26 @@
 
 #pragma once
 
-#include "util/metrics.h"
-
+#include <cstdint>
+#include <iosfwd>
+#include <set>
 #include <string>
 #include <vector>
-#include <set>
-#include <boost/algorithm/string/join.hpp>
+
 #include <boost/accumulators/accumulators.hpp>
+#include <boost/accumulators/framework/accumulator_set.hpp>
+#include <boost/accumulators/framework/features.hpp>
 #include <boost/accumulators/statistics/count.hpp>
+#include <boost/accumulators/statistics/max.hpp>
 #include <boost/accumulators/statistics/mean.hpp>
 #include <boost/accumulators/statistics/min.hpp>
-#include <boost/accumulators/statistics/max.hpp>
 #include <boost/accumulators/statistics/variance.hpp>
+#include <boost/thread/lock_guard.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+
+#include "common/logging.h"
+#include "util/metrics-fwd.h"
+#include "util/metrics.h"
 
 namespace impala {
 
diff --git a/be/src/util/common-metrics.cc b/be/src/util/common-metrics.cc
index 114e0e0..e1f944b 100644
--- a/be/src/util/common-metrics.cc
+++ b/be/src/util/common-metrics.cc
@@ -16,9 +16,12 @@
 // under the License.
 
 #include "util/common-metrics.h"
-#include "util/time.h"
+
 #include <kudu/client/client.h>
 
+#include "util/metrics.h"
+#include "util/time.h"
+
 namespace impala {
 
 StringProperty* CommonMetrics::PROCESS_START_TIME = nullptr;
diff --git a/be/src/util/common-metrics.h b/be/src/util/common-metrics.h
index 141fe1e..91df6b1 100644
--- a/be/src/util/common-metrics.h
+++ b/be/src/util/common-metrics.h
@@ -15,7 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "util/metrics.h"
+#pragma once
+
+#include <string>
+
+#include "util/metrics-fwd.h"
 
 namespace impala {
 
@@ -31,8 +35,8 @@ public:
   static void InitCommonMetrics(MetricGroup* metric_group);
 
 private:
-  static string PROCESS_START_TIME_METRIC_NAME;
-  static string KUDU_CLIENT_VERSION_METRIC_NAME;
+ static std::string PROCESS_START_TIME_METRIC_NAME;
+ static std::string KUDU_CLIENT_VERSION_METRIC_NAME;
 };
 
 }
diff --git a/be/src/util/compress.cc b/be/src/util/compress.cc
index d5c279c..9166d83 100644
--- a/be/src/util/compress.cc
+++ b/be/src/util/compress.cc
@@ -17,17 +17,24 @@
 
 #include "util/compress.h"
 
+#include <strings.h>
+
+#include <ostream>
+#include <vector>
+
 #include <bzlib.h>
 #include <zlib.h>
 #include <boost/crc.hpp>
-#include <gutil/strings/substitute.h>
 #undef DISALLOW_COPY_AND_ASSIGN // Snappy redefines this.
 #include <lz4.h>
 #include <snappy.h>
+#include <zconf.h>
 #include <zstd.h>
 #include <zstd_errors.h>
 
+#include "common/logging.h"
 #include "exec/read-write-util.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/mem-pool.h"
 
 #include "common/names.h"
diff --git a/be/src/util/compress.h b/be/src/util/compress.h
index 14e6527..c854a84 100644
--- a/be/src/util/compress.h
+++ b/be/src/util/compress.h
@@ -15,14 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_COMPRESS_H
-#define IMPALA_UTIL_COMPRESS_H
+#include <cstdint>
+#include <string>
 
 /// We need zlib.h here to declare stream_ below.
 #include <zlib.h>
 #include <zstd.h>
 
+#include "common/status.h"
 #include "util/codec.h"
 
 namespace impala {
@@ -164,4 +166,3 @@ class Lz4BlockCompressor : public Codec {
   virtual std::string file_extension() const override { return "lz4"; }
 };
 }
-#endif
diff --git a/be/src/util/condition-variable.h b/be/src/util/condition-variable.h
index e463790..07d710e 100644
--- a/be/src/util/condition-variable.h
+++ b/be/src/util/condition-variable.h
@@ -15,14 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_CONDITION_VARIABLE_H
-#define IMPALA_UTIL_CONDITION_VARIABLE_H
+#pragma once
 
-#include <boost/thread/pthread/timespec.hpp>
-#include <boost/thread/mutex.hpp>
-#include <boost/thread/thread_time.hpp>
 #include <pthread.h>
 #include <unistd.h>
+#include <boost/thread/pthread/mutex.hpp>
+#include <boost/thread/pthread/timespec.hpp>
+#include <boost/thread/thread_time.hpp>
 
 #include "util/time.h"
 
@@ -79,6 +78,4 @@ class ConditionVariable {
   pthread_cond_t cv_;
 
 };
-
 }
-#endif
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index e486479..25ffa13 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -21,23 +21,21 @@
 #include <sys/sysctl.h>
 #endif
 
-#include <mmintrin.h>
 #include <sched.h>
 #include <stdlib.h>
-#include <string.h>
 #include <unistd.h>
 #include <algorithm>
 #include <fstream>
-#include <iostream>
-#include <sstream>
-#include <boost/algorithm/string.hpp>
+#include <boost/algorithm/string/predicate.hpp>
+#include <boost/algorithm/string/trim.hpp>
 #include <boost/filesystem.hpp>
 #include <sys/sysinfo.h>
 
+#include "common/compiler-util.h"
 #include "common/config.h"
+#include "gen-cpp/Metrics_types.h"
 #include "gutil/strings/substitute.h"
 #include "util/pretty-printer.h"
-#include "util/string-parser.h"
 
 #include "common/names.h"
 
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index e60babc..c82f98c 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -15,14 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_CPU_INFO_H
-#define IMPALA_UTIL_CPU_INFO_H
-
+#include <cstdint>
 #include <memory>
 #include <string>
 #include <vector>
-#include <boost/cstdint.hpp>
 
 #include "common/logging.h"
 
@@ -223,4 +221,3 @@ class CpuInfo {
   static std::vector<int> numa_node_core_idx_;
 };
 }
-#endif
diff --git a/be/src/util/cyclic-barrier.cc b/be/src/util/cyclic-barrier.cc
index 2eb3640..38b9cc0 100644
--- a/be/src/util/cyclic-barrier.cc
+++ b/be/src/util/cyclic-barrier.cc
@@ -17,6 +17,9 @@
 
 #include "util/cyclic-barrier.h"
 
+#include <boost/thread/lock_guard.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+
 #include "common/names.h"
 
 namespace impala {
diff --git a/be/src/util/cyclic-barrier.h b/be/src/util/cyclic-barrier.h
index 83c0f89..0854464 100644
--- a/be/src/util/cyclic-barrier.h
+++ b/be/src/util/cyclic-barrier.h
@@ -17,8 +17,12 @@
 
 #pragma once
 
-#include <boost/thread.hpp>
+#include <cstdint>
 
+#include <boost/thread/lock_types.hpp>
+#include <boost/thread/pthread/mutex.hpp>
+
+#include "common/logging.h"
 #include "common/status.h"
 #include "util/condition-variable.h"
 
diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index 822098e..dfeb067 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -17,26 +17,35 @@
 
 #include "util/debug-util.h"
 
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
 #include <iomanip>
-#include <random>
 #include <sstream>
-#include <boost/algorithm/string.hpp>
+#include <utility>
+
+#include <boost/algorithm/string/classification.hpp>
+#include <boost/algorithm/string/constants.hpp>
+#include <boost/algorithm/string/predicate.hpp>
+#include <boost/algorithm/string/split.hpp>
 #include <boost/tokenizer.hpp>
+#include <gflags/gflags.h>
 
 #include "common/version.h"
 #include "runtime/collection-value.h"
 #include "runtime/descriptors.h"
-#include "runtime/exec-env.h"
-#include "runtime/raw-value.inline.h"
-#include "runtime/tuple-row.h"
+#include "runtime/raw-value.h"
 #include "runtime/row-batch.h"
-#include "util/cpu-info.h"
+#include "runtime/tuple-row.h"
+#include "runtime/tuple.h"
+#include "runtime/types.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
 #include "util/string-parser.h"
-#include "util/uid-util.h"
 #include "util/time.h"
 
+#include "common/names.h"
+
 // / WARNING this uses a private API of GLog: DumpStackTraceToString().
 namespace google {
 namespace glog_internal_namespace_ {
@@ -300,7 +309,7 @@ string GetBackendString() {
 
 DebugActionTokens TokenizeDebugActions(const string& debug_actions) {
   DebugActionTokens results;
-  list<string> actions;
+  vector<string> actions;
   split(actions, debug_actions, is_any_of("|"), token_compress_on);
   for (const string& a : actions) {
     vector<string> components;
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index d0fcad8..e53a105 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -15,25 +15,27 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_DEBUG_UTIL_H
-#define IMPALA_UTIL_DEBUG_UTIL_H
+#pragma once
 
-#include <ostream>
 #include <string>
-#include <sstream>
+#include <vector>
 
 #include <thrift/protocol/TDebugProtocol.h>
 
+#include "common/compiler-util.h"
 #include "common/config.h"
-#include "gen-cpp/JniCatalog_types.h"
-#include "gen-cpp/Descriptors_types.h"
-#include "gen-cpp/Exprs_types.h"
+#include "common/status.h"
+#include "gen-cpp/CatalogObjects_types.h"
 #include "gen-cpp/Frontend_types.h"
-#include "gen-cpp/PlanNodes_types.h"
-#include "gen-cpp/RuntimeProfile_types.h"
+#include "gen-cpp/ImpalaInternalService_types.h"
 #include "gen-cpp/ImpalaService_types.h"
+#include "gen-cpp/JniCatalog_types.h"
+#include "gen-cpp/Metrics_types.h"
+#include "gen-cpp/PlanNodes_types.h"
+#include "gen-cpp/Types_types.h"
+#include "gen-cpp/beeswax_types.h"
 #include "gen-cpp/parquet_types.h"
-
+#include "gutil/macros.h"
 #include "runtime/descriptors.h" // for SchemaPath
 
 namespace impala {
@@ -134,7 +136,7 @@ std::string GetBackendString();
 /// Tokenize 'debug_actions' into a list of tokenized rows, where columns are separated
 /// by ':' and rows by '|'. i.e. if debug_actions="a:b:c|x:y", then the returned
 /// structure is {{"a", "b", "c"}, {"x", "y"}}
-typedef std::list<std::vector<string>> DebugActionTokens;
+typedef std::vector<std::vector<string>> DebugActionTokens;
 DebugActionTokens TokenizeDebugActions(const string& debug_actions);
 
 /// Tokenize 'action' which has an optional parameter separated by '@'. i.e. "x@y"
@@ -189,7 +191,4 @@ static inline void DebugActionNoFail(
 #define FILE_CHECK_LT(a, b) DCHECK_LT(a, b)
 #define FILE_CHECK_GE(a, b) DCHECK_GE(a, b)
 #define FILE_CHECK_LE(a, b) DCHECK_LE(a, b)
-
 }
-
-#endif
diff --git a/be/src/util/decimal-util.cc b/be/src/util/decimal-util.cc
index b6196f2..031c490 100644
--- a/be/src/util/decimal-util.cc
+++ b/be/src/util/decimal-util.cc
@@ -16,7 +16,6 @@
 // under the License.
 
 #include "util/decimal-util.h"
-#include "runtime/types.h"
 
 namespace impala {
 
diff --git a/be/src/util/decimal-util.h b/be/src/util/decimal-util.h
index 5b79a4b..d35d1f5 100644
--- a/be/src/util/decimal-util.h
+++ b/be/src/util/decimal-util.h
@@ -15,17 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_DECIMAL_UTIL_H
-#define IMPALA_UTIL_DECIMAL_UTIL_H
+#include <endian.h>
 
+#include <cstdint>
 #include <functional>
-#include <ostream>
-#include <string>
-#include <boost/cstdint.hpp>
 
+#include "common/compiler-util.h"
+#include "common/logging.h"
 #include "runtime/multi-precision.h"
 #include "runtime/types.h"
+#include "util/arithmetic-util.h"
 #include "util/bit-util.h"
 
 namespace impala {
@@ -260,5 +261,3 @@ inline int128_t DecimalUtil::GetScaleMultiplier<int128_t>(int scale) {
   return -1;  // Overflow
 }
 }
-
-#endif
diff --git a/be/src/util/decompress.cc b/be/src/util/decompress.cc
index 000b585..3b81a6b 100644
--- a/be/src/util/decompress.cc
+++ b/be/src/util/decompress.cc
@@ -17,6 +17,10 @@
 
 #include "util/decompress.h"
 
+#include <strings.h>
+
+#include <sstream>
+
 // Codec libraries
 #include <zlib.h>
 #include <bzlib.h>
@@ -31,6 +35,9 @@
 #include "runtime/mem-pool.h"
 #include "runtime/mem-tracker.h"
 
+#include "common/compiler-util.h"
+#include "gen-cpp/ErrorCodes_types.h"
+
 #include "common/names.h"
 
 using namespace impala;
diff --git a/be/src/util/decompress.h b/be/src/util/decompress.h
index 0c41bcd..ee02b60 100644
--- a/be/src/util/decompress.h
+++ b/be/src/util/decompress.h
@@ -15,14 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_DECOMPRESS_H
-#define IMPALA_UTIL_DECOMPRESS_H
+#include <cstdint>
+#include <string>
 
 // We need zlib.h here to declare stream_ below.
 #include <zlib.h>
 #include <bzlib.h>
 
+#include "common/status.h"
+#include "gutil/strings/substitute.h"
 #include "util/codec.h"
 
 namespace impala {
@@ -159,4 +162,3 @@ class Lz4BlockDecompressor : public Codec {
   virtual std::string file_extension() const override { return "lz4"; }
 };
 }
-#endif
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index 4d811fe..f9c0cd5 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -37,6 +37,7 @@
 #include "util/jni-util.h"
 #include "util/mem-info.h"
 #include "util/memusage-path-handlers.h"
+#include "util/metrics.h"
 #include "util/pprof-path-handlers.h"
 #include "util/process-state-info.h"
 #include "util/runtime-profile-counters.h"
diff --git a/be/src/util/disk-info.cc b/be/src/util/disk-info.cc
index 25d9f02..6b55127 100644
--- a/be/src/util/disk-info.cc
+++ b/be/src/util/disk-info.cc
@@ -17,24 +17,20 @@
 
 #include "util/disk-info.h"
 
+#include <stdlib.h>
 #include <regex>
-#ifdef __APPLE__
-#include <sys/mount.h>
-#else
-#include <sys/vfs.h>
-#endif
-#include <sys/types.h>
-#include <sys/sysmacros.h>
 #include <sys/stat.h>
-#include <unistd.h>
+#include <sys/sysmacros.h>
+#include <sys/types.h>
 
-#include <boost/algorithm/string.hpp>
-#include <boost/algorithm/string/join.hpp>
-#include <iostream>
 #include <fstream>
-#include <sstream>
+#include <utility>
+#include <boost/algorithm/string/classification.hpp>
+#include <boost/algorithm/string/constants.hpp>
+#include <boost/algorithm/string/split.hpp>
+#include <boost/algorithm/string/trim.hpp>
 
-#include "util/debug-util.h"
+#include "gutil/strings/substitute.h"
 
 #include "common/names.h"
 
diff --git a/be/src/util/disk-info.h b/be/src/util/disk-info.h
index 42c188e..00425d4 100644
--- a/be/src/util/disk-info.h
+++ b/be/src/util/disk-info.h
@@ -15,14 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_DISK_INFO_H
-#define IMPALA_UTIL_DISK_INFO_H
-
+#include <cstdint>
 #include <map>
 #include <string>
+#include <vector>
 
-#include <boost/cstdint.hpp>
 #include "common/logging.h"
 
 namespace impala {
@@ -108,4 +107,3 @@ class DiskInfo {
   static bool TryNVMETrim(const std::string& name_in, std::string* basename_out);
 };
 }
-#endif
diff --git a/be/src/util/dynamic-util.cc b/be/src/util/dynamic-util.cc
index b57ed47..a6c178b 100644
--- a/be/src/util/dynamic-util.cc
+++ b/be/src/util/dynamic-util.cc
@@ -19,7 +19,6 @@
 
 #include <dlfcn.h>
 #include <sstream>
-#include "runtime/exec-env.h"
 #include "util/test-info.h"
 
 #include "common/names.h"
diff --git a/be/src/util/event-metrics.cc b/be/src/util/event-metrics.cc
index 451cbcd..c878f0c 100644
--- a/be/src/util/event-metrics.cc
+++ b/be/src/util/event-metrics.cc
@@ -17,6 +17,15 @@
 
 #include "util/event-metrics.h"
 
+#include <ostream>
+#include <unordered_map>
+
+#include <gflags/gflags_declare.h>
+
+#include "common/logging.h"
+#include "gen-cpp/JniCatalog_types.h"
+#include "util/metrics.h"
+
 DECLARE_int32(hms_event_polling_interval_s);
 
 namespace impala {
@@ -113,4 +122,4 @@ void MetastoreEventMetrics::refresh(TEventProcessorMetrics* response) {
     LAST_SYNCED_EVENT_ID->SetValue(response->last_synced_event_id);
   }
 }
-} // namespace impala
\ No newline at end of file
+} // namespace impala
diff --git a/be/src/util/event-metrics.h b/be/src/util/event-metrics.h
index ffbfa66..e1a13c6 100644
--- a/be/src/util/event-metrics.h
+++ b/be/src/util/event-metrics.h
@@ -15,10 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "util/metrics.h"
+#pragma once
+
+#include <string>
+
+#include "gutil/strings/substitute.h"
+#include "util/metrics-fwd.h"
 
 namespace impala {
 
+class TEventProcessorMetrics;
+
 /// class which is used to refresh the metastore event related metrics from catalog
 class MetastoreEventMetrics {
  public:
@@ -62,31 +69,31 @@ class MetastoreEventMetrics {
   /// Following metric names must match with the key in metrics.json
 
   /// metric name for events received counter.
-  static string NUMBER_EVENTS_RECEIVED_METRIC_NAME;
+  static std::string NUMBER_EVENTS_RECEIVED_METRIC_NAME;
 
   /// metric name for events skipped counter
-  static string NUMBER_EVENTS_SKIPPED_METRIC_NAME;
+  static std::string NUMBER_EVENTS_SKIPPED_METRIC_NAME;
 
   /// metric name for event processor status
-  static string EVENT_PROCESSOR_STATUS_METRIC_NAME;
+  static std::string EVENT_PROCESSOR_STATUS_METRIC_NAME;
 
   /// metric name for the mean time taken for events fetch metric
-  static string EVENTS_FETCH_DURATION_MEAN_METRIC_NAME;
+  static std::string EVENTS_FETCH_DURATION_MEAN_METRIC_NAME;
 
   /// metric name for the mean time taken for events processing metric
-  static string EVENTS_PROCESS_DURATION_MEAN_METRIC_NAME;
+  static std::string EVENTS_PROCESS_DURATION_MEAN_METRIC_NAME;
 
   /// metric name for EWMA of number of events in last 1 min
-  static string EVENTS_RECEIVED_1MIN_METRIC_NAME;
+  static std::string EVENTS_RECEIVED_1MIN_METRIC_NAME;
 
   /// metric name for EWMA of number of events in last 5 min
-  static string EVENTS_RECEIVED_5MIN_METRIC_NAME;
+  static std::string EVENTS_RECEIVED_5MIN_METRIC_NAME;
 
   /// metric name for EWMA of number of events in last 15 min
-  static string EVENTS_RECEIVED_15MIN_METRIC_NAME;
+  static std::string EVENTS_RECEIVED_15MIN_METRIC_NAME;
 
   /// Metric name for last metastore event id that the catalog server synced to.
-  static string LAST_SYNCED_EVENT_ID_METRIC_NAME;
+  static std::string LAST_SYNCED_EVENT_ID_METRIC_NAME;
 };
 
 } // namespace impala
diff --git a/be/src/util/filesystem-util.cc b/be/src/util/filesystem-util.cc
index bc9d7fd..ea0a10a 100644
--- a/be/src/util/filesystem-util.cc
+++ b/be/src/util/filesystem-util.cc
@@ -15,22 +15,37 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <dirent.h>
+#include <errno.h>
 #include <fcntl.h>
+#include <limits.h>
+#include <stdlib.h>
 #include <string.h>
+#include <unistd.h>
 #include <sys/resource.h>
 #include <sys/stat.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
 #include <boost/filesystem.hpp>
-#include <gutil/strings/numbers.h>
-#include <gutil/strings/substitute.h>
-#include <gutil/strings/util.h>
 
-#include "exec/kudu-util.h"
+#include "common/logging.h"
+#include "common/status.h"
+#include "gen-cpp/ErrorCodes_types.h"
 #include "gutil/macros.h"
-#include "runtime/io/error-converter.h"
+#include "gutil/strings/numbers.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/util.h"
 #include "kudu/util/env.h"
 #include "kudu/util/path_util.h"
-#include "util/filesystem-util.h"
+#include "kudu/util/slice.h"
+#include "runtime/io/error-converter.h"
+#include "util/debug-util.h"
 #include "util/error-util.h"
+#include "util/filesystem-util.h"
+#include "util/kudu-status-util.h"
 #include "util/scope-exit-trigger.h"
 #include "util/uid-util.h"
 
diff --git a/be/src/util/flat_buffer.h b/be/src/util/flat_buffer.h
index 916db4e..885e97d 100644
--- a/be/src/util/flat_buffer.h
+++ b/be/src/util/flat_buffer.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_FLAT_BUFFER_H
-#define IMPALA_UTIL_FLAT_BUFFER_H
+#pragma once
 
 #include "common/status.h"
 
 #include "gen-cpp/CatalogObjects_generated.h"
-#include "gen-cpp/ImpalaInternalService_constants.h"
+#include "gen-cpp/CatalogObjects_types.h"
 
 namespace impala {
 
@@ -30,4 +29,3 @@ namespace impala {
 Status FromFbCompression(org::apache::impala::fb::FbCompression fb_compression,
     THdfsCompression::type* thrift_compression);
 } // namespace impala
-#endif // IMPALA_UTIL_FLAT_BUFFER_H
diff --git a/be/src/util/hdfs-util.cc b/be/src/util/hdfs-util.cc
index d344309..a98394e 100644
--- a/be/src/util/hdfs-util.cc
+++ b/be/src/util/hdfs-util.cc
@@ -20,11 +20,12 @@
 #include <sstream>
 #include <string.h>
 
+#include "common/logging.h"
+#include "kudu/util/path_util.h"
+#include "runtime/exec-env.h"
 #include "util/error-util.h"
 
 #include "common/names.h"
-#include "runtime/exec-env.h"
-#include "kudu/util/path_util.h"
 
 namespace impala {
 
diff --git a/be/src/util/hdfs-util.h b/be/src/util/hdfs-util.h
index f57efd9..c1c253e 100644
--- a/be/src/util/hdfs-util.h
+++ b/be/src/util/hdfs-util.h
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_HDFS_UTIL_H
-#define IMPALA_UTIL_HDFS_UTIL_H
+#pragma once
 
 #include <string>
 #include <hdfs.h>
+
 #include "common/status.h"
 
 namespace impala {
@@ -62,6 +61,5 @@ bool FilesystemsMatch(const char* pathA, const char* pathB);
 /// Returns the terminal component of 'path'.
 /// E.g. if 'path' is "hdfs://localhost:8020/a/b/c", "c" is returned.
 /// If the terminal component is empty string or "/", the function returns ".".
-string GetBaseName(const char* path);
+std::string GetBaseName(const char* path);
 }
-#endif // IMPALA_UTIL_HDFS_UTIL_H
diff --git a/be/src/util/internal-queue-test.cc b/be/src/util/internal-queue-test.cc
index 6739e4a..aaa49e0 100644
--- a/be/src/util/internal-queue-test.cc
+++ b/be/src/util/internal-queue-test.cc
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
+#include <unistd.h>
 #include <boost/thread.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
-#include <unistd.h>
 
 #include "common/init.h"
 #include "common/atomic.h"
diff --git a/be/src/util/mem-info.h b/be/src/util/mem-info.h
index 2e32d94..cf4d23e 100644
--- a/be/src/util/mem-info.h
+++ b/be/src/util/mem-info.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_MEM_INFO_H
-#define IMPALA_UTIL_MEM_INFO_H
+#pragma once
 
+#include <cstdint>
 #include <string>
-#include <boost/cstdint.hpp>
 
 #include "common/logging.h"
 
@@ -121,6 +120,4 @@ class MemInfo {
   /// If overcommit is turned off the maximum allocatable memory
   static int64_t commit_limit_;
 };
-
 }
-#endif
diff --git a/be/src/util/memory-metrics.h b/be/src/util/memory-metrics.h
index bdb083f..9dc9eab 100644
--- a/be/src/util/memory-metrics.h
+++ b/be/src/util/memory-metrics.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_MEM_METRICS_H
-#define IMPALA_UTIL_MEM_METRICS_H
+#pragma once
 
 #include "util/metrics.h"
 
 #include <boost/bind.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <gperftools/malloc_extension.h>
 #if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER)
 #include <sanitizer/allocator_interface.h>
@@ -338,5 +337,3 @@ class MemTrackerMetric : public IntGauge {
 Status RegisterMemoryMetrics(MetricGroup* metrics, bool register_jvm_metrics,
     ReservationTracker* global_reservations, BufferPool* buffer_pool);
 }
-
-#endif
diff --git a/be/src/util/memusage-path-handlers.cc b/be/src/util/memusage-path-handlers.cc
index d45fded..d8bc889 100644
--- a/be/src/util/memusage-path-handlers.cc
+++ b/be/src/util/memusage-path-handlers.cc
@@ -22,6 +22,7 @@
 #include "runtime/mem-tracker.h"
 #include "util/common-metrics.h"
 #include "util/mem-info.h"
+#include "util/metrics.h"
 #include "util/pretty-printer.h"
 
 #include "common/names.h"
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index 4f5fcb4..a2b174d 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -17,8 +17,8 @@
 
 #pragma once
 
+#include <iosfwd>
 #include <map>
-#include <sstream>
 #include <string>
 #include <vector>
 #include <boost/function.hpp>
diff --git a/be/src/util/parse-util.h b/be/src/util/parse-util.h
index 7cd12d5..40660e5 100644
--- a/be/src/util/parse-util.h
+++ b/be/src/util/parse-util.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_PARSE_UTIL_H
-#define IMPALA_UTIL_PARSE_UTIL_H
+#pragma once
 
+#include <cstdint>
 #include <string>
-#include <boost/cstdint.hpp>
 
 namespace impala {
 
@@ -42,7 +41,4 @@ class ParseUtil {
   static int64_t ParseMemSpec(const std::string& mem_spec_str,
       bool* is_percent, int64_t relative_reference);
 };
-
 }
-
-#endif
diff --git a/be/src/util/periodic-counter-updater.h b/be/src/util/periodic-counter-updater.h
index 3c63d6d..ed3676e 100644
--- a/be/src/util/periodic-counter-updater.h
+++ b/be/src/util/periodic-counter-updater.h
@@ -15,13 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_PERIODIC_COUNTER_UPDATER_H
-#define IMPALA_UTIL_PERIODIC_COUNTER_UPDATER_H
+#pragma once
 
 #include <boost/function.hpp>
 #include <boost/scoped_ptr.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <boost/thread/thread.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/unordered_set.hpp>
@@ -155,7 +153,4 @@ class PeriodicCounterUpdater {
   /// for updating them.
   static PeriodicCounterUpdater* instance_;
 };
-
 }
-
-#endif
diff --git a/be/src/util/process-state-info.h b/be/src/util/process-state-info.h
index 702861d..1a59ad5 100644
--- a/be/src/util/process-state-info.h
+++ b/be/src/util/process-state-info.h
@@ -15,12 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_PROCESS_STATE_INFO_H
-#define IMPALA_UTIL_PROCESS_STATE_INFO_H
+#pragma once
 
+#include <cstdint>
 #include <map>
 #include <string>
-#include <boost/cstdint.hpp>
 
 #include "common/logging.h"
 
@@ -111,7 +110,4 @@ class ProcessStateInfo {
   /// Read the number of currently open file descriptors from /proc/self/fd.
   void ReadProcFileDescriptorCount();
 };
-
 }
-
-#endif
diff --git a/be/src/util/progress-updater.h b/be/src/util/progress-updater.h
index af2e0da..b3e9262 100644
--- a/be/src/util/progress-updater.h
+++ b/be/src/util/progress-updater.h
@@ -15,12 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_PROGRESS_UPDATER_H
-#define IMPALA_UTIL_PROGRESS_UPDATER_H
-
+#include <cstdint>
 #include <string>
-#include <boost/cstdint.hpp>
 
 #include "common/atomic.h"
 
@@ -83,7 +81,4 @@ class ProgressUpdater {
   /// Percentage when the last output was generated.
   AtomicInt32 last_output_percentage_;
 };
-
 }
-
-#endif
diff --git a/be/src/util/runtime-profile-counters.h b/be/src/util/runtime-profile-counters.h
index c006b64..2c0b740 100644
--- a/be/src/util/runtime-profile-counters.h
+++ b/be/src/util/runtime-profile-counters.h
@@ -350,7 +350,7 @@ class RuntimeProfile::HighWaterMarkCounter : public RuntimeProfile::Counter {
     UpdateMax(v);
   }
 
-  string CounterType() const override {
+  std::string CounterType() const override {
     return "HighWaterMarkCounter";
   }
 
@@ -385,7 +385,7 @@ class RuntimeProfile::DerivedCounter : public RuntimeProfile::Counter {
     return counter_fn_();
   }
 
-  string CounterType() const override {
+  std::string CounterType() const override {
     return "DerivedCounter";
   }
 
@@ -433,7 +433,7 @@ class RuntimeProfile::AveragedCounter : public RuntimeProfile::Counter {
     }
   }
 
-  string CounterType() const override {
+  std::string CounterType() const override {
     return "AveragedCounter";
   }
 
@@ -506,7 +506,7 @@ class RuntimeProfile::SummaryStatsCounter : public RuntimeProfile::Counter {
     val->AddMember("num_of_samples", total_num_values_, document.GetAllocator());
   }
 
-  string CounterType() const override {
+  std::string CounterType() const override {
     return "SummaryStatsCounter";
   }
 
@@ -812,7 +812,7 @@ class RuntimeProfile::ConcurrentTimerCounter : public Counter {
     DCHECK(false);
   }
 
-  string CounterType() const override {
+  std::string CounterType() const override {
     return "ConcurrentTimerCounter";
   }
 
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index 65e97d7..1b086f6 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -15,13 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_RUNTIME_PROFILE_H
-#define IMPALA_UTIL_RUNTIME_PROFILE_H
+#pragma once
 
 #include <boost/function.hpp>
 #include <boost/thread/lock_guard.hpp>
-#include <iostream>
+#include <iosfwd>
 #include <rapidjson/document.h>
 #include "common/atomic.h"
 #include "common/status.h"
@@ -625,7 +623,4 @@ class RuntimeProfile { // NOLINT: This struct is not packed, but there are not s
       const string& counter_name, const CounterMap& counter_map,
       const ChildCounterMap& child_counter_map) const;
 };
-
 }
-
-#endif
diff --git a/be/src/util/sharded-query-map-util.h b/be/src/util/sharded-query-map-util.h
index 59dc2d5..1c66cfe 100644
--- a/be/src/util/sharded-query-map-util.h
+++ b/be/src/util/sharded-query-map-util.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef SHARDED_QUERY_MAP_UTIL_H
-#define SHARDED_QUERY_MAP_UTIL_H
+#pragma once
 
 #include <boost/thread/lock_guard.hpp>
 #include <unordered_map>
@@ -24,7 +23,7 @@
 #include "gen-cpp/Types_types.h"
 #include "util/aligned-new.h"
 #include "util/spinlock.h"
-#include "util/uid-util.h"
+#include "util/unique-id-hash.h"
 
 namespace impala {
 
@@ -131,5 +130,3 @@ class ScopedShardedMapRef {
 };
 
 } // namespace impala
-
-#endif /* SHARDED_QUERY_MAP_UTIL_H */
diff --git a/be/src/util/simple-logger.h b/be/src/util/simple-logger.h
index 7018ac6..6e101c5 100644
--- a/be/src/util/simple-logger.h
+++ b/be/src/util/simple-logger.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_SERVICE_SIMPLE_LOGGER_H
-#define IMPALA_SERVICE_SIMPLE_LOGGER_H
+#pragma once
 
 #include <fstream>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "common/status.h"
 
@@ -87,4 +86,3 @@ class SimpleLogger {
   void RotateLogFiles();
 };
 }
-#endif
diff --git a/be/src/util/stopwatch.h b/be/src/util/stopwatch.h
index 60cdf42..0c51a4f 100644
--- a/be/src/util/stopwatch.h
+++ b/be/src/util/stopwatch.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_STOPWATCH_H
-#define IMPALA_UTIL_STOPWATCH_H
+#include <cstdint>
 
-#include <boost/cstdint.hpp>
 #include <boost/thread/lock_guard.hpp>
 #include <util/os-info.h>
 #include <util/spinlock.h>
@@ -266,7 +265,4 @@ class ScopedStopWatch {
   T* sw_;
   bool enabled_;
 };
-
 }
-
-#endif
diff --git a/be/src/util/streaming-sampler.h b/be/src/util/streaming-sampler.h
index d517d37..b395ace 100644
--- a/be/src/util/streaming-sampler.h
+++ b/be/src/util/streaming-sampler.h
@@ -15,11 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_STREAMING_SAMPLER_H
-#define IMPALA_UTIL_STREAMING_SAMPLER_H
+#pragma once
 
 #include <string.h>
-#include <iostream>
 
 #include "util/ubsan.h"
 
@@ -105,7 +103,4 @@ class StreamingSampler {
   /// The total time that current_sample_sum_ represents
   int current_sample_total_time_;
 };
-
 }
-
-#endif
diff --git a/be/src/util/string-parser-test.cc b/be/src/util/string-parser-test.cc
index aa4de67..0eaef06 100644
--- a/be/src/util/string-parser-test.cc
+++ b/be/src/util/string-parser-test.cc
@@ -15,14 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdlib.h>
+#include "util/string-parser.h"
 #include <stdio.h>
+#include <stdlib.h>
+#include <cstdint>
 #include <iostream>
 #include <limits>
-#include <boost/cstdint.hpp>
 #include <boost/lexical_cast.hpp>
 #include "testutil/gtest-util.h"
-#include "util/string-parser.h"
 
 #include "common/names.h"
 
diff --git a/be/src/util/system-state-info.h b/be/src/util/system-state-info.h
index c17f562..a4c9b88 100644
--- a/be/src/util/system-state-info.h
+++ b/be/src/util/system-state-info.h
@@ -20,7 +20,6 @@
 #include <array>
 #include <cstdint>
 #include <string>
-#include <sstream>
 
 #include <gtest/gtest_prod.h> // for FRIEND_TEST
 
diff --git a/be/src/util/thread-pool-test.cc b/be/src/util/thread-pool-test.cc
index 3633ae9..4aa2d94 100644
--- a/be/src/util/thread-pool-test.cc
+++ b/be/src/util/thread-pool-test.cc
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <unistd.h>
 #include <boost/thread.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 #include <glog/logging.h>
-#include <unistd.h>
 
 #include "common/logging.h"
 #include "testutil/gtest-util.h"
diff --git a/be/src/util/thread-pool.h b/be/src/util/thread-pool.h
index 9d5d8c9..dca55a3 100644
--- a/be/src/util/thread-pool.h
+++ b/be/src/util/thread-pool.h
@@ -15,13 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_THREAD_POOL_H
-#define IMPALA_UTIL_THREAD_POOL_H
+#pragma once
 
 #include "util/blocking-queue.h"
 
-#include <boost/thread/mutex.hpp>
 #include <boost/bind/mem_fn.hpp>
+#include <boost/thread/pthread/mutex.hpp>
 
 #include "util/aligned-new.h"
 #include "util/condition-variable.h"
@@ -347,7 +346,4 @@ class SynchronousThreadPool : public ThreadPool<std::shared_ptr<SynchronousWorkI
     work->WorkerExecute();
   }
 };
-
 }
-
-#endif
diff --git a/be/src/util/uid-util-test.cc b/be/src/util/uid-util-test.cc
index 0be9f7f..1f4c9ee 100644
--- a/be/src/util/uid-util-test.cc
+++ b/be/src/util/uid-util-test.cc
@@ -19,6 +19,8 @@
 #include <stdio.h>
 #include <iostream>
 
+#include <boost/uuid/uuid_generators.hpp>
+
 #include "testutil/gtest-util.h"
 #include "util/container-util.h"
 #include "util/uid-util.h"
diff --git a/be/src/util/decimal-util.cc b/be/src/util/uid-util.cc
similarity index 75%
copy from be/src/util/decimal-util.cc
copy to be/src/util/uid-util.cc
index b6196f2..fac5be7 100644
--- a/be/src/util/decimal-util.cc
+++ b/be/src/util/uid-util.cc
@@ -15,13 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "util/decimal-util.h"
-#include "runtime/types.h"
+#include "util/uid-util.h"
 
-namespace impala {
+#include <boost/uuid/uuid_generators.hpp>
 
-const int32_t DecimalUtil::MAX_UNSCALED_DECIMAL4;
-const int64_t DecimalUtil::MAX_UNSCALED_DECIMAL8;
-const int128_t DecimalUtil::MAX_UNSCALED_DECIMAL16;
+namespace impala {
 
+string GenerateUUIDString() {
+  boost::uuids::basic_random_generator<boost::mt19937> gen;
+  boost::uuids::uuid u = gen();
+  string uuid(u.begin(), u.end());
+  return uuid;
 }
+} // namespace impala
diff --git a/be/src/util/uid-util.h b/be/src/util/uid-util.h
index 03b335a..aaa6f09 100644
--- a/be/src/util/uid-util.h
+++ b/be/src/util/uid-util.h
@@ -15,13 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_UID_UTIL_H
-#define IMPALA_UTIL_UID_UTIL_H
-
-#include <boost/functional/hash.hpp>
 #include <boost/uuid/uuid.hpp>
-#include <boost/uuid/uuid_generators.hpp>
 
 #include "gen-cpp/Types_types.h"  // for TUniqueId
 #include "gen-cpp/control_service.pb.h"
@@ -29,28 +25,6 @@
 
 namespace impala {
 
-inline std::size_t hash_value(const TUniqueId& id) {
-  std::size_t seed = 0;
-  boost::hash_combine(seed, id.lo);
-  boost::hash_combine(seed, id.hi);
-  return seed;
-}
-
-}
-
-/// Hash function for std:: containers
-namespace std {
-
-template<> struct hash<impala::TUniqueId> {
-  std::size_t operator()(const impala::TUniqueId& id) const {
-    return impala::hash_value(id);
-  }
-};
-
-}
-
-namespace impala {
-
 inline void UUIDToTUniqueId(const boost::uuids::uuid& uuid, TUniqueId* unique_id) {
   memcpy(&(unique_id->hi), &uuid.data[0], 8);
   memcpy(&(unique_id->lo), &uuid.data[8], 8);
@@ -119,12 +93,7 @@ inline T CastTUniqueId(const F& from) {
 }
 
 /// generates a 16 byte UUID
-inline string GenerateUUIDString() {
-  boost::uuids::basic_random_generator<boost::mt19937> gen;
-  boost::uuids::uuid u = gen();
-  string uuid(u.begin(), u.end());
-  return uuid;
-}
+std::string GenerateUUIDString();
 
 /// generates a 16 byte UUID
 inline TUniqueId GenerateUUID() {
@@ -134,6 +103,4 @@ inline TUniqueId GenerateUUID() {
   memcpy(&uid.lo, u.data() + sizeof(int64_t), sizeof(int64_t));
   return uid;
 }
-
 } // namespace impala
-#endif
diff --git a/be/src/util/cyclic-barrier.cc b/be/src/util/unique-id-hash.h
similarity index 61%
copy from be/src/util/cyclic-barrier.cc
copy to be/src/util/unique-id-hash.h
index 2eb3640..1a73d8d 100644
--- a/be/src/util/cyclic-barrier.cc
+++ b/be/src/util/unique-id-hash.h
@@ -15,21 +15,33 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "util/cyclic-barrier.h"
+// Defines hash functions for TUniqueId.
 
-#include "common/names.h"
+#pragma once
 
-namespace impala {
+#include <boost/functional/hash.hpp>
 
-CyclicBarrier::CyclicBarrier(int num_threads) : num_threads_(num_threads) {}
+#include "gen-cpp/Types_types.h" // for TUniqueId
 
-void CyclicBarrier::Cancel(const Status& err) {
-  DCHECK(!err.ok());
-  {
-    lock_guard<mutex> l(lock_);
-    if (!cancel_status_.ok()) return; // Already cancelled.
-    cancel_status_ = err;
-  }
-  barrier_cv_.NotifyAll();
+namespace impala {
+
+inline std::size_t hash_value(const TUniqueId& id) {
+  std::size_t seed = 0;
+  boost::hash_combine(seed, id.lo);
+  boost::hash_combine(seed, id.hi);
+  return seed;
 }
+
 } // namespace impala
+
+/// Hash function for std:: containers
+namespace std {
+
+template <>
+struct hash<impala::TUniqueId> {
+  std::size_t operator()(const impala::TUniqueId& id) const {
+    return impala::hash_value(id);
+  }
+};
+
+} // namespace std
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index 06cae82..24ebbcc 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -20,7 +20,7 @@
 #include <map>
 #include <string>
 #include <boost/function.hpp>
-#include <boost/thread/shared_mutex.hpp>
+#include <boost/thread/pthread/shared_mutex.hpp>
 #include <rapidjson/fwd.h>
 
 #include "common/status.h"
@@ -119,8 +119,8 @@ class Webserver {
   bool IsSecure() const;
 
   /// Returns the URL to the webserver as a string.
-  string url() { return url_; }
-  string hostname() { return hostname_; }
+  std::string url() { return url_; }
+  std::string hostname() { return hostname_; }
   int port() { return http_address_.port; }
 
   /// Returns the appropriate MIME type for a given ContentType.