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

[impala] 04/04: IMPALA-9335 (part 2): Fix rebased KRPC to compile

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

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

commit 19a4d8fe794c9b17e69d6c65473f9a68084916bb
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Wed Jan 22 17:10:09 2020 -0800

    IMPALA-9335 (part 2): Fix rebased KRPC to compile
    
    This patch applies various fixes to Impala and to the copied Kudu
    source code in be/src/kudu/* to allow everything to compile.
    
    Some highlights of the changes made:
    - Various Kudu files were removed from compilation due to issues like
      relying on libraries that Impala does not provide. The linking of
      some executable is also changed for similar reasons.
    - The Kudu Cache implementation changed to support unique_ptr,
      allowing us to remove various uses of MakeScopeExitTrigger.
    - Some flags that have a DEFINE in both Kudu and Impala are modified
      to change one of the DEFINEs to a DECLARE.
    
    This patch was in part based on the patches that were applied the last
    time we rebased the Kudu code in IMPALA-7006, and I ensured that all
    changes from those commits that are still relevant were included here.
    
    I also went through all commits that have been applied to the
    be/src/kudu directory since the last rebase and ensured that all
    relevant changes from those are included here.
    
    Testing:
    - Passed an exhaustive DEBUG build and a core ASAN build.
    
    Change-Id: I1eb4caf927c729109426fb50a28b5e15d6ac46cb
    Reviewed-on: http://gerrit.cloudera.org:8080/15144
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Joe McDonnell <jo...@cloudera.com>
---
 be/src/kudu/rpc/CMakeLists.txt                | 17 ++++---
 be/src/kudu/rpc/transfer.cc                   |  4 +-
 be/src/kudu/security/CMakeLists.txt           |  9 ++++
 be/src/kudu/security/test/mini_kdc.cc         |  4 +-
 be/src/kudu/util/CMakeLists.txt               | 68 +++++++++++++++------------
 be/src/kudu/util/flags.cc                     |  5 +-
 be/src/kudu/util/kudu_export.h                | 62 ++++++++++++++++++++++++
 be/src/kudu/util/logging.cc                   | 13 +++--
 be/src/kudu/util/logging.h                    |  2 +-
 be/src/rpc/impala-service-pool.cc             |  7 ++-
 be/src/rpc/impala-service-pool.h              |  4 +-
 be/src/runtime/io/data-cache.cc               | 48 +++++++++----------
 be/src/runtime/io/data-cache.h                |  5 +-
 be/src/util/webserver.cc                      |  2 +
 bin/rat_exclude_files.txt                     |  2 +
 cmake_modules/kudu_cmake_fns.txt              |  9 ++++
 tests/custom_cluster/test_restart_services.py |  2 +-
 17 files changed, 177 insertions(+), 86 deletions(-)

diff --git a/be/src/kudu/rpc/CMakeLists.txt b/be/src/kudu/rpc/CMakeLists.txt
index 1190968..b666de5 100644
--- a/be/src/kudu/rpc/CMakeLists.txt
+++ b/be/src/kudu/rpc/CMakeLists.txt
@@ -90,13 +90,16 @@ ADD_EXPORTABLE_LIBRARY(krpc
   DEPS ${KRPC_LIBS})
 
 ### RPC generator tool
-add_executable(protoc-gen-krpc protoc-gen-krpc.cc)
-target_link_libraries(protoc-gen-krpc
-    ${KUDU_BASE_LIBS}
-    rpc_header_proto
-    protoc
-    protobuf
-    gutil)
+add_executable(protoc-gen-krpc protoc-gen-krpc.cc
+  # Impala - add stub for kudu::VersionInfo
+  ${CMAKE_CURRENT_SOURCE_DIR}/../../common/kudu_version.cc
+  # Impala - add definition for any flag names shared between Impala / Kudu.
+  # TODO: Consider either removing code that depends on these flags, or namespacing them
+  # somehow.
+  ${CMAKE_CURRENT_SOURCE_DIR}/../../common/global-flags.cc)
+# IMPALA-8642: kudu_version.cc depends on gen-cpp/Status_types.h in target thrift-deps
+add_dependencies(protoc-gen-krpc thrift-deps)
+target_link_libraries(protoc-gen-krpc gutil glog gflags protoc protobuf rpc_header_proto ${KUDU_BASE_LIBS})
 
 #### RPC test
 PROTOBUF_GENERATE_CPP(
diff --git a/be/src/kudu/rpc/transfer.cc b/be/src/kudu/rpc/transfer.cc
index b268e77..674a9f6 100644
--- a/be/src/kudu/rpc/transfer.cc
+++ b/be/src/kudu/rpc/transfer.cc
@@ -41,7 +41,9 @@ DEFINE_bool(rpc_max_message_size_enable_validation, true,
             "This is a test-only flag.");
 TAG_FLAG(rpc_max_message_size_enable_validation, unsafe);
 
-DEFINE_int64(rpc_max_message_size, (50 * 1024 * 1024),
+// Hidden in Impala because we require a particular value and override the user specified
+// value anyways, see RpcMgr::Init() and IMPALA-4874.
+DEFINE_int64_hidden(rpc_max_message_size, (50 * 1024 * 1024),
              "The maximum size of a message that any RPC that the server will accept. "
              "Must be at least 1MB.");
 TAG_FLAG(rpc_max_message_size, advanced);
diff --git a/be/src/kudu/security/CMakeLists.txt b/be/src/kudu/security/CMakeLists.txt
index 7abaabb..ed087c0 100644
--- a/be/src/kudu/security/CMakeLists.txt
+++ b/be/src/kudu/security/CMakeLists.txt
@@ -96,6 +96,15 @@ ADD_EXPORTABLE_LIBRARY(security
   SRCS ${SECURITY_SRCS}
   DEPS ${SECURITY_LIBS})
 
+# Since Kudu tests are explicitly disabled, we want to expose some of their sources
+# to Impala using another variable.
+set(SECURITY_TEST_SRCS_FOR_IMPALA test/mini_kdc.cc)
+add_library(security-test-for-impala ${SECURITY_TEST_SRCS_FOR_IMPALA})
+target_link_libraries(security-test-for-impala
+  gutil
+  kudu_test_util
+  kudu_util
+  security)
 
 ##############################
 # mini_kdc
diff --git a/be/src/kudu/security/test/mini_kdc.cc b/be/src/kudu/security/test/mini_kdc.cc
index 1662770..63e834b 100644
--- a/be/src/kudu/security/test/mini_kdc.cc
+++ b/be/src/kudu/security/test/mini_kdc.cc
@@ -63,7 +63,9 @@ MiniKdc::MiniKdc(MiniKdcOptions options)
     options_.realm = "KRBTEST.COM";
   }
   if (options_.data_root.empty()) {
-    options_.data_root = JoinPathSegments(GetTestDataDirectory(), "krb5kdc");
+    // We hardcode "/tmp" here since the original function which initializes a random test
+    // directory (GetTestDataDirectory()), depends on gmock.
+    options_.data_root = JoinPathSegments("/tmp", "krb5kdc");
   }
   if (options_.ticket_lifetime.empty()) {
     options_.ticket_lifetime = "24h";
diff --git a/be/src/kudu/util/CMakeLists.txt b/be/src/kudu/util/CMakeLists.txt
index 4c576bb..7f894bd 100644
--- a/be/src/kudu/util/CMakeLists.txt
+++ b/be/src/kudu/util/CMakeLists.txt
@@ -151,7 +151,7 @@ set(UTIL_SRCS
   block_bloom_filter.cc
   bloom_filter.cc
   cache.cc
-  char_util.cc
+  #char_util.cc
   coding.cc
   condition_variable.cc
   cow_object.cc
@@ -167,7 +167,7 @@ set(UTIL_SRCS
   errno.cc
   faststring.cc
   fault_injection.cc
-  file_cache.cc
+  #file_cache.cc
   file_cache_metrics.cc
   flags.cc
   flag_tags.cc
@@ -190,7 +190,7 @@ set(UTIL_SRCS
   memory/overwrite.cc
   mem_tracker.cc
   metrics.cc
-  minidump.cc
+  #minidump.cc
   monotime.cc
   mutex.cc
   net/dns_resolver.cc
@@ -229,11 +229,13 @@ set(UTIL_SRCS
   trace_metrics.cc
   user.cc
   url-coding.cc
-  version_info.cc
+  # Remove from compilation, as it depends on generated method calls. Replaced by
+  # kudu_version.cc in Impala's common library.
+  #version_info.cc
   version_util.cc
   web_callback_registry.cc
   website_util.cc
-  yamlreader.cc
+  #yamlreader.cc
   zlib.cc
 )
 
@@ -277,7 +279,7 @@ set(UTIL_LIBS
   pb_util_proto
   protobuf
   version_info_proto
-  yaml
+  #yaml
   zlib)
 
 if(NOT APPLE)
@@ -320,27 +322,29 @@ ADD_EXPORTABLE_LIBRARY(kudu_util_compression
 # Define LZ4_DISABLE_DEPRECATE_WARNINGS to mute warnings like:
 # "'int LZ4_compress(const char*, char*, int)' is deprecated".
 target_compile_definitions(kudu_util_compression PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS)
-target_compile_definitions(kudu_util_compression_exported PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS)
+#target_compile_definitions(kudu_util_compression_exported PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS)
 
 #######################################
 # kudu_curl_util
 #######################################
-add_library(kudu_curl_util
-  curl_util.cc)
-target_link_libraries(kudu_curl_util
-  security
-  ${CURL_LIBRARIES}
-  glog
-  gutil)
+# Impala doesn't have curl in its toolchain. It relies instead on the system curl, which
+# is too old on some OSes we compile on and doesn't include macros Kudu needs.
+#add_library(kudu_curl_util
+#  curl_util.cc)
+#target_link_libraries(kudu_curl_util
+#  security
+#  ${CURL_LIBRARIES}
+#  glog
+#  gutil)
 
 #######################################
 # kudu_cloud_util
 #######################################
-add_library(kudu_cloud_util
-  cloud/instance_detector.cc
-  cloud/instance_metadata.cc)
-target_link_libraries(kudu_cloud_util
-  kudu_curl_util)
+#add_library(kudu_cloud_util
+#  cloud/instance_detector.cc
+#  cloud/instance_metadata.cc)
+#target_link_libraries(kudu_cloud_util
+#  kudu_curl_util)
 
 # See the comment in sanitizer_options.cc for details on this library's usage.
 # The top-level CMakeLists sets a ${SANITIZER_OPTIONS_OVERRIDE} variable which
@@ -375,7 +379,9 @@ add_library(kudu_test_util
 target_link_libraries(kudu_test_util
   gflags
   glog
-  gmock
+  # Impala doesn't have gmock in its toolchain
+  gtest
+  #gmock
   kudu_util)
 
 #######################################
@@ -404,7 +410,7 @@ endif()
 #######################################
 
 add_executable(protoc-gen-insertions protoc-gen-insertions.cc)
-target_link_libraries(protoc-gen-insertions gutil protobuf protoc ${KUDU_BASE_LIBS})
+target_link_libraries(protoc-gen-insertions gutil glog gflags protoc protobuf ${KUDU_BASE_LIBS})
 
 #######################################
 # Unit tests
@@ -565,17 +571,17 @@ endif()
 #######################################
 # curl_util-test
 #######################################
-ADD_KUDU_TEST(curl_util-test)
-if(NOT NO_TESTS)
-  target_link_libraries(curl_util-test
-    kudu_curl_util)
-endif()
+#ADD_KUDU_TEST(curl_util-test)
+#if(NOT NO_TESTS)
+#  target_link_libraries(curl_util-test
+#    kudu_curl_util)
+#endif()
 
 #######################################
 # instance_detector-test
 #######################################
-ADD_KUDU_TEST(cloud/instance_detector-test)
-if(NOT NO_TESTS)
-  target_link_libraries(instance_detector-test
-    kudu_cloud_util)
-endif()
+#ADD_KUDU_TEST(cloud/instance_detector-test)
+#if(NOT NO_TESTS)
+#  target_link_libraries(instance_detector-test
+#    kudu_cloud_util)
+#endif()
diff --git a/be/src/kudu/util/flags.cc b/be/src/kudu/util/flags.cc
index b4eaf7e..97519c8 100644
--- a/be/src/kudu/util/flags.cc
+++ b/be/src/kudu/util/flags.cc
@@ -73,9 +73,8 @@ DEFINE_bool(dump_metrics_json, false,
 TAG_FLAG(dump_metrics_json, hidden);
 
 #ifdef TCMALLOC_ENABLED
-DEFINE_bool(enable_process_lifetime_heap_profiling, false, "Enables heap "
-    "profiling for the lifetime of the process. Profile output will be stored in the "
-    "directory specified by -heap_profile_path.");
+// Defined in Impala in common/global-flags.cc
+DECLARE_bool(enable_process_lifetime_heap_profiling);
 TAG_FLAG(enable_process_lifetime_heap_profiling, stable);
 TAG_FLAG(enable_process_lifetime_heap_profiling, advanced);
 
diff --git a/be/src/kudu/util/kudu_export.h b/be/src/kudu/util/kudu_export.h
new file mode 100644
index 0000000..3cbdf11
--- /dev/null
+++ b/be/src/kudu/util/kudu_export.h
@@ -0,0 +1,62 @@
+// 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.
+
+// This file is generated during Kudu's build. Instead of recreating the necessary steps
+// in Impala's build process, we copy it into our repository. See
+// kudu/client/CMakeLists.txt in Kudu's repository for details.
+
+#ifndef KUDU_EXPORT_H
+#define KUDU_EXPORT_H
+
+#ifdef KUDU_STATIC_DEFINE
+#  define KUDU_EXPORT
+#  define KUDU_NO_EXPORT
+#else
+#  ifndef KUDU_EXPORT
+#    ifdef kudu_client_exported_EXPORTS
+        /* We are building this library */
+#      define KUDU_EXPORT __attribute__((visibility("default")))
+#    else
+        /* We are using this library */
+#      define KUDU_EXPORT __attribute__((visibility("default")))
+#    endif
+#  endif
+
+#  ifndef KUDU_NO_EXPORT
+#    define KUDU_NO_EXPORT __attribute__((visibility("hidden")))
+#  endif
+#endif
+
+#ifndef KUDU_DEPRECATED
+#  define KUDU_DEPRECATED __attribute__ ((__deprecated__))
+#endif
+
+#ifndef KUDU_DEPRECATED_EXPORT
+#  define KUDU_DEPRECATED_EXPORT KUDU_EXPORT KUDU_DEPRECATED
+#endif
+
+#ifndef KUDU_DEPRECATED_NO_EXPORT
+#  define KUDU_DEPRECATED_NO_EXPORT KUDU_NO_EXPORT KUDU_DEPRECATED
+#endif
+
+#if 0 /* DEFINE_NO_DEPRECATED */
+#  ifndef KUDU_NO_DEPRECATED
+#    define KUDU_NO_DEPRECATED
+#  endif
+#endif
+
+#endif
diff --git a/be/src/kudu/util/logging.cc b/be/src/kudu/util/logging.cc
index 36850c3..e366abc 100644
--- a/be/src/kudu/util/logging.cc
+++ b/be/src/kudu/util/logging.cc
@@ -50,9 +50,8 @@
 #include "kudu/util/signal.h"
 #include "kudu/util/status.h"
 
-DEFINE_string(log_filename, "",
-    "Prefix of log filename - "
-    "full path is <log_dir>/<log_filename>.[INFO|WARN|ERROR|FATAL]");
+// Defined in Impala.
+DECLARE_string(log_filename);
 TAG_FLAG(log_filename, stable);
 
 DEFINE_bool(log_async, true,
@@ -65,9 +64,8 @@ DEFINE_int32(log_async_buffer_bytes_per_level, 2 * 1024 * 1024,
              "level. Only relevant when --log_async is enabled.");
 TAG_FLAG(log_async_buffer_bytes_per_level, hidden);
 
-DEFINE_int32(max_log_files, 10,
-    "Maximum number of log files to retain per severity level. The most recent "
-    "log files are retained. If set to 0, all log files are retained.");
+// Defined in Impala.
+DECLARE_int32(max_log_files);
 TAG_FLAG(max_log_files, runtime);
 TAG_FLAG(max_log_files, experimental);
 
@@ -275,7 +273,8 @@ void InitGoogleLoggingSafe(const char* arg) {
   IgnoreSigPipe();
 
   // For minidump support. Must be called before logging threads started.
-  CHECK_OK(BlockSigUSR1());
+  // Disabled by Impala, which does not link Kudu's minidump library.
+  //CHECK_OK(BlockSigUSR1());
 
   if (FLAGS_log_async) {
     EnableAsyncLogging();
diff --git a/be/src/kudu/util/logging.h b/be/src/kudu/util/logging.h
index f8b03b5..6db2bba 100644
--- a/be/src/kudu/util/logging.h
+++ b/be/src/kudu/util/logging.h
@@ -162,7 +162,7 @@ class ScopedDisableRedaction {
       &google::LogMessage::SendToLog).stream()
 
 #define KLOG_EVERY_N_SECS(severity, n_secs) \
-  static logging::LogThrottler LOG_THROTTLER;  \
+  static ::kudu::logging::LogThrottler LOG_THROTTLER;                   \
   KLOG_EVERY_N_SECS_THROTTLER(severity, n_secs, LOG_THROTTLER, "no-tag")
 
 #define WARN_NOT_OK_EVERY_N_SECS(to_call, warning_prefix, n_secs) do {                 \
diff --git a/be/src/rpc/impala-service-pool.cc b/be/src/rpc/impala-service-pool.cc
index 74fe2df..a4f28cb 100644
--- a/be/src/rpc/impala-service-pool.cc
+++ b/be/src/rpc/impala-service-pool.cc
@@ -43,11 +43,10 @@
 #include "common/names.h"
 #include "common/status.h"
 
-METRIC_DEFINE_histogram(server, impala_incoming_queue_time,
-    "RPC Queue Time",
+METRIC_DEFINE_histogram(server, impala_incoming_queue_time, "RPC Queue Time",
     kudu::MetricUnit::kMicroseconds,
     "Number of microseconds incoming RPC requests spend in the worker queue",
-    60000000LU, 3);
+    kudu::MetricLevel::kInfo, 60000000LU, 3);
 
 using namespace rapidjson;
 
@@ -149,7 +148,7 @@ kudu::rpc::RpcMethodInfo* ImpalaServicePool::LookupMethod(
 }
 
 kudu::Status ImpalaServicePool::QueueInboundCall(
-    gscoped_ptr<kudu::rpc::InboundCall> call) {
+    std::unique_ptr<kudu::rpc::InboundCall> call) {
   kudu::rpc::InboundCall* c = call.release();
 
   vector<uint32_t> unsupported_features;
diff --git a/be/src/rpc/impala-service-pool.h b/be/src/rpc/impala-service-pool.h
index 7771358..01b4301 100644
--- a/be/src/rpc/impala-service-pool.h
+++ b/be/src/rpc/impala-service-pool.h
@@ -62,8 +62,8 @@ class ImpalaServicePool : public kudu::rpc::RpcService {
   virtual kudu::rpc::RpcMethodInfo* LookupMethod(const kudu::rpc::RemoteMethod& method)
     override;
 
-  virtual kudu::Status
-      QueueInboundCall(gscoped_ptr<kudu::rpc::InboundCall> call) OVERRIDE;
+  virtual kudu::Status QueueInboundCall(
+      std::unique_ptr<kudu::rpc::InboundCall> call) OVERRIDE;
 
   const std::string service_name() const;
 
diff --git a/be/src/runtime/io/data-cache.cc b/be/src/runtime/io/data-cache.cc
index e4d01a2..3e5a8fc 100644
--- a/be/src/runtime/io/data-cache.cc
+++ b/be/src/runtime/io/data-cache.cc
@@ -439,12 +439,14 @@ struct DataCache::CacheKey {
   faststring key_;
 };
 
-DataCache::Partition::Partition(const string& path, int64_t capacity,
-    int max_opened_files)
-  : path_(path), capacity_(max<int64_t>(capacity, PAGE_SIZE)),
+DataCache::Partition::Partition(
+    const string& path, int64_t capacity, int max_opened_files)
+  : path_(path),
+    capacity_(max<int64_t>(capacity, PAGE_SIZE)),
     max_opened_files_(max_opened_files),
-    meta_cache_(NewLRUCache(kudu::DRAM_CACHE, capacity_, path_)) {
-}
+    meta_cache_(
+        kudu::NewCache<kudu::Cache::EvictionPolicy::LRU, kudu::Cache::MemoryType::DRAM>(
+            capacity_, path_)) {}
 
 DataCache::Partition::~Partition() {
   if (!closed_) ReleaseResources();
@@ -550,18 +552,15 @@ int64_t DataCache::Partition::Lookup(const CacheKey& cache_key, int64_t bytes_to
     uint8_t* buffer) {
   DCHECK(!closed_);
   Slice key = cache_key.ToSlice();
-  kudu::Cache::Handle* handle =
-      meta_cache_->Lookup(key, kudu::Cache::EXPECT_IN_CACHE);
-
+  kudu::Cache::UniqueHandle handle(
+      meta_cache_->Lookup(key, kudu::Cache::EXPECT_IN_CACHE));
 
-  if (handle == nullptr) {
+  if (handle.get() == nullptr) {
     if (tracer_ != nullptr) {
       tracer_->Trace(Tracer::MISS, cache_key, bytes_to_read, /*entry_len=*/-1);
     }
     return 0;
   }
-  auto handle_release =
-      MakeScopeExitTrigger([this, &handle]() { meta_cache_->Release(handle); });
 
   // Read from the backing file.
   CacheEntry entry(meta_cache_->Value(handle));
@@ -589,7 +588,7 @@ int64_t DataCache::Partition::Lookup(const CacheKey& cache_key, int64_t bytes_to
 }
 
 bool DataCache::Partition::HandleExistingEntry(const Slice& key,
-    kudu::Cache::Handle* handle, const uint8_t* buffer, int64_t buffer_len) {
+    const kudu::Cache::UniqueHandle& handle, const uint8_t* buffer, int64_t buffer_len) {
   // Unpack the cache entry.
   CacheEntry entry(meta_cache_->Value(handle));
 
@@ -612,12 +611,9 @@ bool DataCache::Partition::InsertIntoCache(const Slice& key, CacheFile* cache_fi
   const int64_t charge_len = BitUtil::RoundUp(buffer_len, PAGE_SIZE);
 
   // Allocate a cache handle
-  kudu::Cache::PendingHandle* pending_handle =
-      meta_cache_->Allocate(key, sizeof(CacheEntry), charge_len);
-  if (UNLIKELY(pending_handle == nullptr)) return false;
-  auto release_pending_handle = MakeScopeExitTrigger([this, &pending_handle]() {
-    if (pending_handle != nullptr) meta_cache_->Free(pending_handle);
-  });
+  kudu::Cache::UniquePendingHandle pending_handle(
+      meta_cache_->Allocate(key, sizeof(CacheEntry), charge_len));
+  if (UNLIKELY(pending_handle.get() == nullptr)) return false;
 
   // Compute checksum if necessary.
   int64_t checksum = FLAGS_data_cache_checksum ? Checksum(buffer, buffer_len) : 0;
@@ -631,9 +627,8 @@ bool DataCache::Partition::InsertIntoCache(const Slice& key, CacheFile* cache_fi
 
   // Insert the new entry into the cache.
   CacheEntry entry(cache_file, insertion_offset, buffer_len, checksum);
-  memcpy(meta_cache_->MutableValue(pending_handle), &entry, sizeof(CacheEntry));
-  kudu::Cache::Handle* handle = meta_cache_->Insert(pending_handle, this);
-  meta_cache_->Release(handle);
+  memcpy(meta_cache_->MutableValue(&pending_handle), &entry, sizeof(CacheEntry));
+  kudu::Cache::UniqueHandle handle(meta_cache_->Insert(std::move(pending_handle), this));
   pending_handle = nullptr;
   ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_TOTAL_BYTES->Increment(charge_len);
   return true;
@@ -648,11 +643,12 @@ bool DataCache::Partition::Store(const CacheKey& cache_key, const uint8_t* buffe
   if (charge_len > capacity_) return false;
 
   // Check for existing entry.
-  kudu::Cache::Handle* handle = meta_cache_->Lookup(key, kudu::Cache::EXPECT_IN_CACHE);
-  if (handle != nullptr) {
-    auto handle_release =
-        MakeScopeExitTrigger([this, &handle]() { meta_cache_->Release(handle); });
-    if (HandleExistingEntry(key, handle, buffer, buffer_len)) return false;
+  {
+    kudu::Cache::UniqueHandle handle(
+        meta_cache_->Lookup(key, kudu::Cache::EXPECT_IN_CACHE));
+    if (handle.get() != nullptr) {
+      if (HandleExistingEntry(key, handle, buffer, buffer_len)) return false;
+    }
   }
 
   CacheFile* cache_file;
diff --git a/be/src/runtime/io/data-cache.h b/be/src/runtime/io/data-cache.h
index 30cdaf8..fde1e97 100644
--- a/be/src/runtime/io/data-cache.h
+++ b/be/src/runtime/io/data-cache.h
@@ -324,8 +324,9 @@ class DataCache {
     /// Returns true iff the existing entry already covers the range of 'buffer' so no
     /// work needs to be done. Returns false otherwise. In which case, the existing entry
     /// will be overwritten.
-    bool HandleExistingEntry(const kudu::Slice& key, kudu::Cache::Handle* handle,
-        const uint8_t* buffer, int64_t buffer_len);
+    bool HandleExistingEntry(const kudu::Slice& key,
+        const kudu::Cache::UniqueHandle& handle, const uint8_t* buffer,
+        int64_t buffer_len);
 
     /// Helper function to insert a new entry with key 'key' into the LRU cache.
     /// The content in 'buffer' of length 'buffer_len' in bytes will be written to
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index deaf822..65f7555 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -165,6 +165,8 @@ string HttpStatusCodeToString(HttpStatusCode code) {
       return "200 OK";
     case HttpStatusCode::BadRequest:
       return "400 Bad Request";
+    case HttpStatusCode::AuthenticationRequired:
+      return "401 Authentication Required";
     case HttpStatusCode::NotFound:
       return "404 Not Found";
     case HttpStatusCode::LengthRequired:
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 29798f8..c92142b 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -54,6 +54,7 @@ be/src/kudu/util/array_view.h
 be/src/kudu/util/cache-test.cc
 be/src/kudu/util/cache.cc
 be/src/kudu/util/cache.h
+be/src/kudu/util/cloud/instance_detector-test.cc
 be/src/kudu/util/coding.cc
 be/src/kudu/util/coding.h
 be/src/kudu/util/condition_variable.cc
@@ -102,6 +103,7 @@ shell/packaging/README.md
 testdata/*.csv
 testdata/*.sql
 testdata/*.test
+be/src/kudu/util/testdata/*.txt
 be/src/testutil/*.pem
 *.json
 fe/src/test/resources/*.xml
diff --git a/cmake_modules/kudu_cmake_fns.txt b/cmake_modules/kudu_cmake_fns.txt
index 20e281d..2c5250b 100644
--- a/cmake_modules/kudu_cmake_fns.txt
+++ b/cmake_modules/kudu_cmake_fns.txt
@@ -131,3 +131,12 @@ function(ADD_THIRDPARTY_LIB LIB_NAME)
       PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES "${ARG_DEPS}")
   endif()
 endfunction()
+
+
+# This macro initializes KUDU_MIN_TEST_LIBS to KUDU_MIN_TEST_LIBS and
+# appends the passed list of libraries to the end. This ensures that
+# KUDU_MIN_TEST_LIBS is linked first.
+macro(SET_KUDU_TEST_LINK_LIBS)
+  set(KUDU_TEST_LINK_LIBS ${KUDU_MIN_TEST_LIBS})
+  list(APPEND KUDU_TEST_LINK_LIBS ${ARGN})
+endmacro()
diff --git a/tests/custom_cluster/test_restart_services.py b/tests/custom_cluster/test_restart_services.py
index ea92561..0b22141 100644
--- a/tests/custom_cluster/test_restart_services.py
+++ b/tests/custom_cluster/test_restart_services.py
@@ -250,7 +250,7 @@ class TestGracefulShutdown(CustomClusterTestSuite, HS2TestSuite):
         ":shutdown('e6c00ca5cd67b567eb96c6ecfb26f05')")
     assert "Could not find IPv4 address for:" in str(ex)
     ex = self.execute_query_expect_failure(self.client, ":shutdown('localhost:100000')")
-    assert "Invalid port:" in str(ex)
+    assert "invalid port:" in str(ex)
     assert ("This may be because the port specified is wrong.") not in str(ex)
 
     # Test that pointing to the wrong thrift service (the HS2 port) fails gracefully-ish.