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 2018/07/13 06:03:13 UTC

[01/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Repository: impala
Updated Branches:
  refs/heads/master 80c009631 -> 540611e86


http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_util.cc b/be/src/kudu/util/version_util.cc
new file mode 100644
index 0000000..bd298f8
--- /dev/null
+++ b/be/src/kudu/util/version_util.cc
@@ -0,0 +1,83 @@
+// 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 "kudu/util/version_util.h"
+
+#include <iostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+using std::ostream;
+using std::string;
+using std::vector;
+using strings::Split;
+using strings::Substitute;
+
+namespace kudu {
+
+bool Version::operator==(const Version& other) const {
+  return this->major == other.major &&
+         this->minor == other.minor &&
+         this->maintenance == other.maintenance &&
+         this->extra == other.extra;
+}
+
+string Version::ToString() const {
+  return raw_version;
+}
+
+ostream& operator<<(ostream& os, const Version& v) {
+  return os << v.ToString();
+}
+
+Status ParseVersion(const string& version_str,
+                    Version* v) {
+  CHECK(v);
+  const Status invalid_ver_err =
+      Status::InvalidArgument("invalid version string", version_str);
+  Version temp_v;
+  const vector<string> main_and_extra = Split(version_str, "-");
+  if (main_and_extra.empty() || main_and_extra.size() > 2) {
+    return invalid_ver_err;
+  }
+  if (main_and_extra.size() == 2) {
+    temp_v.extra = main_and_extra[1];
+  }
+  const auto& main_ver_str = main_and_extra[0];
+  const vector<string> maj_min_maint = Split(main_ver_str, ".");
+  if (maj_min_maint.size() != 3) {
+    return invalid_ver_err;
+  }
+  if (!SimpleAtoi(maj_min_maint[0], &temp_v.major) ||
+      !SimpleAtoi(maj_min_maint[1], &temp_v.minor) ||
+      !SimpleAtoi(maj_min_maint[2], &temp_v.maintenance)) {
+    return invalid_ver_err;
+  }
+  temp_v.raw_version = version_str;
+  *v = std::move(temp_v);
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_util.h b/be/src/kudu/util/version_util.h
new file mode 100644
index 0000000..5cde6cc
--- /dev/null
+++ b/be/src/kudu/util/version_util.h
@@ -0,0 +1,58 @@
+// 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.
+#pragma once
+
+#include <iostream>
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// A struct representing a parsed version. Versions are expected to look like
+//
+//  <major>.<minor>.<maintenance>[-<extra>]
+//
+// e.g. 1.6.0 or 1.7.1-SNAPSHOT.
+//
+// This struct can be used with versions reported by ksck to determine if and
+// how certain tools should function depending on what versions are running in
+// the cluster.
+struct Version {
+  bool operator==(const Version& other) const;
+
+  std::string ToString() const;
+
+  // The original version string.
+  std::string raw_version;
+
+  // The parsed version numbers.
+  int major;
+  int minor;
+  int maintenance;
+
+  // The extra component. Empty if there was no extra component.
+  std::string extra;
+};
+
+std::ostream& operator<<(std::ostream& os, const Version& v);
+
+// Parse 'version_str' into 'v'. 'v' must not be null.
+Status ParseVersion(const std::string& version_str,
+                    Version* v);
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/web_callback_registry.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/web_callback_registry.h b/be/src/kudu/util/web_callback_registry.h
new file mode 100644
index 0000000..600bddd
--- /dev/null
+++ b/be/src/kudu/util/web_callback_registry.h
@@ -0,0 +1,129 @@
+// 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.
+#ifndef KUDU_UTIL_WEB_CALLBACK_REGISTRY_H
+#define KUDU_UTIL_WEB_CALLBACK_REGISTRY_H
+
+#include <iosfwd>
+#include <string>
+#include <unordered_map>
+
+#include <boost/function.hpp>
+
+#include "kudu/util/easy_json.h"
+
+namespace kudu {
+
+enum class HttpStatusCode {
+  Ok, // 200
+  BadRequest, // 400
+  NotFound, // 404
+  LengthRequired, // 411
+  RequestEntityTooLarge, // 413
+  InternalServerError, // 500
+  ServiceUnavailable, // 503
+};
+
+// Interface for registering webserver callbacks.
+//
+// To register a webserver callback for /example/path:
+//
+// 1. Define a PathHandlerCallback that accepts an EasyJson
+//    object and fills out its fields with relevant information.
+// 2. Call RegisterPathHandler("/example/path", ...)
+// 3. Create the file $KUDU_HOME/www/example/path.mustache.
+class WebCallbackRegistry {
+ public:
+  typedef std::unordered_map<std::string, std::string> ArgumentMap;
+
+  struct WebRequest {
+    // The query string, parsed into key/value argument pairs.
+    ArgumentMap parsed_args;
+
+    // The raw query string passed in the URL. May be empty.
+    std::string query_string;
+
+    // The method (POST/GET/etc).
+    std::string request_method;
+
+    // In the case of a POST, the posted data.
+    std::string post_data;
+  };
+
+  typedef std::unordered_map<std::string, std::string> HttpResponseHeaders;
+
+  // A response to an HTTP request whose body is rendered by template.
+  struct WebResponse {
+    // Determines the status code of the HTTP response.
+    HttpStatusCode status_code;
+
+    // Additional headers added to the HTTP response.
+    HttpResponseHeaders response_headers;
+
+    // A JSON object to be rendered to HTML by a mustache template.
+    EasyJson* output;
+  };
+
+  // A response to an HTTP request.
+  struct PrerenderedWebResponse {
+    // Determines the status code of the HTTP response.
+    HttpStatusCode status_code;
+
+    // Additional headers added to the HTTP response.
+    HttpResponseHeaders response_headers;
+
+    // The fully-rendered HTML response body.
+    std::ostringstream* output;
+  };
+
+  // A function that handles an HTTP request where the response body will be rendered
+  // with a mustache template from the JSON object held by 'resp'.
+  typedef boost::function<void (const WebRequest& args, WebResponse* resp)>
+      PathHandlerCallback;
+
+  // A function that handles an HTTP request, where the response body is the contents
+  // of the 'output' member of 'resp'.
+  typedef boost::function<void (const WebRequest& args, PrerenderedWebResponse* resp)>
+      PrerenderedPathHandlerCallback;
+
+  virtual ~WebCallbackRegistry() {}
+
+  // Register a callback for a URL path. Path should not include the
+  // http://hostname/ prefix. If is_styled is true, the page is meant to be for
+  // people to look at and is styled.  If false, it is meant to be for machines to
+  // scrape.  If is_on_nav_bar is true,  a link to this page is
+  // printed in the navigation bar at the top of each debug page. Otherwise the
+  // link does not appear, and the page is rendered without HTML headers and
+  // footers.
+  // The first registration's choice of is_styled overrides all
+  // subsequent registrations for that URL.
+  // For each call to RegisterPathHandler(), the file $KUDU_HOME/www<path>.mustache
+  // should exist.
+  virtual void RegisterPathHandler(const std::string& path, const std::string& alias,
+                                   const PathHandlerCallback& callback,
+                                   bool is_styled, bool is_on_nav_bar) = 0;
+
+  // Same as RegisterPathHandler(), except that callback produces prerendered HTML.
+  // Use RegisterPathHandler() with a mustache template instead.
+  virtual void RegisterPrerenderedPathHandler(const std::string& path, const std::string& alias,
+                                              const PrerenderedPathHandlerCallback& callback,
+                                              bool is_styled,
+                                              bool is_on_nav_bar) = 0;
+};
+
+} // namespace kudu
+
+#endif /* KUDU_UTIL_WEB_CALLBACK_REGISTRY_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/website_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/website_util.cc b/be/src/kudu/util/website_util.cc
new file mode 100644
index 0000000..b7d14e5
--- /dev/null
+++ b/be/src/kudu/util/website_util.cc
@@ -0,0 +1,43 @@
+// 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 "kudu/util/website_util.h"
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/version_info.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+const char* const kKuduUrl = "https://kudu.apache.org";
+
+// Returns a URL for the Kudu website.
+string KuduUrl() {
+  return kKuduUrl;
+}
+
+string KuduDocsUrl() {
+  return Substitute("$0/releases/$1/docs", kKuduUrl, VersionInfo::GetShortVersionInfo());
+}
+
+string KuduDocsTroubleshootingUrl() {
+  return Substitute("$0/troubleshooting.html", KuduDocsUrl());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/website_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/website_util.h b/be/src/kudu/util/website_util.h
new file mode 100644
index 0000000..6dcf810
--- /dev/null
+++ b/be/src/kudu/util/website_util.h
@@ -0,0 +1,35 @@
+// 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.
+#pragma once
+
+#include <string>
+
+namespace kudu {
+
+// Returns a URL for the Kudu website.
+std::string KuduUrl();
+
+// Returns the base URL for this Kudu version's documentation.
+// Of course, if this version of Kudu isn't released, the link won't work.
+std::string KuduDocsUrl();
+
+// Returns a link to this Kudu version's troubleshooting docs. Useful to put in
+// error messages for common problems covered in the troubleshooting docs,
+// but whose solutions are too complex or varied to put in a log message.
+std::string KuduDocsTroubleshootingUrl();
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/zlib.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/zlib.cc b/be/src/kudu/util/zlib.cc
new file mode 100644
index 0000000..4805885
--- /dev/null
+++ b/be/src/kudu/util/zlib.cc
@@ -0,0 +1,127 @@
+// 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 "kudu/util/zlib.h"
+
+#include <zconf.h>
+#include <zlib.h>
+
+#include <cstdint>
+#include <cstring>
+#include <string>
+#include <memory>
+#include <ostream>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+using std::ostream;
+using std::string;
+using std::unique_ptr;
+
+#define ZRETURN_NOT_OK(call) \
+  RETURN_NOT_OK(ZlibResultToStatus(call))
+
+namespace kudu {
+namespace zlib {
+
+namespace  {
+Status ZlibResultToStatus(int rc) {
+  switch (rc) {
+    case Z_OK:
+      return Status::OK();
+    case Z_STREAM_END:
+      return Status::EndOfFile("zlib EOF");
+    case Z_NEED_DICT:
+      return Status::Corruption("zlib error: NEED_DICT");
+    case Z_ERRNO:
+      return Status::IOError("zlib error: Z_ERRNO");
+    case Z_STREAM_ERROR:
+      return Status::Corruption("zlib error: STREAM_ERROR");
+    case Z_DATA_ERROR:
+      return Status::Corruption("zlib error: DATA_ERROR");
+    case Z_MEM_ERROR:
+      return Status::RuntimeError("zlib error: MEM_ERROR");
+    case Z_BUF_ERROR:
+      return Status::RuntimeError("zlib error: BUF_ERROR");
+    case Z_VERSION_ERROR:
+      return Status::RuntimeError("zlib error: VERSION_ERROR");
+    default:
+      return Status::RuntimeError(
+          strings::Substitute("zlib error: unknown error $0", rc));
+  }
+}
+} // anonymous namespace
+
+Status Compress(Slice input, ostream* out) {
+  z_stream zs;
+  memset(&zs, 0, sizeof(zs));
+  ZRETURN_NOT_OK(deflateInit2(&zs, Z_DEFAULT_COMPRESSION, Z_DEFLATED,
+                              15 + 16 /* 15 window bits, enable gzip */,
+                              8 /* memory level, max is 9 */,
+                              Z_DEFAULT_STRATEGY));
+
+  zs.avail_in = input.size();
+  zs.next_in = const_cast<uint8_t*>(input.data());
+  const int kChunkSize = 256 * 1024;
+  unique_ptr<unsigned char[]> chunk(new unsigned char[kChunkSize]);
+  int flush;
+  do {
+    zs.avail_out = kChunkSize;
+    zs.next_out = chunk.get();
+    flush = (zs.avail_in == 0) ? Z_FINISH : Z_NO_FLUSH;
+    Status s = ZlibResultToStatus(deflate(&zs, flush));
+    if (!s.ok() && !s.IsEndOfFile()) {
+      return s;
+    }
+    int out_size = zs.next_out - chunk.get();
+    if (out_size > 0) {
+      out->write(reinterpret_cast<char *>(chunk.get()), out_size);
+    }
+  } while (flush != Z_FINISH);
+  ZRETURN_NOT_OK(deflateEnd(&zs));
+  return Status::OK();
+}
+
+Status Uncompress(Slice compressed, std::ostream* out) {
+  z_stream zs;
+  memset(&zs, 0, sizeof(zs));
+  zs.next_in = const_cast<uint8_t*>(compressed.data());
+  zs.avail_in = compressed.size();
+  ZRETURN_NOT_OK(inflateInit2(&zs, 15 + 16 /* 15 window bits, enable zlib */));
+  int flush;
+  Status s;
+  do {
+    unsigned char buf[4096];
+    zs.next_out = buf;
+    zs.avail_out = arraysize(buf);
+    flush = zs.avail_in > 0 ? Z_NO_FLUSH : Z_FINISH;
+    s = ZlibResultToStatus(inflate(&zs, flush));
+    if (!s.ok() && !s.IsEndOfFile()) {
+      return s;
+    }
+    out->write(reinterpret_cast<char *>(buf), zs.next_out - buf);
+  } while (flush == Z_NO_FLUSH);
+  ZRETURN_NOT_OK(inflateEnd(&zs));
+
+  return Status::OK();
+}
+
+} // namespace zlib
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/zlib.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/zlib.h b/be/src/kudu/util/zlib.h
new file mode 100644
index 0000000..35330fd
--- /dev/null
+++ b/be/src/kudu/util/zlib.h
@@ -0,0 +1,39 @@
+// 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.
+#pragma once
+
+#include <iosfwd>
+
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace zlib {
+
+// Zlib-compress the data in 'input', appending the result to 'out'.
+//
+// In case of an error, some data may still be appended to 'out'.
+Status Compress(Slice input, std::ostream* out);
+
+// Uncompress the zlib-compressed data in 'compressed', appending the result
+// to 'out'.
+//
+// In case of an error, some data may still be appended to 'out'.
+Status Uncompress(Slice compressed, std::ostream* out);
+
+} // namespace zlib
+} // namespace kudu


[21/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/errno-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/errno-test.cc b/be/src/kudu/util/errno-test.cc
new file mode 100644
index 0000000..f628b55
--- /dev/null
+++ b/be/src/kudu/util/errno-test.cc
@@ -0,0 +1,50 @@
+// 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 <cerrno>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/errno.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(OsUtilTest, TestErrnoToString) {
+  int err = ENOENT;
+
+  // Non-truncated result.
+  ASSERT_EQ("No such file or directory", ErrnoToString(err));
+
+  // Truncated because of a short buffer.
+  char buf[2];
+  ErrnoToCString(err, buf, arraysize(buf));
+  ASSERT_EQ("N", string(buf));
+
+  // Unknown error.
+  string expected = "Unknown error";
+  ASSERT_EQ(ErrnoToString(-1).compare(0, expected.length(), expected), 0);
+
+  // Unknown error (truncated).
+  ErrnoToCString(-1, buf, arraysize(buf));
+  ASSERT_EQ("U", string(buf));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/errno.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/errno.cc b/be/src/kudu/util/errno.cc
new file mode 100644
index 0000000..cc00d0f
--- /dev/null
+++ b/be/src/kudu/util/errno.cc
@@ -0,0 +1,52 @@
+// 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 "kudu/util/errno.h"
+
+#include <cstring>
+#include <glog/logging.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+// IWYU pragma: no_include <features.h>
+
+namespace kudu {
+
+void ErrnoToCString(int err, char *buf, size_t buf_len) {
+  CHECK_GT(buf_len, 0);
+#if !defined(__GLIBC__) || \
+  ((_POSIX_C_SOURCE >= 200112 || _XOPEN_SOURCE >= 600) && !defined(_GNU_SOURCE))
+  // Using POSIX version 'int strerror_r(...)'.
+  int ret = strerror_r(err, buf, buf_len);
+  if (ret && ret != ERANGE && ret != EINVAL) {
+    strncpy(buf, "unknown error", buf_len);
+    buf[buf_len - 1] = '\0';
+  }
+#else
+  // Using GLIBC version
+
+  // KUDU-1515: TSAN in Clang 3.9 has an incorrect interceptor for strerror_r:
+  // https://github.com/google/sanitizers/issues/696
+  ANNOTATE_IGNORE_WRITES_BEGIN();
+  char* ret = strerror_r(err, buf, buf_len);
+  ANNOTATE_IGNORE_WRITES_END();
+  if (ret != buf) {
+    strncpy(buf, ret, buf_len);
+    buf[buf_len - 1] = '\0';
+  }
+#endif
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/errno.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/errno.h b/be/src/kudu/util/errno.h
new file mode 100644
index 0000000..89802de
--- /dev/null
+++ b/be/src/kudu/util/errno.h
@@ -0,0 +1,36 @@
+// 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.
+#ifndef KUDU_ERRNO_H
+#define KUDU_ERRNO_H
+
+#include <cstddef>
+#include <string>
+
+namespace kudu {
+
+void ErrnoToCString(int err, char *buf, size_t buf_len);
+
+// Return a string representing an errno.
+inline static std::string ErrnoToString(int err) {
+  char buf[512];
+  ErrnoToCString(err, buf, sizeof(buf));
+  return std::string(buf);
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/faststring-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/faststring-test.cc b/be/src/kudu/util/faststring-test.cc
new file mode 100644
index 0000000..07c5697
--- /dev/null
+++ b/be/src/kudu/util/faststring-test.cc
@@ -0,0 +1,65 @@
+// 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 <algorithm>
+#include <cstring>
+#include <memory>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/faststring.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+class FaststringTest : public KuduTest {};
+
+TEST_F(FaststringTest, TestShrinkToFit_Empty) {
+  faststring s;
+  s.shrink_to_fit();
+  ASSERT_EQ(faststring::kInitialCapacity, s.capacity());
+}
+
+// Test that, if the string contents is shorter than the initial capacity
+// of the faststring, shrink_to_fit() leaves the string in the built-in
+// array.
+TEST_F(FaststringTest, TestShrinkToFit_SmallerThanInitialCapacity) {
+  faststring s;
+  s.append("hello");
+  s.shrink_to_fit();
+  ASSERT_EQ(faststring::kInitialCapacity, s.capacity());
+}
+
+TEST_F(FaststringTest, TestShrinkToFit_Random) {
+  Random r(GetRandomSeed32());
+  int kMaxSize = faststring::kInitialCapacity * 2;
+  std::unique_ptr<char[]> random_bytes(new char[kMaxSize]);
+  RandomString(random_bytes.get(), kMaxSize, &r);
+
+  faststring s;
+  for (int i = 0; i < 100; i++) {
+    int new_size = r.Uniform(kMaxSize);
+    s.resize(new_size);
+    memcpy(s.data(), random_bytes.get(), new_size);
+    s.shrink_to_fit();
+    ASSERT_EQ(0, memcmp(s.data(), random_bytes.get(), new_size));
+    ASSERT_EQ(std::max<int>(faststring::kInitialCapacity, new_size), s.capacity());
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/faststring.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/faststring.cc b/be/src/kudu/util/faststring.cc
new file mode 100644
index 0000000..a1cd26b
--- /dev/null
+++ b/be/src/kudu/util/faststring.cc
@@ -0,0 +1,72 @@
+// 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 "kudu/util/faststring.h"
+
+#include <glog/logging.h>
+#include <memory>
+
+namespace kudu {
+
+void faststring::GrowByAtLeast(size_t count) {
+  // Not enough space, need to reserve more.
+  // Don't reserve exactly enough space for the new string -- that makes it
+  // too easy to write perf bugs where you get O(n^2) append.
+  // Instead, alwayhs expand by at least 50%.
+
+  size_t to_reserve = len_ + count;
+  if (len_ + count < len_ * 3 / 2) {
+    to_reserve = len_ *  3 / 2;
+  }
+  GrowArray(to_reserve);
+}
+
+void faststring::GrowArray(size_t newcapacity) {
+  DCHECK_GE(newcapacity, capacity_);
+  std::unique_ptr<uint8_t[]> newdata(new uint8_t[newcapacity]);
+  if (len_ > 0) {
+    memcpy(&newdata[0], &data_[0], len_);
+  }
+  capacity_ = newcapacity;
+  if (data_ != initial_data_) {
+    delete[] data_;
+  } else {
+    ASAN_POISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
+  }
+
+  data_ = newdata.release();
+  ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
+}
+
+void faststring::ShrinkToFitInternal() {
+  DCHECK_NE(data_, initial_data_);
+  if (len_ <= kInitialCapacity) {
+    ASAN_UNPOISON_MEMORY_REGION(initial_data_, len_);
+    memcpy(initial_data_, &data_[0], len_);
+    delete[] data_;
+    data_ = initial_data_;
+    capacity_ = kInitialCapacity;
+  } else {
+    std::unique_ptr<uint8_t[]> newdata(new uint8_t[len_]);
+    memcpy(&newdata[0], &data_[0], len_);
+    delete[] data_;
+    data_ = newdata.release();
+    capacity_ = len_;
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/faststring.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/faststring.h b/be/src/kudu/util/faststring.h
new file mode 100644
index 0000000..992060b
--- /dev/null
+++ b/be/src/kudu/util/faststring.h
@@ -0,0 +1,259 @@
+// 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.
+#ifndef KUDU_UTIL_FASTSTRING_H
+#define KUDU_UTIL_FASTSTRING_H
+
+#include <cstdint>
+#include <cstring>
+#include <string>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/fastmem.h"
+
+namespace kudu {
+
+// A faststring is similar to a std::string, except that it is faster for many
+// common use cases (in particular, resize() will fill with uninitialized data
+// instead of memsetting to \0)
+class faststring {
+ public:
+  enum {
+    kInitialCapacity = 32
+  };
+
+  faststring() :
+    data_(initial_data_),
+    len_(0),
+    capacity_(kInitialCapacity) {
+  }
+
+  // Construct a string with the given capacity, in bytes.
+  explicit faststring(size_t capacity)
+    : data_(initial_data_),
+      len_(0),
+      capacity_(kInitialCapacity) {
+    if (capacity > capacity_) {
+      data_ = new uint8_t[capacity];
+      capacity_ = capacity;
+    }
+    ASAN_POISON_MEMORY_REGION(data_, capacity_);
+  }
+
+  ~faststring() {
+    ASAN_UNPOISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
+    if (data_ != initial_data_) {
+      delete[] data_;
+    }
+  }
+
+  // Reset the valid length of the string to 0.
+  //
+  // This does not free up any memory. The capacity of the string remains unchanged.
+  void clear() {
+    resize(0);
+    ASAN_POISON_MEMORY_REGION(data_, capacity_);
+  }
+
+  // Resize the string to the given length.
+  // If the new length is larger than the old length, the capacity is expanded as necessary.
+  //
+  // NOTE: in contrast to std::string's implementation, Any newly "exposed" bytes of data are
+  // not cleared.
+  void resize(size_t newsize) {
+    if (newsize > capacity_) {
+      reserve(newsize);
+    }
+    len_ = newsize;
+    ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
+    ASAN_UNPOISON_MEMORY_REGION(data_, len_);
+  }
+
+  // Releases the underlying array; after this, the buffer is left empty.
+  //
+  // NOTE: the data pointer returned by release() is not necessarily the pointer
+  uint8_t *release() WARN_UNUSED_RESULT {
+    uint8_t *ret = data_;
+    if (ret == initial_data_) {
+      ret = new uint8_t[len_];
+      memcpy(ret, data_, len_);
+    }
+    len_ = 0;
+    capacity_ = kInitialCapacity;
+    data_ = initial_data_;
+    ASAN_POISON_MEMORY_REGION(data_, capacity_);
+    return ret;
+  }
+
+  // Reserve space for the given total amount of data. If the current capacity is already
+  // larger than the newly requested capacity, this is a no-op (i.e. it does not ever free memory).
+  //
+  // NOTE: even though the new capacity is reserved, it is illegal to begin writing into that memory
+  // directly using pointers. If ASAN is enabled, this is ensured using manual memory poisoning.
+  void reserve(size_t newcapacity) {
+    if (PREDICT_TRUE(newcapacity <= capacity_)) return;
+    GrowArray(newcapacity);
+  }
+
+  // Append the given data to the string, resizing capacity as necessary.
+  void append(const void *src_v, size_t count) {
+    const uint8_t *src = reinterpret_cast<const uint8_t *>(src_v);
+    EnsureRoomForAppend(count);
+    ASAN_UNPOISON_MEMORY_REGION(data_ + len_, count);
+
+    // appending short values is common enough that this
+    // actually helps, according to benchmarks. In theory
+    // memcpy_inlined should already be just as good, but this
+    // was ~20% faster for reading a large prefix-coded string file
+    // where each string was only a few chars different
+    if (count <= 4) {
+      uint8_t *p = &data_[len_];
+      for (int i = 0; i < count; i++) {
+        *p++ = *src++;
+      }
+    } else {
+      strings::memcpy_inlined(&data_[len_], src, count);
+    }
+    len_ += count;
+  }
+
+  // Append the given string to this string.
+  void append(const std::string &str) {
+    append(str.data(), str.size());
+  }
+
+  // Append the given character to this string.
+  void push_back(const char byte) {
+    EnsureRoomForAppend(1);
+    ASAN_UNPOISON_MEMORY_REGION(data_ + len_, 1);
+    data_[len_] = byte;
+    len_++;
+  }
+
+  // Return the valid length of this string.
+  size_t length() const {
+    return len_;
+  }
+
+  // Return the valid length of this string (identical to length())
+  size_t size() const {
+    return len_;
+  }
+
+  // Return the allocated capacity of this string.
+  size_t capacity() const {
+    return capacity_;
+  }
+
+  // Return a pointer to the data in this string. Note that this pointer
+  // may be invalidated by any later non-const operation.
+  const uint8_t *data() const {
+    return &data_[0];
+  }
+
+  // Return a pointer to the data in this string. Note that this pointer
+  // may be invalidated by any later non-const operation.
+  uint8_t *data() {
+    return &data_[0];
+  }
+
+  // Return the given element of this string. Note that this does not perform
+  // any bounds checking.
+  const uint8_t &at(size_t i) const {
+    return data_[i];
+  }
+
+  // Return the given element of this string. Note that this does not perform
+  // any bounds checking.
+  const uint8_t &operator[](size_t i) const {
+    return data_[i];
+  }
+
+  // Return the given element of this string. Note that this does not perform
+  // any bounds checking.
+  uint8_t &operator[](size_t i) {
+    return data_[i];
+  }
+
+  // Reset the contents of this string by copying 'len' bytes from 'src'.
+  void assign_copy(const uint8_t *src, size_t len) {
+    // Reset length so that the first resize doesn't need to copy the current
+    // contents of the array.
+    len_ = 0;
+    resize(len);
+    memcpy(data(), src, len);
+  }
+
+  // Reset the contents of this string by copying from the given std::string.
+  void assign_copy(const std::string &str) {
+    assign_copy(reinterpret_cast<const uint8_t *>(str.c_str()),
+                str.size());
+  }
+
+  // Reallocates the internal storage to fit only the current data.
+  //
+  // This may revert to using internal storage if the current length is shorter than
+  // kInitialCapacity. Note that, in that case, after this call, capacity() will return
+  // a capacity larger than the data length.
+  //
+  // Any pointers within this instance are invalidated.
+  void shrink_to_fit() {
+    if (data_ == initial_data_ || capacity_ == len_) return;
+    ShrinkToFitInternal();
+  }
+
+  // Return a copy of this string as a std::string.
+  std::string ToString() const {
+    return std::string(reinterpret_cast<const char *>(data()),
+                       len_);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(faststring);
+
+  // If necessary, expand the buffer to fit at least 'count' more bytes.
+  // If the array has to be grown, it is grown by at least 50%.
+  void EnsureRoomForAppend(size_t count) {
+    if (PREDICT_TRUE(len_ + count <= capacity_)) {
+      return;
+    }
+
+    // Call the non-inline slow path - this reduces the number of instructions
+    // on the hot path.
+    GrowByAtLeast(count);
+  }
+
+  // The slow path of MakeRoomFor. Grows the buffer by either
+  // 'count' bytes, or 50%, whichever is more.
+  void GrowByAtLeast(size_t count);
+
+  // Grow the array to the given capacity, which must be more than
+  // the current capacity.
+  void GrowArray(size_t newcapacity);
+
+  void ShrinkToFitInternal();
+
+  uint8_t* data_;
+  uint8_t initial_data_[kInitialCapacity];
+  size_t len_;
+  size_t capacity_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/fault_injection.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/fault_injection.cc b/be/src/kudu/util/fault_injection.cc
new file mode 100644
index 0000000..6638bb6
--- /dev/null
+++ b/be/src/kudu/util/fault_injection.cc
@@ -0,0 +1,78 @@
+// 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 "kudu/util/fault_injection.h"
+
+#include <unistd.h>
+
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/once.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+
+namespace kudu {
+namespace fault_injection {
+
+namespace {
+GoogleOnceType g_random_once;
+Random* g_random;
+
+void InitRandom() {
+  LOG(WARNING) << "FAULT INJECTION ENABLED!";
+  LOG(WARNING) << "THIS SERVER MAY CRASH!";
+
+  debug::ScopedLeakCheckDisabler d;
+  g_random = new Random(GetRandomSeed32());
+  ANNOTATE_BENIGN_RACE_SIZED(g_random, sizeof(Random),
+                             "Racy random numbers are OK");
+}
+
+} // anonymous namespace
+
+void DoMaybeFault(const char* fault_str, double fraction) {
+  GoogleOnceInit(&g_random_once, InitRandom);
+  if (PREDICT_TRUE(g_random->NextDoubleFraction() >= fraction)) {
+    return;
+  }
+  LOG(ERROR) << "Injecting fault: " << fault_str << " (process will exit)";
+  // _exit will exit the program without running atexit handlers. This more
+  // accurately simulates a crash.
+  _exit(kExitStatus);
+}
+
+void DoInjectRandomLatency(double max_latency_ms) {
+  GoogleOnceInit(&g_random_once, InitRandom);
+  SleepFor(MonoDelta::FromMilliseconds(g_random->NextDoubleFraction() * max_latency_ms));
+}
+
+void DoInjectFixedLatency(int32_t latency_ms) {
+  SleepFor(MonoDelta::FromMilliseconds(latency_ms));
+}
+
+bool DoMaybeTrue(double fraction) {
+  GoogleOnceInit(&g_random_once, InitRandom);
+  return PREDICT_FALSE(g_random->NextDoubleFraction() <= fraction);
+}
+
+} // namespace fault_injection
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/fault_injection.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/fault_injection.h b/be/src/kudu/util/fault_injection.h
new file mode 100644
index 0000000..7a71698
--- /dev/null
+++ b/be/src/kudu/util/fault_injection.h
@@ -0,0 +1,98 @@
+// 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.
+#ifndef KUDU_UTIL_FAULT_INJECTION_H
+#define KUDU_UTIL_FAULT_INJECTION_H
+
+#include <stdint.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+// Macros for injecting various kinds of faults with varying probability. If
+// configured with 0 probability, each of these macros is evaluated inline and
+// is fast enough to run even in hot code paths.
+
+// With some probability, crash at the current point in the code
+// by issuing LOG(FATAL).
+//
+// The probability is determined by the 'fraction_flag' argument.
+//
+// Typical usage:
+//
+//   DEFINE_double(fault_crash_before_foo, 0.0,
+//                 "Fraction of the time when we will crash before doing foo");
+//   TAG_FLAG(fault_crash_before_foo, unsafe);
+#define MAYBE_FAULT(fraction_flag) \
+  kudu::fault_injection::MaybeFault(AS_STRING(fraction_flag), fraction_flag)
+
+// Inject a uniformly random amount of latency between 0 and the configured
+// number of milliseconds.
+#define MAYBE_INJECT_RANDOM_LATENCY(max_ms_flag) \
+  kudu::fault_injection::MaybeInjectRandomLatency(max_ms_flag)
+
+// Inject a specific amount of latency.
+#define MAYBE_INJECT_FIXED_LATENCY(ms_flag) \
+  kudu::fault_injection::MaybeInjectFixedLatency(ms_flag)
+
+// With some probability, return the status described by 'status_expr'.
+// This will not evaluate 'status_expr' if 'fraction_flag' is zero.
+#define MAYBE_RETURN_FAILURE(fraction_flag, status_expr) \
+  if (kudu::fault_injection::MaybeTrue(fraction_flag)) { \
+    RETURN_NOT_OK((status_expr)); \
+  }
+
+// Implementation details below.
+// Use the MAYBE_FAULT macro instead.
+namespace kudu {
+namespace fault_injection {
+
+// The exit status returned from a process exiting due to a fault.
+// The choice of value here is arbitrary: just needs to be something
+// wouldn't normally be returned by a non-fault-injection code path.
+constexpr int kExitStatus = 85;
+
+// Out-of-line implementation.
+void DoMaybeFault(const char* fault_str, double fraction);
+void DoInjectRandomLatency(double max_latency_ms);
+void DoInjectFixedLatency(int32_t latency_ms);
+bool DoMaybeTrue(double fraction);
+
+inline bool MaybeTrue(double fraction) {
+  if (PREDICT_TRUE(fraction <= 0)) return false;
+  return DoMaybeTrue(fraction);
+}
+
+inline void MaybeFault(const char* fault_str, double fraction) {
+  if (PREDICT_TRUE(fraction <= 0)) return;
+  DoMaybeFault(fault_str, fraction);
+}
+
+inline void MaybeInjectRandomLatency(double max_latency) {
+  if (PREDICT_TRUE(max_latency <= 0)) return;
+  DoInjectRandomLatency(max_latency);
+}
+
+inline void MaybeInjectFixedLatency(int32_t latency) {
+  if (PREDICT_TRUE(latency <= 0)) return;
+  DoInjectFixedLatency(latency);
+}
+
+
+} // namespace fault_injection
+} // namespace kudu
+#endif /* KUDU_UTIL_FAULT_INJECTION_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/file_cache-stress-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/file_cache-stress-test.cc b/be/src/kudu/util/file_cache-stress-test.cc
new file mode 100644
index 0000000..9c51a52
--- /dev/null
+++ b/be/src/kudu/util/file_cache-stress-test.cc
@@ -0,0 +1,402 @@
+// 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 "kudu/util/file_cache.h"
+
+#include <cstddef>
+#include <cstdint>
+#include <deque>
+#include <iterator>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/file_cache-test-util.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/oid_generator.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/random.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DEFINE_int32(test_num_producer_threads, 1, "Number of producer threads");
+DEFINE_int32(test_num_consumer_threads, 4, "Number of consumer threads");
+DEFINE_int32(test_duration_secs, 2, "Number of seconds to run the test");
+
+DECLARE_bool(cache_force_single_shard);
+
+using std::deque;
+using std::shared_ptr;
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::unordered_map;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+// FD limit to enforce during the test.
+static const int kTestMaxOpenFiles = 100;
+
+template <class FileType>
+class FileCacheStressTest : public KuduTest {
+
+// Like CHECK_OK(), but dumps the contents of the cache before failing.
+//
+// The output of ToDebugString() tends to be long enough that LOG() truncates
+// it, so we must split it ourselves before logging.
+#define TEST_CHECK_OK(to_call) do {                                       \
+    const Status& _s = (to_call);                                         \
+    if (!_s.ok()) {                                                       \
+      LOG(INFO) << "Dumping cache contents";                              \
+      vector<string> lines = strings::Split(cache_->ToDebugString(), "\n",\
+                                            strings::SkipEmpty());        \
+      for (const auto& l : lines) {                                       \
+        LOG(INFO) << l;                                                   \
+      }                                                                   \
+    }                                                                     \
+    CHECK(_s.ok()) << "Bad status: " << _s.ToString();                    \
+  } while (0);
+
+ public:
+  typedef unordered_map<string, unordered_map<string, int>> MetricMap;
+
+  FileCacheStressTest()
+      : rand_(SeedRandom()),
+        running_(1) {
+    // Use a single shard. Otherwise, the cache can be a little bit "sloppy"
+    // depending on the number of CPUs on the system.
+    FLAGS_cache_force_single_shard = true;
+    cache_.reset(new FileCache<FileType>("test",
+                                         env_,
+                                         kTestMaxOpenFiles,
+                                         scoped_refptr<MetricEntity>()));
+  }
+
+  void SetUp() override {
+    ASSERT_OK(cache_->Init());
+  }
+
+  void ProducerThread() {
+    Random rand(rand_.Next32());
+    ObjectIdGenerator oid_generator;
+    MetricMap metrics;
+
+    do {
+      // Create a new file with some (0-32k) random data in it.
+      string next_file_name = GetTestPath(oid_generator.Next());
+      {
+        unique_ptr<WritableFile> next_file;
+        CHECK_OK(env_->NewWritableFile(next_file_name, &next_file));
+        uint8_t buf[rand.Uniform((32 * 1024) - 1) + 1];
+        CHECK_OK(next_file->Append(GenerateRandomChunk(buf, sizeof(buf), &rand)));
+        CHECK_OK(next_file->Close());
+      }
+      {
+        std::lock_guard<simple_spinlock> l(lock_);
+        InsertOrDie(&available_files_, next_file_name, 0);
+      }
+      metrics[BaseName(next_file_name)]["create"] = 1;
+    } while (!running_.WaitFor(MonoDelta::FromMilliseconds(1)));
+
+    // Update the global metrics map.
+    MergeNewMetrics(std::move(metrics));
+  }
+
+  void ConsumerThread() {
+    // Each thread has its own PRNG to minimize contention on the main one.
+    Random rand(rand_.Next32());
+
+    // Active opened files in this thread.
+    deque<shared_ptr<FileType>> files;
+
+    // Metrics generated by this thread. They will be merged into the main
+    // metrics map when the thread is done.
+    MetricMap metrics;
+
+    do {
+      // Pick an action to perform. Distribution:
+      // 20% open
+      // 15% close
+      // 35% read
+      // 20% write
+      // 10% delete
+      int next_action = rand.Uniform(100);
+
+      if (next_action < 20) {
+        // Open an existing file.
+        string to_open;
+        if (!GetRandomFile(OPEN, &rand, &to_open)) {
+          continue;
+        }
+        shared_ptr<FileType> new_file;
+        TEST_CHECK_OK(cache_->OpenExistingFile(to_open, &new_file));
+        FinishedOpen(to_open);
+        metrics[BaseName(to_open)]["open"]++;
+        files.emplace_back(new_file);
+      } else if (next_action < 35) {
+        // Close a file.
+        if (files.empty()) {
+          continue;
+        }
+        shared_ptr<FileType> file = files.front();
+        files.pop_front();
+        metrics[BaseName(file->filename())]["close"]++;
+      } else if (next_action < 70) {
+        // Read a random chunk from a file.
+        TEST_CHECK_OK(ReadRandomChunk(files, &metrics, &rand));
+      } else if (next_action < 90) {
+        // Write a random chunk to a file.
+        TEST_CHECK_OK(WriteRandomChunk(files, &metrics, &rand));
+      } else if (next_action < 100) {
+        // Delete a file.
+        string to_delete;
+        if (!GetRandomFile(DELETE, &rand, &to_delete)) {
+          continue;
+        }
+        TEST_CHECK_OK(cache_->DeleteFile(to_delete));
+        metrics[BaseName(to_delete)]["delete"]++;
+      }
+    } while (!running_.WaitFor(MonoDelta::FromMilliseconds(1)));
+
+    // Update the global metrics map.
+    MergeNewMetrics(std::move(metrics));
+  }
+
+ protected:
+  void NotifyThreads() { running_.CountDown(); }
+
+  const MetricMap& metrics() const { return metrics_; }
+
+ private:
+  enum GetMode {
+    OPEN,
+    DELETE
+  };
+
+  // Retrieve a random file name to be either opened or deleted. If deleting,
+  // the file name is made inaccessible to future operations.
+  bool GetRandomFile(GetMode mode, Random* rand, string* out) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    if (available_files_.empty()) {
+      return false;
+    }
+
+    // This is linear time, but it's simpler than managing multiple data
+    // structures.
+    auto it = available_files_.begin();
+    std::advance(it, rand->Uniform(available_files_.size()));
+
+    // It's unsafe to delete a file that is still being opened.
+    if (mode == DELETE && it->second > 0) {
+      return false;
+    }
+
+    *out = it->first;
+    if (mode == OPEN) {
+      it->second++;
+    } else {
+      available_files_.erase(it);
+    }
+    return true;
+  }
+
+  // Signal that a previously in-progress open has finished, allowing the file
+  // in question to be deleted.
+  void FinishedOpen(const string& opened) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    int& openers = FindOrDie(available_files_, opened);
+    openers--;
+  }
+
+  // Reads a random chunk of data from a random file in 'files'. On success,
+  // writes to 'metrics'.
+  static Status ReadRandomChunk(const deque<shared_ptr<FileType>>& files,
+                                MetricMap* metrics,
+                                Random* rand) {
+    if (files.empty()) {
+      return Status::OK();
+    }
+    const shared_ptr<FileType>& file = files[rand->Uniform(files.size())];
+
+    uint64_t file_size;
+    RETURN_NOT_OK(file->Size(&file_size));
+    uint64_t off = file_size > 0 ? rand->Uniform(file_size) : 0;
+    size_t len = file_size > 0 ? rand->Uniform(file_size - off) : 0;
+    unique_ptr<uint8_t[]> scratch(new uint8_t[len]);
+    RETURN_NOT_OK(file->Read(off, Slice(scratch.get(), len)));
+
+    (*metrics)[BaseName(file->filename())]["read"]++;
+    return Status::OK();
+  }
+
+  // Writes a random chunk of data to a random file in 'files'. On success,
+  // updates 'metrics'.
+  //
+  // No-op for file implementations that don't support writing.
+  static Status WriteRandomChunk(const deque<shared_ptr<FileType>>& files,
+                                 MetricMap* metrics,
+                                 Random* rand);
+
+  static Slice GenerateRandomChunk(uint8_t* buffer, size_t max_length, Random* rand) {
+    size_t len = rand->Uniform(max_length);
+    len -= len % sizeof(uint32_t);
+    for (int i = 0; i < (len / sizeof(uint32_t)); i += sizeof(uint32_t)) {
+      reinterpret_cast<uint32_t*>(buffer)[i] = rand->Next32();
+    }
+    return Slice(buffer, len);
+  }
+
+  // Merge the metrics in 'new_metrics' into the global metric map.
+  void MergeNewMetrics(MetricMap new_metrics) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    for (const auto& file_action_pair : new_metrics) {
+      for (const auto& action_count_pair : file_action_pair.second) {
+        metrics_[file_action_pair.first][action_count_pair.first] += action_count_pair.second;
+      }
+    }
+  }
+
+  unique_ptr<FileCache<FileType>> cache_;
+
+  // Used to seed per-thread PRNGs.
+  ThreadSafeRandom rand_;
+
+  // Drops to zero when the test ends.
+  CountDownLatch running_;
+
+  // Protects 'available_files_' and 'metrics_'.
+  simple_spinlock lock_;
+
+  // Contains files produced by producer threads and ready for consumption by
+  // consumer threads.
+  //
+  // Each entry is a file name and the number of in-progress openers. To delete
+  // a file, there must be no openers.
+  unordered_map<string, int> available_files_;
+
+  // For each file name, tracks the count of consumer actions performed.
+  //
+  // Only updated at test end.
+  MetricMap metrics_;
+};
+
+template <>
+Status FileCacheStressTest<RWFile>::WriteRandomChunk(
+    const deque<shared_ptr<RWFile>>& files,
+    MetricMap* metrics,
+    Random* rand) {
+  if (files.empty()) {
+    return Status::OK();
+  }
+  const shared_ptr<RWFile>& file = files[rand->Uniform(files.size())];
+
+  uint64_t file_size;
+  RETURN_NOT_OK(file->Size(&file_size));
+  uint64_t off = file_size > 0 ? rand->Uniform(file_size) : 0;
+  uint8_t buf[64];
+  RETURN_NOT_OK(file->Write(off, GenerateRandomChunk(buf, sizeof(buf), rand)));
+  (*metrics)[BaseName(file->filename())]["write"]++;
+  return Status::OK();
+}
+
+template <>
+Status FileCacheStressTest<RandomAccessFile>::WriteRandomChunk(
+    const deque<shared_ptr<RandomAccessFile>>& /* unused */,
+    MetricMap* /* unused */,
+    Random* /* unused */) {
+  return Status::OK();
+}
+
+typedef ::testing::Types<RWFile, RandomAccessFile> FileTypes;
+TYPED_TEST_CASE(FileCacheStressTest, FileTypes);
+
+TYPED_TEST(FileCacheStressTest, TestStress) {
+  OverrideFlagForSlowTests("test_num_producer_threads", "2");
+  OverrideFlagForSlowTests("test_num_consumer_threads", "8");
+  OverrideFlagForSlowTests("test_duration_secs", "30");
+
+  // Start the threads.
+  PeriodicOpenFdChecker checker(
+      this->env_,
+      this->GetTestPath("*"),           // only count within our test dir
+      kTestMaxOpenFiles +               // cache capacity
+      FLAGS_test_num_producer_threads + // files being written
+      FLAGS_test_num_consumer_threads); // files being opened
+  checker.Start();
+  vector<thread> producers;
+  for (int i = 0; i < FLAGS_test_num_producer_threads; i++) {
+    producers.emplace_back(&FileCacheStressTest<TypeParam>::ProducerThread, this);
+  }
+  vector<thread> consumers;
+  for (int i = 0; i < FLAGS_test_num_consumer_threads; i++) {
+    consumers.emplace_back(&FileCacheStressTest<TypeParam>::ConsumerThread, this);
+  }
+
+  // Let the test run.
+  SleepFor(MonoDelta::FromSeconds(FLAGS_test_duration_secs));
+
+  // Stop the threads.
+  this->NotifyThreads();
+  checker.Stop();
+  for (auto& p : producers) {
+    p.join();
+  }
+  for (auto& c : consumers) {
+    c.join();
+  }
+
+  // Log the metrics.
+  unordered_map<string, int> action_counts;
+  for (const auto& file_action_pair : this->metrics()) {
+    for (const auto& action_count_pair : file_action_pair.second) {
+      VLOG(2) << Substitute("$0: $1: $2",
+                            file_action_pair.first,
+                            action_count_pair.first,
+                            action_count_pair.second);
+      action_counts[action_count_pair.first] += action_count_pair.second;
+    }
+  }
+  for (const auto& action_count_pair : action_counts) {
+    LOG(INFO) << Substitute("$0: $1",
+                            action_count_pair.first,
+                            action_count_pair.second);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/file_cache-test-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/file_cache-test-util.h b/be/src/kudu/util/file_cache-test-util.h
new file mode 100644
index 0000000..09acb68
--- /dev/null
+++ b/be/src/kudu/util/file_cache-test-util.h
@@ -0,0 +1,92 @@
+// 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.
+
+#pragma once
+
+#include <thread>
+
+#include <glog/logging.h>
+#include <string>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+// Periodically checks the number of open file descriptors belonging to this
+// process, crashing if it exceeds some upper bound.
+class PeriodicOpenFdChecker {
+ public:
+  // path_pattern: a glob-style pattern of which paths should be included while
+  //               counting file descriptors
+  // upper_bound:  the maximum number of file descriptors that should be open
+  //               at any point in time
+  PeriodicOpenFdChecker(Env* env, std::string path_pattern, int upper_bound)
+    : env_(env),
+      path_pattern_(std::move(path_pattern)),
+      initial_fd_count_(CountOpenFds(env, path_pattern_)),
+      max_fd_count_(upper_bound + initial_fd_count_),
+      running_(1),
+      started_(false) {}
+
+  ~PeriodicOpenFdChecker() { Stop(); }
+
+  void Start() {
+    DCHECK(!started_);
+    running_.Reset(1);
+    check_thread_ = std::thread(&PeriodicOpenFdChecker::CheckThread, this);
+    started_ = true;
+  }
+
+  void Stop() {
+    if (started_) {
+      running_.CountDown();
+      check_thread_.join();
+      started_ = false;
+    }
+  }
+
+ private:
+  void CheckThread() {
+    LOG(INFO) << strings::Substitute(
+        "Periodic open fd checker starting for path pattern $0"
+        "(initial: $1 max: $2)",
+        path_pattern_, initial_fd_count_, max_fd_count_);
+    do {
+      int open_fd_count = CountOpenFds(env_, path_pattern_);
+      KLOG_EVERY_N_SECS(INFO, 1) << strings::Substitute("Open fd count: $0/$1",
+                                                        open_fd_count,
+                                                        max_fd_count_);
+      CHECK_LE(open_fd_count, max_fd_count_);
+    } while (!running_.WaitFor(MonoDelta::FromMilliseconds(100)));
+  }
+
+  Env* env_;
+  const std::string path_pattern_;
+  const int initial_fd_count_;
+  const int max_fd_count_;
+
+  CountDownLatch running_;
+  std::thread check_thread_;
+  bool started_;
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/file_cache-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/file_cache-test.cc b/be/src/kudu/util/file_cache-test.cc
new file mode 100644
index 0000000..94c09eb
--- /dev/null
+++ b/be/src/kudu/util/file_cache-test.cc
@@ -0,0 +1,361 @@
+// 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 "kudu/util/file_cache.h"
+
+#include <unistd.h>
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/metrics.h"  // IWYU pragma: keep
+#include "kudu/util/random.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_bool(cache_force_single_shard);
+DECLARE_int32(file_cache_expiry_period_ms);
+
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+template <class FileType>
+class FileCacheTest : public KuduTest {
+ public:
+  FileCacheTest()
+      : rand_(SeedRandom()) {
+    // Simplify testing of the actual cache capacity.
+    FLAGS_cache_force_single_shard = true;
+
+    // Speed up tests that check the number of descriptors.
+    FLAGS_file_cache_expiry_period_ms = 1;
+
+    // libunwind internally uses two file descriptors as a pipe.
+    // Make sure it gets initialized early so that our fd count
+    // doesn't get affected by it.
+    ignore_result(GetStackTraceHex());
+    initial_open_fds_ = CountOpenFds();
+  }
+
+  int CountOpenFds() const {
+    // Only count files in the test working directory so that we don't
+    // accidentally count other fds that might be opened or closed in
+    // the background by other threads.
+    return kudu::CountOpenFds(env_, GetTestPath("*"));
+  }
+
+  void SetUp() override {
+    KuduTest::SetUp();
+    ASSERT_OK(ReinitCache(1));
+  }
+
+ protected:
+  Status ReinitCache(int max_open_files) {
+    cache_.reset(new FileCache<FileType>("test",
+                                         env_,
+                                         max_open_files,
+                                         nullptr));
+    return cache_->Init();
+  }
+
+  Status WriteTestFile(const string& name, const string& data) {
+    unique_ptr<RWFile> f;
+    RETURN_NOT_OK(env_->NewRWFile(name, &f));
+    RETURN_NOT_OK(f->Write(0, data));
+    return Status::OK();
+  }
+
+  void AssertFdsAndDescriptors(int num_expected_fds,
+                               int num_expected_descriptors) {
+    ASSERT_EQ(initial_open_fds_ + num_expected_fds, CountOpenFds());
+
+    // The expiry thread may take some time to run.
+    ASSERT_EVENTUALLY([&]() {
+      ASSERT_EQ(num_expected_descriptors, cache_->NumDescriptorsForTests());
+    });
+  }
+
+  Random rand_;
+  int initial_open_fds_;
+  unique_ptr<FileCache<FileType>> cache_;
+};
+
+typedef ::testing::Types<RWFile, RandomAccessFile> FileTypes;
+TYPED_TEST_CASE(FileCacheTest, FileTypes);
+
+TYPED_TEST(FileCacheTest, TestBasicOperations) {
+  // Open a non-existent file.
+  {
+    shared_ptr<TypeParam> f;
+    ASSERT_TRUE(this->cache_->OpenExistingFile(
+        "/does/not/exist", &f).IsNotFound());
+    NO_FATALS(this->AssertFdsAndDescriptors(0, 0));
+  }
+
+  const string kFile1 = this->GetTestPath("foo");
+  const string kFile2 = this->GetTestPath("bar");
+  const string kData1 = "test data 1";
+  const string kData2 = "test data 2";
+
+  // Create some test files.
+  ASSERT_OK(this->WriteTestFile(kFile1, kData1));
+  ASSERT_OK(this->WriteTestFile(kFile2, kData2));
+  NO_FATALS(this->AssertFdsAndDescriptors(0, 0));
+
+  {
+    // Open a test file. It should open an fd and create a descriptor.
+    shared_ptr<TypeParam> f1;
+    ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f1));
+    NO_FATALS(this->AssertFdsAndDescriptors(1, 1));
+
+    // Spot check the test data by comparing sizes.
+    for (int i = 0; i < 3; i++) {
+      uint64_t size;
+      ASSERT_OK(f1->Size(&size));
+      ASSERT_EQ(kData1.size(), size);
+      NO_FATALS(this->AssertFdsAndDescriptors(1, 1));
+    }
+
+    // Open the same file a second time. It should reuse the existing
+    // descriptor and not open a second fd.
+    shared_ptr<TypeParam> f2;
+    ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f2));
+    NO_FATALS(this->AssertFdsAndDescriptors(1, 1));
+    {
+      Cache::UniqueHandle uh(
+          this->cache_->cache_->Lookup(kFile1, Cache::EXPECT_IN_CACHE),
+          Cache::HandleDeleter(this->cache_->cache_.get()));
+      ASSERT_TRUE(uh.get());
+    }
+
+    // Open a second file. This will create a new descriptor, but evict the fd
+    // opened for the first file, so the fd count should remain constant.
+    shared_ptr<TypeParam> f3;
+    ASSERT_OK(this->cache_->OpenExistingFile(kFile2, &f3));
+    NO_FATALS(this->AssertFdsAndDescriptors(1, 2));
+    {
+      Cache::UniqueHandle uh(
+          this->cache_->cache_->Lookup(kFile1, Cache::EXPECT_IN_CACHE),
+          Cache::HandleDeleter(this->cache_->cache_.get()));
+      ASSERT_FALSE(uh.get());
+    }
+    {
+      Cache::UniqueHandle uh(
+          this->cache_->cache_->Lookup(kFile2, Cache::EXPECT_IN_CACHE),
+          Cache::HandleDeleter(this->cache_->cache_.get()));
+      ASSERT_TRUE(uh.get());
+    }
+  }
+
+  // The descriptors are all out of scope, but the open fds remain in the cache.
+  NO_FATALS(this->AssertFdsAndDescriptors(1, 0));
+
+  // With the cache gone, so are the cached fds.
+  this->cache_.reset();
+  ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
+}
+
+TYPED_TEST(FileCacheTest, TestDeletion) {
+  // Deleting a file that doesn't exist does nothing/
+  ASSERT_TRUE(this->cache_->DeleteFile("/does/not/exist").IsNotFound());
+
+  // Create a test file, then delete it. It will be deleted immediately.
+  const string kFile1 = this->GetTestPath("foo");
+  const string kData1 = "test data 1";
+  ASSERT_OK(this->WriteTestFile(kFile1, kData1));
+  ASSERT_TRUE(this->env_->FileExists(kFile1));
+  ASSERT_OK(this->cache_->DeleteFile(kFile1));
+  ASSERT_FALSE(this->env_->FileExists(kFile1));
+
+  // Trying to delete it again fails.
+  ASSERT_TRUE(this->cache_->DeleteFile(kFile1).IsNotFound());
+
+  // Create another test file, open it, then delete it. The delete is not
+  // effected until the last open descriptor is closed. In between, the
+  // cache won't allow the file to be opened again.
+  const string kFile2 = this->GetTestPath("bar");
+  const string kData2 = "test data 2";
+  ASSERT_OK(this->WriteTestFile(kFile2, kData2));
+  ASSERT_TRUE(this->env_->FileExists(kFile2));
+  {
+    shared_ptr<TypeParam> f1;
+    ASSERT_OK(this->cache_->OpenExistingFile(kFile2, &f1));
+    ASSERT_EQ(this->initial_open_fds_ + 1, this->CountOpenFds());
+    ASSERT_OK(this->cache_->DeleteFile(kFile2));
+    {
+      shared_ptr<TypeParam> f2;
+      ASSERT_TRUE(this->cache_->OpenExistingFile(kFile2, &f2).IsNotFound());
+    }
+    ASSERT_TRUE(this->cache_->DeleteFile(kFile2).IsNotFound());
+    ASSERT_TRUE(this->env_->FileExists(kFile2));
+    ASSERT_EQ(this->initial_open_fds_ + 1, this->CountOpenFds());
+  }
+  ASSERT_FALSE(this->env_->FileExists(kFile2));
+  ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
+
+  // Create a test file, open it, and let it go out of scope before
+  // deleting it. The deletion should evict the fd and close it, despite
+  // happening after the descriptor is gone.
+  const string kFile3 = this->GetTestPath("baz");
+  const string kData3 = "test data 3";
+  ASSERT_OK(this->WriteTestFile(kFile3, kData3));
+  {
+    shared_ptr<TypeParam> f3;
+    ASSERT_OK(this->cache_->OpenExistingFile(kFile3, &f3));
+  }
+  ASSERT_TRUE(this->env_->FileExists(kFile3));
+  ASSERT_EQ(this->initial_open_fds_ + 1, this->CountOpenFds());
+  ASSERT_OK(this->cache_->DeleteFile(kFile3));
+  ASSERT_FALSE(this->env_->FileExists(kFile3));
+  ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
+}
+
+TYPED_TEST(FileCacheTest, TestInvalidation) {
+  const string kFile1 = this->GetTestPath("foo");
+  const string kData1 = "test data 1";
+  ASSERT_OK(this->WriteTestFile(kFile1, kData1));
+
+  // Open the file.
+  shared_ptr<TypeParam> f;
+  ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f));
+
+  // Write a new file and rename it in place on top of file1.
+  const string kFile2 = this->GetTestPath("foo2");
+  const string kData2 = "test data 2 (longer than original)";
+  ASSERT_OK(this->WriteTestFile(kFile2, kData2));
+  ASSERT_OK(this->env_->RenameFile(kFile2, kFile1));
+
+  // We should still be able to access the file, since it has a cached fd.
+  uint64_t size;
+  ASSERT_OK(f->Size(&size));
+  ASSERT_EQ(kData1.size(), size);
+
+  // If we invalidate it from the cache and try again, it should crash because
+  // the existing descriptor was invalidated.
+  this->cache_->Invalidate(kFile1);
+  ASSERT_DEATH({ f->Size(&size); }, "invalidated");
+
+  // But if we re-open the path again, the new descriptor should read the
+  // new data.
+  shared_ptr<TypeParam> f2;
+  ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f2));
+  ASSERT_OK(f2->Size(&size));
+  ASSERT_EQ(kData2.size(), size);
+}
+
+
+TYPED_TEST(FileCacheTest, TestHeavyReads) {
+  const int kNumFiles = 20;
+  const int kNumIterations = 100;
+  const int kCacheCapacity = 5;
+
+  ASSERT_OK(this->ReinitCache(kCacheCapacity));
+
+  // Randomly generate some data.
+  string data;
+  for (int i = 0; i < 1000; i++) {
+    data += Substitute("$0", this->rand_.Next());
+  }
+
+  // Write that data to a bunch of files and open them through the cache.
+  vector<shared_ptr<TypeParam>> opened_files;
+  for (int i = 0; i < kNumFiles; i++) {
+    string filename = this->GetTestPath(Substitute("$0", i));
+    ASSERT_OK(this->WriteTestFile(filename, data));
+    shared_ptr<TypeParam> f;
+    ASSERT_OK(this->cache_->OpenExistingFile(filename, &f));
+    opened_files.push_back(f);
+  }
+
+  // Read back the data at random through the cache.
+  unique_ptr<uint8_t[]> buf(new uint8_t[data.length()]);
+  for (int i = 0; i < kNumIterations; i++) {
+    int idx = this->rand_.Uniform(opened_files.size());
+    const auto& f = opened_files[idx];
+    uint64_t size;
+    ASSERT_OK(f->Size(&size));
+    Slice s(buf.get(), size);
+    ASSERT_OK(f->Read(0, s));
+    ASSERT_EQ(data, s);
+    ASSERT_LE(this->CountOpenFds(),
+              this->initial_open_fds_ + kCacheCapacity);
+  }
+}
+
+TYPED_TEST(FileCacheTest, TestNoRecursiveDeadlock) {
+  // This test triggered a deadlock in a previous implementation, when expired
+  // weak_ptrs were removed from the descriptor map in the descriptor's
+  // destructor.
+  alarm(60);
+  auto cleanup = MakeScopedCleanup([]() {
+    alarm(0);
+  });
+
+  const string kFile = this->GetTestPath("foo");
+  ASSERT_OK(this->WriteTestFile(kFile, "test data"));
+
+  vector<std::thread> threads;
+  for (int i = 0; i < 2; i++) {
+    threads.emplace_back([&]() {
+      for (int i = 0; i < 10000; i++) {
+        shared_ptr<TypeParam> f;
+        CHECK_OK(this->cache_->OpenExistingFile(kFile, &f));
+      }
+    });
+  }
+
+  for (auto& t : threads) {
+    t.join();
+  }
+}
+
+class RandomAccessFileCacheTest : public FileCacheTest<RandomAccessFile> {
+};
+
+TEST_F(RandomAccessFileCacheTest, TestMemoryFootprintDoesNotCrash) {
+  const string kFile = this->GetTestPath("foo");
+  ASSERT_OK(this->WriteTestFile(kFile, "test data"));
+
+  shared_ptr<RandomAccessFile> f;
+  ASSERT_OK(this->cache_->OpenExistingFile(kFile, &f));
+
+  // This used to crash due to a kudu_malloc_usable_size() call on a memory
+  // address that wasn't the start of an actual heap allocation.
+  LOG(INFO) << f->memory_footprint();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/file_cache.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/file_cache.cc b/be/src/kudu/util/file_cache.cc
new file mode 100644
index 0000000..a1ab814
--- /dev/null
+++ b/be/src/kudu/util/file_cache.cc
@@ -0,0 +1,654 @@
+// 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 "kudu/util/file_cache.h"
+
+#include <atomic>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/array_view.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"  // IWYU pragma: keep
+#include "kudu/util/monotime.h"
+#include "kudu/util/once.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+DEFINE_int32(file_cache_expiry_period_ms, 60 * 1000,
+             "Period of time (in ms) between removing expired file cache descriptors");
+TAG_FLAG(file_cache_expiry_period_ms, advanced);
+
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+namespace {
+
+template <class FileType>
+FileType* CacheValueToFileType(Slice s) {
+  return reinterpret_cast<FileType*>(*reinterpret_cast<void**>(
+      s.mutable_data()));
+}
+
+template <class FileType>
+class EvictionCallback : public Cache::EvictionCallback {
+ public:
+  EvictionCallback() {}
+
+  void EvictedEntry(Slice key, Slice value) override {
+    VLOG(2) << "Evicted fd belonging to " << key.ToString();
+    delete CacheValueToFileType<FileType>(value);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(EvictionCallback);
+};
+
+} // anonymous namespace
+
+namespace internal {
+
+template <class FileType>
+class ScopedOpenedDescriptor;
+
+// Encapsulates common descriptor fields and methods.
+template <class FileType>
+class BaseDescriptor {
+ public:
+  BaseDescriptor(FileCache<FileType>* file_cache,
+                 string filename)
+      : file_cache_(file_cache),
+        file_name_(std::move(filename)) {}
+
+  ~BaseDescriptor() {
+    VLOG(2) << "Out of scope descriptor with file name: " << filename();
+
+    // The (now expired) weak_ptr remains in 'descriptors_', to be removed by
+    // the next call to RunDescriptorExpiry(). Removing it here would risk a
+    // deadlock on recursive acquisition of 'lock_'.
+
+    if (deleted()) {
+      cache()->Erase(filename());
+
+      VLOG(1) << "Deleting file: " << filename();
+      WARN_NOT_OK(env()->DeleteFile(filename()), "");
+    }
+  }
+
+  // Insert a pointer to an open file object into the file cache with the
+  // filename as the cache key.
+  //
+  // Returns a handle to the inserted entry. The handle always contains an open
+  // file.
+  ScopedOpenedDescriptor<FileType> InsertIntoCache(void* file_ptr) const {
+    // The allocated charge is always one byte. This is incorrect with respect
+    // to memory tracking, but it's necessary if the cache capacity is to be
+    // equivalent to the max number of fds.
+    Cache::PendingHandle* pending = CHECK_NOTNULL(cache()->Allocate(
+        filename(), sizeof(file_ptr), 1));
+    memcpy(cache()->MutableValue(pending),
+           &file_ptr,
+           sizeof(file_ptr));
+    return ScopedOpenedDescriptor<FileType>(this, Cache::UniqueHandle(
+        cache()->Insert(pending, file_cache_->eviction_cb_.get()),
+        Cache::HandleDeleter(cache())));
+  }
+
+  // Retrieves a pointer to an open file object from the file cache with the
+  // filename as the cache key.
+  //
+  // Returns a handle to the looked up entry. The handle may or may not contain
+  // an open file, depending on whether the cache hit or missed.
+  ScopedOpenedDescriptor<FileType> LookupFromCache() const {
+    return ScopedOpenedDescriptor<FileType>(this, Cache::UniqueHandle(
+        cache()->Lookup(filename(), Cache::EXPECT_IN_CACHE),
+        Cache::HandleDeleter(cache())));
+  }
+
+  // Mark this descriptor as to-be-deleted later.
+  void MarkDeleted() {
+    DCHECK(!deleted());
+    while (true) {
+      auto v = flags_.load();
+      if (flags_.compare_exchange_weak(v, v | FILE_DELETED)) return;
+    }
+  }
+
+  // Mark this descriptor as invalidated. No further access is allowed
+  // to this file.
+  void MarkInvalidated() {
+    DCHECK(!invalidated());
+    while (true) {
+      auto v = flags_.load();
+      if (flags_.compare_exchange_weak(v, v | INVALIDATED)) return;
+    }
+  }
+
+  Cache* cache() const { return file_cache_->cache_.get(); }
+
+  Env* env() const { return file_cache_->env_; }
+
+  const string& filename() const { return file_name_; }
+
+  bool deleted() const { return flags_.load() & FILE_DELETED; }
+  bool invalidated() const { return flags_.load() & INVALIDATED; }
+
+ private:
+  FileCache<FileType>* file_cache_;
+  const string file_name_;
+  enum Flags {
+    FILE_DELETED = 1 << 0,
+    INVALIDATED = 1 << 1
+  };
+  std::atomic<uint8_t> flags_ {0};
+
+  DISALLOW_COPY_AND_ASSIGN(BaseDescriptor);
+};
+
+// A "smart" retrieved LRU cache handle.
+//
+// The cache handle is released when this object goes out of scope, possibly
+// closing the opened file if it is no longer in the cache.
+template <class FileType>
+class ScopedOpenedDescriptor {
+ public:
+  // A not-yet-but-soon-to-be opened descriptor.
+  explicit ScopedOpenedDescriptor(const BaseDescriptor<FileType>* desc)
+      : desc_(desc),
+        handle_(nullptr, Cache::HandleDeleter(desc_->cache())) {
+  }
+
+  // An opened descriptor. Its handle may or may not contain an open file.
+  ScopedOpenedDescriptor(const BaseDescriptor<FileType>* desc,
+                         Cache::UniqueHandle handle)
+      : desc_(desc),
+        handle_(std::move(handle)) {
+  }
+
+  bool opened() const { return handle_.get(); }
+
+  FileType* file() const {
+    DCHECK(opened());
+    return CacheValueToFileType<FileType>(desc_->cache()->Value(handle_.get()));
+  }
+
+ private:
+  const BaseDescriptor<FileType>* desc_;
+  Cache::UniqueHandle handle_;
+};
+
+// Reference to an on-disk file that may or may not be opened (and thus
+// cached) in the file cache.
+//
+// This empty template is just a specification; actual descriptor classes must
+// be fully specialized.
+template <class FileType>
+class Descriptor : public FileType {
+};
+
+// A descriptor adhering to the RWFile interface (i.e. when opened, provides
+// a read-write interface to the underlying file).
+template <>
+class Descriptor<RWFile> : public RWFile {
+ public:
+  Descriptor(FileCache<RWFile>* file_cache, const string& filename)
+      : base_(file_cache, filename) {}
+
+  ~Descriptor() = default;
+
+  Status Read(uint64_t offset, Slice result) const override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Read(offset, result);
+  }
+
+  Status ReadV(uint64_t offset, ArrayView<Slice> results) const override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->ReadV(offset, results);
+  }
+
+  Status Write(uint64_t offset, const Slice& data) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Write(offset, data);
+  }
+
+  Status WriteV(uint64_t offset, ArrayView<const Slice> data) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->WriteV(offset, data);
+  }
+
+  Status PreAllocate(uint64_t offset, size_t length, PreAllocateMode mode) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->PreAllocate(offset, length, mode);
+  }
+
+  Status Truncate(uint64_t length) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Truncate(length);
+  }
+
+  Status PunchHole(uint64_t offset, size_t length) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->PunchHole(offset, length);
+  }
+
+  Status Flush(FlushMode mode, uint64_t offset, size_t length) override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Flush(mode, offset, length);
+  }
+
+  Status Sync() override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Sync();
+  }
+
+  Status Close() override {
+    // Intentional no-op; actual closing is deferred to LRU cache eviction.
+    return Status::OK();
+  }
+
+  Status Size(uint64_t* size) const override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Size(size);
+  }
+
+  Status GetExtentMap(ExtentMap* out) const override {
+    ScopedOpenedDescriptor<RWFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->GetExtentMap(out);
+  }
+
+  const string& filename() const override {
+    return base_.filename();
+  }
+
+ private:
+  friend class FileCache<RWFile>;
+
+  Status Init() {
+    return once_.Init(&Descriptor<RWFile>::InitOnce, this);
+  }
+
+  Status InitOnce() {
+    return ReopenFileIfNecessary(nullptr);
+  }
+
+  Status ReopenFileIfNecessary(ScopedOpenedDescriptor<RWFile>* out) const {
+    ScopedOpenedDescriptor<RWFile> found(base_.LookupFromCache());
+    CHECK(!base_.invalidated());
+    if (found.opened()) {
+      // The file is already open in the cache, return it.
+      if (out) {
+        *out = std::move(found);
+      }
+      return Status::OK();
+    }
+
+    // The file was evicted, reopen it.
+    //
+    // Because the file may be evicted at any time we must use 'sync_on_close'
+    // (note: sync is a no-op if the file isn't dirty).
+    RWFileOptions opts;
+    opts.sync_on_close = true;
+    opts.mode = Env::OPEN_EXISTING;
+    unique_ptr<RWFile> f;
+    RETURN_NOT_OK(base_.env()->NewRWFile(opts, base_.filename(), &f));
+
+    // The cache will take ownership of the newly opened file.
+    ScopedOpenedDescriptor<RWFile> opened(base_.InsertIntoCache(f.release()));
+    if (out) {
+      *out = std::move(opened);
+    }
+    return Status::OK();
+  }
+
+  BaseDescriptor<RWFile> base_;
+  KuduOnceDynamic once_;
+
+  DISALLOW_COPY_AND_ASSIGN(Descriptor);
+};
+
+// A descriptor adhering to the RandomAccessFile interface (i.e. when opened,
+// provides a read-only interface to the underlying file).
+template <>
+class Descriptor<RandomAccessFile> : public RandomAccessFile {
+ public:
+  Descriptor(FileCache<RandomAccessFile>* file_cache, const string& filename)
+      : base_(file_cache, filename) {}
+
+  ~Descriptor() = default;
+
+  Status Read(uint64_t offset, Slice result) const override {
+    ScopedOpenedDescriptor<RandomAccessFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Read(offset, result);
+  }
+
+  Status ReadV(uint64_t offset, ArrayView<Slice> results) const override {
+    ScopedOpenedDescriptor<RandomAccessFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->ReadV(offset, results);
+  }
+
+  Status Size(uint64_t *size) const override {
+    ScopedOpenedDescriptor<RandomAccessFile> opened(&base_);
+    RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
+    return opened.file()->Size(size);
+  }
+
+  const string& filename() const override {
+    return base_.filename();
+  }
+
+  size_t memory_footprint() const override {
+    // Normally we would use kudu_malloc_usable_size(this). However, that's
+    // not safe because 'this' was allocated via std::make_shared(), which
+    // means it isn't necessarily the base of the memory allocation; it may be
+    // preceded by the shared_ptr control block.
+    //
+    // It doesn't appear possible to get the base of the allocation via any
+    // shared_ptr APIs, so we'll use sizeof(*this) + 16 instead. The 16 bytes
+    // represent the shared_ptr control block. Overall the object size is still
+    // undercounted as it doesn't account for any internal heap fragmentation,
+    // but at least it's safe.
+    //
+    // Some anecdotal memory measurements taken inside gdb:
+    // - glibc 2.23 malloc_usable_size() on make_shared<FileType>: 88 bytes.
+    // - tcmalloc malloc_usable_size() on make_shared<FileType>: 96 bytes.
+    // - sizeof(std::_Sp_counted_base<>) with libstdc++ 5.4: 16 bytes.
+    // - sizeof(std::__1::__shared_ptr_emplace<>) with libc++ 3.9: 16 bytes.
+    // - sizeof(*this): 72 bytes.
+    return sizeof(*this) +
+        16 + // shared_ptr control block
+        once_.memory_footprint_excluding_this() +
+        base_.filename().capacity();
+  }
+
+ private:
+  friend class FileCache<RandomAccessFile>;
+
+  Status Init() {
+    return once_.Init(&Descriptor<RandomAccessFile>::InitOnce, this);
+  }
+
+  Status InitOnce() {
+    return ReopenFileIfNecessary(nullptr);
+  }
+
+  Status ReopenFileIfNecessary(
+      ScopedOpenedDescriptor<RandomAccessFile>* out) const {
+    ScopedOpenedDescriptor<RandomAccessFile> found(base_.LookupFromCache());
+    CHECK(!base_.invalidated());
+    if (found.opened()) {
+      // The file is already open in the cache, return it.
+      if (out) {
+        *out = std::move(found);
+      }
+      return Status::OK();
+    }
+
+    // The file was evicted, reopen it.
+    unique_ptr<RandomAccessFile> f;
+    RETURN_NOT_OK(base_.env()->NewRandomAccessFile(base_.filename(), &f));
+
+    // The cache will take ownership of the newly opened file.
+    ScopedOpenedDescriptor<RandomAccessFile> opened(
+        base_.InsertIntoCache(f.release()));
+    if (out) {
+      *out = std::move(opened);
+    }
+    return Status::OK();
+  }
+
+  BaseDescriptor<RandomAccessFile> base_;
+  KuduOnceDynamic once_;
+
+  DISALLOW_COPY_AND_ASSIGN(Descriptor);
+};
+
+} // namespace internal
+
+template <class FileType>
+FileCache<FileType>::FileCache(const string& cache_name,
+                               Env* env,
+                               int max_open_files,
+                               const scoped_refptr<MetricEntity>& entity)
+    : env_(env),
+      cache_name_(cache_name),
+      eviction_cb_(new EvictionCallback<FileType>()),
+      cache_(NewLRUCache(DRAM_CACHE, max_open_files, cache_name)),
+      running_(1) {
+  if (entity) {
+    cache_->SetMetrics(entity);
+  }
+  LOG(INFO) << Substitute("Constructed file cache $0 with capacity $1",
+                          cache_name, max_open_files);
+}
+
+template <class FileType>
+FileCache<FileType>::~FileCache() {
+  running_.CountDown();
+  if (descriptor_expiry_thread_) {
+    descriptor_expiry_thread_->Join();
+  }
+}
+
+template <class FileType>
+Status FileCache<FileType>::Init() {
+  return Thread::Create("cache", Substitute("$0-evict", cache_name_),
+                        &FileCache::RunDescriptorExpiry, this,
+                        &descriptor_expiry_thread_);
+}
+
+template <class FileType>
+Status FileCache<FileType>::OpenExistingFile(const string& file_name,
+                                             shared_ptr<FileType>* file) {
+  shared_ptr<internal::Descriptor<FileType>> desc;
+  {
+    // Find an existing descriptor, or create one if none exists.
+    std::lock_guard<simple_spinlock> l(lock_);
+    RETURN_NOT_OK(FindDescriptorUnlocked(file_name, &desc));
+    if (desc) {
+      VLOG(2) << "Found existing descriptor: " << desc->filename();
+    } else {
+      desc = std::make_shared<internal::Descriptor<FileType>>(this, file_name);
+      InsertOrDie(&descriptors_, file_name, desc);
+      VLOG(2) << "Created new descriptor: " << desc->filename();
+    }
+  }
+
+  // Check that the underlying file can be opened (no-op for found
+  // descriptors). Done outside the lock.
+  RETURN_NOT_OK(desc->Init());
+  *file = std::move(desc);
+  return Status::OK();
+}
+
+template <class FileType>
+Status FileCache<FileType>::DeleteFile(const string& file_name) {
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    shared_ptr<internal::Descriptor<FileType>> desc;
+    RETURN_NOT_OK(FindDescriptorUnlocked(file_name, &desc));
+
+    if (desc) {
+      VLOG(2) << "Marking file for deletion: " << file_name;
+      desc->base_.MarkDeleted();
+      return Status::OK();
+    }
+  }
+
+  // There is no outstanding descriptor. Delete the file now.
+  //
+  // Make sure it's been fully evicted from the cache (perhaps it was opened
+  // previously?) so that the filesystem can reclaim the file data instantly.
+  cache_->Erase(file_name);
+  return env_->DeleteFile(file_name);
+}
+
+template <class FileType>
+void FileCache<FileType>::Invalidate(const string& file_name) {
+  // Ensure that there is an invalidated descriptor in the map for this filename.
+  //
+  // This ensures that any concurrent OpenExistingFile() during this method wil
+  // see the invalidation and issue a CHECK failure.
+  shared_ptr<internal::Descriptor<FileType>> desc;
+  {
+    // Find an existing descriptor, or create one if none exists.
+    std::lock_guard<simple_spinlock> l(lock_);
+    auto it = descriptors_.find(file_name);
+    if (it != descriptors_.end()) {
+      desc = it->second.lock();
+    }
+    if (!desc) {
+      desc = std::make_shared<internal::Descriptor<FileType>>(this, file_name);
+      descriptors_.emplace(file_name, desc);
+    }
+
+    desc->base_.MarkInvalidated();
+  }
+  // Remove it from the cache so that if the same path is opened again, we
+  // will re-open a new FD rather than retrieving one that might have been
+  // cached prior to invalidation.
+  cache_->Erase(file_name);
+
+  // Remove the invalidated descriptor from the map. We are guaranteed it
+  // is still there because we've held a strong reference to it for
+  // the duration of this method, and no other methods erase strong
+  // references from the map.
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    CHECK_EQ(1, descriptors_.erase(file_name));
+  }
+}
+
+template <class FileType>
+int FileCache<FileType>::NumDescriptorsForTests() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return descriptors_.size();
+}
+
+template <class FileType>
+string FileCache<FileType>::ToDebugString() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  string ret;
+  for (const auto& e : descriptors_) {
+    bool strong = false;
+    bool deleted = false;
+    bool opened = false;
+    shared_ptr<internal::Descriptor<FileType>> desc = e.second.lock();
+    if (desc) {
+      strong = true;
+      if (desc->base_.deleted()) {
+        deleted = true;
+      }
+      internal::ScopedOpenedDescriptor<FileType> o(
+          desc->base_.LookupFromCache());
+      if (o.opened()) {
+        opened = true;
+      }
+    }
+    if (strong) {
+      ret += Substitute("$0 (S$1$2)\n", e.first,
+                        deleted ? "D" : "", opened ? "O" : "");
+    } else {
+      ret += Substitute("$0\n", e.first);
+    }
+  }
+  return ret;
+}
+
+template <class FileType>
+Status FileCache<FileType>::FindDescriptorUnlocked(
+    const string& file_name,
+    shared_ptr<internal::Descriptor<FileType>>* file) {
+  DCHECK(lock_.is_locked());
+
+  auto it = descriptors_.find(file_name);
+  if (it != descriptors_.end()) {
+    // Found the descriptor. Has it expired?
+    shared_ptr<internal::Descriptor<FileType>> desc = it->second.lock();
+    if (desc) {
+      CHECK(!desc->base_.invalidated());
+      if (desc->base_.deleted()) {
+        return Status::NotFound("File already marked for deletion", file_name);
+      }
+
+      // Descriptor is still valid, return it.
+      if (file) {
+        *file = desc;
+      }
+      return Status::OK();
+    }
+    // Descriptor has expired; erase it and pretend we found nothing.
+    descriptors_.erase(it);
+  }
+  return Status::OK();
+}
+
+template <class FileType>
+void FileCache<FileType>::RunDescriptorExpiry() {
+  while (!running_.WaitFor(MonoDelta::FromMilliseconds(
+      FLAGS_file_cache_expiry_period_ms))) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    for (auto it = descriptors_.begin(); it != descriptors_.end();) {
+      if (it->second.expired()) {
+        it = descriptors_.erase(it);
+      } else {
+        it++;
+      }
+    }
+  }
+}
+
+// Explicit specialization for callers outside this compilation unit.
+template class FileCache<RWFile>;
+template class FileCache<RandomAccessFile>;
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/file_cache.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/file_cache.h b/be/src/kudu/util/file_cache.h
new file mode 100644
index 0000000..021c758
--- /dev/null
+++ b/be/src/kudu/util/file_cache.h
@@ -0,0 +1,209 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Env;
+
+namespace internal {
+
+template <class FileType>
+class BaseDescriptor;
+
+template <class FileType>
+class Descriptor;
+
+} // namespace internal
+
+class MetricEntity;
+class Thread;
+
+// Cache of open files.
+//
+// The purpose of this cache is to enforce an upper bound on the maximum number
+// of files open at a time. Files opened through the cache may be closed at any
+// time, only to be reopened upon next use.
+//
+// The file cache can be viewed as having two logical parts: the client-facing
+// API and the LRU cache.
+//
+// Client-facing API
+// -----------------
+// The core of the client-facing API is the cache descriptor. A descriptor
+// uniquely identifies an opened file. To a client, a descriptor is just an
+// open file interface of the variety defined in util/env.h. Clients open
+// descriptors via the OpenExistingFile() cache method.
+//
+// Descriptors are shared objects; an existing descriptor is handed back to a
+// client if a file with the same name is already opened. To facilitate
+// descriptor sharing, the file cache maintains a by-file-name descriptor map.
+// The values are weak references to the descriptors so that map entries don't
+// affect the descriptor lifecycle.
+//
+// LRU cache
+// ---------
+// The lower half of the file cache is a standard LRU cache whose keys are file
+// names and whose values are pointers to opened file objects allocated on the
+// heap. Unlike the descriptor map, this cache has an upper bound on capacity,
+// and handles are evicted (and closed) according to an LRU algorithm.
+//
+// Whenever a descriptor is used by a client in file I/O, its file name is used
+// in an LRU cache lookup. If found, the underlying file is still open and the
+// file access is performed. Otherwise, the file must have been evicted and
+// closed, so it is reopened and reinserted (possibly evicting a different open
+// file) before the file access is performed.
+//
+// Other notes
+// -----------
+// In a world where files are opened and closed transparently, file deletion
+// demands special care if UNIX semantics are to be preserved. When a call to
+// DeleteFile() is made to a file with an opened descriptor, the descriptor is
+// simply "marked" as to-be-deleted-later. Only when all references to the
+// descriptor are dropped is the file actually deleted. If there is no open
+// descriptor, the file is deleted immediately.
+//
+// Every public method in the file cache is thread safe.
+template <class FileType>
+class FileCache {
+ public:
+  // Creates a new file cache.
+  //
+  // The 'cache_name' is used to disambiguate amongst other file cache
+  // instances. The cache will use 'max_open_files' as a soft upper bound on
+  // the number of files open at any given time.
+  FileCache(const std::string& cache_name,
+            Env* env,
+            int max_open_files,
+            const scoped_refptr<MetricEntity>& entity);
+
+  // Destroys the file cache.
+  ~FileCache();
+
+  // Initializes the file cache. Initialization done here may fail.
+  Status Init();
+
+  // Opens an existing file by name through the cache.
+  //
+  // The returned 'file' is actually an object called a descriptor. It adheres
+  // to a file-like interface but interfaces with the cache under the hood to
+  // reopen a file as needed during file operations.
+  //
+  // The descriptor is opened immediately to verify that the on-disk file can
+  // be opened, but may be closed later if the cache reaches its upper bound on
+  // the number of open files.
+  Status OpenExistingFile(const std::string& file_name,
+                          std::shared_ptr<FileType>* file);
+
+  // Deletes a file by name through the cache.
+  //
+  // If there is an outstanding descriptor for the file, the deletion will be
+  // deferred until the last referent is dropped. Otherwise, the file is
+  // deleted immediately.
+  Status DeleteFile(const std::string& file_name);
+
+  // Invalidate the given path in the cache if present. This removes the
+  // path from the cache, and invalidates any previously-opened descriptors
+  // associated with this file.
+  //
+  // If a file with the same path is opened again, the actual path will be opened from
+  // disk.
+  //
+  // This operation should be used during 'rename-to-replace' patterns, eg:
+  //
+  //    WriteNewDataTo(tmp_path);
+  //    env->RenameFile(tmp_path, p);
+  //    file_cache->Invalidate(p);
+  //
+  // NOTE: if any reader of 'p' holds an open descriptor from the cache
+  // prior to this operation, that descriptor is invalidated and any
+  // further operations on that descriptor will result in a CHECK failure.
+  // Hence this is not safe to use without some external synchronization
+  // which prevents concurrent access to the same file.
+  //
+  // NOTE: this function must not be called concurrently on the same file name
+  // from multiple threads.
+  void Invalidate(const std::string& file_name);
+
+  // Returns the number of entries in the descriptor map.
+  //
+  // Only intended for unit tests.
+  int NumDescriptorsForTests() const;
+
+  // Dumps the contents of the file cache. Intended for debugging.
+  std::string ToDebugString() const;
+
+ private:
+  friend class internal::BaseDescriptor<FileType>;
+
+  template<class FileType2>
+  FRIEND_TEST(FileCacheTest, TestBasicOperations);
+
+  // Looks up a descriptor by file name.
+  //
+  // Must be called with 'lock_' held.
+  Status FindDescriptorUnlocked(
+      const std::string& file_name,
+      std::shared_ptr<internal::Descriptor<FileType>>* file);
+
+  // Periodically removes expired descriptors from 'descriptors_'.
+  void RunDescriptorExpiry();
+
+  // Interface to the underlying filesystem.
+  Env* env_;
+
+  // Name of the cache.
+  const std::string cache_name_;
+
+  // Invoked whenever a cached file reaches zero references (i.e. it was
+  // removed from the cache and is no longer in use by any file operations).
+  std::unique_ptr<Cache::EvictionCallback> eviction_cb_;
+
+  // Underlying cache instance. Caches opened files.
+  std::unique_ptr<Cache> cache_;
+
+  // Protects the descriptor map.
+  mutable simple_spinlock lock_;
+
+  // Maps filenames to descriptors.
+  std::unordered_map<std::string,
+                     std::weak_ptr<internal::Descriptor<FileType>>> descriptors_;
+
+  // Calls RunDescriptorExpiry() in a loop until 'running_' isn't set.
+  scoped_refptr<Thread> descriptor_expiry_thread_;
+
+  // Tracks whether or not 'descriptor_expiry_thread_' should be running.
+  CountDownLatch running_;
+
+  DISALLOW_COPY_AND_ASSIGN(FileCache);
+};
+
+} // namespace kudu


[47/51] [abbrv] impala git commit: IMPALA-7006: Pick parts of recent Kudu gutil changes

Posted by ta...@apache.org.
IMPALA-7006: Pick parts of recent Kudu gutil changes

- Include some ASAN macros from gutil (Kudu commit c8724c61)
- Pick parts of KUDU-2427 (Kudu commit b7cf3b2e)
- Rename constants (Kudu commit e719b5ef)

These changes will be subsumed by a proper rebase of GUTIL.

Change-Id: Id2dc8c70425e3ac030427ebeb1ec18a44d14d5cb
Reviewed-on: http://gerrit.cloudera.org:8080/10769
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Lars Volker <lv...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/0459721c
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/0459721c
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/0459721c

Branch: refs/heads/master
Commit: 0459721ccd59deae34548d44c684def4e04b31a6
Parents: 1ac9a3f
Author: Lars Volker <lv...@cloudera.com>
Authored: Tue Jul 3 17:05:37 2018 -0700
Committer: Lars Volker <lv...@cloudera.com>
Committed: Thu Jul 12 21:35:42 2018 +0000

----------------------------------------------------------------------
 be/src/gutil/macros.h              | 22 +++++++++++++++++
 be/src/gutil/port.h                | 26 ++++++++++++++++++++
 be/src/gutil/strings/substitute.cc |  4 ++--
 be/src/gutil/strings/substitute.h  | 42 ++++++++++++++++-----------------
 be/src/gutil/sysinfo.cc            | 26 ++++++++++++++------
 be/src/util/error-util.h           | 20 ++++++++--------
 6 files changed, 100 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/gutil/macros.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/macros.h b/be/src/gutil/macros.h
index 0318008..da7ea13 100644
--- a/be/src/gutil/macros.h
+++ b/be/src/gutil/macros.h
@@ -262,4 +262,26 @@ enum LinkerInitialized { LINKER_INITIALIZED };
 #define FALLTHROUGH_INTENDED do { } while (0)
 #endif
 
+// Retry on EINTR for functions like read() that return -1 on error.
+#define RETRY_ON_EINTR(err, expr) do { \
+  static_assert(std::is_signed<decltype(err)>::value, \
+                #err " must be a signed integer"); \
+  (err) = (expr); \
+} while ((err) == -1 && errno == EINTR)
+
+// Same as above but for stream API calls like fread() and fwrite().
+#define STREAM_RETRY_ON_EINTR(nread, stream, expr) do { \
+  static_assert(std::is_unsigned<decltype(nread)>::value == true, \
+                #nread " must be an unsigned integer"); \
+  (nread) = (expr); \
+} while ((nread) == 0 && ferror(stream) == EINTR)
+
+// Same as above but for functions that return pointer types (like
+// fopen() and freopen()).
+#define POINTER_RETRY_ON_EINTR(ptr, expr) do { \
+  static_assert(std::is_pointer<decltype(ptr)>::value == true, \
+                #ptr " must be a pointer"); \
+  (ptr) = (expr); \
+} while ((ptr) == nullptr && errno == EINTR)
+
 #endif  // BASE_MACROS_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/gutil/port.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/port.h b/be/src/gutil/port.h
index e204de7..e258dba 100644
--- a/be/src/gutil/port.h
+++ b/be/src/gutil/port.h
@@ -423,6 +423,32 @@ inline void* memrchr(const void* bytes, int find_char, size_t len) {
 #define ATTRIBUTE_NO_SANITIZE_THREAD
 #endif
 
+// Tell UBSAN to ignore a given function completely. There is no
+// __has_feature(undefined_sanitizer) or equivalent, so ASAN support is used as
+// a proxy.
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+#  define ATTRIBUTE_NO_SANITIZE_UNDEFINED \
+      __attribute__((no_sanitize("undefined")))
+#  endif
+#endif
+#ifndef ATTRIBUTE_NO_SANITIZE_UNDEFINED
+#define ATTRIBUTE_NO_SANITIZE_UNDEFINED
+#endif
+
+// Tell UBSAN to ignore integer overflows in a given function. There is no
+// __has_feature(undefined_sanitizer) or equivalent, so ASAN support is used as
+// a proxy.
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+#  define ATTRIBUTE_NO_SANITIZE_INTEGER \
+      __attribute__((no_sanitize("integer")))
+#  endif
+#endif
+#ifndef ATTRIBUTE_NO_SANITIZE_INTEGER
+#define ATTRIBUTE_NO_SANITIZE_INTEGER
+#endif
+
 #ifndef HAVE_ATTRIBUTE_SECTION  // may have been pre-set to 0, e.g. for Darwin
 #define HAVE_ATTRIBUTE_SECTION 1
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/gutil/strings/substitute.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/substitute.cc b/be/src/gutil/strings/substitute.cc
index 76ca151..5c69f04 100644
--- a/be/src/gutil/strings/substitute.cc
+++ b/be/src/gutil/strings/substitute.cc
@@ -13,13 +13,13 @@ namespace strings {
 
 using internal::SubstituteArg;
 
-const SubstituteArg SubstituteArg::NoArg;
+const SubstituteArg SubstituteArg::kNoArg;
 
 // Returns the number of args in arg_array which were passed explicitly
 // to Substitute().
 static int CountSubstituteArgs(const SubstituteArg* const* args_array) {
   int count = 0;
-  while (args_array[count] != &SubstituteArg::NoArg) {
+  while (args_array[count] != &SubstituteArg::kNoArg) {
     ++count;
   }
   return count;

http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/gutil/strings/substitute.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/substitute.h b/be/src/gutil/strings/substitute.h
index 84d362a..902a073 100644
--- a/be/src/gutil/strings/substitute.h
+++ b/be/src/gutil/strings/substitute.h
@@ -131,7 +131,7 @@ class SubstituteArg {
   inline int size() const { return size_; }
 
   // Indicates that no argument was given.
-  static const SubstituteArg NoArg;
+  static const SubstituteArg kNoArg;
 
  private:
   inline SubstituteArg() : text_(NULL), size_(-1) {}
@@ -158,29 +158,29 @@ char* SubstituteToBuffer(StringPiece format,
 
 void SubstituteAndAppend(
   string* output, StringPiece format,
-  const internal::SubstituteArg& arg0 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg1 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg2 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg3 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg4 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg5 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg6 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg7 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg8 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg9 = internal::SubstituteArg::NoArg);
+  const internal::SubstituteArg& arg0 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg1 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg2 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg3 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg4 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg5 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg6 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg7 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg8 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg9 = internal::SubstituteArg::kNoArg);
 
 inline string Substitute(
   StringPiece format,
-  const internal::SubstituteArg& arg0 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg1 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg2 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg3 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg4 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg5 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg6 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg7 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg8 = internal::SubstituteArg::NoArg,
-  const internal::SubstituteArg& arg9 = internal::SubstituteArg::NoArg) {
+  const internal::SubstituteArg& arg0 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg1 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg2 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg3 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg4 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg5 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg6 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg7 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg8 = internal::SubstituteArg::kNoArg,
+  const internal::SubstituteArg& arg9 = internal::SubstituteArg::kNoArg) {
   string result;
   SubstituteAndAppend(&result, format, arg0, arg1, arg2, arg3, arg4,
                                        arg5, arg6, arg7, arg8, arg9);

http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/gutil/sysinfo.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/sysinfo.cc b/be/src/gutil/sysinfo.cc
index a2b6077..dddfa74 100644
--- a/be/src/gutil/sysinfo.cc
+++ b/be/src/gutil/sysinfo.cc
@@ -42,8 +42,8 @@
 #include <unistd.h>   // for read()
 #endif
 #if defined __MACH__          // Mac OS X, almost certainly
-#include <sys/types.h>
 #include <sys/sysctl.h>       // how we figure out numcpu's on OS X
+#include <sys/types.h>
 #elif defined __FreeBSD__
 #include <sys/sysctl.h>
 #elif defined __sun__         // Solaris
@@ -114,18 +114,25 @@ static int64 EstimateCyclesPerSecond(const int estimate_time_ms) {
 // issue a FATAL error.
 static bool SlurpSmallTextFile(const char* file, char* buf, int buflen) {
   bool ret = false;
-  int fd = open(file, O_RDONLY);
+  int fd;
+  RETRY_ON_EINTR(fd, open(file, O_RDONLY));
   if (fd == -1) return ret;
 
   memset(buf, '\0', buflen);
-  int n = read(fd, buf, buflen - 1);
+  int n;
+  RETRY_ON_EINTR(n, read(fd, buf, buflen - 1));
   CHECK_NE(n, buflen - 1) << "buffer of len " << buflen << " not large enough to store "
                           << "contents of " << file;
   if (n > 0) {
     ret = true;
   }
 
-  close(fd);
+  int close_ret;
+  RETRY_ON_EINTR(close_ret, close(fd));
+  if (PREDICT_FALSE(close_ret != 0)) {
+    PLOG(WARNING) << "Failed to close fd " << fd;
+  }
+
   return ret;
 }
 
@@ -220,7 +227,8 @@ static void InitializeSystemInfo() {
 
   // Read /proc/cpuinfo for other values, and if there is no cpuinfo_max_freq.
   const char* pname = "/proc/cpuinfo";
-  int fd = open(pname, O_RDONLY);
+  int fd;
+  RETRY_ON_EINTR(fd, open(pname, O_RDONLY));
   if (fd == -1) {
     PLOG(FATAL) << "Unable to read CPU info from /proc. procfs must be mounted.";
   }
@@ -243,7 +251,7 @@ static void InitializeSystemInfo() {
       const int linelen = strlen(line);
       const int bytes_to_read = sizeof(line)-1 - linelen;
       CHECK(bytes_to_read > 0);  // because the memmove recovered >=1 bytes
-      chars_read = read(fd, line + linelen, bytes_to_read);
+      RETRY_ON_EINTR(chars_read, read(fd, line + linelen, bytes_to_read));
       line[linelen + chars_read] = '\0';
       newline = strchr(line, '\n');
     }
@@ -288,7 +296,11 @@ static void InitializeSystemInfo() {
       num_cpus++;  // count up every time we see an "processor :" entry
     }
   } while (chars_read > 0);
-  close(fd);
+  int ret;
+  RETRY_ON_EINTR(ret, close(fd));
+  if (PREDICT_FALSE(ret != 0)) {
+    PLOG(WARNING) << "Failed to close fd " << fd;
+  }
 
   if (!saw_mhz) {
     if (saw_bogo) {

http://git-wip-us.apache.org/repos/asf/impala/blob/0459721c/be/src/util/error-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/error-util.h b/be/src/util/error-util.h
index 44dcb26..3975548 100644
--- a/be/src/util/error-util.h
+++ b/be/src/util/error-util.h
@@ -93,16 +93,16 @@ class ErrorMsg {
   /// the cost of this method is proportional to the number of entries in the global error
   /// message list.
   /// WARNING: DO NOT CALL THIS METHOD IN A NON STATIC CONTEXT
-  static ErrorMsg Init(TErrorCode::type error, const ArgType& arg0 = ArgType::NoArg,
-      const ArgType& arg1 = ArgType::NoArg,
-      const ArgType& arg2 = ArgType::NoArg,
-      const ArgType& arg3 = ArgType::NoArg,
-      const ArgType& arg4 = ArgType::NoArg,
-      const ArgType& arg5 = ArgType::NoArg,
-      const ArgType& arg6 = ArgType::NoArg,
-      const ArgType& arg7 = ArgType::NoArg,
-      const ArgType& arg8 = ArgType::NoArg,
-      const ArgType& arg9 = ArgType::NoArg);
+  static ErrorMsg Init(TErrorCode::type error, const ArgType& arg0 = ArgType::kNoArg,
+      const ArgType& arg1 = ArgType::kNoArg,
+      const ArgType& arg2 = ArgType::kNoArg,
+      const ArgType& arg3 = ArgType::kNoArg,
+      const ArgType& arg4 = ArgType::kNoArg,
+      const ArgType& arg5 = ArgType::kNoArg,
+      const ArgType& arg6 = ArgType::kNoArg,
+      const ArgType& arg7 = ArgType::kNoArg,
+      const ArgType& arg8 = ArgType::kNoArg,
+      const ArgType& arg9 = ArgType::kNoArg);
 
   TErrorCode::type error() const { return error_; }
 


[03/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadpool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadpool-test.cc b/be/src/kudu/util/threadpool-test.cc
new file mode 100644
index 0000000..23fc45c
--- /dev/null
+++ b/be/src/kudu/util/threadpool-test.cc
@@ -0,0 +1,941 @@
+// 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 <unistd.h>
+
+#include <atomic>
+#include <cstdint>
+#include <iterator>
+#include <limits>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/smart_ptr/shared_ptr.hpp>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/util/barrier.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/promise.h"
+#include "kudu/util/random.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/trace.h"
+
+using std::atomic;
+using std::shared_ptr;
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+using strings::Substitute;
+
+DECLARE_int32(thread_inject_start_latency_ms);
+
+namespace kudu {
+
+static const char* kDefaultPoolName = "test";
+
+class ThreadPoolTest : public KuduTest {
+ public:
+
+  virtual void SetUp() override {
+    KuduTest::SetUp();
+    ASSERT_OK(ThreadPoolBuilder(kDefaultPoolName).Build(&pool_));
+  }
+
+  Status RebuildPoolWithBuilder(const ThreadPoolBuilder& builder) {
+    return builder.Build(&pool_);
+  }
+
+  Status RebuildPoolWithMinMax(int min_threads, int max_threads) {
+    return ThreadPoolBuilder(kDefaultPoolName)
+        .set_min_threads(min_threads)
+        .set_max_threads(max_threads)
+        .Build(&pool_);
+  }
+
+ protected:
+  gscoped_ptr<ThreadPool> pool_;
+};
+
+TEST_F(ThreadPoolTest, TestNoTaskOpenClose) {
+  ASSERT_OK(RebuildPoolWithMinMax(4, 4));
+  pool_->Shutdown();
+}
+
+static void SimpleTaskMethod(int n, Atomic32 *counter) {
+  while (n--) {
+    base::subtle::NoBarrier_AtomicIncrement(counter, 1);
+    boost::detail::yield(n);
+  }
+}
+
+class SimpleTask : public Runnable {
+ public:
+  SimpleTask(int n, Atomic32 *counter)
+    : n_(n), counter_(counter) {
+  }
+
+  void Run() OVERRIDE {
+    SimpleTaskMethod(n_, counter_);
+  }
+
+ private:
+  int n_;
+  Atomic32 *counter_;
+};
+
+TEST_F(ThreadPoolTest, TestSimpleTasks) {
+  ASSERT_OK(RebuildPoolWithMinMax(4, 4));
+
+  Atomic32 counter(0);
+  std::shared_ptr<Runnable> task(new SimpleTask(15, &counter));
+
+  ASSERT_OK(pool_->SubmitFunc(boost::bind(&SimpleTaskMethod, 10, &counter)));
+  ASSERT_OK(pool_->Submit(task));
+  ASSERT_OK(pool_->SubmitFunc(boost::bind(&SimpleTaskMethod, 20, &counter)));
+  ASSERT_OK(pool_->Submit(task));
+  ASSERT_OK(pool_->SubmitClosure(Bind(&SimpleTaskMethod, 123, &counter)));
+  pool_->Wait();
+  ASSERT_EQ(10 + 15 + 20 + 15 + 123, base::subtle::NoBarrier_Load(&counter));
+  pool_->Shutdown();
+}
+
+static void IssueTraceStatement() {
+  TRACE("hello from task");
+}
+
+// Test that the thread-local trace is propagated to tasks
+// submitted to the threadpool.
+TEST_F(ThreadPoolTest, TestTracePropagation) {
+  ASSERT_OK(RebuildPoolWithMinMax(1, 1));
+
+  scoped_refptr<Trace> t(new Trace);
+  {
+    ADOPT_TRACE(t.get());
+    ASSERT_OK(pool_->SubmitFunc(&IssueTraceStatement));
+  }
+  pool_->Wait();
+  ASSERT_STR_CONTAINS(t->DumpToString(), "hello from task");
+}
+
+TEST_F(ThreadPoolTest, TestSubmitAfterShutdown) {
+  ASSERT_OK(RebuildPoolWithMinMax(1, 1));
+  pool_->Shutdown();
+  Status s = pool_->SubmitFunc(&IssueTraceStatement);
+  ASSERT_EQ("Service unavailable: The pool has been shut down.",
+            s.ToString());
+}
+
+class SlowTask : public Runnable {
+ public:
+  explicit SlowTask(CountDownLatch* latch)
+    : latch_(latch) {
+  }
+
+  void Run() OVERRIDE {
+    latch_->Wait();
+  }
+
+  static shared_ptr<Runnable> NewSlowTask(CountDownLatch* latch) {
+    return std::make_shared<SlowTask>(latch);
+  }
+
+ private:
+  CountDownLatch* latch_;
+};
+
+TEST_F(ThreadPoolTest, TestThreadPoolWithNoMinimum) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(0)
+                                   .set_max_threads(3)
+                                   .set_idle_timeout(MonoDelta::FromMilliseconds(1))));
+
+  // There are no threads to start with.
+  ASSERT_TRUE(pool_->num_threads() == 0);
+  // We get up to 3 threads when submitting work.
+  CountDownLatch latch(1);
+  SCOPED_CLEANUP({
+    latch.CountDown();
+  });
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(2, pool_->num_threads());
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(3, pool_->num_threads());
+  // The 4th piece of work gets queued.
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(3, pool_->num_threads());
+  // Finish all work
+  latch.CountDown();
+  pool_->Wait();
+  ASSERT_EQ(0, pool_->active_threads_);
+  pool_->Shutdown();
+  ASSERT_EQ(0, pool_->num_threads());
+}
+
+TEST_F(ThreadPoolTest, TestThreadPoolWithNoMaxThreads) {
+  // By default a threadpool's max_threads is set to the number of CPUs, so
+  // this test submits more tasks than that to ensure that the number of CPUs
+  // isn't some kind of upper bound.
+  const int kNumCPUs = base::NumCPUs();
+
+  // Build a threadpool with no limit on the maximum number of threads.
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_threads(std::numeric_limits<int>::max())));
+  CountDownLatch latch(1);
+  auto cleanup_latch = MakeScopedCleanup([&]() {
+    latch.CountDown();
+  });
+
+  // Submit tokenless tasks. Each should create a new thread.
+  for (int i = 0; i < kNumCPUs * 2; i++) {
+    ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  }
+  ASSERT_EQ((kNumCPUs * 2), pool_->num_threads());
+
+  // Submit tasks on two tokens. Only two threads should be created.
+  unique_ptr<ThreadPoolToken> t1 = pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
+  unique_ptr<ThreadPoolToken> t2 = pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
+  for (int i = 0; i < kNumCPUs * 2; i++) {
+    ThreadPoolToken* t = (i % 2 == 0) ? t1.get() : t2.get();
+    ASSERT_OK(t->Submit(SlowTask::NewSlowTask(&latch)));
+  }
+  ASSERT_EQ((kNumCPUs * 2) + 2, pool_->num_threads());
+
+  // Submit more tokenless tasks. Each should create a new thread.
+  for (int i = 0; i < kNumCPUs; i++) {
+    ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  }
+  ASSERT_EQ((kNumCPUs * 3) + 2, pool_->num_threads());
+
+  latch.CountDown();
+  pool_->Wait();
+  pool_->Shutdown();
+}
+
+// Regression test for a bug where a task is submitted exactly
+// as a thread is about to exit. Previously this could hang forever.
+TEST_F(ThreadPoolTest, TestRace) {
+  alarm(60);
+  auto cleanup = MakeScopedCleanup([]() {
+    alarm(0); // Disable alarm on test exit.
+  });
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(0)
+                                   .set_max_threads(1)
+                                   .set_idle_timeout(MonoDelta::FromMicroseconds(1))));
+
+  for (int i = 0; i < 500; i++) {
+    CountDownLatch l(1);
+    ASSERT_OK(pool_->SubmitFunc(boost::bind(&CountDownLatch::CountDown, &l)));
+    l.Wait();
+    // Sleeping a different amount in each iteration makes it more likely to hit
+    // the bug.
+    SleepFor(MonoDelta::FromMicroseconds(i));
+  }
+}
+
+TEST_F(ThreadPoolTest, TestVariableSizeThreadPool) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(1)
+                                   .set_max_threads(4)
+                                   .set_idle_timeout(MonoDelta::FromMilliseconds(1))));
+
+  // There is 1 thread to start with.
+  ASSERT_EQ(1, pool_->num_threads());
+  // We get up to 4 threads when submitting work.
+  CountDownLatch latch(1);
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(1, pool_->num_threads());
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(2, pool_->num_threads());
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(3, pool_->num_threads());
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(4, pool_->num_threads());
+  // The 5th piece of work gets queued.
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_EQ(4, pool_->num_threads());
+  // Finish all work
+  latch.CountDown();
+  pool_->Wait();
+  ASSERT_EQ(0, pool_->active_threads_);
+  pool_->Shutdown();
+  ASSERT_EQ(0, pool_->num_threads());
+}
+
+TEST_F(ThreadPoolTest, TestMaxQueueSize) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(1)
+                                   .set_max_threads(1)
+                                   .set_max_queue_size(1)));
+
+  CountDownLatch latch(1);
+  // We will be able to submit two tasks: one for max_threads == 1 and one for
+  // max_queue_size == 1.
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  Status s = pool_->Submit(SlowTask::NewSlowTask(&latch));
+  CHECK(s.IsServiceUnavailable()) << "Expected failure due to queue blowout:" << s.ToString();
+  latch.CountDown();
+  pool_->Wait();
+  pool_->Shutdown();
+}
+
+// Test that when we specify a zero-sized queue, the maximum number of threads
+// running is used for enforcement.
+TEST_F(ThreadPoolTest, TestZeroQueueSize) {
+  const int kMaxThreads = 4;
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_queue_size(0)
+                                   .set_max_threads(kMaxThreads)));
+
+  CountDownLatch latch(1);
+  for (int i = 0; i < kMaxThreads; i++) {
+    ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  }
+  Status s = pool_->Submit(SlowTask::NewSlowTask(&latch));
+  ASSERT_TRUE(s.IsServiceUnavailable()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "Thread pool is at capacity");
+  latch.CountDown();
+  pool_->Wait();
+  pool_->Shutdown();
+}
+
+// Regression test for KUDU-2187:
+//
+// If a threadpool thread is slow to start up, it shouldn't block progress of
+// other tasks on the same pool.
+TEST_F(ThreadPoolTest, TestSlowThreadStart) {
+  // Start a pool of threads from which we'll submit tasks.
+  gscoped_ptr<ThreadPool> submitter_pool;
+  ASSERT_OK(ThreadPoolBuilder("submitter")
+            .set_min_threads(5)
+            .set_max_threads(5)
+            .Build(&submitter_pool));
+
+  // Start the actual test pool, which starts with one thread
+  // but will start a second one on-demand.
+  ASSERT_OK(RebuildPoolWithMinMax(1, 2));
+  // Ensure that the second thread will take a long time to start.
+  FLAGS_thread_inject_start_latency_ms = 3000;
+
+  // Now submit 10 tasks to the 'submitter' pool, each of which
+  // submits a single task to 'pool_'. The 'pool_' task sleeps
+  // for 10ms.
+  //
+  // Because the 'submitter' tasks submit faster than they can be
+  // processed on a single thread (due to the sleep), we expect that
+  // this will trigger 'pool_' to start up its second worker thread.
+  // The thread startup will have some latency injected.
+  //
+  // We expect that the thread startup will block only one of the
+  // tasks in the 'submitter' pool after it submits its task. Other
+  // tasks will continue to be processed by the other (already-running)
+  // thread on 'pool_'.
+  std::atomic<int32_t> total_queue_time_ms(0);
+  for (int i = 0; i < 10; i++) {
+    ASSERT_OK(submitter_pool->SubmitFunc([&]() {
+          auto submit_time = MonoTime::Now();
+          CHECK_OK(pool_->SubmitFunc([&,submit_time]() {
+                auto queue_time = MonoTime::Now() - submit_time;
+                total_queue_time_ms += queue_time.ToMilliseconds();
+                SleepFor(MonoDelta::FromMilliseconds(10));
+              }));
+        }));
+  }
+  submitter_pool->Wait();
+  pool_->Wait();
+
+  // Since the total amount of work submitted was only 100ms, we expect
+  // that the performance would be equivalent to a single-threaded
+  // threadpool. So, we expect the total queue time to be approximately
+  // 0 + 10 + 20 ... + 80 + 90 = 450ms.
+  //
+  // If, instead, throughput had been blocked while starting threads,
+  // we'd get something closer to 18000ms (3000ms delay * 5 submitter threads).
+  ASSERT_GE(total_queue_time_ms, 400);
+  ASSERT_LE(total_queue_time_ms, 10000);
+}
+
+// Test that setting a promise from another thread yields
+// a value on the current thread.
+TEST_F(ThreadPoolTest, TestPromises) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(1)
+                                   .set_max_threads(1)
+                                   .set_max_queue_size(1)));
+
+  Promise<int> my_promise;
+  ASSERT_OK(pool_->SubmitClosure(
+                     Bind(&Promise<int>::Set, Unretained(&my_promise), 5)));
+  ASSERT_EQ(5, my_promise.Get());
+  pool_->Shutdown();
+}
+
+METRIC_DEFINE_entity(test_entity);
+METRIC_DEFINE_histogram(test_entity, queue_length, "queue length",
+                        MetricUnit::kTasks, "queue length", 1000, 1);
+
+METRIC_DEFINE_histogram(test_entity, queue_time, "queue time",
+                        MetricUnit::kMicroseconds, "queue time", 1000000, 1);
+
+METRIC_DEFINE_histogram(test_entity, run_time, "run time",
+                        MetricUnit::kMicroseconds, "run time", 1000, 1);
+
+TEST_F(ThreadPoolTest, TestMetrics) {
+  MetricRegistry registry;
+  vector<ThreadPoolMetrics> all_metrics;
+  for (int i = 0; i < 3; i++) {
+    scoped_refptr<MetricEntity> entity = METRIC_ENTITY_test_entity.Instantiate(
+        &registry, Substitute("test $0", i));
+    all_metrics.emplace_back(ThreadPoolMetrics{
+      METRIC_queue_length.Instantiate(entity),
+      METRIC_queue_time.Instantiate(entity),
+      METRIC_run_time.Instantiate(entity)
+    });
+  }
+
+  // Enable metrics for the thread pool.
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(1)
+                                   .set_max_threads(1)
+                                   .set_metrics(all_metrics[0])));
+
+  unique_ptr<ThreadPoolToken> t1 = pool_->NewTokenWithMetrics(
+      ThreadPool::ExecutionMode::SERIAL, all_metrics[1]);
+  unique_ptr<ThreadPoolToken> t2 = pool_->NewTokenWithMetrics(
+      ThreadPool::ExecutionMode::SERIAL, all_metrics[2]);
+
+  // Submit once to t1, twice to t2, and three times without a token.
+  ASSERT_OK(t1->SubmitFunc([](){}));
+  ASSERT_OK(t2->SubmitFunc([](){}));
+  ASSERT_OK(t2->SubmitFunc([](){}));
+  ASSERT_OK(pool_->SubmitFunc([](){}));
+  ASSERT_OK(pool_->SubmitFunc([](){}));
+  ASSERT_OK(pool_->SubmitFunc([](){}));
+  pool_->Wait();
+
+  // The total counts should reflect the number of submissions to each token.
+  ASSERT_EQ(1, all_metrics[1].queue_length_histogram->TotalCount());
+  ASSERT_EQ(1, all_metrics[1].queue_time_us_histogram->TotalCount());
+  ASSERT_EQ(1, all_metrics[1].run_time_us_histogram->TotalCount());
+  ASSERT_EQ(2, all_metrics[2].queue_length_histogram->TotalCount());
+  ASSERT_EQ(2, all_metrics[2].queue_time_us_histogram->TotalCount());
+  ASSERT_EQ(2, all_metrics[2].run_time_us_histogram->TotalCount());
+
+  // And the counts on the pool-wide metrics should reflect all submissions.
+  ASSERT_EQ(6, all_metrics[0].queue_length_histogram->TotalCount());
+  ASSERT_EQ(6, all_metrics[0].queue_time_us_histogram->TotalCount());
+  ASSERT_EQ(6, all_metrics[0].run_time_us_histogram->TotalCount());
+}
+
+// Test that a thread pool will crash if asked to run its own blocking
+// functions in a pool thread.
+//
+// In a multi-threaded application, TSAN is unsafe to use following a fork().
+// After a fork(), TSAN will:
+// 1. Disable verification, expecting an exec() soon anyway, and
+// 2. Die on future thread creation.
+// For some reason, this test triggers behavior #2. We could disable it with
+// the TSAN option die_after_fork=0, but this can (supposedly) lead to
+// deadlocks, so we'll disable the entire test instead.
+#ifndef THREAD_SANITIZER
+TEST_F(ThreadPoolTest, TestDeadlocks) {
+  const char* death_msg = "called pool function that would result in deadlock";
+  ASSERT_DEATH({
+    ASSERT_OK(RebuildPoolWithMinMax(1, 1));
+    ASSERT_OK(pool_->SubmitClosure(
+        Bind(&ThreadPool::Shutdown, Unretained(pool_.get()))));
+    pool_->Wait();
+  }, death_msg);
+
+  ASSERT_DEATH({
+    ASSERT_OK(RebuildPoolWithMinMax(1, 1));
+    ASSERT_OK(pool_->SubmitClosure(
+        Bind(&ThreadPool::Wait, Unretained(pool_.get()))));
+    pool_->Wait();
+  }, death_msg);
+}
+#endif
+
+class SlowDestructorRunnable : public Runnable {
+ public:
+  void Run() override {}
+
+  virtual ~SlowDestructorRunnable() {
+    SleepFor(MonoDelta::FromMilliseconds(100));
+  }
+};
+
+// Test that if a tasks's destructor is slow, it doesn't cause serialization of the tasks
+// in the queue.
+TEST_F(ThreadPoolTest, TestSlowDestructor) {
+  ASSERT_OK(RebuildPoolWithMinMax(1, 20));
+  MonoTime start = MonoTime::Now();
+  for (int i = 0; i < 100; i++) {
+    shared_ptr<Runnable> task(new SlowDestructorRunnable());
+    ASSERT_OK(pool_->Submit(std::move(task)));
+  }
+  pool_->Wait();
+  ASSERT_LT((MonoTime::Now() - start).ToSeconds(), 5);
+}
+
+// For test cases that should run with both kinds of tokens.
+class ThreadPoolTestTokenTypes : public ThreadPoolTest,
+                                 public testing::WithParamInterface<ThreadPool::ExecutionMode> {};
+
+INSTANTIATE_TEST_CASE_P(Tokens, ThreadPoolTestTokenTypes,
+                        ::testing::Values(ThreadPool::ExecutionMode::SERIAL,
+                                          ThreadPool::ExecutionMode::CONCURRENT));
+
+
+TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitAndWait) {
+  unique_ptr<ThreadPoolToken> t = pool_->NewToken(GetParam());
+  int i = 0;
+  ASSERT_OK(t->SubmitFunc([&]() {
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    i++;
+  }));
+  t->Wait();
+  ASSERT_EQ(1, i);
+}
+
+TEST_F(ThreadPoolTest, TestTokenSubmitsProcessedSerially) {
+  unique_ptr<ThreadPoolToken> t = pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
+  Random r(SeedRandom());
+  string result;
+  for (char c = 'a'; c < 'f'; c++) {
+    // Sleep a little first so that there's a higher chance of out-of-order
+    // appends if the submissions did execute in parallel.
+    int sleep_ms = r.Next() % 5;
+    ASSERT_OK(t->SubmitFunc([&result, c, sleep_ms]() {
+      SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+      result += c;
+    }));
+  }
+  t->Wait();
+  ASSERT_EQ("abcde", result);
+}
+
+TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitsProcessedConcurrently) {
+  const int kNumTokens = 5;
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_threads(kNumTokens)));
+  vector<unique_ptr<ThreadPoolToken>> tokens;
+
+  // A violation to the tested invariant would yield a deadlock, so let's set
+  // up an alarm to bail us out.
+  alarm(60);
+  SCOPED_CLEANUP({
+      alarm(0); // Disable alarm on test exit.
+  });
+  shared_ptr<Barrier> b = std::make_shared<Barrier>(kNumTokens + 1);
+  for (int i = 0; i < kNumTokens; i++) {
+    tokens.emplace_back(pool_->NewToken(GetParam()));
+    ASSERT_OK(tokens.back()->SubmitFunc([b]() {
+      b->Wait();
+    }));
+  }
+
+  // This will deadlock if the above tasks weren't all running concurrently.
+  b->Wait();
+}
+
+TEST_F(ThreadPoolTest, TestTokenSubmitsNonSequential) {
+  const int kNumSubmissions = 5;
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_threads(kNumSubmissions)));
+
+  // A violation to the tested invariant would yield a deadlock, so let's set
+  // up an alarm to bail us out.
+  alarm(60);
+  SCOPED_CLEANUP({
+      alarm(0); // Disable alarm on test exit.
+  });
+  shared_ptr<Barrier> b = std::make_shared<Barrier>(kNumSubmissions + 1);
+  unique_ptr<ThreadPoolToken> t = pool_->NewToken(ThreadPool::ExecutionMode::CONCURRENT);
+  for (int i = 0; i < kNumSubmissions; i++) {
+    ASSERT_OK(t->SubmitFunc([b]() {
+      b->Wait();
+    }));
+  }
+
+  // This will deadlock if the above tasks weren't all running concurrently.
+  b->Wait();
+}
+
+TEST_P(ThreadPoolTestTokenTypes, TestTokenShutdown) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_threads(4)));
+
+  unique_ptr<ThreadPoolToken> t1(pool_->NewToken(GetParam()));
+  unique_ptr<ThreadPoolToken> t2(pool_->NewToken(GetParam()));
+  CountDownLatch l1(1);
+  CountDownLatch l2(1);
+
+  // A violation to the tested invariant would yield a deadlock, so let's set
+  // up an alarm to bail us out.
+  alarm(60);
+  SCOPED_CLEANUP({
+      alarm(0); // Disable alarm on test exit.
+  });
+
+  for (int i = 0; i < 3; i++) {
+    ASSERT_OK(t1->SubmitFunc([&]() {
+      l1.Wait();
+    }));
+  }
+  for (int i = 0; i < 3; i++) {
+    ASSERT_OK(t2->SubmitFunc([&]() {
+      l2.Wait();
+    }));
+  }
+
+  // Unblock all of t1's tasks, but not t2's tasks.
+  l1.CountDown();
+
+  // If this also waited for t2's tasks, it would deadlock.
+  t1->Shutdown();
+
+  // We can no longer submit to t1 but we can still submit to t2.
+  ASSERT_TRUE(t1->SubmitFunc([](){}).IsServiceUnavailable());
+  ASSERT_OK(t2->SubmitFunc([](){}));
+
+  // Unblock t2's tasks.
+  l2.CountDown();
+  t2->Shutdown();
+}
+
+TEST_P(ThreadPoolTestTokenTypes, TestTokenWaitForAll) {
+  const int kNumTokens = 3;
+  const int kNumSubmissions = 20;
+  Random r(SeedRandom());
+  vector<unique_ptr<ThreadPoolToken>> tokens;
+  for (int i = 0; i < kNumTokens; i++) {
+    tokens.emplace_back(pool_->NewToken(GetParam()));
+  }
+
+  atomic<int32_t> v(0);
+  for (int i = 0; i < kNumSubmissions; i++) {
+    // Sleep a little first to raise the likelihood of the test thread
+    // reaching Wait() before the submissions finish.
+    int sleep_ms = r.Next() % 5;
+
+    auto task = [&v, sleep_ms]() {
+      SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+      v++;
+    };
+
+    // Half of the submissions will be token-less, and half will use a token.
+    if (i % 2 == 0) {
+      ASSERT_OK(pool_->SubmitFunc(task));
+    } else {
+      int token_idx = r.Next() % tokens.size();
+      ASSERT_OK(tokens[token_idx]->SubmitFunc(task));
+    }
+  }
+  pool_->Wait();
+  ASSERT_EQ(kNumSubmissions, v);
+}
+
+TEST_F(ThreadPoolTest, TestFuzz) {
+  const int kNumOperations = 1000;
+  Random r(SeedRandom());
+  vector<unique_ptr<ThreadPoolToken>> tokens;
+
+  for (int i = 0; i < kNumOperations; i++) {
+    // Operation distribution:
+    //
+    // - Submit without a token: 40%
+    // - Submit with a randomly selected token: 35%
+    // - Allocate a new token: 10%
+    // - Wait on a randomly selected token: 7%
+    // - Shutdown a randomly selected token: 4%
+    // - Deallocate a randomly selected token: 2%
+    // - Wait for all submissions: 2%
+    int op = r.Next() % 100;
+    if (op < 40) {
+      // Submit without a token.
+      int sleep_ms = r.Next() % 5;
+      ASSERT_OK(pool_->SubmitFunc([sleep_ms]() {
+        // Sleep a little first to increase task overlap.
+        SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+      }));
+    } else if (op < 75) {
+      // Submit with a randomly selected token.
+      if (tokens.empty()) {
+        continue;
+      }
+      int sleep_ms = r.Next() % 5;
+      int token_idx = r.Next() % tokens.size();
+      Status s = tokens[token_idx]->SubmitFunc([sleep_ms]() {
+        // Sleep a little first to increase task overlap.
+        SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+      });
+      ASSERT_TRUE(s.ok() || s.IsServiceUnavailable());
+    } else if (op < 85) {
+      // Allocate a token with a randomly selected policy.
+      ThreadPool::ExecutionMode mode = r.Next() % 2 ?
+          ThreadPool::ExecutionMode::SERIAL :
+          ThreadPool::ExecutionMode::CONCURRENT;
+      tokens.emplace_back(pool_->NewToken(mode));
+    } else if (op < 92) {
+      // Wait on a randomly selected token.
+      if (tokens.empty()) {
+        continue;
+      }
+      int token_idx = r.Next() % tokens.size();
+      tokens[token_idx]->Wait();
+    } else if (op < 96) {
+      // Shutdown a randomly selected token.
+      if (tokens.empty()) {
+        continue;
+      }
+      int token_idx = r.Next() % tokens.size();
+      tokens[token_idx]->Shutdown();
+    } else if (op < 98) {
+      // Deallocate a randomly selected token.
+      if (tokens.empty()) {
+        continue;
+      }
+      auto it = tokens.begin();
+      int token_idx = r.Next() % tokens.size();
+      std::advance(it, token_idx);
+      tokens.erase(it);
+    } else {
+      // Wait on everything.
+      ASSERT_LT(op, 100);
+      ASSERT_GE(op, 98);
+      pool_->Wait();
+    }
+  }
+
+  // Some test runs will shut down the pool before the tokens, and some won't.
+  // Either way should be safe.
+  if (r.Next() % 2 == 0) {
+    pool_->Shutdown();
+  }
+}
+
+TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmissionsAdhereToMaxQueueSize) {
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_min_threads(1)
+                                   .set_max_threads(1)
+                                   .set_max_queue_size(1)));
+
+  CountDownLatch latch(1);
+  unique_ptr<ThreadPoolToken> t = pool_->NewToken(GetParam());
+  SCOPED_CLEANUP({
+    latch.CountDown();
+  });
+  // We will be able to submit two tasks: one for max_threads == 1 and one for
+  // max_queue_size == 1.
+  ASSERT_OK(t->Submit(SlowTask::NewSlowTask(&latch)));
+  ASSERT_OK(t->Submit(SlowTask::NewSlowTask(&latch)));
+  Status s = t->Submit(SlowTask::NewSlowTask(&latch));
+  ASSERT_TRUE(s.IsServiceUnavailable());
+}
+
+TEST_F(ThreadPoolTest, TestTokenConcurrency) {
+  const int kNumTokens = 20;
+  const int kTestRuntimeSecs = 1;
+  const int kCycleThreads = 2;
+  const int kShutdownThreads = 2;
+  const int kWaitThreads = 2;
+  const int kSubmitThreads = 8;
+
+  vector<shared_ptr<ThreadPoolToken>> tokens;
+  Random rng(SeedRandom());
+
+  // Protects 'tokens' and 'rng'.
+  simple_spinlock lock;
+
+  // Fetch a token from 'tokens' at random.
+  auto GetRandomToken = [&]() -> shared_ptr<ThreadPoolToken> {
+    std::lock_guard<simple_spinlock> l(lock);
+    int idx = rng.Uniform(kNumTokens);
+    return tokens[idx];
+  };
+
+  // Preallocate all of the tokens.
+  for (int i = 0; i < kNumTokens; i++) {
+    ThreadPool::ExecutionMode mode;
+    {
+      std::lock_guard<simple_spinlock> l(lock);
+      mode = rng.Next() % 2 ?
+          ThreadPool::ExecutionMode::SERIAL :
+          ThreadPool::ExecutionMode::CONCURRENT;
+    }
+    tokens.emplace_back(pool_->NewToken(mode).release());
+  }
+
+  atomic<int64_t> total_num_tokens_cycled(0);
+  atomic<int64_t> total_num_tokens_shutdown(0);
+  atomic<int64_t> total_num_tokens_waited(0);
+  atomic<int64_t> total_num_tokens_submitted(0);
+
+  CountDownLatch latch(1);
+  vector<thread> threads;
+
+  for (int i = 0; i < kCycleThreads; i++) {
+    // Pick a token at random and replace it.
+    //
+    // The replaced token is only destroyed when the last ref is dropped,
+    // possibly by another thread.
+    threads.emplace_back([&]() {
+      int num_tokens_cycled = 0;
+      while (latch.count()) {
+        {
+          std::lock_guard<simple_spinlock> l(lock);
+          int idx = rng.Uniform(kNumTokens);
+          ThreadPool::ExecutionMode mode = rng.Next() % 2 ?
+              ThreadPool::ExecutionMode::SERIAL :
+              ThreadPool::ExecutionMode::CONCURRENT;
+          tokens[idx] = shared_ptr<ThreadPoolToken>(pool_->NewToken(mode).release());
+        }
+        num_tokens_cycled++;
+
+        // Sleep a bit, otherwise this thread outpaces the other threads and
+        // nothing interesting happens to most tokens.
+        SleepFor(MonoDelta::FromMicroseconds(10));
+      }
+      total_num_tokens_cycled += num_tokens_cycled;
+    });
+  }
+
+  for (int i = 0; i < kShutdownThreads; i++) {
+    // Pick a token at random and shut it down. Submitting a task to a shut
+    // down token will return a ServiceUnavailable error.
+    threads.emplace_back([&]() {
+      int num_tokens_shutdown = 0;
+      while (latch.count()) {
+        GetRandomToken()->Shutdown();
+        num_tokens_shutdown++;
+      }
+      total_num_tokens_shutdown += num_tokens_shutdown;
+    });
+  }
+
+  for (int i = 0; i < kWaitThreads; i++) {
+    // Pick a token at random and wait for any outstanding tasks.
+    threads.emplace_back([&]() {
+      int num_tokens_waited  = 0;
+      while (latch.count()) {
+        GetRandomToken()->Wait();
+        num_tokens_waited++;
+      }
+      total_num_tokens_waited += num_tokens_waited;
+    });
+  }
+
+  for (int i = 0; i < kSubmitThreads; i++) {
+    // Pick a token at random and submit a task to it.
+    threads.emplace_back([&]() {
+      int num_tokens_submitted = 0;
+      Random rng(SeedRandom());
+      while (latch.count()) {
+        int sleep_ms = rng.Next() % 5;
+        Status s = GetRandomToken()->SubmitFunc([sleep_ms]() {
+          // Sleep a little first so that tasks are running during other events.
+          SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+        });
+        CHECK(s.ok() || s.IsServiceUnavailable());
+        num_tokens_submitted++;
+      }
+      total_num_tokens_submitted += num_tokens_submitted;
+    });
+  }
+
+  SleepFor(MonoDelta::FromSeconds(kTestRuntimeSecs));
+  latch.CountDown();
+  for (auto& t : threads) {
+    t.join();
+  }
+
+  LOG(INFO) << Substitute("Tokens cycled ($0 threads): $1",
+                          kCycleThreads, total_num_tokens_cycled.load());
+  LOG(INFO) << Substitute("Tokens shutdown ($0 threads): $1",
+                          kShutdownThreads, total_num_tokens_shutdown.load());
+  LOG(INFO) << Substitute("Tokens waited ($0 threads): $1",
+                          kWaitThreads, total_num_tokens_waited.load());
+  LOG(INFO) << Substitute("Tokens submitted ($0 threads): $1",
+                          kSubmitThreads, total_num_tokens_submitted.load());
+}
+
+TEST_F(ThreadPoolTest, TestLIFOThreadWakeUps) {
+  const int kNumThreads = 10;
+
+  // Test with a pool that allows for kNumThreads concurrent threads.
+  ASSERT_OK(RebuildPoolWithBuilder(ThreadPoolBuilder(kDefaultPoolName)
+                                   .set_max_threads(kNumThreads)));
+
+  // Submit kNumThreads slow tasks and unblock them, in order to produce
+  // kNumThreads worker threads.
+  CountDownLatch latch(1);
+  SCOPED_CLEANUP({
+    latch.CountDown();
+  });
+  for (int i = 0; i < kNumThreads; i++) {
+    ASSERT_OK(pool_->Submit(SlowTask::NewSlowTask(&latch)));
+  }
+  ASSERT_EQ(kNumThreads, pool_->num_threads());
+  latch.CountDown();
+  pool_->Wait();
+
+  // The kNumThreads threads are idle and waiting for the idle timeout.
+
+  // Submit a slow trickle of lightning fast tasks.
+  //
+  // If the threads are woken up in FIFO order, this trickle is enough to
+  // prevent all of them from idling and the AssertEventually will time out.
+  //
+  // If LIFO order is used, the same thread will be reused for each task and
+  // the other threads will eventually time out.
+  AssertEventually([&]() {
+    ASSERT_OK(pool_->SubmitFunc([](){}));
+    SleepFor(MonoDelta::FromMilliseconds(10));
+    ASSERT_EQ(1, pool_->num_threads());
+  }, MonoDelta::FromSeconds(10), AssertBackoff::NONE);
+  NO_PENDING_FATALS();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadpool.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadpool.cc b/be/src/kudu/util/threadpool.cc
new file mode 100644
index 0000000..23dda3d
--- /dev/null
+++ b/be/src/kudu/util/threadpool.cc
@@ -0,0 +1,766 @@
+// 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 "kudu/util/threadpool.h"
+
+#include <cstdint>
+#include <deque>
+#include <limits>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <boost/function.hpp> // IWYU pragma: keep
+#include <glog/logging.h>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/trace_metrics.h"
+
+namespace kudu {
+
+using std::deque;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+////////////////////////////////////////////////////////
+// FunctionRunnable
+////////////////////////////////////////////////////////
+
+class FunctionRunnable : public Runnable {
+ public:
+  explicit FunctionRunnable(boost::function<void()> func) : func_(std::move(func)) {}
+
+  void Run() OVERRIDE {
+    func_();
+  }
+
+ private:
+  boost::function<void()> func_;
+};
+
+////////////////////////////////////////////////////////
+// ClosureRunnable
+////////////////////////////////////////////////////////
+
+class ClosureRunnable : public Runnable {
+ public:
+  explicit ClosureRunnable(Closure cl) : cl_(std::move(cl)) {}
+
+  void Run() OVERRIDE {
+    cl_.Run();
+  }
+
+ private:
+  Closure cl_;
+};
+
+////////////////////////////////////////////////////////
+// ThreadPoolBuilder
+////////////////////////////////////////////////////////
+
+ThreadPoolBuilder::ThreadPoolBuilder(string name)
+    : name_(std::move(name)),
+      min_threads_(0),
+      max_threads_(base::NumCPUs()),
+      max_queue_size_(std::numeric_limits<int>::max()),
+      idle_timeout_(MonoDelta::FromMilliseconds(500)) {}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_trace_metric_prefix(const string& prefix) {
+  trace_metric_prefix_ = prefix;
+  return *this;
+}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_min_threads(int min_threads) {
+  CHECK_GE(min_threads, 0);
+  min_threads_ = min_threads;
+  return *this;
+}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_max_threads(int max_threads) {
+  CHECK_GT(max_threads, 0);
+  max_threads_ = max_threads;
+  return *this;
+}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_max_queue_size(int max_queue_size) {
+  max_queue_size_ = max_queue_size;
+  return *this;
+}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_idle_timeout(const MonoDelta& idle_timeout) {
+  idle_timeout_ = idle_timeout;
+  return *this;
+}
+
+ThreadPoolBuilder& ThreadPoolBuilder::set_metrics(ThreadPoolMetrics metrics) {
+  metrics_ = std::move(metrics);
+  return *this;
+}
+
+Status ThreadPoolBuilder::Build(gscoped_ptr<ThreadPool>* pool) const {
+  pool->reset(new ThreadPool(*this));
+  RETURN_NOT_OK((*pool)->Init());
+  return Status::OK();
+}
+
+////////////////////////////////////////////////////////
+// ThreadPoolToken
+////////////////////////////////////////////////////////
+
+ThreadPoolToken::ThreadPoolToken(ThreadPool* pool,
+                                 ThreadPool::ExecutionMode mode,
+                                 ThreadPoolMetrics metrics)
+    : mode_(mode),
+      metrics_(std::move(metrics)),
+      pool_(pool),
+      state_(State::IDLE),
+      not_running_cond_(&pool->lock_),
+      active_threads_(0) {
+}
+
+ThreadPoolToken::~ThreadPoolToken() {
+  Shutdown();
+  pool_->ReleaseToken(this);
+}
+
+Status ThreadPoolToken::SubmitClosure(Closure c) {
+  return Submit(std::make_shared<ClosureRunnable>(std::move(c)));
+}
+
+Status ThreadPoolToken::SubmitFunc(boost::function<void()> f) {
+  return Submit(std::make_shared<FunctionRunnable>(std::move(f)));
+}
+
+Status ThreadPoolToken::Submit(shared_ptr<Runnable> r) {
+  return pool_->DoSubmit(std::move(r), this);
+}
+
+void ThreadPoolToken::Shutdown() {
+  MutexLock unique_lock(pool_->lock_);
+  pool_->CheckNotPoolThreadUnlocked();
+
+  // Clear the queue under the lock, but defer the releasing of the tasks
+  // outside the lock, in case there are concurrent threads wanting to access
+  // the ThreadPool. The task's destructors may acquire locks, etc, so this
+  // also prevents lock inversions.
+  std::deque<ThreadPool::Task> to_release = std::move(entries_);
+  pool_->total_queued_tasks_ -= to_release.size();
+
+  switch (state()) {
+    case State::IDLE:
+      // There were no tasks outstanding; we can quiesce the token immediately.
+      Transition(State::QUIESCED);
+      break;
+    case State::RUNNING:
+      // There were outstanding tasks. If any are still running, switch to
+      // QUIESCING and wait for them to finish (the worker thread executing
+      // the token's last task will switch the token to QUIESCED). Otherwise,
+      // we can quiesce the token immediately.
+
+      // Note: this is an O(n) operation, but it's expected to be infrequent.
+      // Plus doing it this way (rather than switching to QUIESCING and waiting
+      // for a worker thread to process the queue entry) helps retain state
+      // transition symmetry with ThreadPool::Shutdown.
+      for (auto it = pool_->queue_.begin(); it != pool_->queue_.end();) {
+        if (*it == this) {
+          it = pool_->queue_.erase(it);
+        } else {
+          it++;
+        }
+      }
+
+      if (active_threads_ == 0) {
+        Transition(State::QUIESCED);
+        break;
+      }
+      Transition(State::QUIESCING);
+      FALLTHROUGH_INTENDED;
+    case State::QUIESCING:
+      // The token is already quiescing. Just wait for a worker thread to
+      // switch it to QUIESCED.
+      while (state() != State::QUIESCED) {
+        not_running_cond_.Wait();
+      }
+      break;
+    default:
+      break;
+  }
+
+  // Finally release the queued tasks, outside the lock.
+  unique_lock.Unlock();
+  for (auto& t : to_release) {
+    if (t.trace) {
+      t.trace->Release();
+    }
+  }
+}
+
+void ThreadPoolToken::Wait() {
+  MutexLock unique_lock(pool_->lock_);
+  pool_->CheckNotPoolThreadUnlocked();
+  while (IsActive()) {
+    not_running_cond_.Wait();
+  }
+}
+
+bool ThreadPoolToken::WaitUntil(const MonoTime& until) {
+  MutexLock unique_lock(pool_->lock_);
+  pool_->CheckNotPoolThreadUnlocked();
+  while (IsActive()) {
+    if (!not_running_cond_.WaitUntil(until)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+bool ThreadPoolToken::WaitFor(const MonoDelta& delta) {
+  return WaitUntil(MonoTime::Now() + delta);
+}
+
+void ThreadPoolToken::Transition(State new_state) {
+#ifndef NDEBUG
+  CHECK_NE(state_, new_state);
+
+  switch (state_) {
+    case State::IDLE:
+      CHECK(new_state == State::RUNNING ||
+            new_state == State::QUIESCED);
+      if (new_state == State::RUNNING) {
+        CHECK(!entries_.empty());
+      } else {
+        CHECK(entries_.empty());
+        CHECK_EQ(active_threads_, 0);
+      }
+      break;
+    case State::RUNNING:
+      CHECK(new_state == State::IDLE ||
+            new_state == State::QUIESCING ||
+            new_state == State::QUIESCED);
+      CHECK(entries_.empty());
+      if (new_state == State::QUIESCING) {
+        CHECK_GT(active_threads_, 0);
+      }
+      break;
+    case State::QUIESCING:
+      CHECK(new_state == State::QUIESCED);
+      CHECK_EQ(active_threads_, 0);
+      break;
+    case State::QUIESCED:
+      CHECK(false); // QUIESCED is a terminal state
+      break;
+    default:
+      LOG(FATAL) << "Unknown token state: " << state_;
+  }
+#endif
+
+  // Take actions based on the state we're entering.
+  switch (new_state) {
+    case State::IDLE:
+    case State::QUIESCED:
+      not_running_cond_.Broadcast();
+      break;
+    default:
+      break;
+  }
+
+  state_ = new_state;
+}
+
+const char* ThreadPoolToken::StateToString(State s) {
+  switch (s) {
+    case State::IDLE: return "IDLE"; break;
+    case State::RUNNING: return "RUNNING"; break;
+    case State::QUIESCING: return "QUIESCING"; break;
+    case State::QUIESCED: return "QUIESCED"; break;
+  }
+  return "<cannot reach here>";
+}
+
+////////////////////////////////////////////////////////
+// ThreadPool
+////////////////////////////////////////////////////////
+
+ThreadPool::ThreadPool(const ThreadPoolBuilder& builder)
+  : name_(builder.name_),
+    min_threads_(builder.min_threads_),
+    max_threads_(builder.max_threads_),
+    max_queue_size_(builder.max_queue_size_),
+    idle_timeout_(builder.idle_timeout_),
+    pool_status_(Status::Uninitialized("The pool was not initialized.")),
+    idle_cond_(&lock_),
+    no_threads_cond_(&lock_),
+    num_threads_(0),
+    num_threads_pending_start_(0),
+    active_threads_(0),
+    total_queued_tasks_(0),
+    tokenless_(NewToken(ExecutionMode::CONCURRENT)),
+    metrics_(builder.metrics_) {
+  string prefix = !builder.trace_metric_prefix_.empty() ?
+      builder.trace_metric_prefix_ : builder.name_;
+
+  queue_time_trace_metric_name_ = TraceMetrics::InternName(
+      prefix + ".queue_time_us");
+  run_wall_time_trace_metric_name_ = TraceMetrics::InternName(
+      prefix + ".run_wall_time_us");
+  run_cpu_time_trace_metric_name_ = TraceMetrics::InternName(
+      prefix + ".run_cpu_time_us");
+}
+
+ThreadPool::~ThreadPool() {
+  // There should only be one live token: the one used in tokenless submission.
+  CHECK_EQ(1, tokens_.size()) << Substitute(
+      "Threadpool $0 destroyed with $1 allocated tokens",
+      name_, tokens_.size());
+  Shutdown();
+}
+
+Status ThreadPool::Init() {
+  if (!pool_status_.IsUninitialized()) {
+    return Status::NotSupported("The thread pool is already initialized");
+  }
+  pool_status_ = Status::OK();
+  num_threads_pending_start_ = min_threads_;
+  for (int i = 0; i < min_threads_; i++) {
+    Status status = CreateThread();
+    if (!status.ok()) {
+      Shutdown();
+      return status;
+    }
+  }
+  return Status::OK();
+}
+
+void ThreadPool::Shutdown() {
+  MutexLock unique_lock(lock_);
+  CheckNotPoolThreadUnlocked();
+
+  // Note: this is the same error seen at submission if the pool is at
+  // capacity, so clients can't tell them apart. This isn't really a practical
+  // concern though because shutting down a pool typically requires clients to
+  // be quiesced first, so there's no danger of a client getting confused.
+  pool_status_ = Status::ServiceUnavailable("The pool has been shut down.");
+
+  // Clear the various queues under the lock, but defer the releasing
+  // of the tasks outside the lock, in case there are concurrent threads
+  // wanting to access the ThreadPool. The task's destructors may acquire
+  // locks, etc, so this also prevents lock inversions.
+  queue_.clear();
+  std::deque<std::deque<Task>> to_release;
+  for (auto* t : tokens_) {
+    if (!t->entries_.empty()) {
+      to_release.emplace_back(std::move(t->entries_));
+    }
+    switch (t->state()) {
+      case ThreadPoolToken::State::IDLE:
+        // The token is idle; we can quiesce it immediately.
+        t->Transition(ThreadPoolToken::State::QUIESCED);
+        break;
+      case ThreadPoolToken::State::RUNNING:
+        // The token has tasks associated with it. If they're merely queued
+        // (i.e. there are no active threads), the tasks will have been removed
+        // above and we can quiesce immediately. Otherwise, we need to wait for
+        // the threads to finish.
+        t->Transition(t->active_threads_ > 0 ?
+            ThreadPoolToken::State::QUIESCING :
+            ThreadPoolToken::State::QUIESCED);
+        break;
+      default:
+        break;
+    }
+  }
+
+  // The queues are empty. Wake any sleeping worker threads and wait for all
+  // of them to exit. Some worker threads will exit immediately upon waking,
+  // while others will exit after they finish executing an outstanding task.
+  total_queued_tasks_ = 0;
+  while (!idle_threads_.empty()) {
+    idle_threads_.front().not_empty.Signal();
+    idle_threads_.pop_front();
+  }
+  while (num_threads_ + num_threads_pending_start_ > 0) {
+    no_threads_cond_.Wait();
+  }
+
+  // All the threads have exited. Check the state of each token.
+  for (auto* t : tokens_) {
+    DCHECK(t->state() == ThreadPoolToken::State::IDLE ||
+           t->state() == ThreadPoolToken::State::QUIESCED);
+  }
+
+  // Finally release the queued tasks, outside the lock.
+  unique_lock.Unlock();
+  for (auto& token : to_release) {
+    for (auto& t : token) {
+      if (t.trace) {
+        t.trace->Release();
+      }
+    }
+  }
+}
+
+unique_ptr<ThreadPoolToken> ThreadPool::NewToken(ExecutionMode mode) {
+  return NewTokenWithMetrics(mode, {});
+}
+
+unique_ptr<ThreadPoolToken> ThreadPool::NewTokenWithMetrics(
+    ExecutionMode mode, ThreadPoolMetrics metrics) {
+  MutexLock guard(lock_);
+  unique_ptr<ThreadPoolToken> t(new ThreadPoolToken(this,
+                                                    mode,
+                                                    std::move(metrics)));
+  InsertOrDie(&tokens_, t.get());
+  return t;
+}
+
+void ThreadPool::ReleaseToken(ThreadPoolToken* t) {
+  MutexLock guard(lock_);
+  CHECK(!t->IsActive()) << Substitute("Token with state $0 may not be released",
+                                      ThreadPoolToken::StateToString(t->state()));
+  CHECK_EQ(1, tokens_.erase(t));
+}
+
+Status ThreadPool::SubmitClosure(Closure c) {
+  return Submit(std::make_shared<ClosureRunnable>(std::move(c)));
+}
+
+Status ThreadPool::SubmitFunc(boost::function<void()> f) {
+  return Submit(std::make_shared<FunctionRunnable>(std::move(f)));
+}
+
+Status ThreadPool::Submit(shared_ptr<Runnable> r) {
+  return DoSubmit(std::move(r), tokenless_.get());
+}
+
+Status ThreadPool::DoSubmit(shared_ptr<Runnable> r, ThreadPoolToken* token) {
+  DCHECK(token);
+  MonoTime submit_time = MonoTime::Now();
+
+  MutexLock guard(lock_);
+  if (PREDICT_FALSE(!pool_status_.ok())) {
+    return pool_status_;
+  }
+
+  if (PREDICT_FALSE(!token->MaySubmitNewTasks())) {
+    return Status::ServiceUnavailable("Thread pool token was shut down");
+  }
+
+  // Size limit check.
+  int64_t capacity_remaining = static_cast<int64_t>(max_threads_) - active_threads_ +
+                               static_cast<int64_t>(max_queue_size_) - total_queued_tasks_;
+  if (capacity_remaining < 1) {
+    return Status::ServiceUnavailable(
+        Substitute("Thread pool is at capacity ($0/$1 tasks running, $2/$3 tasks queued)",
+                   num_threads_ + num_threads_pending_start_, max_threads_,
+                   total_queued_tasks_, max_queue_size_));
+  }
+
+  // Should we create another thread?
+
+  // We assume that each current inactive thread will grab one item from the
+  // queue.  If it seems like we'll need another thread, we create one.
+  //
+  // Rather than creating the thread here, while holding the lock, we defer
+  // it to down below. This is because thread creation can be rather slow
+  // (hundreds of milliseconds in some cases) and we'd like to allow the
+  // existing threads to continue to process tasks while we do so.
+  //
+  // In theory, a currently active thread could finish immediately after this
+  // calculation but before our new worker starts running. This would mean we
+  // created a thread we didn't really need. However, this race is unavoidable
+  // and harmless.
+  //
+  // Of course, we never create more than max_threads_ threads no matter what.
+  int threads_from_this_submit =
+      token->IsActive() && token->mode() == ExecutionMode::SERIAL ? 0 : 1;
+  int inactive_threads = num_threads_ + num_threads_pending_start_ - active_threads_;
+  int additional_threads = static_cast<int>(queue_.size())
+                         + threads_from_this_submit
+                         - inactive_threads;
+  bool need_a_thread = false;
+  if (additional_threads > 0 && num_threads_ + num_threads_pending_start_ < max_threads_) {
+    need_a_thread = true;
+    num_threads_pending_start_++;
+  }
+
+  Task task;
+  task.runnable = std::move(r);
+  task.trace = Trace::CurrentTrace();
+  // Need to AddRef, since the thread which submitted the task may go away,
+  // and we don't want the trace to be destructed while waiting in the queue.
+  if (task.trace) {
+    task.trace->AddRef();
+  }
+  task.submit_time = submit_time;
+
+  // Add the task to the token's queue.
+  ThreadPoolToken::State state = token->state();
+  DCHECK(state == ThreadPoolToken::State::IDLE ||
+         state == ThreadPoolToken::State::RUNNING);
+  token->entries_.emplace_back(std::move(task));
+  if (state == ThreadPoolToken::State::IDLE ||
+      token->mode() == ExecutionMode::CONCURRENT) {
+    queue_.emplace_back(token);
+    if (state == ThreadPoolToken::State::IDLE) {
+      token->Transition(ThreadPoolToken::State::RUNNING);
+    }
+  }
+  int length_at_submit = total_queued_tasks_++;
+
+  // Wake up an idle thread for this task. Choosing the thread at the front of
+  // the list ensures LIFO semantics as idling threads are also added to the front.
+  //
+  // If there are no idle threads, the new task remains on the queue and is
+  // processed by an active thread (or a thread we're about to create) at some
+  // point in the future.
+  if (!idle_threads_.empty()) {
+    idle_threads_.front().not_empty.Signal();
+    idle_threads_.pop_front();
+  }
+  guard.Unlock();
+
+  if (metrics_.queue_length_histogram) {
+    metrics_.queue_length_histogram->Increment(length_at_submit);
+  }
+  if (token->metrics_.queue_length_histogram) {
+    token->metrics_.queue_length_histogram->Increment(length_at_submit);
+  }
+
+  if (need_a_thread) {
+    Status status = CreateThread();
+    if (!status.ok()) {
+      guard.Lock();
+      num_threads_pending_start_--;
+      if (num_threads_ + num_threads_pending_start_ == 0) {
+        // If we have no threads, we can't do any work.
+        return status;
+      }
+      // If we failed to create a thread, but there are still some other
+      // worker threads, log a warning message and continue.
+      LOG(ERROR) << "Thread pool failed to create thread: "
+                 << status.ToString();
+    }
+  }
+
+
+  return Status::OK();
+}
+
+void ThreadPool::Wait() {
+  MutexLock unique_lock(lock_);
+  CheckNotPoolThreadUnlocked();
+  while (total_queued_tasks_ > 0 || active_threads_ > 0) {
+    idle_cond_.Wait();
+  }
+}
+
+bool ThreadPool::WaitUntil(const MonoTime& until) {
+  MutexLock unique_lock(lock_);
+  CheckNotPoolThreadUnlocked();
+  while (total_queued_tasks_ > 0 || active_threads_ > 0) {
+    if (!idle_cond_.WaitUntil(until)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+bool ThreadPool::WaitFor(const MonoDelta& delta) {
+  return WaitUntil(MonoTime::Now() + delta);
+}
+
+void ThreadPool::DispatchThread() {
+  MutexLock unique_lock(lock_);
+  InsertOrDie(&threads_, Thread::current_thread());
+  DCHECK_GT(num_threads_pending_start_, 0);
+  num_threads_++;
+  num_threads_pending_start_--;
+  // If we are one of the first 'min_threads_' to start, we must be
+  // a "permanent" thread.
+  bool permanent = num_threads_ <= min_threads_;
+
+  // Owned by this worker thread and added/removed from idle_threads_ as needed.
+  IdleThread me(&lock_);
+
+  while (true) {
+    // Note: Status::Aborted() is used to indicate normal shutdown.
+    if (!pool_status_.ok()) {
+      VLOG(2) << "DispatchThread exiting: " << pool_status_.ToString();
+      break;
+    }
+
+    if (queue_.empty()) {
+      // There's no work to do, let's go idle.
+      //
+      // Note: if FIFO behavior is desired, it's as simple as changing this to push_back().
+      idle_threads_.push_front(me);
+      SCOPED_CLEANUP({
+        // For some wake ups (i.e. Shutdown or DoSubmit) this thread is
+        // guaranteed to be unlinked after being awakened. In others (i.e.
+        // spurious wake-up or Wait timeout), it'll still be linked.
+        if (me.is_linked()) {
+          idle_threads_.erase(idle_threads_.iterator_to(me));
+        }
+      });
+      if (permanent) {
+        me.not_empty.Wait();
+      } else {
+        if (!me.not_empty.WaitFor(idle_timeout_)) {
+          // After much investigation, it appears that pthread condition variables have
+          // a weird behavior in which they can return ETIMEDOUT from timed_wait even if
+          // another thread did in fact signal. Apparently after a timeout there is some
+          // brief period during which another thread may actually grab the internal mutex
+          // protecting the state, signal, and release again before we get the mutex. So,
+          // we'll recheck the empty queue case regardless.
+          if (queue_.empty()) {
+            VLOG(3) << "Releasing worker thread from pool " << name_ << " after "
+                    << idle_timeout_.ToMilliseconds() << "ms of idle time.";
+            break;
+          }
+        }
+      }
+      continue;
+    }
+
+    // Get the next token and task to execute.
+    ThreadPoolToken* token = queue_.front();
+    queue_.pop_front();
+    DCHECK_EQ(ThreadPoolToken::State::RUNNING, token->state());
+    DCHECK(!token->entries_.empty());
+    Task task = std::move(token->entries_.front());
+    token->entries_.pop_front();
+    token->active_threads_++;
+    --total_queued_tasks_;
+    ++active_threads_;
+
+    unique_lock.Unlock();
+
+    // Release the reference which was held by the queued item.
+    ADOPT_TRACE(task.trace);
+    if (task.trace) {
+      task.trace->Release();
+    }
+
+    // Update metrics
+    MonoTime now(MonoTime::Now());
+    int64_t queue_time_us = (now - task.submit_time).ToMicroseconds();
+    TRACE_COUNTER_INCREMENT(queue_time_trace_metric_name_, queue_time_us);
+    if (metrics_.queue_time_us_histogram) {
+      metrics_.queue_time_us_histogram->Increment(queue_time_us);
+    }
+    if (token->metrics_.queue_time_us_histogram) {
+      token->metrics_.queue_time_us_histogram->Increment(queue_time_us);
+    }
+
+    // Execute the task
+    {
+      MicrosecondsInt64 start_wall_us = GetMonoTimeMicros();
+      MicrosecondsInt64 start_cpu_us = GetThreadCpuTimeMicros();
+
+      task.runnable->Run();
+
+      int64_t wall_us = GetMonoTimeMicros() - start_wall_us;
+      int64_t cpu_us = GetThreadCpuTimeMicros() - start_cpu_us;
+
+      if (metrics_.run_time_us_histogram) {
+        metrics_.run_time_us_histogram->Increment(wall_us);
+      }
+      if (token->metrics_.run_time_us_histogram) {
+        token->metrics_.run_time_us_histogram->Increment(wall_us);
+      }
+      TRACE_COUNTER_INCREMENT(run_wall_time_trace_metric_name_, wall_us);
+      TRACE_COUNTER_INCREMENT(run_cpu_time_trace_metric_name_, cpu_us);
+    }
+    // Destruct the task while we do not hold the lock.
+    //
+    // The task's destructor may be expensive if it has a lot of bound
+    // objects, and we don't want to block submission of the threadpool.
+    // In the worst case, the destructor might even try to do something
+    // with this threadpool, and produce a deadlock.
+    task.runnable.reset();
+    unique_lock.Lock();
+
+    // Possible states:
+    // 1. The token was shut down while we ran its task. Transition to QUIESCED.
+    // 2. The token has no more queued tasks. Transition back to IDLE.
+    // 3. The token has more tasks. Requeue it and transition back to RUNNABLE.
+    ThreadPoolToken::State state = token->state();
+    DCHECK(state == ThreadPoolToken::State::RUNNING ||
+           state == ThreadPoolToken::State::QUIESCING);
+    if (--token->active_threads_ == 0) {
+      if (state == ThreadPoolToken::State::QUIESCING) {
+        DCHECK(token->entries_.empty());
+        token->Transition(ThreadPoolToken::State::QUIESCED);
+      } else if (token->entries_.empty()) {
+        token->Transition(ThreadPoolToken::State::IDLE);
+      } else if (token->mode() == ExecutionMode::SERIAL) {
+        queue_.emplace_back(token);
+      }
+    }
+    if (--active_threads_ == 0) {
+      idle_cond_.Broadcast();
+    }
+  }
+
+  // It's important that we hold the lock between exiting the loop and dropping
+  // num_threads_. Otherwise it's possible someone else could come along here
+  // and add a new task just as the last running thread is about to exit.
+  CHECK(unique_lock.OwnsLock());
+
+  CHECK_EQ(threads_.erase(Thread::current_thread()), 1);
+  num_threads_--;
+  if (num_threads_ + num_threads_pending_start_ == 0) {
+    no_threads_cond_.Broadcast();
+
+    // Sanity check: if we're the last thread exiting, the queue ought to be
+    // empty. Otherwise it will never get processed.
+    CHECK(queue_.empty());
+    DCHECK_EQ(0, total_queued_tasks_);
+  }
+}
+
+Status ThreadPool::CreateThread() {
+  return kudu::Thread::Create("thread pool", strings::Substitute("$0 [worker]", name_),
+                              &ThreadPool::DispatchThread, this, nullptr);
+}
+
+void ThreadPool::CheckNotPoolThreadUnlocked() {
+  Thread* current = Thread::current_thread();
+  if (ContainsKey(threads_, current)) {
+    LOG(FATAL) << Substitute("Thread belonging to thread pool '$0' with "
+        "name '$1' called pool function that would result in deadlock",
+        name_, current->name());
+  }
+}
+
+std::ostream& operator<<(std::ostream& o, ThreadPoolToken::State s) {
+  return o << ThreadPoolToken::StateToString(s);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadpool.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadpool.h b/be/src/kudu/util/threadpool.h
new file mode 100644
index 0000000..1557486
--- /dev/null
+++ b/be/src/kudu/util/threadpool.h
@@ -0,0 +1,505 @@
+// 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.
+#ifndef KUDU_UTIL_THREAD_POOL_H
+#define KUDU_UTIL_THREAD_POOL_H
+
+#include <deque>
+#include <iosfwd>
+#include <memory>
+#include <string>
+#include <unordered_set>
+
+#include <boost/intrusive/list.hpp>
+#include <boost/intrusive/list_hook.hpp>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+
+namespace boost {
+template <typename Signature>
+class function;
+} // namespace boost
+
+namespace kudu {
+
+class Thread;
+class ThreadPool;
+class ThreadPoolToken;
+class Trace;
+
+class Runnable {
+ public:
+  virtual void Run() = 0;
+  virtual ~Runnable() {}
+};
+
+// Interesting thread pool metrics. Can be applied to the entire pool (see
+// ThreadPoolBuilder) or to individual tokens.
+struct ThreadPoolMetrics {
+  // Measures the queue length seen by tasks when they enter the queue.
+  scoped_refptr<Histogram> queue_length_histogram;
+
+  // Measures the amount of time that tasks spend waiting in a queue.
+  scoped_refptr<Histogram> queue_time_us_histogram;
+
+  // Measures the amount of time that tasks spend running.
+  scoped_refptr<Histogram> run_time_us_histogram;
+};
+
+// ThreadPool takes a lot of arguments. We provide sane defaults with a builder.
+//
+// name: Used for debugging output and default names of the worker threads.
+//    Since thread names are limited to 16 characters on Linux, it's good to
+//    choose a short name here.
+//    Required.
+//
+// trace_metric_prefix: used to prefix the names of TraceMetric counters.
+//    When a task on a thread pool has an associated trace, the thread pool
+//    implementation will increment TraceMetric counters to indicate the
+//    amount of time spent waiting in the queue as well as the amount of wall
+//    and CPU time spent executing. By default, these counters are prefixed
+//    with the name of the thread pool. For example, if the pool is named
+//    'apply', then counters such as 'apply.queue_time_us' will be
+//    incremented.
+//
+//    The TraceMetrics implementation relies on the number of distinct counter
+//    names being small. Thus, if the thread pool name itself is dynamically
+//    generated, the default behavior described above would result in an
+//    unbounded number of distinct counter names. The 'trace_metric_prefix'
+//    setting can be used to override the prefix used in generating the trace
+//    metric names.
+//
+//    For example, the Raft thread pools are named "<tablet id>-raft" which
+//    has unbounded cardinality (a server may have thousands of different
+//    tablet IDs over its lifetime). In that case, setting the prefix to
+//    "raft" will avoid any issues.
+//
+// min_threads: Minimum number of threads we'll have at any time.
+//    Default: 0.
+//
+// max_threads: Maximum number of threads we'll have at any time.
+//    Default: Number of CPUs detected on the system.
+//
+// max_queue_size: Maximum number of items to enqueue before returning a
+//    Status::ServiceUnavailable message from Submit().
+//    Default: INT_MAX.
+//
+// idle_timeout: How long we'll keep around an idle thread before timing it out.
+//    We always keep at least min_threads.
+//    Default: 500 milliseconds.
+//
+// metrics: Histograms, counters, etc. to update on various threadpool events.
+//    Default: not set.
+//
+class ThreadPoolBuilder {
+ public:
+  explicit ThreadPoolBuilder(std::string name);
+
+  // Note: We violate the style guide by returning mutable references here
+  // in order to provide traditional Builder pattern conveniences.
+  ThreadPoolBuilder& set_trace_metric_prefix(const std::string& prefix);
+  ThreadPoolBuilder& set_min_threads(int min_threads);
+  ThreadPoolBuilder& set_max_threads(int max_threads);
+  ThreadPoolBuilder& set_max_queue_size(int max_queue_size);
+  ThreadPoolBuilder& set_idle_timeout(const MonoDelta& idle_timeout);
+  ThreadPoolBuilder& set_metrics(ThreadPoolMetrics metrics);
+
+  // Instantiate a new ThreadPool with the existing builder arguments.
+  Status Build(gscoped_ptr<ThreadPool>* pool) const;
+
+ private:
+  friend class ThreadPool;
+  const std::string name_;
+  std::string trace_metric_prefix_;
+  int min_threads_;
+  int max_threads_;
+  int max_queue_size_;
+  MonoDelta idle_timeout_;
+  ThreadPoolMetrics metrics_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadPoolBuilder);
+};
+
+// Thread pool with a variable number of threads.
+//
+// Tasks submitted directly to the thread pool enter a FIFO queue and are
+// dispatched to a worker thread when one becomes free. Tasks may also be
+// submitted via ThreadPoolTokens. The token Wait() and Shutdown() functions
+// can then be used to block on logical groups of tasks.
+//
+// A token operates in one of two ExecutionModes, determined at token
+// construction time:
+// 1. SERIAL: submitted tasks are run one at a time.
+// 2. CONCURRENT: submitted tasks may be run in parallel. This isn't unlike
+//    tasks submitted without a token, but the logical grouping that tokens
+//    impart can be useful when a pool is shared by many contexts (e.g. to
+//    safely shut down one context, to derive context-specific metrics, etc.).
+//
+// Tasks submitted without a token or via ExecutionMode::CONCURRENT tokens are
+// processed in FIFO order. On the other hand, ExecutionMode::SERIAL tokens are
+// processed in a round-robin fashion, one task at a time. This prevents them
+// from starving one another. However, tokenless (and CONCURRENT token-based)
+// tasks can starve SERIAL token-based tasks.
+//
+// Usage Example:
+//    static void Func(int n) { ... }
+//    class Task : public Runnable { ... }
+//
+//    gscoped_ptr<ThreadPool> thread_pool;
+//    CHECK_OK(
+//        ThreadPoolBuilder("my_pool")
+//            .set_min_threads(0)
+//            .set_max_threads(5)
+//            .set_max_queue_size(10)
+//            .set_idle_timeout(MonoDelta::FromMilliseconds(2000))
+//            .Build(&thread_pool));
+//    thread_pool->Submit(shared_ptr<Runnable>(new Task()));
+//    thread_pool->SubmitFunc(boost::bind(&Func, 10));
+class ThreadPool {
+ public:
+  ~ThreadPool();
+
+  // Wait for the running tasks to complete and then shutdown the threads.
+  // All the other pending tasks in the queue will be removed.
+  // NOTE: That the user may implement an external abort logic for the
+  //       runnables, that must be called before Shutdown(), if the system
+  //       should know about the non-execution of these tasks, or the runnable
+  //       require an explicit "abort" notification to exit from the run loop.
+  void Shutdown();
+
+  // Submits a function using the kudu Closure system.
+  Status SubmitClosure(Closure c) WARN_UNUSED_RESULT;
+
+  // Submits a function bound using boost::bind(&FuncName, args...).
+  Status SubmitFunc(boost::function<void()> f) WARN_UNUSED_RESULT;
+
+  // Submits a Runnable class.
+  Status Submit(std::shared_ptr<Runnable> r) WARN_UNUSED_RESULT;
+
+  // Waits until all the tasks are completed.
+  void Wait();
+
+  // Waits for the pool to reach the idle state, or until 'until' time is reached.
+  // Returns true if the pool reached the idle state, false otherwise.
+  bool WaitUntil(const MonoTime& until);
+
+  // Waits for the pool to reach the idle state, or until 'delta' time elapses.
+  // Returns true if the pool reached the idle state, false otherwise.
+  bool WaitFor(const MonoDelta& delta);
+
+  // Allocates a new token for use in token-based task submission. All tokens
+  // must be destroyed before their ThreadPool is destroyed.
+  //
+  // There is no limit on the number of tokens that may be allocated.
+  enum class ExecutionMode {
+    // Tasks submitted via this token will be executed serially.
+    SERIAL,
+
+    // Tasks submitted via this token may be executed concurrently.
+    CONCURRENT,
+  };
+  std::unique_ptr<ThreadPoolToken> NewToken(ExecutionMode mode);
+
+  // Like NewToken(), but lets the caller provide metrics for the token. These
+  // metrics are incremented/decremented in addition to the configured
+  // pool-wide metrics (if any).
+  std::unique_ptr<ThreadPoolToken> NewTokenWithMetrics(ExecutionMode mode,
+                                                       ThreadPoolMetrics metrics);
+
+  // Return the number of threads currently running (or in the process of starting up)
+  // for this thread pool.
+  int num_threads() const {
+    MutexLock l(lock_);
+    return num_threads_ + num_threads_pending_start_;
+  }
+
+ private:
+  FRIEND_TEST(ThreadPoolTest, TestThreadPoolWithNoMinimum);
+  FRIEND_TEST(ThreadPoolTest, TestVariableSizeThreadPool);
+
+  friend class ThreadPoolBuilder;
+  friend class ThreadPoolToken;
+
+  // Client-provided task to be executed by this pool.
+  struct Task {
+    std::shared_ptr<Runnable> runnable;
+    Trace* trace;
+
+    // Time at which the entry was submitted to the pool.
+    MonoTime submit_time;
+  };
+
+  // Creates a new thread pool using a builder.
+  explicit ThreadPool(const ThreadPoolBuilder& builder);
+
+  // Initializes the thread pool by starting the minimum number of threads.
+  Status Init();
+
+  // Dispatcher responsible for dequeueing and executing the tasks
+  void DispatchThread();
+
+  // Create new thread.
+  //
+  // REQUIRES: caller has incremented 'num_threads_pending_start_' ahead of this call.
+  // NOTE: For performance reasons, lock_ should not be held.
+  Status CreateThread();
+
+  // Aborts if the current thread is a member of this thread pool.
+  void CheckNotPoolThreadUnlocked();
+
+  // Submits a task to be run via token.
+  Status DoSubmit(std::shared_ptr<Runnable> r, ThreadPoolToken* token);
+
+  // Releases token 't' and invalidates it.
+  void ReleaseToken(ThreadPoolToken* t);
+
+  const std::string name_;
+  const int min_threads_;
+  const int max_threads_;
+  const int max_queue_size_;
+  const MonoDelta idle_timeout_;
+
+  // Overall status of the pool. Set to an error when the pool is shut down.
+  //
+  // Protected by 'lock_'.
+  Status pool_status_;
+
+  // Synchronizes many of the members of the pool and all of its
+  // condition variables.
+  mutable Mutex lock_;
+
+  // Condition variable for "pool is idling". Waiters wake up when
+  // active_threads_ reaches zero.
+  ConditionVariable idle_cond_;
+
+  // Condition variable for "pool has no threads". Waiters wake up when
+  // num_threads_ and num_pending_threads_ are both 0.
+  ConditionVariable no_threads_cond_;
+
+  // Number of threads currently running.
+  //
+  // Protected by lock_.
+  int num_threads_;
+
+  // Number of threads which are in the process of starting.
+  // When these threads start, they will decrement this counter and
+  // accordingly increment 'num_threads_'.
+  //
+  // Protected by lock_.
+  int num_threads_pending_start_;
+
+  // Number of threads currently running and executing client tasks.
+  //
+  // Protected by lock_.
+  int active_threads_;
+
+  // Total number of client tasks queued, either directly (queue_) or
+  // indirectly (tokens_).
+  //
+  // Protected by lock_.
+  int total_queued_tasks_;
+
+  // All allocated tokens.
+  //
+  // Protected by lock_.
+  std::unordered_set<ThreadPoolToken*> tokens_;
+
+  // FIFO of tokens from which tasks should be executed. Does not own the
+  // tokens; they are owned by clients and are removed from the FIFO on shutdown.
+  //
+  // Protected by lock_.
+  std::deque<ThreadPoolToken*> queue_;
+
+  // Pointers to all running threads. Raw pointers are safe because a Thread
+  // may only go out of scope after being removed from threads_.
+  //
+  // Protected by lock_.
+  std::unordered_set<Thread*> threads_;
+
+  // List of all threads currently waiting for work.
+  //
+  // A thread is added to the front of the list when it goes idle and is
+  // removed from the front and signaled when new work arrives. This produces a
+  // LIFO usage pattern that is more efficient than idling on a single
+  // ConditionVariable (which yields FIFO semantics).
+  //
+  // Protected by lock_.
+  struct IdleThread : public boost::intrusive::list_base_hook<> {
+    explicit IdleThread(Mutex* m)
+        : not_empty(m) {}
+
+    // Condition variable for "queue is not empty". Waiters wake up when a new
+    // task is queued.
+    ConditionVariable not_empty;
+
+    DISALLOW_COPY_AND_ASSIGN(IdleThread);
+  };
+  boost::intrusive::list<IdleThread> idle_threads_; // NOLINT(build/include_what_you_use)
+
+  // ExecutionMode::CONCURRENT token used by the pool for tokenless submission.
+  std::unique_ptr<ThreadPoolToken> tokenless_;
+
+  // Metrics for the entire thread pool.
+  const ThreadPoolMetrics metrics_;
+
+  const char* queue_time_trace_metric_name_;
+  const char* run_wall_time_trace_metric_name_;
+  const char* run_cpu_time_trace_metric_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadPool);
+};
+
+// Entry point for token-based task submission and blocking for a particular
+// thread pool. Tokens can only be created via ThreadPool::NewToken().
+//
+// All functions are thread-safe. Mutable members are protected via the
+// ThreadPool's lock.
+class ThreadPoolToken {
+ public:
+  // Destroys the token.
+  //
+  // May be called on a token with outstanding tasks, as Shutdown() will be
+  // called first to take care of them.
+  ~ThreadPoolToken();
+
+  // Submits a function using the kudu Closure system.
+  Status SubmitClosure(Closure c) WARN_UNUSED_RESULT;
+
+  // Submits a function bound using boost::bind(&FuncName, args...).
+  Status SubmitFunc(boost::function<void()> f) WARN_UNUSED_RESULT;
+
+  // Submits a Runnable class.
+  Status Submit(std::shared_ptr<Runnable> r) WARN_UNUSED_RESULT;
+
+  // Marks the token as unusable for future submissions. Any queued tasks not
+  // yet running are destroyed. If tasks are in flight, Shutdown() will wait
+  // on their completion before returning.
+  void Shutdown();
+
+  // Waits until all the tasks submitted via this token are completed.
+  void Wait();
+
+  // Waits for all submissions using this token are complete, or until 'until'
+  // time is reached.
+  //
+  // Returns true if all submissions are complete, false otherwise.
+  bool WaitUntil(const MonoTime& until);
+
+  // Waits for all submissions using this token are complete, or until 'delta'
+  // time elapses.
+  //
+  // Returns true if all submissions are complete, false otherwise.
+  bool WaitFor(const MonoDelta& delta);
+
+ private:
+  // All possible token states. Legal state transitions:
+  //   IDLE      -> RUNNING: task is submitted via token
+  //   IDLE      -> QUIESCED: token or pool is shut down
+  //   RUNNING   -> IDLE: worker thread finishes executing a task and
+  //                      there are no more tasks queued to the token
+  //   RUNNING   -> QUIESCING: token or pool is shut down while worker thread
+  //                           is executing a task
+  //   RUNNING   -> QUIESCED: token or pool is shut down
+  //   QUIESCING -> QUIESCED:  worker thread finishes executing a task
+  //                           belonging to a shut down token or pool
+  enum class State {
+    // Token has no queued tasks.
+    IDLE,
+
+    // A worker thread is running one of the token's previously queued tasks.
+    RUNNING,
+
+    // No new tasks may be submitted to the token. A worker thread is still
+    // running a previously queued task.
+    QUIESCING,
+
+    // No new tasks may be submitted to the token. There are no active tasks
+    // either. At this state, the token may only be destroyed.
+    QUIESCED,
+  };
+
+  // Writes a textual representation of the token state in 's' to 'o'.
+  friend std::ostream& operator<<(std::ostream& o, ThreadPoolToken::State s);
+
+  friend class ThreadPool;
+
+  // Returns a textual representation of 's' suitable for debugging.
+  static const char* StateToString(State s);
+
+  // Constructs a new token.
+  //
+  // The token may not outlive its thread pool ('pool').
+  ThreadPoolToken(ThreadPool* pool,
+                  ThreadPool::ExecutionMode mode,
+                  ThreadPoolMetrics metrics);
+
+  // Changes this token's state to 'new_state' taking actions as needed.
+  void Transition(State new_state);
+
+  // Returns true if this token has a task queued and ready to run, or if a
+  // task belonging to this token is already running.
+  bool IsActive() const {
+    return state_ == State::RUNNING ||
+           state_ == State::QUIESCING;
+  }
+
+  // Returns true if new tasks may be submitted to this token.
+  bool MaySubmitNewTasks() const {
+    return state_ != State::QUIESCING &&
+           state_ != State::QUIESCED;
+  }
+
+  State state() const { return state_; }
+  ThreadPool::ExecutionMode mode() const { return mode_; }
+
+  // Token's configured execution mode.
+  const ThreadPool::ExecutionMode mode_;
+
+  // Metrics for just this token.
+  const ThreadPoolMetrics metrics_;
+
+  // Pointer to the token's thread pool.
+  ThreadPool* pool_;
+
+  // Token state machine.
+  State state_;
+
+  // Queued client tasks.
+  std::deque<ThreadPool::Task> entries_;
+
+  // Condition variable for "token is idle". Waiters wake up when the token
+  // transitions to IDLE or QUIESCED.
+  ConditionVariable not_running_cond_;
+
+  // Number of worker threads currently executing tasks belonging to this
+  // token.
+  int active_threads_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadPoolToken);
+};
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/throttler-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/throttler-test.cc b/be/src/kudu/util/throttler-test.cc
new file mode 100644
index 0000000..ff97eb5
--- /dev/null
+++ b/be/src/kudu/util/throttler-test.cc
@@ -0,0 +1,76 @@
+// 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 "kudu/util/throttler.h"
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/monotime.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+class ThrottlerTest : public KuduTest {
+};
+
+TEST_F(ThrottlerTest, TestOpThrottle) {
+  // Check operation rate throttling
+  MonoTime now = MonoTime::Now();
+  Throttler t0(now, 1000, 1000*1000, 1);
+  // Fill up bucket
+  now += MonoDelta::FromMilliseconds(2000);
+  // Check throttle behavior for 1 second.
+  for (int p = 0; p < 10; p++) {
+    for (int i = 0; i < 100; i++) {
+      ASSERT_TRUE(t0.Take(now, 1, 1));
+    }
+    ASSERT_FALSE(t0.Take(now, 1, 1));
+    now += MonoDelta::FromMilliseconds(100);
+  }
+}
+
+TEST_F(ThrottlerTest, TestIOThrottle) {
+  // Check operation rate throttling
+  MonoTime now = MonoTime::Now();
+  Throttler t0(now, 50000, 1000*1000, 1);
+  // Fill up bucket
+  now += MonoDelta::FromMilliseconds(2000);
+  // Check throttle behavior for 1 second.
+  for (int p = 0; p < 10; p++) {
+    for (int i = 0; i < 100; i++) {
+      ASSERT_TRUE(t0.Take(now, 1, 1000));
+    }
+    ASSERT_FALSE(t0.Take(now, 1, 1000));
+    now += MonoDelta::FromMilliseconds(100);
+  }
+}
+
+TEST_F(ThrottlerTest, TestBurst) {
+  // Check IO rate throttling
+  MonoTime now = MonoTime::Now();
+  Throttler t0(now, 2000, 1000*1000, 5);
+  // Fill up bucket
+  now += MonoDelta::FromMilliseconds(2000);
+  for (int i = 0; i < 100; i++) {
+    now += MonoDelta::FromMilliseconds(1);
+    ASSERT_TRUE(t0.Take(now, 1, 5000));
+  }
+  ASSERT_TRUE(t0.Take(now, 1, 100000));
+  ASSERT_FALSE(t0.Take(now, 1, 1));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/throttler.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/throttler.cc b/be/src/kudu/util/throttler.cc
new file mode 100644
index 0000000..69e0f99
--- /dev/null
+++ b/be/src/kudu/util/throttler.cc
@@ -0,0 +1,67 @@
+// 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 "kudu/util/throttler.h"
+
+#include <algorithm>
+#include <mutex>
+
+namespace kudu {
+
+Throttler::Throttler(MonoTime now, uint64_t op_rate, uint64_t byte_rate, double burst_factor) :
+    next_refill_(now) {
+  op_refill_ = op_rate / (MonoTime::kMicrosecondsPerSecond / kRefillPeriodMicros);
+  op_token_ = 0;
+  op_token_max_ = static_cast<uint64_t>(op_refill_ * burst_factor);
+  byte_refill_ = byte_rate / (MonoTime::kMicrosecondsPerSecond / kRefillPeriodMicros);
+  byte_token_ = 0;
+  byte_token_max_ = static_cast<uint64_t>(byte_refill_ * burst_factor);
+}
+
+bool Throttler::Take(MonoTime now, uint64_t op, uint64_t byte) {
+  if (op_refill_ == 0 && byte_refill_ == 0) {
+    return true;
+  }
+  std::lock_guard<simple_spinlock> lock(lock_);
+  Refill(now);
+  if ((op_refill_ == 0 || op <= op_token_) &&
+      (byte_refill_ == 0 || byte <= byte_token_)) {
+    if (op_refill_ > 0) {
+      op_token_ -= op;
+    }
+    if (byte_refill_ > 0) {
+      byte_token_ -= byte;
+    }
+    return true;
+  }
+  return false;
+}
+
+void Throttler::Refill(MonoTime now) {
+  int64_t d = (now - next_refill_).ToMicroseconds();
+  if (d < 0) {
+    return;
+  }
+  uint64_t num_period = d / kRefillPeriodMicros + 1;
+  next_refill_ += MonoDelta::FromMicroseconds(num_period * kRefillPeriodMicros);
+  op_token_ += num_period * op_refill_;
+  op_token_ = std::min(op_token_, op_token_max_);
+  byte_token_ += num_period * byte_refill_;
+  byte_token_ = std::min(byte_token_, byte_token_max_);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/throttler.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/throttler.h b/be/src/kudu/util/throttler.h
new file mode 100644
index 0000000..5594091
--- /dev/null
+++ b/be/src/kudu/util/throttler.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.
+#ifndef KUDU_UTIL_THROTTLER_H
+#define KUDU_UTIL_THROTTLER_H
+
+#include <cstdint>
+
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+
+// A throttler to throttle both operation/s and IO byte/s.
+class Throttler {
+ public:
+  // Refill period is 100ms.
+  enum {
+    kRefillPeriodMicros = 100000
+  };
+
+  // Construct a throttler with max operation per second, max IO bytes per second
+  // and burst factor (burst_rate = rate * burst), burst rate means maximum
+  // throughput within one refill period.
+  // Set op_per_sec to 0 to disable operation throttling.
+  // Set byte_per_sec to 0 to disable IO bytes throttling.
+  Throttler(MonoTime now, uint64_t op_per_sec, uint64_t byte_per_sec, double burst_factor);
+
+  // Throttle an "operation group" by taking 'op' operation tokens and 'byte' byte tokens.
+  // Return true if there are enough tokens, and operation is allowed.
+  // Return false if there are not enough tokens, and operation is throttled.
+  bool Take(MonoTime now, uint64_t op, uint64_t byte);
+
+ private:
+  void Refill(MonoTime now);
+
+  MonoTime next_refill_;
+  uint64_t op_refill_;
+  uint64_t op_token_;
+  uint64_t op_token_max_;
+  uint64_t byte_refill_;
+  uint64_t byte_token_;
+  uint64_t byte_token_max_;
+  simple_spinlock lock_;
+};
+
+} // namespace kudu
+
+#endif


[50/51] [abbrv] impala git commit: IMPALA-7251: Fix QueryMaintenance calls in Aggregators

Posted by ta...@apache.org.
IMPALA-7251: Fix QueryMaintenance calls in Aggregators

A recent change, IMPALA-110 (part 2), refactored
PartitionedAggregationNode into several classes, including a new type
'Aggregator'. During this refactor, code that makes local allocations
while evaluating exprs was moved from the ExecNode (now
AggregationNode/StreamingAggregationNode) into the Aggregators, but
code related to cleaning these allocations up (ie QueryMaintenance())
was not, resulting in some queries using an excessive amount of
memory.

This patch removes all calls to QueryMaintenance() from the exec nodes
and moves them into the Aggregators.

Testing:
- Added new test cases with a mem limit that fails if the expr
  allocations aren't released in a timely manner.
- Passed a full exhaustive run.

Change-Id: I4dac2bb0a15cdd7315ee15608bae409c125c82f5
Reviewed-on: http://gerrit.cloudera.org:8080/10871
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/e2aafae2
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/e2aafae2
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/e2aafae2

Branch: refs/heads/master
Commit: e2aafae204a695ddc6a9745f879f6164af19027a
Parents: 0459721
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Thu Jul 5 18:28:04 2018 +0000
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Fri Jul 13 03:04:35 2018 +0000

----------------------------------------------------------------------
 be/src/exec/aggregation-node.cc                 |  5 ++--
 be/src/exec/aggregator.cc                       |  5 ++++
 be/src/exec/aggregator.h                        |  6 ++++
 be/src/exec/grouping-aggregator.cc              |  7 ++---
 be/src/exec/grouping-aggregator.h               |  6 ----
 be/src/exec/non-grouping-aggregator.cc          |  2 ++
 be/src/exec/streaming-aggregation-node.cc       |  5 ++--
 .../spilling-regression-exhaustive.test         | 30 ++++++++++++++++++++
 8 files changed, 51 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node.cc b/be/src/exec/aggregation-node.cc
index d25284d..2c95590 100644
--- a/be/src/exec/aggregation-node.cc
+++ b/be/src/exec/aggregation-node.cc
@@ -78,7 +78,6 @@ Status AggregationNode::Open(RuntimeState* state) {
   bool eos = false;
   do {
     RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(QueryMaintenance(state));
     RETURN_IF_ERROR(children_[0]->GetNext(state, &batch, &eos));
     RETURN_IF_ERROR(aggregator_->AddBatch(state, &batch));
     batch.Reset();
@@ -98,7 +97,6 @@ Status AggregationNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool*
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
   RETURN_IF_CANCELLED(state);
-  RETURN_IF_ERROR(QueryMaintenance(state));
 
   if (ReachedLimit()) {
     *eos = true;
@@ -118,6 +116,9 @@ Status AggregationNode::Reset(RuntimeState* state) {
 
 void AggregationNode::Close(RuntimeState* state) {
   if (is_closed()) return;
+  // All expr mem allocations should happen in the Aggregator.
+  DCHECK(expr_results_pool() == nullptr
+      || expr_results_pool()->total_allocated_bytes() == 0);
   aggregator_->Close(state);
   ExecNode::Close(state);
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/aggregator.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregator.cc b/be/src/exec/aggregator.cc
index 70178cc..87abc52 100644
--- a/be/src/exec/aggregator.cc
+++ b/be/src/exec/aggregator.cc
@@ -196,6 +196,11 @@ Tuple* Aggregator::GetOutputTuple(
   return dst;
 }
 
+Status Aggregator::QueryMaintenance(RuntimeState* state) {
+  expr_results_pool_->Clear();
+  return state->CheckQueryState();
+}
+
 // IR Generation for updating a single aggregation slot. Signature is:
 // void UpdateSlot(AggFnEvaluator* agg_expr_eval, AggTuple* agg_tuple, char** row)
 //

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/aggregator.h
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregator.h b/be/src/exec/aggregator.h
index ab13d45..f415606 100644
--- a/be/src/exec/aggregator.h
+++ b/be/src/exec/aggregator.h
@@ -188,6 +188,12 @@ class Aggregator {
   Tuple* GetOutputTuple(
       const std::vector<AggFnEvaluator*>& agg_fn_evals, Tuple* tuple, MemPool* pool);
 
+  /// Clears 'expr_results_pool_' and returns the result of state->CheckQueryState().
+  /// Aggregators should call this periodically, e.g. once per input row batch. This
+  /// should not be called outside the main execution thread.
+  /// TODO: IMPALA-2399: replace QueryMaintenance() - see JIRA for more details.
+  Status QueryMaintenance(RuntimeState* state) WARN_UNUSED_RESULT;
+
   /// Codegen for updating aggregate expressions agg_fns_[agg_fn_idx]
   /// and returns the IR function in 'fn'. Returns non-OK status if codegen
   /// is unsuccessful.

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/grouping-aggregator.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/grouping-aggregator.cc b/be/src/exec/grouping-aggregator.cc
index 0eb4a3f..4f3e5cf 100644
--- a/be/src/exec/grouping-aggregator.cc
+++ b/be/src/exec/grouping-aggregator.cc
@@ -228,6 +228,7 @@ Status GroupingAggregator::Open(RuntimeState* state) {
 }
 
 Status GroupingAggregator::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  RETURN_IF_ERROR(QueryMaintenance(state));
   if (!partition_eos_) {
     RETURN_IF_ERROR(GetRowsFromPartition(state, row_batch));
   }
@@ -405,6 +406,7 @@ void GroupingAggregator::Close(RuntimeState* state) {
 
 Status GroupingAggregator::AddBatch(RuntimeState* state, RowBatch* batch) {
   SCOPED_TIMER(build_timer_);
+  RETURN_IF_ERROR(QueryMaintenance(state));
   num_input_rows_ += batch->num_rows();
 
   TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
@@ -952,11 +954,6 @@ int64_t GroupingAggregator::MinReservation() const {
       + resource_profile_.max_row_buffer_size * 2;
 }
 
-Status GroupingAggregator::QueryMaintenance(RuntimeState* state) {
-  expr_results_pool_->Clear();
-  return state->CheckQueryState();
-}
-
 BufferPool::ClientHandle* GroupingAggregator::buffer_pool_client() {
   return reservation_manager_.buffer_pool_client();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/grouping-aggregator.h
----------------------------------------------------------------------
diff --git a/be/src/exec/grouping-aggregator.h b/be/src/exec/grouping-aggregator.h
index 0d1b893..b766a1e 100644
--- a/be/src/exec/grouping-aggregator.h
+++ b/be/src/exec/grouping-aggregator.h
@@ -587,12 +587,6 @@ class GroupingAggregator : public Aggregator {
   void CleanupHashTbl(
       const std::vector<AggFnEvaluator*>& agg_fn_evals, HashTable::Iterator it);
 
-  /// Clears 'expr_results_pool_' and returns the result of state->CheckQueryState().
-  /// Aggregators should call this periodically, e.g. once per input row batch. This
-  /// should not be called outside the main execution thread.
-  /// TODO: IMPALA-2399: replace QueryMaintenance() - see JIRA for more details.
-  Status QueryMaintenance(RuntimeState* state) WARN_UNUSED_RESULT;
-
   /// Codegen the non-streaming add row batch loop. The loop has already been compiled to
   /// IR and loaded into the codegen object. UpdateAggTuple has also been codegen'd to IR.
   /// This function will modify the loop subsituting the statically compiled functions

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/non-grouping-aggregator.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/non-grouping-aggregator.cc b/be/src/exec/non-grouping-aggregator.cc
index 585c264..1ee4e46 100644
--- a/be/src/exec/non-grouping-aggregator.cc
+++ b/be/src/exec/non-grouping-aggregator.cc
@@ -72,6 +72,7 @@ Status NonGroupingAggregator::Open(RuntimeState* state) {
 
 Status NonGroupingAggregator::GetNext(
     RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  RETURN_IF_ERROR(QueryMaintenance(state));
   // There was no grouping, so evaluate the conjuncts and return the single result row.
   // We allow calling GetNext() after eos, so don't return this row again.
   if (!singleton_output_tuple_returned_) GetSingletonOutput(row_batch);
@@ -116,6 +117,7 @@ void NonGroupingAggregator::Close(RuntimeState* state) {
 
 Status NonGroupingAggregator::AddBatch(RuntimeState* state, RowBatch* batch) {
   SCOPED_TIMER(build_timer_);
+  RETURN_IF_ERROR(QueryMaintenance(state));
 
   if (add_batch_impl_fn_ != nullptr) {
     RETURN_IF_ERROR(add_batch_impl_fn_(this, batch));

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/be/src/exec/streaming-aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/streaming-aggregation-node.cc b/be/src/exec/streaming-aggregation-node.cc
index 4ad7820..c1e9184 100644
--- a/be/src/exec/streaming-aggregation-node.cc
+++ b/be/src/exec/streaming-aggregation-node.cc
@@ -80,7 +80,6 @@ Status StreamingAggregationNode::GetNext(
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
   RETURN_IF_CANCELLED(state);
-  RETURN_IF_ERROR(QueryMaintenance(state));
 
   if (ReachedLimit()) {
     *eos = true;
@@ -113,7 +112,6 @@ Status StreamingAggregationNode::GetRowsStreaming(
   do {
     DCHECK_EQ(out_batch->num_rows(), 0);
     RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(QueryMaintenance(state));
 
     RETURN_IF_ERROR(child(0)->GetNext(state, child_batch_.get(), &child_eos_));
 
@@ -137,6 +135,9 @@ Status StreamingAggregationNode::Reset(RuntimeState* state) {
 
 void StreamingAggregationNode::Close(RuntimeState* state) {
   if (is_closed()) return;
+  // All expr mem allocations should happen in the Aggregator.
+  DCHECK(expr_results_pool() == nullptr
+      || expr_results_pool()->total_allocated_bytes() == 0);
   aggregator_->Close(state);
   child_batch_.reset();
   ExecNode::Close(state);

http://git-wip-us.apache.org/repos/asf/impala/blob/e2aafae2/testdata/workloads/functional-query/queries/QueryTest/spilling-regression-exhaustive.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-regression-exhaustive.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-regression-exhaustive.test
index 03a1010..1f3bf24 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-regression-exhaustive.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-regression-exhaustive.test
@@ -245,6 +245,36 @@ BIGINT
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
+# Same as above, except disable streaming preaggs to ensure that AggregationNode is also
+# releasing local memory allocations as appropriate.
+set mem_limit=800m;
+set num_scanner_threads=1;
+set disable_streaming_preaggregations=true;
+select count(distinct concat(cast(l_comment as char(120)), cast(l_comment as char(120)),
+                             cast(l_comment as char(120)), cast(l_comment as char(120)),
+                             cast(l_comment as char(120)), cast(l_comment as char(120))))
+from lineitem
+---- RESULTS
+4502054
+---- TYPES
+BIGINT
+---- RUNTIME_PROFILE
+# Verify that the agg spilled.
+row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# Same as above, except use a non-grouping aggregate function to ensure that
+# NonGroupingAggregator is also releasing local memory allocations as appropriate.
+set mem_limit=50m;
+set num_scanner_threads=1;
+select min(regexp_replace(l_comment, ".", "x"))
+from lineitem
+---- RESULTS
+'xxxxxxxxxx'
+---- TYPES
+STRING
+====
+---- QUERY
 # IMPALA-3304: test that avg() can spill with a query mem limit.
 # This test only covers that use FIXED_UDA_INTERMEDIATE, not functions that allocate
 # strings for intermediate values. mem_limit is tuned to reproduce the issue on a 3-node


[34/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/cert.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/cert.h b/be/src/kudu/security/cert.h
new file mode 100644
index 0000000..4629883
--- /dev/null
+++ b/be/src/kudu/security/cert.h
@@ -0,0 +1,119 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <openssl/asn1.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/security/openssl_util.h"
+
+typedef struct X509_name_st X509_NAME;
+
+namespace boost {
+template <class T>
+class optional;
+}
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+class PrivateKey;
+class PublicKey;
+
+// Convert an X509_NAME object to a human-readable string.
+std::string X509NameToString(X509_NAME* name);
+
+// Return the OpenSSL NID for the custom X509 extension where we store
+// our Kerberos principal in IPKI certs.
+int GetKuduKerberosPrincipalOidNid();
+
+// A wrapper class around the STACK_OF(X509) object. This can either hold one certificate or
+// a chain of certificates.
+// TODO(unknown): Currently, there isn't a mechanism to add to the chain. Implement it when needed.
+class Cert : public RawDataWrapper<STACK_OF(X509)> {
+ public:
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+  Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
+
+  int chain_len() const { return sk_X509_num(data_.get()); }
+
+  std::string SubjectName() const;
+  std::string IssuerName() const;
+
+  // Return DNS names from the SAN extension field of the end-user cert.
+  std::vector<std::string> Hostnames() const;
+
+  // Return the 'userId' extension of the end-user cert, if set.
+  boost::optional<std::string> UserId() const;
+
+  // Return the Kerberos principal encoded in the end-user certificate, if set.
+  boost::optional<std::string> KuduKerberosPrincipal() const;
+
+  // Check whether the specified private key matches the end-user certificate.
+  // Return Status::OK() if key match the end-user certificate.
+  Status CheckKeyMatch(const PrivateKey& key) const WARN_UNUSED_RESULT;
+
+  // Returns the 'tls-server-end-point' channel bindings for the end-user certificate as
+  // specified in RFC 5929.
+  Status GetServerEndPointChannelBindings(std::string* channel_bindings) const WARN_UNUSED_RESULT;
+
+  // Adopts the provided STACK_OF(X509), and increments the reference count of the X509 cert
+  // contained within it. Currently, only one certificate should be contained in the stack.
+  void AdoptAndAddRefRawData(RawDataType* data);
+
+  // Adopts the provided X509 certificate, and replaces the current underlying STACK_OF(X509).
+  void AdoptX509(X509* cert);
+
+  // Adopts the provided X509 certificate, increments its reference count and replaces the current
+  // underlying STACK_OF(X509).
+  void AdoptAndAddRefX509(X509* cert);
+
+  // Returns the end-user certificate's public key.
+  Status GetPublicKey(PublicKey* key) const WARN_UNUSED_RESULT;
+
+  // Get the first certificate in the chain, otherwise known as the 'end-user' certificate.
+  X509* GetTopOfChainX509() const;
+};
+
+class CertSignRequest : public RawDataWrapper<X509_REQ> {
+ public:
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+  Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
+
+  // Returns a clone of the CSR.
+  //
+  // Whether this clone is deep or shallow (i.e. only a reference count is
+  // incremented) depends on the version of OpenSSL. Either way, the right
+  // thing happens when the clone goes out of scope.
+  CertSignRequest Clone() const;
+
+  // Returns the CSR's public key.
+  Status GetPublicKey(PublicKey* key) const WARN_UNUSED_RESULT;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/crypto-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/crypto-test.cc b/be/src/kudu/security/crypto-test.cc
new file mode 100644
index 0000000..c1e32df
--- /dev/null
+++ b/be/src/kudu/security/crypto-test.cc
@@ -0,0 +1,257 @@
+// 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 <cstring>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/url-coding.h"
+
+using std::pair;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+// Test for various crypto-related functionality in the security library.
+class CryptoTest : public KuduTest {
+ public:
+  CryptoTest() :
+      pem_dir_(GetTestPath("pem")),
+      private_key_file_(JoinPathSegments(pem_dir_, "private_key.pem")),
+      public_key_file_(JoinPathSegments(pem_dir_, "public_key.pem")),
+      corrupted_private_key_file_(JoinPathSegments(pem_dir_,
+          "corrupted.private_key.pem")),
+      corrupted_public_key_file_(JoinPathSegments(pem_dir_,
+          "corrupted.public_key.pem")) {
+  }
+
+  void SetUp() override {
+    ASSERT_OK(env_->CreateDir(pem_dir_));
+    ASSERT_OK(WriteStringToFile(env_, kCaPrivateKey, private_key_file_));
+    ASSERT_OK(WriteStringToFile(env_, kCaPublicKey, public_key_file_));
+    ASSERT_OK(WriteStringToFile(env_,
+        string(kCaPrivateKey, strlen(kCaPrivateKey) / 2),
+        corrupted_private_key_file_));
+    ASSERT_OK(WriteStringToFile(env_,
+        string(kCaPublicKey, strlen(kCaPublicKey) / 2),
+        corrupted_public_key_file_));
+  }
+
+ protected:
+  template<typename Key>
+  void CheckToAndFromString(const Key& key_ref, DataFormat format) {
+    SCOPED_TRACE(Substitute("ToAndFromString for $0 format",
+                            DataFormatToString(format)));
+    string key_ref_str;
+    ASSERT_OK(key_ref.ToString(&key_ref_str, format));
+    Key key;
+    ASSERT_OK(key.FromString(key_ref_str, format));
+    string key_str;
+    ASSERT_OK(key.ToString(&key_str, format));
+    ASSERT_EQ(key_ref_str, key_str);
+  }
+
+  const string pem_dir_;
+
+  const string private_key_file_;
+  const string public_key_file_;
+  const string corrupted_private_key_file_;
+  const string corrupted_public_key_file_;
+};
+
+// Check input/output of RSA private keys in PEM format.
+TEST_F(CryptoTest, RsaPrivateKeyInputOutputPEM) {
+  PrivateKey key;
+  ASSERT_OK(key.FromFile(private_key_file_, DataFormat::PEM));
+  string key_str;
+  ASSERT_OK(key.ToString(&key_str, DataFormat::PEM));
+  RemoveExtraWhitespace(&key_str);
+
+  string ref_key_str(kCaPrivateKey);
+  RemoveExtraWhitespace(&ref_key_str);
+  EXPECT_EQ(ref_key_str, key_str);
+}
+
+// Check input of corrupted RSA private keys in PEM format.
+TEST_F(CryptoTest, CorruptedRsaPrivateKeyInputPEM) {
+  static const string kFiles[] = {
+      corrupted_private_key_file_,
+      public_key_file_,
+      corrupted_public_key_file_,
+      "/bin/sh"
+  };
+  for (const auto& file : kFiles) {
+    PrivateKey key;
+    const Status s = key.FromFile(file, DataFormat::PEM);
+    EXPECT_TRUE(s.IsRuntimeError()) << s.ToString();
+  }
+}
+
+// Check input/output of RSA public keys in PEM format.
+TEST_F(CryptoTest, RsaPublicKeyInputOutputPEM) {
+  PublicKey key;
+  ASSERT_OK(key.FromFile(public_key_file_, DataFormat::PEM));
+  string key_str;
+  ASSERT_OK(key.ToString(&key_str, DataFormat::PEM));
+  RemoveExtraWhitespace(&key_str);
+
+  string ref_key_str(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_key_str);
+  EXPECT_EQ(ref_key_str, key_str);
+}
+
+// Check input of corrupted RSA public keys in PEM format.
+TEST_F(CryptoTest, CorruptedRsaPublicKeyInputPEM) {
+  static const string kFiles[] = {
+      corrupted_public_key_file_,
+      private_key_file_,
+      corrupted_private_key_file_,
+      "/bin/sh"
+  };
+  for (const auto& file : kFiles) {
+    PublicKey key;
+    const Status s = key.FromFile(file, DataFormat::PEM);
+    EXPECT_TRUE(s.IsRuntimeError()) << s.ToString();
+  }
+}
+
+// Check extraction of the public part from RSA private keys par.
+TEST_F(CryptoTest, RsaExtractPublicPartFromPrivateKey) {
+  // Load the reference RSA private key.
+  PrivateKey private_key;
+  ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
+
+  PublicKey public_key;
+  ASSERT_OK(private_key.GetPublicKey(&public_key));
+  string str_public_key;
+  ASSERT_OK(public_key.ToString(&str_public_key, DataFormat::PEM));
+  RemoveExtraWhitespace(&str_public_key);
+
+  string ref_str_public_key(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_str_public_key);
+  EXPECT_EQ(ref_str_public_key, str_public_key);
+}
+
+class CryptoKeySerDesTest :
+    public CryptoTest,
+    public ::testing::WithParamInterface<DataFormat> {
+};
+
+// Check the transformation chains for RSA public/private keys:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_P(CryptoKeySerDesTest, ToAndFromString) {
+  const auto format = GetParam();
+
+  // Generate private RSA key.
+  PrivateKey private_key;
+  ASSERT_OK(GeneratePrivateKey(2048, &private_key));
+  NO_FATALS(CheckToAndFromString(private_key, format));
+
+  // Extract public part of the key.
+  PublicKey public_key;
+  ASSERT_OK(private_key.GetPublicKey(&public_key));
+  NO_FATALS(CheckToAndFromString(public_key, format));
+}
+
+INSTANTIATE_TEST_CASE_P(
+    DataFormats, CryptoKeySerDesTest,
+    ::testing::Values(DataFormat::DER, DataFormat::PEM));
+
+// Check making crypto signatures against the reference data.
+TEST_F(CryptoTest, MakeVerifySignatureRef) {
+  static const vector<pair<string, string>> kRefElements = {
+    { kDataTiny,    kSignatureTinySHA512 },
+    { kDataShort,   kSignatureShortSHA512 },
+    { kDataLong,    kSignatureLongSHA512 },
+  };
+
+  // Load the reference RSA private key.
+  PrivateKey private_key;
+  ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
+
+  // Load the reference RSA public key.
+  PublicKey public_key;
+  ASSERT_OK(public_key.FromString(kCaPublicKey, DataFormat::PEM));
+
+  for (const auto& e : kRefElements) {
+    string sig;
+    ASSERT_OK(private_key.MakeSignature(DigestType::SHA512, e.first, &sig));
+
+    // Ad-hoc verification: check the produced signature matches the reference.
+    string sig_base64;
+    Base64Encode(sig, &sig_base64);
+    EXPECT_EQ(e.second, sig_base64);
+
+    // Verify the signature cryptographically.
+    EXPECT_OK(public_key.VerifySignature(DigestType::SHA512, e.first, sig));
+  }
+}
+
+TEST_F(CryptoTest, VerifySignatureWrongData) {
+  static const vector<string> kRefSignatures = {
+    kSignatureTinySHA512,
+    kSignatureShortSHA512,
+    kSignatureLongSHA512,
+  };
+
+  // Load the reference RSA public key.
+  PublicKey key;
+  ASSERT_OK(key.FromString(kCaPublicKey, DataFormat::PEM));
+
+  for (const auto& e : kRefSignatures) {
+    string signature;
+    ASSERT_TRUE(Base64Decode(e, &signature));
+    Status s = key.VerifySignature(DigestType::SHA512,
+        "non-expected-data", signature);
+    EXPECT_TRUE(s.IsCorruption()) << s.ToString();
+  }
+}
+
+TEST_F(CryptoTest, TestGenerateNonce) {
+  string nonce;
+  ASSERT_OK(GenerateNonce(&nonce));
+
+  // Do some basic validation on the returned nonce.
+  ASSERT_EQ(kNonceSize, nonce.size());
+  ASSERT_NE(string(kNonceSize, '\0'), nonce);
+
+  // Nonces should be unique, by definition.
+  string another_nonce;
+  ASSERT_OK(GenerateNonce(&another_nonce));
+  ASSERT_NE(nonce, another_nonce);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/crypto.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/crypto.cc b/be/src/kudu/security/crypto.cc
new file mode 100644
index 0000000..234d193
--- /dev/null
+++ b/be/src/kudu/security/crypto.cc
@@ -0,0 +1,276 @@
+// 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 "kudu/security/crypto.h"
+
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/bio.h>
+#include <openssl/bn.h>
+#include <openssl/err.h>
+#include <openssl/evp.h>
+#include <openssl/opensslv.h>
+#include <openssl/ossl_typ.h>
+#include <openssl/pem.h>
+#include <openssl/rand.h>
+#include <openssl/rsa.h>
+#include <openssl/x509.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/openssl_util_bio.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+const size_t kNonceSize = 16;
+
+namespace {
+
+// Writing the private key from an EVP_PKEY has a different
+// signature than the rest of the write functions, so we
+// have to provide this wrapper.
+int PemWritePrivateKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return PEM_write_bio_RSAPrivateKey(
+      bio, rsa.get(), nullptr, nullptr, 0, nullptr, nullptr);
+}
+
+int PemWritePublicKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return PEM_write_bio_RSA_PUBKEY(bio, rsa.get());
+}
+
+int DerWritePublicKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return i2d_RSA_PUBKEY_bio(bio, rsa.get());
+}
+
+} // anonymous namespace
+
+template<> struct SslTypeTraits<BIGNUM> {
+  static constexpr auto kFreeFunc = &BN_free;
+};
+struct RsaPrivateKeyTraits : public SslTypeTraits<EVP_PKEY> {
+  static constexpr auto kReadPemFunc = &PEM_read_bio_PrivateKey;
+  static constexpr auto kReadDerFunc = &d2i_PrivateKey_bio;
+  static constexpr auto kWritePemFunc = &PemWritePrivateKey;
+  static constexpr auto kWriteDerFunc = &i2d_PrivateKey_bio;
+};
+struct RsaPublicKeyTraits : public SslTypeTraits<EVP_PKEY> {
+  static constexpr auto kReadPemFunc = &PEM_read_bio_PUBKEY;
+  static constexpr auto kReadDerFunc = &d2i_PUBKEY_bio;
+  static constexpr auto kWritePemFunc = &PemWritePublicKey;
+  static constexpr auto kWriteDerFunc = &DerWritePublicKey;
+};
+template<> struct SslTypeTraits<RSA> {
+  static constexpr auto kFreeFunc = &RSA_free;
+};
+template<> struct SslTypeTraits<EVP_MD_CTX> {
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  static constexpr auto kFreeFunc = &EVP_MD_CTX_destroy;
+#else
+  static constexpr auto kFreeFunc = &EVP_MD_CTX_free;
+#endif
+};
+
+namespace {
+
+const EVP_MD* GetMessageDigest(DigestType digest_type) {
+  switch (digest_type) {
+    case DigestType::SHA256: return EVP_sha256();
+    case DigestType::SHA512: return EVP_sha512();
+  }
+  LOG(FATAL) << "unknown digest type";
+}
+
+} // anonymous namespace
+
+
+Status PublicKey::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString<RawDataType, RsaPublicKeyTraits>(
+      data, format, &data_);
+}
+
+Status PublicKey::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString<RawDataType, RsaPublicKeyTraits>(
+      data, format, data_.get());
+}
+
+Status PublicKey::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile<RawDataType, RsaPublicKeyTraits>(
+      fpath, format, &data_);
+}
+
+Status PublicKey::FromBIO(BIO* bio, DataFormat format) {
+  return ::kudu::security::FromBIO<RawDataType, RsaPublicKeyTraits>(
+      bio, format, &data_);
+}
+
+// Modeled after code in $OPENSSL_ROOT/apps/dgst.c
+Status PublicKey::VerifySignature(DigestType digest,
+                                  const std::string& data,
+                                  const std::string& signature) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  const EVP_MD* md = GetMessageDigest(digest);
+  auto md_ctx = ssl_make_unique(EVP_MD_CTX_create());
+
+  OPENSSL_RET_NOT_OK(EVP_DigestVerifyInit(md_ctx.get(), nullptr, md, nullptr, GetRawData()),
+                     "error initializing verification digest");
+  OPENSSL_RET_NOT_OK(EVP_DigestVerifyUpdate(md_ctx.get(), data.data(), data.size()),
+                     "error verifying data signature");
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+  unsigned char* sig_data = reinterpret_cast<unsigned char*>(
+      const_cast<char*>(signature.data()));
+#else
+  const unsigned char* sig_data = reinterpret_cast<const unsigned char*>(
+      signature.data());
+#endif
+  // The success is indicated by return code 1. All other values means
+  // either wrong signature or error while performing signature verification.
+  const int rc = EVP_DigestVerifyFinal(md_ctx.get(), sig_data, signature.size());
+  if (rc < 0 || rc > 1) {
+    return Status::RuntimeError(
+        Substitute("error verifying data signature: $0", GetOpenSSLErrors()));
+  }
+  if (rc == 0) {
+    // No sense stringifying the internal OpenSSL error, since a bad verification
+    // is self-explanatory.
+    ERR_clear_error();
+    return Status::Corruption("data signature verification failed");
+  }
+
+  return Status::OK();
+}
+
+Status PublicKey::Equals(const PublicKey& other, bool* equals) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  int cmp = EVP_PKEY_cmp(data_.get(), other.data_.get());
+  switch (cmp) {
+    case -2:
+      return Status::NotSupported("failed to compare public keys");
+    case -1: // Key types are different; treat this as not equal
+    case 0:  // Keys are not equal
+      *equals = false;
+      return Status::OK();
+    case 1:
+      *equals = true;
+      return Status::OK();
+    default:
+      return Status::RuntimeError("unexpected public key comparison result", std::to_string(cmp));
+  }
+}
+
+Status PrivateKey::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString<RawDataType, RsaPrivateKeyTraits>(
+      data, format, &data_);
+}
+
+Status PrivateKey::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString<RawDataType, RsaPrivateKeyTraits>(
+      data, format, data_.get());
+}
+
+Status PrivateKey::FromFile(const std::string& fpath, DataFormat format,
+                            const PasswordCallback& password_cb) {
+  return ::kudu::security::FromFile<RawDataType, RsaPrivateKeyTraits>(
+      fpath, format, &data_, password_cb);
+}
+
+// The code is modeled after $OPENSSL_ROOT/apps/rsa.c code: there is
+// corresponding functionality to read public part from RSA private/public
+// keypair.
+Status PrivateKey::GetPublicKey(PublicKey* public_key) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(public_key);
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(CHECK_NOTNULL(data_.get())));
+  if (PREDICT_FALSE(!rsa)) {
+    return Status::RuntimeError(GetOpenSSLErrors());
+  }
+  auto tmp = ssl_make_unique(BIO_new(BIO_s_mem()));
+  CHECK(tmp);
+  // Export public key in DER format into the temporary buffer.
+  OPENSSL_RET_NOT_OK(i2d_RSA_PUBKEY_bio(tmp.get(), rsa.get()),
+      "error extracting public RSA key");
+  // Read the public key into the result placeholder.
+  RETURN_NOT_OK(public_key->FromBIO(tmp.get(), DataFormat::DER));
+
+  return Status::OK();
+}
+
+// Modeled after code in $OPENSSL_ROOT/apps/dgst.c
+Status PrivateKey::MakeSignature(DigestType digest,
+                                 const std::string& data,
+                                 std::string* signature) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(signature);
+  const EVP_MD* md = GetMessageDigest(digest);
+  auto md_ctx = ssl_make_unique(EVP_MD_CTX_create());
+
+  OPENSSL_RET_NOT_OK(EVP_DigestSignInit(md_ctx.get(), nullptr, md, nullptr, GetRawData()),
+                     "error initializing signing digest");
+  OPENSSL_RET_NOT_OK(EVP_DigestSignUpdate(md_ctx.get(), data.data(), data.size()),
+                     "error signing data");
+  size_t sig_len = EVP_PKEY_size(GetRawData());
+  static const size_t kSigBufSize = 4 * 1024;
+  CHECK(sig_len <= kSigBufSize);
+  unsigned char buf[kSigBufSize];
+  OPENSSL_RET_NOT_OK(EVP_DigestSignFinal(md_ctx.get(), buf, &sig_len),
+                     "error finalizing data signature");
+  *signature = string(reinterpret_cast<char*>(buf), sig_len);
+
+  return Status::OK();
+}
+
+Status GeneratePrivateKey(int num_bits, PrivateKey* ret) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ret);
+  InitializeOpenSSL();
+  auto key = ssl_make_unique(EVP_PKEY_new());
+  {
+    auto bn = ssl_make_unique(BN_new());
+    OPENSSL_CHECK_OK(BN_set_word(bn.get(), RSA_F4));
+    auto rsa = ssl_make_unique(RSA_new());
+    OPENSSL_RET_NOT_OK(
+        RSA_generate_key_ex(rsa.get(), num_bits, bn.get(), nullptr),
+        "error generating RSA key");
+    OPENSSL_RET_NOT_OK(
+        EVP_PKEY_set1_RSA(key.get(), rsa.get()), "error assigning RSA key");
+  }
+  ret->AdoptRawData(key.release());
+
+  return Status::OK();
+}
+
+Status GenerateNonce(string* s) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK_NOTNULL(s);
+  unsigned char buf[kNonceSize];
+  OPENSSL_RET_NOT_OK(RAND_bytes(buf, sizeof(buf)), "failed to generate nonce");
+  s->assign(reinterpret_cast<char*>(buf), kNonceSize);
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/crypto.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/crypto.h b/be/src/kudu/security/crypto.h
new file mode 100644
index 0000000..145c405
--- /dev/null
+++ b/be/src/kudu/security/crypto.h
@@ -0,0 +1,103 @@
+// 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.
+
+#pragma once
+
+#include <cstddef>
+#include <string>
+
+#include <openssl/bio.h>
+#include <openssl/rsa.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/security/openssl_util.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct rsa_st RSA;
+typedef struct bio_st BIO;
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+extern const size_t kNonceSize;
+
+// Supported message digests for data signing and signature verification.
+enum class DigestType {
+  SHA256,
+  SHA512,
+};
+
+// A class with generic public key interface, but actually it represents
+// an RSA key.
+class PublicKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PublicKey() {}
+
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+  Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
+
+  Status FromBIO(BIO* bio, DataFormat format) WARN_UNUSED_RESULT;
+
+  // Using the key, verify data signature using the specified message
+  // digest algorithm for signature verification.
+  // The input signature should be in in raw format (i.e. no base64 encoding).
+  Status VerifySignature(DigestType digest,
+                         const std::string& data,
+                         const std::string& signature) const WARN_UNUSED_RESULT;
+
+  // Sets 'equals' to true if the other public key equals this.
+  Status Equals(const PublicKey& other, bool* equals) const WARN_UNUSED_RESULT;
+};
+
+// A class with generic private key interface, but actually it represents
+// an RSA private key. It's important to have PrivateKey and PublicKey
+// be different types to avoid accidental leakage of private keys.
+class PrivateKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PrivateKey() {}
+
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+
+  // If 'cb' is set, it will be called to obtain the password necessary to decrypt
+  // the private key file in 'fpath'.
+  Status FromFile(const std::string& fpath, DataFormat format,
+                  const PasswordCallback& password_cb = PasswordCallback()) WARN_UNUSED_RESULT;
+
+  // Output the public part of the keypair into the specified placeholder.
+  Status GetPublicKey(PublicKey* public_key) const WARN_UNUSED_RESULT;
+
+  // Using the key, generate data signature using the specified
+  // message digest algorithm. The result signature is in raw format
+  // (i.e. no base64 encoding).
+  Status MakeSignature(DigestType digest,
+                       const std::string& data,
+                       std::string* signature) const WARN_UNUSED_RESULT;
+};
+
+// Utility method to generate private keys.
+Status GeneratePrivateKey(int num_bits, PrivateKey* ret) WARN_UNUSED_RESULT;
+
+// Generates a nonce of size kNonceSize, and writes it to the provided string.
+Status GenerateNonce(std::string* s) WARN_UNUSED_RESULT;
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/init.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/init.cc b/be/src/kudu/security/init.cc
new file mode 100644
index 0000000..ee754eb
--- /dev/null
+++ b/be/src/kudu/security/init.cc
@@ -0,0 +1,465 @@
+// 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 "kudu/security/init.h"
+
+#include <algorithm>
+#include <cctype>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <ctime>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <random>
+#include <string>
+#include <type_traits>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <krb5/krb5.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+#ifndef __APPLE__
+static constexpr bool kDefaultSystemAuthToLocal = true;
+#else
+// macOS's Heimdal library has a no-op implementation of
+// krb5_aname_to_localname, so instead we just use the simple
+// implementation.
+static constexpr bool kDefaultSystemAuthToLocal = false;
+#endif
+DEFINE_bool(use_system_auth_to_local, kDefaultSystemAuthToLocal,
+            "When enabled, use the system krb5 library to map Kerberos principal "
+            "names to local (short) usernames. If not enabled, the first component "
+            "of the principal will be used as the short name. For example, "
+            "'kudu/foo.example.com@EXAMPLE' will map to 'kudu'.");
+TAG_FLAG(use_system_auth_to_local, advanced);
+
+
+using std::mt19937;
+using std::random_device;
+using std::string;
+using std::uniform_int_distribution;
+using std::uniform_real_distribution;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+class KinitContext;
+
+// Global context for usage of the Krb5 library.
+krb5_context g_krb5_ctx;
+
+// Global instance of the context used by the kinit/reacquire thread.
+KinitContext* g_kinit_ctx;
+
+// This lock is used to avoid a race while reacquiring the kerberos ticket.
+// The race can occur between the time we reinitialize the cache and the
+// time when we actually store the new credentials back in the cache.
+RWMutex* g_kerberos_reinit_lock;
+
+class KinitContext {
+ public:
+  KinitContext();
+
+  // Equivalent implementation of 'kinit -kt <keytab path> <principal>'.
+  //
+  // This logs in from the given keytab as the given principal, returning
+  // RuntimeError if any part of this process fails.
+  //
+  // If the log-in is successful, then the default ticket cache is overwritten
+  // with the credentials of the newly logged-in principal.
+  Status Kinit(const string& keytab_path, const string& principal);
+
+  // Acquires a new Ticket Granting Ticket (TGT).
+  //
+  // Renews the existing ticket if possible, or acquires a new Ticket Granting
+  // Ticket (TGT).
+  Status DoRenewal();
+
+  // Calculates the next sleep interval based on the 'ticket_end_timestamp_' and
+  // adds some jitter so that all the nodes do not hit the KDC at the same time.
+  //
+  // If 'num_retries' > 0, it calls GetBackedOffRenewInterval() to return a backed
+  // off interval.
+  int32_t GetNextRenewInterval(uint32_t num_retries);
+
+  // Returns a value based on 'time_remaining' that increases exponentially with
+  // 'num_retries', with a random jitter of +/- 0%-50% of that value.
+  int32_t GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries);
+
+  const string& principal_str() const { return principal_str_; }
+  const string& username_str() const { return username_str_; }
+
+ private:
+  krb5_principal principal_;
+  krb5_keytab keytab_;
+  krb5_ccache ccache_;
+  krb5_get_init_creds_opt* opts_;
+
+  // The stringified principal and username that we are logged in as.
+  string principal_str_, username_str_;
+
+  // This is the time that the current TGT in use expires.
+  int32_t ticket_end_timestamp_;
+};
+
+Status Krb5CallToStatus(krb5_context ctx, krb5_error_code code) {
+  if (code == 0) return Status::OK();
+
+  std::unique_ptr<const char, std::function<void(const char*)>> err_msg(
+      krb5_get_error_message(ctx, code),
+      std::bind(krb5_free_error_message, ctx, std::placeholders::_1));
+  return Status::RuntimeError(err_msg.get());
+}
+#define KRB5_RETURN_NOT_OK_PREPEND(call, prepend) \
+  RETURN_NOT_OK_PREPEND(Krb5CallToStatus(g_krb5_ctx, (call)), (prepend))
+
+
+void InitKrb5Ctx() {
+  static std::once_flag once;
+  std::call_once(once, [&]() {
+      CHECK_EQ(krb5_init_context(&g_krb5_ctx), 0);
+    });
+}
+
+KinitContext::KinitContext() {}
+
+// Port of the data_eq() implementation from krb5/k5-int.h
+inline int data_eq(krb5_data d1, krb5_data d2) {
+    return (d1.length == d2.length && !memcmp(d1.data, d2.data, d1.length));
+}
+
+// Port of the data_eq_string() implementation from krb5/k5-int.h
+inline int data_eq_string(krb5_data d, const char *s) {
+    return (d.length == strlen(s) && !memcmp(d.data, s, d.length));
+}
+
+Status Krb5UnparseName(krb5_principal princ, string* name) {
+  char* c_name;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_unparse_name(g_krb5_ctx, princ, &c_name),
+                             "krb5_unparse_name");
+  SCOPED_CLEANUP({
+      krb5_free_unparsed_name(g_krb5_ctx, c_name);
+    });
+  *name = c_name;
+  return Status::OK();
+}
+
+// Periodically calls DoRenewal().
+void RenewThread() {
+  uint32_t failure_retries = 0;
+  while (true) {
+    // This thread is run immediately after the first Kinit, so sleep first.
+    SleepFor(MonoDelta::FromSeconds(g_kinit_ctx->GetNextRenewInterval(failure_retries)));
+
+    Status s = g_kinit_ctx->DoRenewal();
+    WARN_NOT_OK(s, "Kerberos reacquire error: ");
+    if (!s.ok()) {
+      ++failure_retries;
+    } else {
+      failure_retries = 0;
+    }
+  }
+}
+
+int32_t KinitContext::GetNextRenewInterval(uint32_t num_retries) {
+  int32_t time_remaining = ticket_end_timestamp_ - time(nullptr);
+
+  // If the last ticket reacqusition was a failure, we back off our retry attempts exponentially.
+  if (num_retries > 0) return GetBackedOffRenewInterval(time_remaining, num_retries);
+
+  // If the time remaining between now and ticket expiry is:
+  // * > 10 minutes:   We attempt to reacquire the ticket between 5 seconds and 5 minutes before the
+  //                   ticket expires.
+  // * 5 - 10 minutes: We attempt to reacquire the ticket betwen 5 seconds and 1 minute before the
+  //                   ticket expires.
+  // * < 5 minutes:    Attempt to reacquire the ticket every 'time_remaining'.
+  // The jitter is added to make sure that every server doesn't flood the KDC at the same time.
+  random_device rd;
+  mt19937 generator(rd());
+  if (time_remaining > 600) {
+    uniform_int_distribution<> dist(5, 300);
+    return time_remaining - dist(generator);
+  } else if (time_remaining > 300) {
+    uniform_int_distribution<> dist(5, 60);
+    return time_remaining - dist(generator);
+  }
+  return time_remaining;
+}
+
+int32_t KinitContext::GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries) {
+  // The minimum sleep interval after a failure will be 60 seconds.
+  int32_t next_interval = std::max(time_remaining, 60);
+  int32_t base_time = std::min(next_interval * (1 << num_retries), INT32_MAX);
+  random_device rd;
+  mt19937 generator(rd());
+  uniform_real_distribution<> dist(0.5, 1.5);
+  return static_cast<int32_t>(base_time * dist(generator));
+}
+
+Status KinitContext::DoRenewal() {
+
+  krb5_cc_cursor cursor;
+  // Setup a cursor to iterate through the credential cache.
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_start_seq_get(g_krb5_ctx, ccache_, &cursor),
+                             "Failed to peek into ccache");
+  SCOPED_CLEANUP({
+      krb5_cc_end_seq_get(g_krb5_ctx, ccache_, &cursor); });
+
+  krb5_creds creds;
+  memset(&creds, 0, sizeof(krb5_creds));
+
+  krb5_error_code rc;
+  // Iterate through the credential cache.
+  while (!(rc = krb5_cc_next_cred(g_krb5_ctx, ccache_, &cursor, &creds))) {
+    SCOPED_CLEANUP({
+        krb5_free_cred_contents(g_krb5_ctx, &creds); });
+    if (krb5_is_config_principal(g_krb5_ctx, creds.server)) continue;
+
+    // We only want to reacquire the TGT (Ticket Granting Ticket). Ignore all other tickets.
+    // This follows the same format as is_local_tgt() from krb5:src/clients/klist/klist.c
+    if (creds.server->length != 2 ||
+        data_eq(creds.server->data[1], principal_->realm) == 0 ||
+        data_eq_string(creds.server->data[0], KRB5_TGS_NAME) == 0 ||
+        data_eq(creds.server->realm, principal_->realm) == 0) {
+      continue;
+    }
+
+    krb5_creds new_creds;
+    memset(&new_creds, 0, sizeof(krb5_creds));
+    SCOPED_CLEANUP({
+        krb5_free_cred_contents(g_krb5_ctx, &new_creds); });
+    // Acquire a new ticket using the keytab. This ticket will automatically be put into the
+    // credential cache.
+    {
+      std::lock_guard<RWMutex> l(*g_kerberos_reinit_lock);
+      KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(g_krb5_ctx, &new_creds, principal_,
+                                                            keytab_, 0 /* valid from now */,
+                                                            nullptr /* TKT service name */,
+                                                            opts_),
+                                 "Reacquire error: unable to login from keytab");
+#if !defined(HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
+      // Heimdal krb5 doesn't have the 'krb5_get_init_creds_opt_set_out_ccache' option,
+      // so use this alternate route.
+      KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(g_krb5_ctx, ccache_, principal_),
+                                 "Reacquire error: could not init ccache");
+
+      KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(g_krb5_ctx, ccache_, &new_creds),
+                                 "Reacquire error: could not store creds in cache");
+#endif
+    }
+    LOG(INFO) << "Successfully reacquired a new kerberos TGT";
+    ticket_end_timestamp_ = new_creds.times.endtime;
+    break;
+  }
+  return Status::OK();
+}
+
+Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
+  InitKrb5Ctx();
+
+  // Parse the principal
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal.c_str(), &principal_),
+                             "could not parse principal");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_kt_resolve(g_krb5_ctx, keytab_path.c_str(), &keytab_),
+                             "unable to resolve keytab");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_default(g_krb5_ctx, &ccache_),
+                             "unable to get default credentials cache");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(g_krb5_ctx, &opts_),
+                             "unable to allocate get_init_creds_opt struct");
+
+#if defined(HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_set_out_ccache(g_krb5_ctx, opts_, ccache_),
+                             "unable to set init_creds options");
+#endif
+
+  krb5_creds creds;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(g_krb5_ctx, &creds, principal_, keytab_,
+                                                        0 /* valid from now */,
+                                                        nullptr /* TKT service name */, opts_),
+                             "unable to login from keytab");
+  SCOPED_CLEANUP({
+      krb5_free_cred_contents(g_krb5_ctx, &creds); });
+
+  ticket_end_timestamp_ = creds.times.endtime;
+
+#if !defined(HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
+  // Heimdal krb5 doesn't have the 'krb5_get_init_creds_opt_set_out_ccache' option,
+  // so use this alternate route.
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(g_krb5_ctx, ccache_, principal_),
+                             "could not init ccache");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(g_krb5_ctx, ccache_, &creds),
+                             "could not store creds in cache");
+#endif
+
+  // Convert the logged-in principal back to a string. This may be different than
+  // 'principal', since the default realm will be filled in based on the Kerberos
+  // configuration if not originally specified.
+  RETURN_NOT_OK_PREPEND(Krb5UnparseName(principal_, &principal_str_),
+                        "could not stringify the logged-in principal");
+  RETURN_NOT_OK_PREPEND(MapPrincipalToLocalName(principal_str_, &username_str_),
+                        "could not map own logged-in principal to a short username");
+
+  LOG(INFO) << "Logged in from keytab as " << principal_str_
+            << " (short username " << username_str_ << ")";
+
+  return Status::OK();
+}
+
+// 'in_principal' is the user specified principal to use with Kerberos. It may have a token
+// in the string of the form '_HOST', which if present, needs to be replaced with the FQDN of the
+// current host.
+// 'out_principal' has the final principal with which one may Kinit.
+Status GetConfiguredPrincipal(const std::string& in_principal, string* out_principal) {
+  *out_principal = in_principal;
+  const auto& kHostToken = "_HOST";
+  if (in_principal.find(kHostToken) != string::npos) {
+    string hostname;
+    // Try to fill in either the FQDN or hostname.
+    if (!GetFQDN(&hostname).ok()) {
+      RETURN_NOT_OK(GetHostname(&hostname));
+    }
+    // Hosts in principal names are canonicalized to lower-case.
+    std::transform(hostname.begin(), hostname.end(), hostname.begin(), tolower);
+    GlobalReplaceSubstring(kHostToken, hostname, out_principal);
+  }
+  return Status::OK();
+}
+} // anonymous namespace
+
+
+RWMutex* KerberosReinitLock() {
+  return g_kerberos_reinit_lock;
+}
+
+Status CanonicalizeKrb5Principal(std::string* principal) {
+  InitKrb5Ctx();
+  krb5_principal princ;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal->c_str(), &princ),
+                             "could not parse principal");
+  SCOPED_CLEANUP({
+      krb5_free_principal(g_krb5_ctx, princ);
+    });
+  RETURN_NOT_OK_PREPEND(Krb5UnparseName(princ, principal),
+                        "failed to convert principal back to string");
+  return Status::OK();
+}
+
+Status MapPrincipalToLocalName(const std::string& principal, std::string* local_name) {
+  InitKrb5Ctx();
+  krb5_principal princ;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal.c_str(), &princ),
+                             "could not parse principal");
+  SCOPED_CLEANUP({
+      krb5_free_principal(g_krb5_ctx, princ);
+    });
+  char buf[1024];
+  krb5_error_code rc = KRB5_LNAME_NOTRANS;
+  if (FLAGS_use_system_auth_to_local) {
+    rc = krb5_aname_to_localname(g_krb5_ctx, princ, arraysize(buf), buf);
+  }
+  if (rc == KRB5_LNAME_NOTRANS || rc == KRB5_PLUGIN_NO_HANDLE) {
+    // No name mapping specified, or krb5-based name mapping is disabled.
+    //
+    // We fall back to simply taking the first component of the principal, for
+    // compatibility with the default behavior of Hadoop.
+    //
+    // NOTE: KRB5_PLUGIN_NO_HANDLE isn't typically expected here, but works around
+    // a bug in SSSD's auth_to_local implementation: https://pagure.io/SSSD/sssd/issue/3459
+    //
+    // TODO(todd): we should support custom configured auth-to-local mapping, since
+    // most Hadoop ecosystem components do not load them from krb5.conf.
+    if (princ->length > 0) {
+      local_name->assign(princ->data[0].data, princ->data[0].length);
+      return Status::OK();
+    }
+    return Status::NotFound("unable to find first component of principal");
+  }
+  if (rc == KRB5_CONFIG_NOTENUFSPACE) {
+    return Status::InvalidArgument("mapped username too large");
+  }
+  KRB5_RETURN_NOT_OK_PREPEND(rc, "krb5_aname_to_localname");
+  if (strlen(buf) == 0) {
+    return Status::InvalidArgument("principal mapped to empty username");
+  }
+  local_name->assign(buf);
+  return Status::OK();
+}
+
+boost::optional<string> GetLoggedInPrincipalFromKeytab() {
+  if (!g_kinit_ctx) return boost::none;
+  return g_kinit_ctx->principal_str();
+}
+
+boost::optional<string> GetLoggedInUsernameFromKeytab() {
+  if (!g_kinit_ctx) return boost::none;
+  return g_kinit_ctx->username_str();
+}
+
+Status InitKerberosForServer(const std::string& raw_principal, const std::string& keytab_file,
+    const std::string& krb5ccname, bool disable_krb5_replay_cache) {
+  if (keytab_file.empty()) return Status::OK();
+
+  setenv("KRB5CCNAME", krb5ccname.c_str(), 1);
+  setenv("KRB5_KTNAME", keytab_file.c_str(), 1);
+
+  if (disable_krb5_replay_cache) {
+    // KUDU-1897: disable the Kerberos replay cache. The KRPC protocol includes a
+    // per-connection server-generated nonce to protect against replay attacks
+    // when authenticating via Kerberos. The replay cache has many performance and
+    // implementation issues.
+    setenv("KRB5RCACHETYPE", "none", 1);
+  }
+
+  g_kinit_ctx = new KinitContext();
+  string configured_principal;
+  RETURN_NOT_OK(GetConfiguredPrincipal(raw_principal, &configured_principal));
+  RETURN_NOT_OK_PREPEND(g_kinit_ctx->Kinit(
+      keytab_file, configured_principal), "unable to kinit");
+
+  g_kerberos_reinit_lock = new RWMutex(RWMutex::Priority::PREFER_WRITING);
+  scoped_refptr<Thread> reacquire_thread;
+  // Start the reacquire thread.
+  RETURN_NOT_OK(Thread::Create("kerberos", "reacquire thread", &RenewThread, &reacquire_thread));
+
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/init.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/init.h b/be/src/kudu/security/init.h
new file mode 100644
index 0000000..8b1519a
--- /dev/null
+++ b/be/src/kudu/security/init.h
@@ -0,0 +1,84 @@
+// 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.
+#pragma once
+
+#include <string>
+
+namespace boost {
+template <class T>
+class optional;
+}
+
+namespace kudu {
+
+class RWMutex;
+class Status;
+
+namespace security {
+
+// The default kerberos credential cache name.
+// Have the daemons use an in-memory ticket cache, so they don't accidentally
+// pick up credentials from test cases or any other daemon.
+static const std::string kKrb5CCName = "MEMORY:kudu";
+
+// Initializes Kerberos for a server. In particular, this processes
+// the '--keytab_file' command line flag.
+// 'raw_principal' is the principal to Kinit with after calling GetConfiguredPrincipal()
+// on it.
+// 'keytab_file' is the path to the kerberos keytab file. If it's an empty string, kerberos
+// will not be initialized.
+// 'krb5ccname' is passed into the KRB5CCNAME env var.
+// 'disable_krb5_replay_cache' if set to true, disables the kerberos replay cache by setting
+// the KRB5RCACHETYPE env var to "none".
+Status InitKerberosForServer(const std::string& raw_principal,
+                             const std::string& keytab_file,
+                             const std::string& krb5ccname = kKrb5CCName,
+                             bool disable_krb5_replay_cache = true);
+
+// Returns the process lock 'kerberos_reinit_lock'
+// This lock is taken in write mode while the ticket is being reacquired, and
+// taken in read mode before using the SASL library which might require a ticket.
+RWMutex* KerberosReinitLock();
+
+// Return the full principal (user/host@REALM) that the server has used to
+// log in from the keytab.
+//
+// If the server has not logged in from a keytab, returns boost::none.
+boost::optional<std::string> GetLoggedInPrincipalFromKeytab();
+
+// Same, but returns the mapped short username.
+boost::optional<std::string> GetLoggedInUsernameFromKeytab();
+
+// Canonicalize the given principal name by adding '@DEFAULT_REALM' in the case that
+// the principal has no realm.
+//
+// TODO(todd): move to kerberos_util.h in the later patch in this series (the file doesn't
+// exist yet, and trying to avoid rebase pain).
+Status CanonicalizeKrb5Principal(std::string* principal);
+
+// Map the given Kerberos principal 'principal' to a short username (i.e. with no realm or
+// host component).
+//
+// This respects the "auth-to-local" mappings from the system krb5.conf. However, if no such
+// mapping can be found, we fall back to simply taking the first component of the principal.
+//
+// TODO(todd): move to kerberos_util.h in the later patch in this series (the file doesn't
+// exist yet, and trying to avoid rebase pain).
+Status MapPrincipalToLocalName(const std::string& principal, std::string* local_name);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/kerberos_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/kerberos_util.cc b/be/src/kudu/security/kerberos_util.cc
new file mode 100644
index 0000000..5e6d9b8
--- /dev/null
+++ b/be/src/kudu/security/kerberos_util.cc
@@ -0,0 +1,37 @@
+// 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 "kudu/security/kerberos_util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+
+#include <array>
+#include <utility>
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal) {
+
+  std::pair<StringPiece, StringPiece> user_realm = strings::Split(principal, "@");
+  std::pair<StringPiece, StringPiece> princ_host = strings::Split(user_realm.first, "/");
+  return {{princ_host.first, princ_host.second, user_realm.second}};
+}
+
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/kerberos_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/kerberos_util.h b/be/src/kudu/security/kerberos_util.h
new file mode 100644
index 0000000..4c27a86
--- /dev/null
+++ b/be/src/kudu/security/kerberos_util.h
@@ -0,0 +1,29 @@
+// 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.
+#pragma once
+
+#include <array>
+
+class StringPiece;
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/krb5_realm_override.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/krb5_realm_override.cc b/be/src/kudu/security/krb5_realm_override.cc
new file mode 100644
index 0000000..05e8af8
--- /dev/null
+++ b/be/src/kudu/security/krb5_realm_override.cc
@@ -0,0 +1,105 @@
+// 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 provides a workaround for tests running with Kerberos 1.11 or earlier.
+// These versions of Kerberos are missing a fix which allows service principals
+// to use IP addresses in their host component:
+//
+//   http://krbdev.mit.edu/rt/Ticket/Display.html?id=7603
+//
+// We use such principals in external minicluster tests, where servers have IP addresses
+// like 127.x.y.z that have no corresponding reverse DNS.
+//
+// The file contains an implementation of krb5_get_host_realm which wraps the one
+// in the Kerberos library. It detects the return code that indicates the
+// above problem and falls back to the default realm/
+//
+// The wrapper is injected via linking it into tests as well as the
+// "security" library. The linkage invocation uses the '-Wl,--undefined'
+// linker flag to force linking even though no symbol here is explicitly
+// referenced.
+
+#include <dlfcn.h>
+
+#include <cerrno>
+#include <cstdlib>
+#include <cstring>
+
+#include <krb5/krb5.h>
+#include <glog/logging.h>
+
+extern "C" {
+
+// This symbol is exported from the static library so that other static-linked binaries
+// can reference it and force this compilation unit to be linked. Otherwise the linker
+// thinks it's unused and doesn't link it.
+int krb5_realm_override_loaded = 1;
+
+// Save the original function from the Kerberos library itself.
+// We use dlsym() to load all of them, since this file gets linked into
+// some test binaries that themselves may not link against libkrb5.so at all.
+static void* g_orig_krb5_get_host_realm;
+static void* g_orig_krb5_get_default_realm;
+static void* g_orig_krb5_free_default_realm;
+
+// We only enable our workaround if this environment variable is set.
+constexpr static const char* kEnvVar = "KUDU_ENABLE_KRB5_REALM_FIX";
+
+#define CALL_ORIG(func_name, ...) \
+  ((decltype(&func_name))g_orig_ ## func_name)(__VA_ARGS__)
+
+__attribute__((constructor))
+static void init_orig_func() {
+  g_orig_krb5_get_host_realm = dlsym(RTLD_NEXT, "krb5_get_host_realm");
+  g_orig_krb5_get_default_realm = dlsym(RTLD_NEXT, "krb5_get_default_realm");
+  g_orig_krb5_free_default_realm = dlsym(RTLD_NEXT, "krb5_free_default_realm");
+}
+
+krb5_error_code krb5_get_host_realm(krb5_context context, const char* host, char*** realmsp) {
+  CHECK(g_orig_krb5_get_host_realm);
+  CHECK(g_orig_krb5_get_default_realm);
+  CHECK(g_orig_krb5_free_default_realm);
+
+  krb5_error_code rc = CALL_ORIG(krb5_get_host_realm, context, host, realmsp);
+  if (rc != KRB5_ERR_NUMERIC_REALM || getenv(kEnvVar) == nullptr) {
+    return rc;
+  }
+  // If we get KRB5_ERR_NUMERIC_REALM, this is indicative of a Kerberos version
+  // which has not provided support for numeric addresses as service host names
+  // So, we fill in the default realm instead.
+  char* default_realm;
+  rc = CALL_ORIG(krb5_get_default_realm, context, &default_realm);
+  if (rc != 0) {
+    return rc;
+  }
+
+  char** ret_realms;
+  ret_realms = static_cast<char**>(malloc(2 * sizeof(*ret_realms)));
+  if (ret_realms == nullptr) return ENOMEM;
+  ret_realms[0] = strdup(default_realm);
+  if (ret_realms[0] == nullptr) {
+    free(ret_realms);
+    return ENOMEM;
+  }
+  ret_realms[1] = 0;
+  *realmsp = ret_realms;
+
+  CALL_ORIG(krb5_free_default_realm, context, default_realm);
+  return 0;
+}
+
+} // extern "C"

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util.cc b/be/src/kudu/security/openssl_util.cc
new file mode 100644
index 0000000..a32140f
--- /dev/null
+++ b/be/src/kudu/security/openssl_util.cc
@@ -0,0 +1,322 @@
+// 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 "kudu/security/openssl_util.h"
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdio>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <openssl/crypto.h>
+#include <openssl/err.h>
+#include <openssl/rand.h>
+
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+// Determine whether initialization was ever called.
+//
+// Thread safety:
+// - written by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - read by DisableOpenSSLInitialization (must not be concurrent with above)
+bool g_ssl_is_initialized = false;
+
+// If true, then we expect someone else has initialized SSL.
+//
+// Thread safety:
+// - read by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - written by DisableOpenSSLInitialization (must not be concurrent with above)
+bool g_disable_ssl_init = false;
+
+// Array of locks used by OpenSSL.
+// We use an intentionally-leaked C-style array here to avoid non-POD static data.
+//
+// As of OpenSSL 1.1, locking callbacks are no longer used.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+Mutex* kCryptoLocks = nullptr;
+
+// Lock/Unlock the nth lock. Only to be used by OpenSSL.
+void LockingCB(int mode, int type, const char* /*file*/, int /*line*/) {
+  DCHECK(kCryptoLocks);
+  Mutex* m = &kCryptoLocks[type];
+  if (mode & CRYPTO_LOCK) {
+    m->lock();
+  } else {
+    m->unlock();
+  }
+}
+#endif
+
+Status CheckOpenSSLInitialized() {
+  if (!CRYPTO_get_locking_callback()) {
+    return Status::RuntimeError("Locking callback not initialized");
+  }
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (!ctx) {
+    ERR_clear_error();
+    return Status::RuntimeError("SSL library appears uninitialized (cannot create SSL_CTX)");
+  }
+  return Status::OK();
+}
+
+void DoInitializeOpenSSL() {
+#if OPENSSL_VERSION_NUMBER > 0x10100000L
+  // The OPENSSL_init_ssl manpage [1] says "As of version 1.1.0 OpenSSL will
+  // automatically allocate all resources it needs so no explicit initialisation
+  // is required." However, eliding library initialization leads to a memory
+  // leak in some versions of OpenSSL 1.1 when the first OpenSSL is
+  // ERR_peek_error [2]. In Kudu this is often the
+  // case due to prolific application of SCOPED_OPENSSL_NO_PENDING_ERRORS.
+  //
+  // Rather than determine whether this particular OpenSSL instance is
+  // leak-free, we'll initialize the library explicitly.
+  //
+  // 1. https://www.openssl.org/docs/man1.1.0/ssl/OPENSSL_init_ssl.html
+  // 2. https://github.com/openssl/openssl/issues/5899
+  if (g_disable_ssl_init) {
+    VLOG(2) << "Not initializing OpenSSL (disabled by application)";
+    return;
+  }
+  CHECK_EQ(1, OPENSSL_init_ssl(0, nullptr));
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+#else
+  // In case the user's thread has left some error around, clear it.
+  ERR_clear_error();
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (g_disable_ssl_init) {
+    VLOG(2) << "Not initializing OpenSSL (disabled by application)";
+    return;
+  }
+
+  // Check that OpenSSL isn't already initialized. If it is, it's likely
+  // we are embedded in (or embedding) another application/library which
+  // initializes OpenSSL, and we risk installing conflicting callbacks
+  // or crashing due to concurrent initialization attempts. In that case,
+  // log a warning.
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (ctx) {
+    LOG(WARNING) << "It appears that OpenSSL has been previously initialized by "
+                 << "code outside of Kudu. Please use kudu::client::DisableOpenSSLInitialization() "
+                 << "to avoid potential crashes due to conflicting initialization.";
+    // Continue anyway; all of the below is idempotent, except for the locking callback,
+    // which we check before overriding. They aren't thread-safe, however -- that's why
+    // we try to get embedding applications to do the right thing here rather than risk a
+    // potential initialization race.
+  } else {
+    // As expected, SSL is not initialized, so SSL_CTX_new() failed. Make sure
+    // it didn't leave anything in our error queue.
+    ERR_clear_error();
+  }
+
+  SSL_load_error_strings();
+  SSL_library_init();
+  OpenSSL_add_all_algorithms();
+  RAND_poll();
+
+  if (!CRYPTO_get_locking_callback()) {
+    // Initialize the OpenSSL mutexes. We intentionally leak these, so ignore
+    // LSAN warnings.
+    debug::ScopedLeakCheckDisabler d;
+    int num_locks = CRYPTO_num_locks();
+    CHECK(!kCryptoLocks);
+    kCryptoLocks = new Mutex[num_locks];
+
+    // Callbacks used by OpenSSL required in a multi-threaded setting.
+    CRYPTO_set_locking_callback(LockingCB);
+  }
+#endif
+
+  g_ssl_is_initialized = true;
+}
+
+} // anonymous namespace
+
+// Reads a STACK_OF(X509) from the BIO and returns it.
+STACK_OF(X509)* PEM_read_STACK_OF_X509(BIO* bio, void* /* unused */, pem_password_cb* /* unused */,
+    void* /* unused */) {
+  // Extract information from the chain certificate.
+  STACK_OF(X509_INFO)* info = PEM_X509_INFO_read_bio(bio, nullptr, nullptr, nullptr);
+  if (!info) return nullptr;
+  SCOPED_CLEANUP({
+    sk_X509_INFO_pop_free(info, X509_INFO_free);
+  });
+
+  // Initialize the Stack.
+  STACK_OF(X509)* sk = sk_X509_new_null();
+
+  // Iterate through the chain certificate and add each one to the stack.
+  for (int i = 0; i < sk_X509_INFO_num(info); ++i) {
+    X509_INFO *stack_item = sk_X509_INFO_value(info, i);
+    sk_X509_push(sk, stack_item->x509);
+    // We don't want the ScopedCleanup to free the x509 certificates as well since we will
+    // use it as a part of the STACK_OF(X509) object to be returned, so we set it to nullptr.
+    // We will take the responsibility of freeing it when we are done with the STACK_OF(X509).
+    stack_item->x509 = nullptr;
+  }
+  return sk;
+}
+
+// Writes a STACK_OF(X509) to the BIO.
+int PEM_write_STACK_OF_X509(BIO* bio, STACK_OF(X509)* obj) {
+  int chain_len = sk_X509_num(obj);
+  // Iterate through the stack and add each one to the BIO.
+  for (int i = 0; i < chain_len; ++i) {
+    X509* cert_item = sk_X509_value(obj, i);
+    int ret = PEM_write_bio_X509(bio, cert_item);
+    if (ret <= 0) return ret;
+  }
+  return 1;
+}
+
+// Reads a single X509 certificate and returns a STACK_OF(X509) with the single certificate.
+STACK_OF(X509)* DER_read_STACK_OF_X509(BIO* bio, void* /* unused */) {
+  // We don't support chain certificates written in DER format.
+  auto x = ssl_make_unique(d2i_X509_bio(bio, nullptr));
+  if (!x) return nullptr;
+  STACK_OF(X509)* sk = sk_X509_new_null();
+  if (sk_X509_push(sk, x.get()) == 0) {
+    return nullptr;
+  }
+  x.release();
+  return sk;
+}
+
+// Writes a single X509 certificate that it gets from the STACK_OF(X509) 'obj'.
+int DER_write_STACK_OF_X509(BIO* bio, STACK_OF(X509)* obj) {
+  int chain_len = sk_X509_num(obj);
+  // We don't support chain certificates written in DER format.
+  DCHECK_EQ(chain_len, 1);
+  X509* cert_item = sk_X509_value(obj, 0);
+  if (cert_item == nullptr) return 0;
+  return i2d_X509_bio(bio, cert_item);
+}
+
+void free_STACK_OF_X509(STACK_OF(X509)* sk) {
+  sk_X509_pop_free(sk, X509_free);
+}
+
+Status DisableOpenSSLInitialization() {
+  if (g_disable_ssl_init) return Status::OK();
+  if (g_ssl_is_initialized) {
+    return Status::IllegalState("SSL already initialized. Initialization can only be disabled "
+                                "before first usage.");
+  }
+  RETURN_NOT_OK(CheckOpenSSLInitialized());
+  g_disable_ssl_init = true;
+  return Status::OK();
+}
+
+void InitializeOpenSSL() {
+  static std::once_flag ssl_once;
+  std::call_once(ssl_once, DoInitializeOpenSSL);
+}
+
+string GetOpenSSLErrors() {
+  ostringstream serr;
+  uint32_t l;
+  int line, flags;
+  const char *file, *data;
+  bool is_first = true;
+  while ((l = ERR_get_error_line_data(&file, &line, &data, &flags)) != 0) {
+    if (is_first) {
+      is_first = false;
+    } else {
+      serr << " ";
+    }
+
+    char buf[256];
+    ERR_error_string_n(l, buf, sizeof(buf));
+    serr << buf << ":" << file << ":" << line;
+    if (flags & ERR_TXT_STRING) {
+      serr << ":" << data;
+    }
+  }
+  return serr.str();
+}
+
+string GetSSLErrorDescription(int error_code) {
+  switch (error_code) {
+    case SSL_ERROR_NONE: return "";
+    case SSL_ERROR_ZERO_RETURN: return "SSL_ERROR_ZERO_RETURN";
+    case SSL_ERROR_WANT_READ: return "SSL_ERROR_WANT_READ";
+    case SSL_ERROR_WANT_WRITE: return "SSL_ERROR_WANT_WRITE";
+    case SSL_ERROR_WANT_CONNECT: return "SSL_ERROR_WANT_CONNECT";
+    case SSL_ERROR_WANT_ACCEPT: return "SSL_ERROR_WANT_ACCEPT";
+    case SSL_ERROR_WANT_X509_LOOKUP: return "SSL_ERROR_WANT_X509_LOOKUP";
+    case SSL_ERROR_SYSCALL: {
+      string queued_error = GetOpenSSLErrors();
+      if (!queued_error.empty()) {
+        return queued_error;
+      }
+      return kudu::ErrnoToString(errno);
+    };
+    default: return GetOpenSSLErrors();
+  }
+}
+
+const string& DataFormatToString(DataFormat fmt) {
+  static const string kStrFormatUnknown = "UNKNOWN";
+  static const string kStrFormatDer = "DER";
+  static const string kStrFormatPem = "PEM";
+  switch (fmt) {
+    case DataFormat::DER:
+      return kStrFormatDer;
+    case DataFormat::PEM:
+      return kStrFormatPem;
+    default:
+      return kStrFormatUnknown;
+  }
+}
+
+Status GetPasswordFromShellCommand(const string& cmd, string* password) {
+  vector<string> argv = strings::Split(cmd, " ", strings::SkipEmpty());
+  if (argv.empty()) {
+    return Status::RuntimeError("invalid empty private key password command");
+  }
+  string stderr, stdout;
+  Status s = Subprocess::Call(argv, "" /* stdin */, &stdout, &stderr);
+  if (!s.ok()) {
+    return Status::RuntimeError(strings::Substitute(
+        "failed to run private key password command: $0", s.ToString()), stderr);
+  }
+  StripTrailingWhitespace(&stdout);
+  *password = stdout;
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util.h b/be/src/kudu/security/openssl_util.h
new file mode 100644
index 0000000..00c4ebf
--- /dev/null
+++ b/be/src/kudu/security/openssl_util.h
@@ -0,0 +1,217 @@
+// 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.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/err.h>
+#include <openssl/pem.h>
+#include <openssl/ssl.h>
+#include <openssl/x509.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct X509_req_st X509_REQ;
+typedef struct bio_st BIO;
+typedef struct evp_pkey_st EVP_PKEY;
+typedef struct ssl_ctx_st SSL_CTX;
+typedef struct ssl_st SSL;
+typedef struct x509_st X509;
+
+#define OPENSSL_CHECK_OK(call) \
+  CHECK_GT((call), 0)
+
+#define OPENSSL_RET_NOT_OK(call, msg) \
+  if ((call) <= 0) { \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
+  }
+
+#define OPENSSL_RET_IF_NULL(call, msg) \
+  if ((call) == nullptr) { \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
+  }
+
+// Scoped helper which DCHECKs that on both scope entry and exit, there are no
+// pending OpenSSL errors for the current thread.
+//
+// This allows us to avoid calling ERR_clear_error() defensively before every
+// OpenSSL call, but rather call it only when we get an error code indicating
+// there may be some pending error.
+//
+// Example usage:
+//
+//    void MyFunc() {
+//      SCOPED_OPENSSL_NO_PENDING_ERRORS;
+//      ... use OpenSSL APIs ...
+//    }
+#define SCOPED_OPENSSL_NO_PENDING_ERRORS \
+  kudu::security::internal::ScopedCheckNoPendingSSLErrors _no_ssl_errors(__PRETTY_FUNCTION__)
+
+namespace kudu {
+namespace security {
+
+using PasswordCallback = std::function<std::string(void)>;
+
+// Disable initialization of OpenSSL. Must be called before
+// any call to InitializeOpenSSL().
+Status DisableOpenSSLInitialization() WARN_UNUSED_RESULT;
+
+// Initializes static state required by the OpenSSL library.
+// This is a no-op if DisableOpenSSLInitialization() has been called.
+//
+// Safe to call multiple times.
+void InitializeOpenSSL();
+
+// Fetches errors from the OpenSSL error error queue, and stringifies them.
+//
+// The error queue will be empty after this method returns.
+//
+// See man(3) ERR_get_err for more discussion.
+std::string GetOpenSSLErrors();
+
+// Returns a string representation of the provided error code, which must be
+// from a prior call to the SSL_get_error function.
+//
+// If necessary, the OpenSSL error queue may be inspected and emptied as part of
+// this call, and/or 'errno' may be inspected. As a result, this method should
+// only be used directly after the error occurs, and from the same thread.
+//
+// See man(3) SSL_get_error for more discussion.
+std::string GetSSLErrorDescription(int error_code);
+
+// Runs the shell command 'cmd' which should give a password to a private key file
+// as the output.
+//
+// 'password' is populated with the password string if the command was a success.
+// An error Status object is returned otherwise.
+Status GetPasswordFromShellCommand(const std::string& cmd, std::string* password);
+
+// A generic wrapper for OpenSSL structures.
+template <typename T>
+using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
+
+// For each SSL type, the Traits class provides the important OpenSSL
+// API functions.
+template<typename SSL_TYPE>
+struct SslTypeTraits {};
+
+template<> struct SslTypeTraits<X509> {
+  static constexpr auto kFreeFunc = &X509_free;
+  static constexpr auto kReadPemFunc = &PEM_read_bio_X509;
+  static constexpr auto kReadDerFunc = &d2i_X509_bio;
+  static constexpr auto kWritePemFunc = &PEM_write_bio_X509;
+  static constexpr auto kWriteDerFunc = &i2d_X509_bio;
+};
+
+// SslTypeTraits functions for Type STACK_OF(X509)
+STACK_OF(X509)* PEM_read_STACK_OF_X509(BIO* bio, void* /* unused */,
+    pem_password_cb* /* unused */, void* /* unused */);
+int PEM_write_STACK_OF_X509(BIO* bio, STACK_OF(X509)* obj);
+STACK_OF(X509)* DER_read_STACK_OF_X509(BIO* bio, void* /* unused */);
+int DER_write_STACK_OF_X509(BIO* bio, STACK_OF(X509)* obj);
+void free_STACK_OF_X509(STACK_OF(X509)* sk);
+
+template<> struct SslTypeTraits<STACK_OF(X509)> {
+  static constexpr auto kFreeFunc = &free_STACK_OF_X509;
+  static constexpr auto kReadPemFunc = &PEM_read_STACK_OF_X509;
+  static constexpr auto kReadDerFunc = &DER_read_STACK_OF_X509;
+  static constexpr auto kWritePemFunc = &PEM_write_STACK_OF_X509;
+  static constexpr auto kWriteDerFunc = &DER_write_STACK_OF_X509;
+};
+template<> struct SslTypeTraits<X509_EXTENSION> {
+  static constexpr auto kFreeFunc = &X509_EXTENSION_free;
+};
+template<> struct SslTypeTraits<X509_REQ> {
+  static constexpr auto kFreeFunc = &X509_REQ_free;
+  static constexpr auto kReadPemFunc = &PEM_read_bio_X509_REQ;
+  static constexpr auto kReadDerFunc = &d2i_X509_REQ_bio;
+  static constexpr auto kWritePemFunc = &PEM_write_bio_X509_REQ;
+  static constexpr auto kWriteDerFunc = &i2d_X509_REQ_bio;
+};
+template<> struct SslTypeTraits<EVP_PKEY> {
+  static constexpr auto kFreeFunc = &EVP_PKEY_free;
+};
+template<> struct SslTypeTraits<SSL_CTX> {
+  static constexpr auto kFreeFunc = &SSL_CTX_free;
+};
+
+template<typename SSL_TYPE, typename Traits = SslTypeTraits<SSL_TYPE>>
+c_unique_ptr<SSL_TYPE> ssl_make_unique(SSL_TYPE* d) {
+  return {d, Traits::kFreeFunc};
+}
+
+// Acceptable formats for keys, X509 certificates and X509 CSRs.
+enum class DataFormat {
+  DER = 0,    // DER/ASN1 format (binary): for representing object on the wire
+  PEM = 1,    // PEM format (ASCII): for storing on filesystem, printing, etc.
+};
+
+// Data format representation as a string.
+const std::string& DataFormatToString(DataFormat fmt);
+
+// Template wrapper for dynamically allocated entities with custom deleter.
+// Mostly, using it for xxx_st types from the OpenSSL crypto library.
+template<typename Type>
+class RawDataWrapper {
+ public:
+  typedef Type RawDataType;
+
+  RawDataType* GetRawData() const {
+    return data_.get();
+  }
+
+  void AdoptRawData(RawDataType* d) {
+    data_ = ssl_make_unique(d);
+  }
+
+ protected:
+  c_unique_ptr<RawDataType> data_;
+};
+
+
+namespace internal {
+
+// Implementation of SCOPED_OPENSSL_NO_PENDING_ERRORS. Use the macro form
+// instead of directly instantiating the implementation class.
+struct ScopedCheckNoPendingSSLErrors {
+ public:
+  explicit ScopedCheckNoPendingSSLErrors(const char* func)
+      : func_(func) {
+    DCHECK_EQ(ERR_peek_error(), 0)
+        << "Expected no pending OpenSSL errors on " << func_
+        << " entry, but had: " << GetOpenSSLErrors();
+  }
+  ~ScopedCheckNoPendingSSLErrors() {
+    DCHECK_EQ(ERR_peek_error(), 0)
+        << "Expected no pending OpenSSL errors on " << func_
+        << " exit, but had: " << GetOpenSSLErrors();
+  }
+
+ private:
+  const char* const func_;
+};
+
+} // namespace internal
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/openssl_util_bio.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util_bio.h b/be/src/kudu/security/openssl_util_bio.h
new file mode 100644
index 0000000..c935b0b
--- /dev/null
+++ b/be/src/kudu/security/openssl_util_bio.h
@@ -0,0 +1,129 @@
+// 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.
+
+#pragma once
+
+#include "kudu/security/openssl_util.h"
+
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/bio.h>
+#include <openssl/buffer.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<BIO> {
+  static constexpr auto kFreeFunc = &BIO_free;
+};
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status ToBIO(BIO* bio, DataFormat format, TYPE* obj) {
+  CHECK(bio);
+  CHECK(obj);
+  switch (format) {
+    case DataFormat::DER:
+      OPENSSL_RET_NOT_OK(Traits::kWriteDerFunc(bio, obj),
+          "error exporting data in DER format");
+      break;
+    case DataFormat::PEM:
+      OPENSSL_RET_NOT_OK(Traits::kWritePemFunc(bio, obj),
+          "error exporting data in PEM format");
+      break;
+  }
+  OPENSSL_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+// The callback which is called by the OpenSSL library when trying to decrypt
+// a password protected private key.
+inline int TLSPasswordCB(char* buf, int size, int /* rwflag */, void* userdata) {
+  const auto* cb = reinterpret_cast<const PasswordCallback*>(userdata);
+  std::string pw = (*cb)();
+  if (pw.size() >= size) {
+    LOG(ERROR) << "Provided key password is longer than maximum length "
+               << size;
+    return -1;
+  }
+  strncpy(buf, pw.c_str(), size);
+  return pw.size();
+}
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status FromBIO(BIO* bio, DataFormat format, c_unique_ptr<TYPE>* ret,
+    const PasswordCallback& cb = PasswordCallback()) {
+  CHECK(bio);
+  switch (format) {
+    case DataFormat::DER:
+      *ret = ssl_make_unique(Traits::kReadDerFunc(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      *ret = ssl_make_unique(Traits::kReadPemFunc(bio, nullptr, &TLSPasswordCB,
+          const_cast<PasswordCallback*>(&cb)));
+      break;
+  }
+  if (PREDICT_FALSE(!*ret)) {
+    return Status::RuntimeError(GetOpenSSLErrors());
+  }
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromString(const std::string& data, DataFormat format,
+                  c_unique_ptr<Type>* ret) {
+  const void* mdata = reinterpret_cast<const void*>(data.data());
+  auto bio = ssl_make_unique(BIO_new_mem_buf(
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+      const_cast<void*>(mdata),
+#else
+      mdata,
+#endif
+      data.size()));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret)),
+                        "unable to load data from memory");
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status ToString(std::string* data, DataFormat format, Type* obj) {
+  CHECK(data);
+  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
+  RETURN_NOT_OK_PREPEND((ToBIO<Type, Traits>(bio.get(), format, obj)),
+                        "error serializing data");
+  BUF_MEM* membuf;
+  OPENSSL_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
+  data->assign(membuf->data, membuf->length);
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromFile(const std::string& fpath, DataFormat format,
+                c_unique_ptr<Type>* ret, const PasswordCallback& cb = PasswordCallback()) {
+  auto bio = ssl_make_unique(BIO_new(BIO_s_file()));
+  OPENSSL_RET_NOT_OK(BIO_read_filename(bio.get(), fpath.c_str()),
+      strings::Substitute("could not read data from file '$0'", fpath));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret, cb)),
+      strings::Substitute("unable to load data from file '$0'", fpath));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/security-test-util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security-test-util.cc b/be/src/kudu/security/security-test-util.cc
new file mode 100644
index 0000000..40b0938
--- /dev/null
+++ b/be/src/kudu/security/security-test-util.cc
@@ -0,0 +1,103 @@
+// 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 "kudu/security/security-test-util.h"
+
+#include <cstdint>
+#include <string>
+
+#include <boost/optional/optional.hpp>
+
+#include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+namespace security {
+
+using ca::CaCertRequestGenerator;
+using ca::CertSigner;
+
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert) {
+  static const int64_t kRootCaCertExpirationSeconds = 24 * 60 * 60;
+  // Create a key for the self-signed CA.
+  //
+  // OpenSSL has a concept of "security levels" which, amongst other things,
+  // place certain restrictions on key strength. OpenSSL 1.0 defaults to level
+  // 0 (no restrictions) while 1.1 defaults to level 1, which requires RSA keys
+  // to have at least 1024 bits. For simplicity, we'll just use 1024 bits here,
+  // even though shorter keys would decrease test running time.
+  //
+  // See https://www.openssl.org/docs/man1.1.0/ssl/SSL_CTX_get_security_level.html
+  // for more details.
+  RETURN_NOT_OK(GeneratePrivateKey(1024, ca_key));
+
+  CaCertRequestGenerator::Config config = { "test-ca-cn" };
+  RETURN_NOT_OK(CertSigner::SelfSignCA(*ca_key,
+                                       config,
+                                       kRootCaCertExpirationSeconds,
+                                       ca_cert));
+  return Status::OK();
+}
+
+std::ostream& operator<<(std::ostream& o, PkiConfig c) {
+    switch (c) {
+      case PkiConfig::NONE: o << "NONE"; break;
+      case PkiConfig::SELF_SIGNED: o << "SELF_SIGNED"; break;
+      case PkiConfig::TRUSTED: o << "TRUSTED"; break;
+      case PkiConfig::SIGNED: o << "SIGNED"; break;
+      case PkiConfig::EXTERNALLY_SIGNED: o << "EXTERNALLY_SIGNED"; break;
+    }
+    return o;
+}
+
+Status ConfigureTlsContext(PkiConfig config,
+                           const Cert& ca_cert,
+                           const PrivateKey& ca_key,
+                           TlsContext* tls_context) {
+  switch (config) {
+    case PkiConfig::NONE: break;
+    case PkiConfig::SELF_SIGNED:
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+      break;
+    case PkiConfig::TRUSTED:
+      RETURN_NOT_OK(tls_context->AddTrustedCertificate(ca_cert));
+      break;
+    case PkiConfig::SIGNED: {
+      RETURN_NOT_OK(tls_context->AddTrustedCertificate(ca_cert));
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+      Cert cert;
+      RETURN_NOT_OK(CertSigner(&ca_cert, &ca_key).Sign(*tls_context->GetCsrIfNecessary(), &cert));
+      RETURN_NOT_OK(tls_context->AdoptSignedCert(cert));
+      break;
+    };
+    case PkiConfig::EXTERNALLY_SIGNED: {
+      std::string cert_path, key_path;
+      // Write certificate and private key to file.
+      RETURN_NOT_OK(CreateTestSSLCertWithPlainKey(GetTestDataDirectory(), &cert_path, &key_path));
+      RETURN_NOT_OK(tls_context->LoadCertificateAndKey(cert_path, key_path));
+      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(cert_path));
+    };
+  }
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu


[19/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hdr_histogram.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hdr_histogram.cc b/be/src/kudu/util/hdr_histogram.cc
new file mode 100644
index 0000000..4907444
--- /dev/null
+++ b/be/src/kudu/util/hdr_histogram.cc
@@ -0,0 +1,501 @@
+// 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.
+//
+// Portions of these classes were ported from Java to C++ from the sources
+// available at https://github.com/HdrHistogram/HdrHistogram .
+//
+//   The code in this repository code was Written by Gil Tene, Michael Barker,
+//   and Matt Warren, and released to the public domain, as explained at
+//   http://creativecommons.org/publicdomain/zero/1.0/
+#include "kudu/util/hdr_histogram.h"
+
+#include <algorithm>
+#include <cmath>
+#include <limits>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/bits.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+using base::subtle::Atomic64;
+using base::subtle::NoBarrier_AtomicIncrement;
+using base::subtle::NoBarrier_Store;
+using base::subtle::NoBarrier_Load;
+using base::subtle::NoBarrier_CompareAndSwap;
+using strings::Substitute;
+
+namespace kudu {
+
+HdrHistogram::HdrHistogram(uint64_t highest_trackable_value, int num_significant_digits)
+  : highest_trackable_value_(highest_trackable_value),
+    num_significant_digits_(num_significant_digits),
+    counts_array_length_(0),
+    bucket_count_(0),
+    sub_bucket_count_(0),
+    sub_bucket_half_count_magnitude_(0),
+    sub_bucket_half_count_(0),
+    sub_bucket_mask_(0),
+    total_count_(0),
+    total_sum_(0),
+    min_value_(std::numeric_limits<Atomic64>::max()),
+    max_value_(0),
+    counts_(nullptr) {
+  Init();
+}
+
+HdrHistogram::HdrHistogram(const HdrHistogram& other)
+  : highest_trackable_value_(other.highest_trackable_value_),
+    num_significant_digits_(other.num_significant_digits_),
+    counts_array_length_(0),
+    bucket_count_(0),
+    sub_bucket_count_(0),
+    sub_bucket_half_count_magnitude_(0),
+    sub_bucket_half_count_(0),
+    sub_bucket_mask_(0),
+    total_count_(0),
+    total_sum_(0),
+    min_value_(std::numeric_limits<Atomic64>::max()),
+    max_value_(0),
+    counts_(nullptr) {
+  Init();
+
+  // Not a consistent snapshot but we try to roughly keep it close.
+  // Copy the sum and min first.
+  NoBarrier_Store(&total_sum_, NoBarrier_Load(&other.total_sum_));
+  NoBarrier_Store(&min_value_, NoBarrier_Load(&other.min_value_));
+
+  uint64_t total_copied_count = 0;
+  // Copy the counts in order of ascending magnitude.
+  for (int i = 0; i < counts_array_length_; i++) {
+    uint64_t count = NoBarrier_Load(&other.counts_[i]);
+    NoBarrier_Store(&counts_[i], count);
+    total_copied_count += count;
+  }
+  // Copy the max observed value last.
+  NoBarrier_Store(&max_value_, NoBarrier_Load(&other.max_value_));
+  // We must ensure the total is consistent with the copied counts.
+  NoBarrier_Store(&total_count_, total_copied_count);
+}
+
+bool HdrHistogram::IsValidHighestTrackableValue(uint64_t highest_trackable_value) {
+  return highest_trackable_value >= kMinHighestTrackableValue;
+}
+
+bool HdrHistogram::IsValidNumSignificantDigits(int num_significant_digits) {
+  return num_significant_digits >= kMinValidNumSignificantDigits &&
+         num_significant_digits <= kMaxValidNumSignificantDigits;
+}
+
+void HdrHistogram::Init() {
+  // Verify parameter validity
+  CHECK(IsValidHighestTrackableValue(highest_trackable_value_)) <<
+      Substitute("highest_trackable_value must be >= $0", kMinHighestTrackableValue);
+  CHECK(IsValidNumSignificantDigits(num_significant_digits_)) <<
+      Substitute("num_significant_digits must be between $0 and $1",
+          kMinValidNumSignificantDigits, kMaxValidNumSignificantDigits);
+
+  uint32_t largest_value_with_single_unit_resolution =
+      2 * static_cast<uint32_t>(pow(10.0, num_significant_digits_));
+
+  // We need to maintain power-of-two sub_bucket_count_ (for clean direct
+  // indexing) that is large enough to provide unit resolution to at least
+  // largest_value_with_single_unit_resolution. So figure out
+  // largest_value_with_single_unit_resolution's nearest power-of-two
+  // (rounded up), and use that:
+
+  // The sub-buckets take care of the precision.
+  // Each sub-bucket is sized to have enough bits for the requested
+  // 10^precision accuracy.
+  int sub_bucket_count_magnitude =
+      Bits::Log2Ceiling(largest_value_with_single_unit_resolution);
+  sub_bucket_half_count_magnitude_ =
+      (sub_bucket_count_magnitude >= 1) ? sub_bucket_count_magnitude - 1 : 0;
+
+  // sub_bucket_count_ is approx. 10^num_sig_digits (as a power of 2)
+  sub_bucket_count_ = pow(2.0, sub_bucket_half_count_magnitude_ + 1);
+  sub_bucket_mask_ = sub_bucket_count_ - 1;
+  sub_bucket_half_count_ = sub_bucket_count_ / 2;
+
+  // The buckets take care of the magnitude.
+  // Determine exponent range needed to support the trackable value with no
+  // overflow:
+  uint64_t trackable_value = sub_bucket_count_ - 1;
+  int buckets_needed = 1;
+  while (trackable_value < highest_trackable_value_) {
+    trackable_value <<= 1;
+    buckets_needed++;
+  }
+  bucket_count_ = buckets_needed;
+
+  counts_array_length_ = (bucket_count_ + 1) * sub_bucket_half_count_;
+  counts_.reset(new Atomic64[counts_array_length_]());  // value-initialized
+}
+
+void HdrHistogram::Increment(int64_t value) {
+  IncrementBy(value, 1);
+}
+
+void HdrHistogram::IncrementBy(int64_t value, int64_t count) {
+  DCHECK_GE(value, 0);
+  DCHECK_GE(count, 0);
+
+  // Dissect the value into bucket and sub-bucket parts, and derive index into
+  // counts array:
+  int bucket_index = BucketIndex(value);
+  int sub_bucket_index = SubBucketIndex(value, bucket_index);
+  int counts_index = CountsArrayIndex(bucket_index, sub_bucket_index);
+
+  // Increment bucket, total, and sum.
+  NoBarrier_AtomicIncrement(&counts_[counts_index], count);
+  NoBarrier_AtomicIncrement(&total_count_, count);
+  NoBarrier_AtomicIncrement(&total_sum_, value * count);
+
+  // Update min, if needed.
+  {
+    Atomic64 min_val;
+    while (PREDICT_FALSE(value < (min_val = MinValue()))) {
+      Atomic64 old_val = NoBarrier_CompareAndSwap(&min_value_, min_val, value);
+      if (PREDICT_TRUE(old_val == min_val)) break; // CAS success.
+    }
+  }
+
+  // Update max, if needed.
+  {
+    Atomic64 max_val;
+    while (PREDICT_FALSE(value > (max_val = MaxValue()))) {
+      Atomic64 old_val = NoBarrier_CompareAndSwap(&max_value_, max_val, value);
+      if (PREDICT_TRUE(old_val == max_val)) break; // CAS success.
+    }
+  }
+}
+
+void HdrHistogram::IncrementWithExpectedInterval(int64_t value,
+                                                 int64_t expected_interval_between_samples) {
+  Increment(value);
+  if (expected_interval_between_samples <= 0) {
+    return;
+  }
+  for (int64_t missing_value = value - expected_interval_between_samples;
+      missing_value >= expected_interval_between_samples;
+      missing_value -= expected_interval_between_samples) {
+    Increment(missing_value);
+  }
+}
+
+////////////////////////////////////
+
+int HdrHistogram::BucketIndex(uint64_t value) const {
+  if (PREDICT_FALSE(value > highest_trackable_value_)) {
+    value = highest_trackable_value_;
+  }
+  // Here we are calculating the power-of-2 magnitude of the value with a
+  // correction for precision in the first bucket.
+  // Smallest power of 2 containing value.
+  int pow2ceiling = Bits::Log2Ceiling64(value | sub_bucket_mask_);
+  return pow2ceiling - (sub_bucket_half_count_magnitude_ + 1);
+}
+
+int HdrHistogram::SubBucketIndex(uint64_t value, int bucket_index) const {
+  if (PREDICT_FALSE(value > highest_trackable_value_)) {
+    value = highest_trackable_value_;
+  }
+  // We hack off the magnitude and are left with only the relevant precision
+  // portion, which gives us a direct index into the sub-bucket. TODO: Right??
+  return static_cast<int>(value >> bucket_index);
+}
+
+int HdrHistogram::CountsArrayIndex(int bucket_index, int sub_bucket_index) const {
+  DCHECK(sub_bucket_index < sub_bucket_count_);
+  DCHECK(bucket_index < bucket_count_);
+  DCHECK(bucket_index == 0 || (sub_bucket_index >= sub_bucket_half_count_));
+  // Calculate the index for the first entry in the bucket:
+  // (The following is the equivalent of ((bucket_index + 1) * sub_bucket_half_count_) ):
+  int bucket_base_index = (bucket_index + 1) << sub_bucket_half_count_magnitude_;
+  // Calculate the offset in the bucket:
+  int offset_in_bucket = sub_bucket_index - sub_bucket_half_count_;
+  return bucket_base_index + offset_in_bucket;
+}
+
+uint64_t HdrHistogram::CountAt(int bucket_index, int sub_bucket_index) const {
+  return counts_[CountsArrayIndex(bucket_index, sub_bucket_index)];
+}
+
+uint64_t HdrHistogram::CountInBucketForValue(uint64_t value) const {
+  int bucket_index = BucketIndex(value);
+  int sub_bucket_index = SubBucketIndex(value, bucket_index);
+  return CountAt(bucket_index, sub_bucket_index);
+}
+
+uint64_t HdrHistogram::ValueFromIndex(int bucket_index, int sub_bucket_index) {
+  return static_cast<uint64_t>(sub_bucket_index) << bucket_index;
+}
+
+////////////////////////////////////
+
+uint64_t HdrHistogram::SizeOfEquivalentValueRange(uint64_t value) const {
+  int bucket_index = BucketIndex(value);
+  int sub_bucket_index = SubBucketIndex(value, bucket_index);
+  uint64_t distance_to_next_value =
+    (1 << ((sub_bucket_index >= sub_bucket_count_) ? (bucket_index + 1) : bucket_index));
+  return distance_to_next_value;
+}
+
+uint64_t HdrHistogram::LowestEquivalentValue(uint64_t value) const {
+  int bucket_index = BucketIndex(value);
+  int sub_bucket_index = SubBucketIndex(value, bucket_index);
+  uint64_t this_value_base_level = ValueFromIndex(bucket_index, sub_bucket_index);
+  return this_value_base_level;
+}
+
+uint64_t HdrHistogram::HighestEquivalentValue(uint64_t value) const {
+  return NextNonEquivalentValue(value) - 1;
+}
+
+uint64_t HdrHistogram::MedianEquivalentValue(uint64_t value) const {
+  return (LowestEquivalentValue(value) + (SizeOfEquivalentValueRange(value) >> 1));
+}
+
+uint64_t HdrHistogram::NextNonEquivalentValue(uint64_t value) const {
+  return LowestEquivalentValue(value) + SizeOfEquivalentValueRange(value);
+}
+
+bool HdrHistogram::ValuesAreEquivalent(uint64_t value1, uint64_t value2) const {
+  return (LowestEquivalentValue(value1) == LowestEquivalentValue(value2));
+}
+
+uint64_t HdrHistogram::MinValue() const {
+  if (PREDICT_FALSE(TotalCount() == 0)) return 0;
+  return NoBarrier_Load(&min_value_);
+}
+
+uint64_t HdrHistogram::MaxValue() const {
+  if (PREDICT_FALSE(TotalCount() == 0)) return 0;
+  return NoBarrier_Load(&max_value_);
+}
+
+double HdrHistogram::MeanValue() const {
+  uint64_t count = TotalCount();
+  if (PREDICT_FALSE(count == 0)) return 0.0;
+  return static_cast<double>(TotalSum()) / count;
+}
+
+uint64_t HdrHistogram::ValueAtPercentile(double percentile) const {
+  uint64_t count = TotalCount();
+  if (PREDICT_FALSE(count == 0)) return 0;
+
+  double requested_percentile = std::min(percentile, 100.0); // Truncate down to 100%
+  uint64_t count_at_percentile = static_cast<uint64_t>(
+      ((requested_percentile / 100.0) * count) + 0.5); // NOLINT(misc-incorrect-roundings)
+  // Make sure we at least reach the first recorded entry
+  count_at_percentile = std::max(count_at_percentile, static_cast<uint64_t>(1));
+
+  uint64_t total_to_current_iJ = 0;
+  for (int i = 0; i < bucket_count_; i++) {
+    int j = (i == 0) ? 0 : (sub_bucket_count_ / 2);
+    for (; j < sub_bucket_count_; j++) {
+      total_to_current_iJ += CountAt(i, j);
+      if (total_to_current_iJ >= count_at_percentile) {
+        uint64_t valueAtIndex = ValueFromIndex(i, j);
+        return valueAtIndex;
+      }
+    }
+  }
+
+  LOG(DFATAL) << "Fell through while iterating, likely concurrent modification of histogram";
+  return 0;
+}
+
+///////////////////////////////////////////////////////////////////////
+// AbstractHistogramIterator
+///////////////////////////////////////////////////////////////////////
+
+AbstractHistogramIterator::AbstractHistogramIterator(const HdrHistogram* histogram)
+  : histogram_(CHECK_NOTNULL(histogram)),
+    cur_iter_val_(),
+    histogram_total_count_(histogram_->TotalCount()),
+    current_bucket_index_(0),
+    current_sub_bucket_index_(0),
+    current_value_at_index_(0),
+    next_bucket_index_(0),
+    next_sub_bucket_index_(1),
+    next_value_at_index_(1),
+    prev_value_iterated_to_(0),
+    total_count_to_prev_index_(0),
+    total_count_to_current_index_(0),
+    total_value_to_current_index_(0),
+    count_at_this_value_(0),
+    fresh_sub_bucket_(true) {
+}
+
+bool AbstractHistogramIterator::HasNext() const {
+  return total_count_to_current_index_ < histogram_total_count_;
+}
+
+Status AbstractHistogramIterator::Next(HistogramIterationValue* value) {
+  if (histogram_->TotalCount() != histogram_total_count_) {
+    return Status::IllegalState("Concurrently modified histogram while traversing it");
+  }
+
+  // Move through the sub buckets and buckets until we hit the next reporting level:
+  while (!ExhaustedSubBuckets()) {
+    count_at_this_value_ =
+        histogram_->CountAt(current_bucket_index_, current_sub_bucket_index_);
+    if (fresh_sub_bucket_) { // Don't add unless we've incremented since last bucket...
+      total_count_to_current_index_ += count_at_this_value_;
+      total_value_to_current_index_ +=
+        count_at_this_value_ * histogram_->MedianEquivalentValue(current_value_at_index_);
+      fresh_sub_bucket_ = false;
+    }
+    if (ReachedIterationLevel()) {
+      uint64_t value_iterated_to = ValueIteratedTo();
+
+      // Update iterator value.
+      cur_iter_val_.value_iterated_to = value_iterated_to;
+      cur_iter_val_.value_iterated_from = prev_value_iterated_to_;
+      cur_iter_val_.count_at_value_iterated_to = count_at_this_value_;
+      cur_iter_val_.count_added_in_this_iteration_step =
+          (total_count_to_current_index_ - total_count_to_prev_index_);
+      cur_iter_val_.total_count_to_this_value = total_count_to_current_index_;
+      cur_iter_val_.total_value_to_this_value = total_value_to_current_index_;
+      cur_iter_val_.percentile =
+          ((100.0 * total_count_to_current_index_) / histogram_total_count_);
+      cur_iter_val_.percentile_level_iterated_to = PercentileIteratedTo();
+
+      prev_value_iterated_to_ = value_iterated_to;
+      total_count_to_prev_index_ = total_count_to_current_index_;
+      // Move the next percentile reporting level forward.
+      IncrementIterationLevel();
+
+      *value = cur_iter_val_;
+      return Status::OK();
+    }
+    IncrementSubBucket();
+  }
+  return Status::IllegalState("Histogram array index out of bounds while traversing");
+}
+
+double AbstractHistogramIterator::PercentileIteratedTo() const {
+  return (100.0 * static_cast<double>(total_count_to_current_index_)) / histogram_total_count_;
+}
+
+double AbstractHistogramIterator::PercentileIteratedFrom() const {
+  return (100.0 * static_cast<double>(total_count_to_prev_index_)) / histogram_total_count_;
+}
+
+uint64_t AbstractHistogramIterator::ValueIteratedTo() const {
+  return histogram_->HighestEquivalentValue(current_value_at_index_);
+}
+
+bool AbstractHistogramIterator::ExhaustedSubBuckets() const {
+  return (current_bucket_index_ >= histogram_->bucket_count_);
+}
+
+void AbstractHistogramIterator::IncrementSubBucket() {
+  fresh_sub_bucket_ = true;
+  // Take on the next index:
+  current_bucket_index_ = next_bucket_index_;
+  current_sub_bucket_index_ = next_sub_bucket_index_;
+  current_value_at_index_ = next_value_at_index_;
+  // Figure out the next next index:
+  next_sub_bucket_index_++;
+  if (next_sub_bucket_index_ >= histogram_->sub_bucket_count_) {
+    next_sub_bucket_index_ = histogram_->sub_bucket_half_count_;
+    next_bucket_index_++;
+  }
+  next_value_at_index_ = HdrHistogram::ValueFromIndex(next_bucket_index_, next_sub_bucket_index_);
+}
+
+///////////////////////////////////////////////////////////////////////
+// RecordedValuesIterator
+///////////////////////////////////////////////////////////////////////
+
+RecordedValuesIterator::RecordedValuesIterator(const HdrHistogram* histogram)
+  : AbstractHistogramIterator(histogram),
+    visited_sub_bucket_index_(-1),
+    visited_bucket_index_(-1) {
+}
+
+void RecordedValuesIterator::IncrementIterationLevel() {
+  visited_sub_bucket_index_ = current_sub_bucket_index_;
+  visited_bucket_index_ = current_bucket_index_;
+}
+
+bool RecordedValuesIterator::ReachedIterationLevel() const {
+  uint64_t current_ij_count =
+      histogram_->CountAt(current_bucket_index_, current_sub_bucket_index_);
+  return current_ij_count != 0 &&
+      ((visited_sub_bucket_index_ != current_sub_bucket_index_) ||
+       (visited_bucket_index_ != current_bucket_index_));
+}
+
+///////////////////////////////////////////////////////////////////////
+// PercentileIterator
+///////////////////////////////////////////////////////////////////////
+
+PercentileIterator::PercentileIterator(const HdrHistogram* histogram,
+                                       int percentile_ticks_per_half_distance)
+  : AbstractHistogramIterator(histogram),
+    percentile_ticks_per_half_distance_(percentile_ticks_per_half_distance),
+    percentile_level_to_iterate_to_(0.0),
+    percentile_level_to_iterate_from_(0.0),
+    reached_last_recorded_value_(false) {
+}
+
+bool PercentileIterator::HasNext() const {
+  if (AbstractHistogramIterator::HasNext()) {
+    return true;
+  }
+  // We want one additional last step to 100%
+  if (!reached_last_recorded_value_ && (histogram_total_count_ > 0)) {
+    const_cast<PercentileIterator*>(this)->percentile_level_to_iterate_to_ = 100.0;
+    const_cast<PercentileIterator*>(this)->reached_last_recorded_value_ = true;
+    return true;
+  }
+  return false;
+}
+
+double PercentileIterator::PercentileIteratedTo() const {
+  return percentile_level_to_iterate_to_;
+}
+
+
+double PercentileIterator::PercentileIteratedFrom() const {
+  return percentile_level_to_iterate_from_;
+}
+
+void PercentileIterator::IncrementIterationLevel() {
+  percentile_level_to_iterate_from_ = percentile_level_to_iterate_to_;
+  // TODO: Can this expression be simplified?
+  uint64_t percentile_reporting_ticks = percentile_ticks_per_half_distance_ *
+    static_cast<uint64_t>(pow(2.0,
+          static_cast<int>(log(100.0 / (100.0 - (percentile_level_to_iterate_to_))) / log(2)) + 1));
+  percentile_level_to_iterate_to_ += 100.0 / percentile_reporting_ticks;
+}
+
+bool PercentileIterator::ReachedIterationLevel() const {
+  if (count_at_this_value_ == 0) return false;
+  double current_percentile =
+      (100.0 * static_cast<double>(total_count_to_current_index_)) / histogram_total_count_;
+  return (current_percentile >= percentile_level_to_iterate_to_);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hdr_histogram.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hdr_histogram.h b/be/src/kudu/util/hdr_histogram.h
new file mode 100644
index 0000000..14b5e95
--- /dev/null
+++ b/be/src/kudu/util/hdr_histogram.h
@@ -0,0 +1,351 @@
+// 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.
+#ifndef KUDU_UTIL_HDRHISTOGRAM_H_
+#define KUDU_UTIL_HDRHISTOGRAM_H_
+
+// C++ (TR1) port of HdrHistogram.
+//
+// Portions of these classes were ported from Java to C++ from the sources
+// available at https://github.com/HdrHistogram/HdrHistogram .
+//
+//   The code in this repository code was Written by Gil Tene, Michael Barker,
+//   and Matt Warren, and released to the public domain, as explained at
+//   http://creativecommons.org/publicdomain/zero/1.0/
+// ---------------------------------------------------------------------------
+//
+// A High Dynamic Range (HDR) Histogram
+//
+// HdrHistogram supports the recording and analyzing sampled data value counts
+// across a configurable integer value range with configurable value precision
+// within the range. Value precision is expressed as the number of significant
+// digits in the value recording, and provides control over value quantization
+// behavior across the value range and the subsequent value resolution at any
+// given level.
+//
+// For example, a Histogram could be configured to track the counts of observed
+// integer values between 0 and 3,600,000,000 while maintaining a value
+// precision of 3 significant digits across that range. Value quantization
+// within the range will thus be no larger than 1/1,000th (or 0.1%) of any
+// value. This example Histogram could be used to track and analyze the counts
+// of observed response times ranging between 1 microsecond and 1 hour in
+// magnitude, while maintaining a value resolution of 1 microsecond up to 1
+// millisecond, a resolution of 1 millisecond (or better) up to one second, and
+// a resolution of 1 second (or better) up to 1,000 seconds. At it's maximum
+// tracked value (1 hour), it would still maintain a resolution of 3.6 seconds
+// (or better).
+
+#include <stdint.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+class Status;
+
+// This implementation allows you to specify a range and accuracy (significant
+// digits) to support in an instance of a histogram. The class takes care of
+// the rest. At this time, only uint64_t values are supported.
+//
+// An HdrHistogram consists of a set of buckets, which bucket the magnitude of
+// a value stored, and a set of sub-buckets, which implement the tunable
+// precision of the storage. So if you specify 3 significant digits of
+// precision, then you will get about 10^3 sub-buckets (as a power of 2) for
+// each level of magnitude. Magnitude buckets are tracked in powers of 2.
+//
+// This class is thread-safe.
+class HdrHistogram {
+ public:
+  // Specify the highest trackable value so that the class has a bound on the
+  // number of buckets, and # of significant digits (in decimal) so that the
+  // class can determine the granularity of those buckets.
+  HdrHistogram(uint64_t highest_trackable_value, int num_significant_digits);
+
+  // Copy-construct a (non-consistent) snapshot of other.
+  explicit HdrHistogram(const HdrHistogram& other);
+
+  // Validate your params before trying to construct the object.
+  static bool IsValidHighestTrackableValue(uint64_t highest_trackable_value);
+  static bool IsValidNumSignificantDigits(int num_significant_digits);
+
+  // Record new data.
+  void Increment(int64_t value);
+  void IncrementBy(int64_t value, int64_t count);
+
+  // Record new data, correcting for "coordinated omission".
+  //
+  // See https://groups.google.com/d/msg/mechanical-sympathy/icNZJejUHfE/BfDekfBEs_sJ
+  // for more details.
+  void IncrementWithExpectedInterval(int64_t value,
+                                     int64_t expected_interval_between_samples);
+
+  // Fetch configuration params.
+  uint64_t highest_trackable_value() const { return highest_trackable_value_; }
+  int num_significant_digits() const { return num_significant_digits_; }
+
+  // Get indexes into histogram based on value.
+  int BucketIndex(uint64_t value) const;
+  int SubBucketIndex(uint64_t value, int bucket_index) const;
+
+  // Count of all events recorded.
+  uint64_t TotalCount() const { return base::subtle::NoBarrier_Load(&total_count_); }
+
+  // Sum of all events recorded.
+  uint64_t TotalSum() const { return base::subtle::NoBarrier_Load(&total_sum_); }
+
+  // Return number of items at index.
+  uint64_t CountAt(int bucket_index, int sub_bucket_index) const;
+
+  // Return count of values in bucket with values equivalent to value.
+  uint64_t CountInBucketForValue(uint64_t) const;
+
+  // Return representative value based on index.
+  static uint64_t ValueFromIndex(int bucket_index, int sub_bucket_index);
+
+  // Get the size (in value units) of the range of values that are equivalent
+  // to the given value within the histogram's resolution. Where "equivalent"
+  // means that value samples recorded for any two equivalent values are
+  // counted in a common total count.
+  uint64_t SizeOfEquivalentValueRange(uint64_t value) const;
+
+  // Get the lowest value that is equivalent to the given value within the
+  // histogram's resolution. Where "equivalent" means that value samples
+  // recorded for any two equivalent values are counted in a common total
+  // count.
+  uint64_t LowestEquivalentValue(uint64_t value) const;
+
+  // Get the highest value that is equivalent to the given value within the
+  // histogram's resolution.
+  uint64_t HighestEquivalentValue(uint64_t value) const;
+
+  // Get a value that lies in the middle (rounded up) of the range of values
+  // equivalent the given value.
+  uint64_t MedianEquivalentValue(uint64_t value) const;
+
+  // Get the next value that is not equivalent to the given value within the
+  // histogram's resolution.
+  uint64_t NextNonEquivalentValue(uint64_t value) const;
+
+  // Determine if two values are equivalent with the histogram's resolution.
+  bool ValuesAreEquivalent(uint64_t value1, uint64_t value2) const;
+
+  // Get the exact minimum value (may lie outside the histogram).
+  uint64_t MinValue() const;
+
+  // Get the exact maximum value (may lie outside the histogram).
+  uint64_t MaxValue() const;
+
+  // Get the exact mean value of all recorded values in the histogram.
+  double MeanValue() const;
+
+  // Get the value at a given percentile.
+  // This is a percentile in percents, i.e. 99.99 percentile.
+  uint64_t ValueAtPercentile(double percentile) const;
+
+  // Get the percentile at a given value
+  // TODO: implement
+  // double PercentileAtOrBelowValue(uint64_t value) const;
+
+  // Get the count of recorded values within a range of value levels.
+  // (inclusive to within the histogram's resolution)
+  // TODO: implement
+  //uint64_t CountBetweenValues(uint64_t low_value, uint64_t high_value) const;
+
+ private:
+  friend class AbstractHistogramIterator;
+
+  static const uint64_t kMinHighestTrackableValue = 2;
+  static const int kMinValidNumSignificantDigits = 1;
+  static const int kMaxValidNumSignificantDigits = 5;
+
+  void Init();
+  int CountsArrayIndex(int bucket_index, int sub_bucket_index) const;
+
+  uint64_t highest_trackable_value_;
+  int num_significant_digits_;
+  int counts_array_length_;
+  int bucket_count_;
+  int sub_bucket_count_;
+
+  // "Hot" fields in the write path.
+  uint8_t sub_bucket_half_count_magnitude_;
+  int sub_bucket_half_count_;
+  uint32_t sub_bucket_mask_;
+
+  // Also hot.
+  base::subtle::Atomic64 total_count_;
+  base::subtle::Atomic64 total_sum_;
+  base::subtle::Atomic64 min_value_;
+  base::subtle::Atomic64 max_value_;
+  gscoped_array<base::subtle::Atomic64> counts_;
+
+  HdrHistogram& operator=(const HdrHistogram& other); // Disable assignment operator.
+};
+
+// Value returned from iterators.
+struct HistogramIterationValue {
+  HistogramIterationValue()
+    : value_iterated_to(0),
+      value_iterated_from(0),
+      count_at_value_iterated_to(0),
+      count_added_in_this_iteration_step(0),
+      total_count_to_this_value(0),
+      total_value_to_this_value(0),
+      percentile(0.0),
+      percentile_level_iterated_to(0.0) {
+  }
+
+  void Reset() {
+    value_iterated_to = 0;
+    value_iterated_from = 0;
+    count_at_value_iterated_to = 0;
+    count_added_in_this_iteration_step = 0;
+    total_count_to_this_value = 0;
+    total_value_to_this_value = 0;
+    percentile = 0.0;
+    percentile_level_iterated_to = 0.0;
+  }
+
+  uint64_t value_iterated_to;
+  uint64_t value_iterated_from;
+  uint64_t count_at_value_iterated_to;
+  uint64_t count_added_in_this_iteration_step;
+  uint64_t total_count_to_this_value;
+  uint64_t total_value_to_this_value;
+  double percentile;
+  double percentile_level_iterated_to;
+};
+
+// Base class for iterating through histogram values.
+//
+// The underlying histogram must not be modified or destroyed while this class
+// is iterating over it.
+//
+// This class is not thread-safe.
+class AbstractHistogramIterator {
+ public:
+  // Create iterator with new histogram.
+  // The histogram must not be mutated while the iterator is in use.
+  explicit AbstractHistogramIterator(const HdrHistogram* histogram);
+  virtual ~AbstractHistogramIterator() {
+  }
+
+  // Returns true if the iteration has more elements.
+  virtual bool HasNext() const;
+
+  // Returns the next element in the iteration.
+  Status Next(HistogramIterationValue* value);
+
+  virtual double PercentileIteratedTo() const;
+  virtual double PercentileIteratedFrom() const;
+  uint64_t ValueIteratedTo() const;
+
+ protected:
+  // Implementations must override these methods.
+  virtual void IncrementIterationLevel() = 0;
+  virtual bool ReachedIterationLevel() const = 0;
+
+  const HdrHistogram* histogram_;
+  HistogramIterationValue cur_iter_val_;
+
+  uint64_t histogram_total_count_;
+
+  int current_bucket_index_;
+  int current_sub_bucket_index_;
+  uint64_t current_value_at_index_;
+
+  int next_bucket_index_;
+  int next_sub_bucket_index_;
+  uint64_t next_value_at_index_;
+
+  uint64_t prev_value_iterated_to_;
+  uint64_t total_count_to_prev_index_;
+
+  uint64_t total_count_to_current_index_;
+  uint64_t total_value_to_current_index_;
+
+  uint64_t count_at_this_value_;
+
+ private:
+  bool ExhaustedSubBuckets() const;
+  void IncrementSubBucket();
+
+  bool fresh_sub_bucket_;
+
+  DISALLOW_COPY_AND_ASSIGN(AbstractHistogramIterator);
+};
+
+// Used for iterating through all recorded histogram values using the finest
+// granularity steps supported by the underlying representation. The iteration
+// steps through all non-zero recorded value counts, and terminates when all
+// recorded histogram values are exhausted.
+//
+// The underlying histogram must not be modified or destroyed while this class
+// is iterating over it.
+//
+// This class is not thread-safe.
+class RecordedValuesIterator : public AbstractHistogramIterator {
+ public:
+  explicit RecordedValuesIterator(const HdrHistogram* histogram);
+
+ protected:
+  virtual void IncrementIterationLevel() OVERRIDE;
+  virtual bool ReachedIterationLevel() const OVERRIDE;
+
+ private:
+  int visited_sub_bucket_index_;
+  int visited_bucket_index_;
+
+  DISALLOW_COPY_AND_ASSIGN(RecordedValuesIterator);
+};
+
+// Used for iterating through histogram values according to percentile levels.
+// The iteration is performed in steps that start at 0% and reduce their
+// distance to 100% according to the percentileTicksPerHalfDistance parameter,
+// ultimately reaching 100% when all recorded histogram values are exhausted.
+//
+// The underlying histogram must not be modified or destroyed while this class
+// is iterating over it.
+//
+// This class is not thread-safe.
+class PercentileIterator : public AbstractHistogramIterator {
+ public:
+  // TODO: Explain percentile_ticks_per_half_distance.
+  PercentileIterator(const HdrHistogram* histogram,
+                     int percentile_ticks_per_half_distance);
+  virtual bool HasNext() const OVERRIDE;
+  virtual double PercentileIteratedTo() const OVERRIDE;
+  virtual double PercentileIteratedFrom() const OVERRIDE;
+
+ protected:
+  virtual void IncrementIterationLevel() OVERRIDE;
+  virtual bool ReachedIterationLevel() const OVERRIDE;
+
+ private:
+  int percentile_ticks_per_half_distance_;
+  double percentile_level_to_iterate_to_;
+  double percentile_level_to_iterate_from_;
+  bool reached_last_recorded_value_;
+
+  DISALLOW_COPY_AND_ASSIGN(PercentileIterator);
+};
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_HDRHISTOGRAM_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hexdump.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hexdump.cc b/be/src/kudu/util/hexdump.cc
new file mode 100644
index 0000000..ddecd9c
--- /dev/null
+++ b/be/src/kudu/util/hexdump.cc
@@ -0,0 +1,85 @@
+// 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 "kudu/util/hexdump.h"
+
+#include <algorithm>
+#include <cctype>
+#include <cstdint>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+std::string HexDump(const Slice &slice) {
+  if (KUDU_SHOULD_REDACT()) {
+    return kRedactionMessage;
+  }
+
+  std::string output;
+  output.reserve(slice.size() * 5);
+
+  const uint8_t *p = slice.data();
+
+  int rem = slice.size();
+  while (rem > 0) {
+    const uint8_t *line_p = p;
+    int line_len = std::min(rem, 16);
+    int line_rem = line_len;
+    StringAppendF(&output, "%06lx: ", line_p - slice.data());
+
+    while (line_rem >= 2) {
+      StringAppendF(&output, "%02x%02x ",
+                    p[0] & 0xff, p[1] & 0xff);
+      p += 2;
+      line_rem -= 2;
+    }
+
+    if (line_rem == 1) {
+      StringAppendF(&output, "%02x   ",
+                    p[0] & 0xff);
+      p += 1;
+      line_rem -= 1;
+    }
+    DCHECK_EQ(line_rem, 0);
+
+    int padding = (16 - line_len) / 2;
+
+    for (int i = 0; i < padding; i++) {
+      output.append("     ");
+    }
+
+    for (int i = 0; i < line_len; i++) {
+      char c = line_p[i];
+      if (isprint(c)) {
+        output.push_back(c);
+      } else {
+        output.push_back('.');
+      }
+    }
+
+    output.push_back('\n');
+    rem -= line_len;
+  }
+  return output;
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hexdump.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hexdump.h b/be/src/kudu/util/hexdump.h
new file mode 100644
index 0000000..eacfad2
--- /dev/null
+++ b/be/src/kudu/util/hexdump.h
@@ -0,0 +1,34 @@
+// 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.
+#ifndef KUDU_UTIL_HEXDUMP_H
+#define KUDU_UTIL_HEXDUMP_H
+
+#include <string>
+
+namespace kudu {
+
+class Slice;
+
+// Generate an 'xxd'-style hexdump of the given slice.  This should only be used
+// for debugging, as the format is subject to change and it has not been
+// implemented for speed.
+//
+// The returned string will be redacted if redaction is enabled.
+std::string HexDump(const Slice &slice);
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/high_water_mark.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/high_water_mark.h b/be/src/kudu/util/high_water_mark.h
new file mode 100644
index 0000000..dfc30e4
--- /dev/null
+++ b/be/src/kudu/util/high_water_mark.h
@@ -0,0 +1,85 @@
+// 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.
+#ifndef KUDU_UTIL_HIGH_WATER_MARK_H
+#define KUDU_UTIL_HIGH_WATER_MARK_H
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/atomic.h"
+
+namespace kudu {
+
+// Lock-free integer that keeps track of the highest value seen.
+// Similar to Impala's RuntimeProfile::HighWaterMarkCounter.
+// HighWaterMark::max_value() returns the highest value seen;
+// HighWaterMark::current_value() returns the current value.
+class HighWaterMark {
+ public:
+  explicit HighWaterMark(int64_t initial_value)
+    : current_value_(initial_value),
+      max_value_(initial_value) {
+  }
+
+  // Return the current value.
+  int64_t current_value() const {
+    return current_value_.Load(kMemOrderNoBarrier);
+  }
+
+  // Return the max value.
+  int64_t max_value() const {
+    return max_value_.Load(kMemOrderNoBarrier);
+  }
+
+  // If current value + 'delta' is <= 'max', increment current value
+  // by 'delta' and return true; return false otherwise.
+  bool TryIncrementBy(int64_t delta, int64_t max) {
+    while (true) {
+      int64_t old_val = current_value();
+      int64_t new_val = old_val + delta;
+      if (new_val > max) {
+        return false;
+      }
+      if (PREDICT_TRUE(current_value_.CompareAndSet(old_val,
+                                                    new_val,
+                                                    kMemOrderNoBarrier))) {
+        UpdateMax(new_val);
+        return true;
+      }
+    }
+  }
+
+  void IncrementBy(int64_t amount) {
+    UpdateMax(current_value_.IncrementBy(amount, kMemOrderNoBarrier));
+  }
+
+  void set_value(int64_t v) {
+    current_value_.Store(v, kMemOrderNoBarrier);
+    UpdateMax(v);
+  }
+
+ private:
+  void UpdateMax(int64_t value) {
+    max_value_.StoreMax(value, kMemOrderNoBarrier);
+  }
+
+  AtomicInt<int64_t> current_value_;
+  AtomicInt<int64_t> max_value_;
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_HIGH_WATER_MARK_H */
+
+

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/histogram.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/histogram.proto b/be/src/kudu/util/histogram.proto
new file mode 100644
index 0000000..e4526e7
--- /dev/null
+++ b/be/src/kudu/util/histogram.proto
@@ -0,0 +1,48 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+option java_package = "org.apache.kudu";
+
+// Captures the state of an Histogram.
+message HistogramSnapshotPB {
+  required string type = 1;
+  required string name = 2;
+  optional string description = 3;
+  required string unit = 4;
+  optional string label = 19;
+
+  required uint64 max_trackable_value = 5;
+  required int32 num_significant_digits = 6;
+  required uint64 total_count = 7;
+  optional uint64 total_sum = 18;
+  required uint64 min = 8;
+  required double mean = 9;
+  required uint64 percentile_75 = 10;
+  required uint64 percentile_95 = 11;
+  required uint64 percentile_99 = 12;
+  required uint64 percentile_99_9 = 13;
+  required uint64 percentile_99_99 = 14;
+  required uint64 max = 15;
+  repeated uint64 values = 16 [packed = true];
+  repeated uint64 counts = 17 [packed = true];
+}
+
+message HistogramSnapshotsListPB {
+  repeated HistogramSnapshotPB histograms = 1;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/init.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/init.cc b/be/src/kudu/util/init.cc
new file mode 100644
index 0000000..bd97d79
--- /dev/null
+++ b/be/src/kudu/util/init.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/util/init.h"
+
+#include <fcntl.h>
+#include <unistd.h>
+
+#include <cstdlib>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/cpu.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+
+Status BadCPUStatus(const base::CPU& cpu, const char* instruction_set) {
+  return Status::NotSupported(strings::Substitute(
+      "The CPU on this system ($0) does not support the $1 instruction "
+      "set which is required for running Kudu. If you are running inside a VM, "
+      "you may need to enable SSE4.2 pass-through.",
+      cpu.cpu_brand(), instruction_set));
+}
+
+bool IsFdOpen(int fd) {
+  return fcntl(fd, F_GETFL) != -1;
+}
+
+// Checks that the standard file descriptors are open when the process
+// starts.
+//
+// If these descriptors aren't open, we can run into serious issues:
+// we later might open some other files which end up reusing the same
+// file descriptor numbers as stderr, and then some library like glog
+// may decide to write a log message to what it thinks is stderr. That
+// would then overwrite one of our important data files and cause
+// corruption!
+void CheckStandardFds() {
+  if (!IsFdOpen(STDIN_FILENO) ||
+      !IsFdOpen(STDOUT_FILENO) ||
+      !IsFdOpen(STDERR_FILENO)) {
+    // We can't use LOG(FATAL) here because glog isn't initialized yet, and even if it
+    // were, it would try to write to stderr, which might end up writing the log message
+    // into some unexpected place. This is a rare enough issue that people can deal with
+    // the core dump.
+    abort();
+  }
+}
+
+Status CheckCPUFlags() {
+  base::CPU cpu;
+  if (!cpu.has_sse42()) {
+    return BadCPUStatus(cpu, "SSE4.2");
+  }
+
+  if (!cpu.has_ssse3()) {
+    return BadCPUStatus(cpu, "SSSE3");
+  }
+
+  return Status::OK();
+}
+
+void InitKuduOrDie() {
+  CheckStandardFds();
+  CHECK_OK(CheckCPUFlags());
+  // NOTE: this function is called before flags are parsed.
+  // Do not add anything in here which is flag-dependent.
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/init.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/init.h b/be/src/kudu/util/init.h
new file mode 100644
index 0000000..84e36e1
--- /dev/null
+++ b/be/src/kudu/util/init.h
@@ -0,0 +1,33 @@
+// 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.
+#ifndef KUDU_UTIL_INIT_H
+#define KUDU_UTIL_INIT_H
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Return a NotSupported Status if the current CPU does not support the CPU flags
+// required for Kudu.
+Status CheckCPUFlags();
+
+// Initialize Kudu, checking that the platform we are running on is supported, etc.
+// Issues a FATAL log message if we fail to init.
+void InitKuduOrDie();
+
+} // namespace kudu
+#endif /* KUDU_UTIL_INIT_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/inline_slice-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/inline_slice-test.cc b/be/src/kudu/util/inline_slice-test.cc
new file mode 100644
index 0000000..60a0005
--- /dev/null
+++ b/be/src/kudu/util/inline_slice-test.cc
@@ -0,0 +1,88 @@
+// 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 <cstddef>
+#include <cstdint>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/inline_slice.h"
+#include "kudu/util/memory/arena.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+template<size_t N>
+static void TestRoundTrip(InlineSlice<N> *slice,
+                          Arena *arena,
+                          size_t test_size) {
+  gscoped_ptr<uint8_t[]> buf(new uint8_t[test_size]);
+  for (int i = 0; i < test_size; i++) {
+    buf[i] = i & 0xff;
+  }
+
+  Slice test_input(buf.get(), test_size);
+
+  slice->set(test_input, arena);
+  Slice ret = slice->as_slice();
+  ASSERT_TRUE(ret == test_input)
+    << "test_size  =" << test_size << "\n"
+    << "ret        = " << ret.ToDebugString() << "\n"
+    << "test_input = " << test_input.ToDebugString();
+
+  // If the data is small enough to fit inline, then
+  // the returned slice should point directly into the
+  // InlineSlice object.
+  if (test_size < N) {
+    ASSERT_EQ(reinterpret_cast<const uint8_t *>(slice) + 1,
+              ret.data());
+  }
+}
+
+// Sweep a variety of inputs for a given size of inline
+// data
+template<size_t N>
+static void DoTest() {
+  Arena arena(1024);
+
+  // Test a range of inputs both growing and shrinking
+  InlineSlice<N> my_slice;
+  ASSERT_EQ(N, sizeof(my_slice));
+
+  for (size_t to_test = 0; to_test < 1000; to_test++) {
+    TestRoundTrip(&my_slice, &arena, to_test);
+  }
+  for (size_t to_test = 1000; to_test > 0; to_test--) {
+    TestRoundTrip(&my_slice, &arena, to_test);
+  }
+}
+
+TEST(TestInlineSlice, Test8ByteInline) {
+  DoTest<8>();
+}
+
+TEST(TestInlineSlice, Test12ByteInline) {
+  DoTest<12>();
+}
+
+TEST(TestInlineSlice, Test16ByteInline) {
+  DoTest<16>();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/inline_slice.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/inline_slice.h b/be/src/kudu/util/inline_slice.h
new file mode 100644
index 0000000..248f5b1
--- /dev/null
+++ b/be/src/kudu/util/inline_slice.h
@@ -0,0 +1,181 @@
+// 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.
+#ifndef KUDU_UTIL_INLINE_SLICE_H
+#define KUDU_UTIL_INLINE_SLICE_H
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/casts.h"
+#include "kudu/util/memory/arena.h"
+
+namespace kudu {
+
+#if __BYTE_ORDER != __LITTLE_ENDIAN
+#error This needs to be ported for big endian
+#endif
+
+// Class which represents short strings inline, and stores longer ones
+// by instead storing a pointer.
+//
+// Internal format:
+// The buffer must be at least as large as a pointer (eg 8 bytes for 64-bit).
+// Let ptr = bit-casting the first 8 bytes as a pointer:
+// If buf_[0] < 0xff:
+//   buf_[0] == length of stored data
+//   buf_[1..1 + buf_[0]] == inline data
+// If buf_[0] == 0xff:
+//   buf_[1..sizeof(uint8_t *)] == pointer to indirect data, minus the MSB.
+//   buf_[sizeof(uint8_t *)..] = unused
+//     TODO: we could store a prefix of the indirect data in this unused space
+//     in the future, which might be able to short-circuit some comparisons
+//
+// The indirect data which is pointed to is stored as a 4 byte length followed by
+// the actual data.
+//
+// This class relies on the fact that the most significant bit of any x86 pointer is
+// 0 (i.e pointers only use the bottom 48 bits)
+//
+// If ATOMIC is true, then this class has the semantics that readers will never see
+// invalid pointers, even in the case of concurrent access. However, they _may_ see
+// invalid *data*. That is to say, calling 'as_slice()' will always return a slice
+// which points to a valid memory region -- the memory region may contain garbage
+// but will not cause a segfault on access.
+//
+// These ATOMIC semantics may seem too loose to be useful, but can be used in
+// optimistic concurrency control schemes -- so long as accessing the slice doesn't
+// produce a segfault, it's OK to read bad data on a race because the higher-level
+// concurrency control will cause a retry.
+template<size_t STORAGE_SIZE, bool ATOMIC = false>
+class InlineSlice {
+ private:
+  enum {
+    kPointerByteWidth = sizeof(uintptr_t),
+    kPointerBitWidth = kPointerByteWidth * 8,
+    kMaxInlineData = STORAGE_SIZE - 1
+  };
+
+  static_assert(STORAGE_SIZE >= kPointerByteWidth,
+                "InlineSlice storage size must be greater than the width of a pointer");
+  static_assert(STORAGE_SIZE <= 256,
+                "InlineSlice storage size must be less than 256 bytes");
+ public:
+  InlineSlice() {
+  }
+
+  inline const Slice as_slice() const ATTRIBUTE_ALWAYS_INLINE {
+    DiscriminatedPointer dptr = LoadValue();
+
+    if (dptr.is_indirect()) {
+      const uint8_t *indir_data = reinterpret_cast<const uint8_t *>(dptr.pointer);
+      uint32_t len = *reinterpret_cast<const uint32_t *>(indir_data);
+      indir_data += sizeof(uint32_t);
+      return Slice(indir_data, static_cast<size_t>(len));
+    }
+    uint8_t len = dptr.discriminator;
+    DCHECK_LE(len, STORAGE_SIZE - 1);
+    return Slice(&buf_[1], len);
+  }
+
+  template<class ArenaType>
+  void set(const Slice &src, ArenaType *alloc_arena) {
+    set(src.data(), src.size(), alloc_arena);
+  }
+
+  template<class ArenaType>
+  void set(const uint8_t *src, size_t len,
+           ArenaType *alloc_arena) {
+    if (len <= kMaxInlineData) {
+      if (ATOMIC) {
+        // If atomic, we need to make sure that we store the discriminator
+        // before we copy in any data. Otherwise the data would overwrite
+        // part of a pointer and a reader might see an invalid address.
+        DiscriminatedPointer dptr;
+        dptr.discriminator = len;
+        dptr.pointer = 0; // will be overwritten
+        // "Acquire" ensures that the later memcpy doesn't reorder above the
+        // set of the discriminator bit.
+        base::subtle::Acquire_Store(reinterpret_cast<volatile AtomicWord *>(buf_),
+                                    bit_cast<uintptr_t>(dptr));
+      } else {
+        buf_[0] = len;
+      }
+      memcpy(&buf_[1], src, len);
+
+    } else {
+      // TODO: if already indirect and the current storage has enough space, just reuse that.
+
+      // Set up the pointed-to data before setting a pointer to it. This ensures that readers
+      // never see a pointer to an invalid region (i.e one without a proper length header).
+      void *in_arena = CHECK_NOTNULL(alloc_arena->AllocateBytes(len + sizeof(uint32_t)));
+      *reinterpret_cast<uint32_t *>(in_arena) = len;
+      memcpy(reinterpret_cast<uint8_t *>(in_arena) + sizeof(uint32_t), src, len);
+      set_ptr(in_arena);
+    }
+  }
+
+ private:
+  struct DiscriminatedPointer {
+    uint8_t discriminator : 8;
+    uintptr_t pointer : 54;
+
+    bool is_indirect() const {
+      return discriminator == 0xff;
+    }
+  };
+
+  DiscriminatedPointer LoadValue() const {
+    if (ATOMIC) {
+      // Load with "Acquire" semantics -- if we load a pointer, this ensures
+      // that we also see the pointed-to data.
+      uintptr_t ptr_val = base::subtle::Acquire_Load(
+        reinterpret_cast<volatile const AtomicWord *>(buf_));
+      return bit_cast<DiscriminatedPointer>(ptr_val);
+    } else {
+      DiscriminatedPointer ret;
+      memcpy(&ret, buf_, sizeof(ret));
+      return ret;
+    }
+  }
+
+  // Set the internal storage to be an indirect pointer to the given
+  // address.
+  void set_ptr(void *ptr) {
+    uintptr_t ptr_int = reinterpret_cast<uintptr_t>(ptr);
+    DCHECK_EQ(ptr_int >> (kPointerBitWidth - 8), 0) <<
+      "bad pointer (should have 0x00 MSB): " << ptr;
+
+    DiscriminatedPointer dptr;
+    dptr.discriminator = 0xff;
+    dptr.pointer = ptr_int;
+
+    if (ATOMIC) {
+      // Store with "Release" semantics -- this ensures that the pointed-to data
+      // is visible to any readers who see this pointer.
+      uintptr_t to_store = bit_cast<uintptr_t>(dptr);
+      base::subtle::Release_Store(reinterpret_cast<volatile AtomicWord *>(buf_),
+                                  to_store);
+    } else {
+      memcpy(&buf_[0], &dptr, sizeof(dptr));
+    }
+  }
+
+  uint8_t buf_[STORAGE_SIZE];
+
+} PACKED;
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/int128-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/int128-test.cc b/be/src/kudu/util/int128-test.cc
new file mode 100644
index 0000000..cc1e174
--- /dev/null
+++ b/be/src/kudu/util/int128-test.cc
@@ -0,0 +1,69 @@
+// 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 <cstddef>
+#include <cstdint>
+#include <iosfwd>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/int128.h"
+#include "kudu/util/int128_util.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(TestInt128, TestOstreamSigned) {
+  int128_t INTEGERS[] = {0, -1, 1, -1234567890,
+                         INT64_MIN, UINT64_MAX,
+                         INT128_MIN,
+                         INT128_MAX};
+  std::string STRINGS[] = {"0", "-1", "1", "-1234567890",
+                           "-9223372036854775808", "18446744073709551615",
+                           "-170141183460469231731687303715884105728",
+                           "170141183460469231731687303715884105727"};
+  for (size_t i = 0; i < arraysize(INTEGERS); i++) {
+    std::ostringstream ss;
+    ss << INTEGERS[i];
+    ASSERT_EQ(STRINGS[i], ss.str());
+  }
+}
+
+TEST(TestInt128, TestOstreamUnsigned) {
+  uint128_t INTEGERS[] = {0, 1, 1234567890,
+                          UINT128_MIN, UINT128_MAX};
+  string STRINGS[] = {"0", "1", "1234567890",
+                      "0", "340282366920938463463374607431768211455"};
+  for (size_t i = 0; i < arraysize(INTEGERS); i++) {
+    std::ostringstream ss;
+    ss << INTEGERS[i];
+    ASSERT_EQ(STRINGS[i], ss.str());
+  }
+}
+
+TEST(TestInt128, TestCasting) {
+  uint128_t mathToMax = (static_cast<uint128_t>(INT128_MAX) * 2) + 1;
+  ASSERT_EQ(UINT128_MAX, mathToMax);
+
+  uint128_t castToMax = static_cast<uint128_t>(-1);
+  ASSERT_EQ(UINT128_MAX, castToMax);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/int128.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/int128.h b/be/src/kudu/util/int128.h
new file mode 100644
index 0000000..ac35d08
--- /dev/null
+++ b/be/src/kudu/util/int128.h
@@ -0,0 +1,46 @@
+// 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 the central location for defining the int128 type
+// used by Kudu. Though this file is small it ensures flexibility
+// as choices and standards around int128 change.
+#pragma once
+
+// __int128 is not supported before gcc 4.6
+#if defined(__clang__) || \
+  (defined(__GNUC__) && \
+  (__GNUC__ * 10000 + __GNUC_MINOR__ * 100) >= 40600)
+#define KUDU_INT128_SUPPORTED 1
+#else
+#define KUDU_INT128_SUPPORTED 0
+#endif
+
+#if KUDU_INT128_SUPPORTED
+namespace kudu {
+
+typedef unsigned __int128 uint128_t;
+typedef signed __int128 int128_t;
+
+// Note: We don't use numeric_limits because it can give incorrect
+// values for __int128 and unsigned __int128.
+static const uint128_t UINT128_MIN = (uint128_t) 0;
+static const uint128_t UINT128_MAX = ((uint128_t) -1);
+static const int128_t INT128_MAX = ((int128_t)(UINT128_MAX >> 1));
+static const int128_t INT128_MIN = (-INT128_MAX - 1);
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/int128_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/int128_util.h b/be/src/kudu/util/int128_util.h
new file mode 100644
index 0000000..2d01de7
--- /dev/null
+++ b/be/src/kudu/util/int128_util.h
@@ -0,0 +1,39 @@
+// 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 "kudu/util/int128.h"
+
+#include <iostream>
+#include <string>
+
+#include "kudu/gutil/strings/numbers.h"
+
+namespace std {
+
+// Support the << operator on int128_t and uint128_t types.
+//
+inline std::ostream& operator<<(std::ostream& os, const __int128& val) {
+  os << SimpleItoa(val);
+  return os;
+}
+inline std::ostream& operator<<(std::ostream& os, const unsigned __int128& val) {
+  os << SimpleItoa(val);
+  return os;
+}
+
+} // namespace std
+

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/interval_tree-inl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/interval_tree-inl.h b/be/src/kudu/util/interval_tree-inl.h
new file mode 100644
index 0000000..7637317
--- /dev/null
+++ b/be/src/kudu/util/interval_tree-inl.h
@@ -0,0 +1,444 @@
+// 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.
+#ifndef KUDU_UTIL_INTERVAL_TREE_INL_H
+#define KUDU_UTIL_INTERVAL_TREE_INL_H
+
+#include <algorithm>
+#include <vector>
+
+#include "kudu/util/interval_tree.h"
+
+namespace kudu {
+
+template<class Traits>
+IntervalTree<Traits>::IntervalTree(const IntervalVector &intervals)
+  : root_(NULL) {
+  if (!intervals.empty()) {
+    root_ = CreateNode(intervals);
+  }
+}
+
+template<class Traits>
+IntervalTree<Traits>::~IntervalTree() {
+  delete root_;
+}
+
+template<class Traits>
+template<class QueryPointType>
+void IntervalTree<Traits>::FindContainingPoint(const QueryPointType &query,
+                                               IntervalVector *results) const {
+  if (root_) {
+    root_->FindContainingPoint(query, results);
+  }
+}
+
+template<class Traits>
+template<class Callback, class QueryContainer>
+void IntervalTree<Traits>::ForEachIntervalContainingPoints(
+    const QueryContainer& queries,
+    const Callback& cb) const {
+  if (root_) {
+    root_->ForEachIntervalContainingPoints(queries.begin(), queries.end(), cb);
+  }
+}
+
+
+template<class Traits>
+void IntervalTree<Traits>::FindIntersectingInterval(const interval_type &query,
+                                                    IntervalVector *results) const {
+  if (root_) {
+    root_->FindIntersectingInterval(query, results);
+  }
+}
+
+template<class Traits>
+static bool LessThan(const typename Traits::point_type &a,
+                     const typename Traits::point_type &b) {
+  return Traits::compare(a, b) < 0;
+}
+
+// Select a split point which attempts to evenly divide 'in' into three groups:
+//  (a) those that are fully left of the split point
+//  (b) those that overlap the split point.
+//  (c) those that are fully right of the split point
+// These three groups are stored in the output parameters '*left', '*overlapping',
+// and '*right', respectively. The selected split point is stored in *split_point.
+//
+// For example, the input interval set:
+//
+//   |------1-------|         |-----2-----|
+//       |--3--|    |---4--|    |----5----|
+//                     |
+// Resulting split:    | Partition point
+//                     |
+//
+// *left: intervals 1 and 3
+// *overlapping: interval 4
+// *right: intervals 2 and 5
+template<class Traits>
+void IntervalTree<Traits>::Partition(const IntervalVector &in,
+                                     point_type *split_point,
+                                     IntervalVector *left,
+                                     IntervalVector *overlapping,
+                                     IntervalVector *right) {
+  CHECK(!in.empty());
+
+  // Pick a split point which is the median of all of the interval boundaries.
+  std::vector<point_type> endpoints;
+  endpoints.reserve(in.size() * 2);
+  for (const interval_type &interval : in) {
+    endpoints.push_back(Traits::get_left(interval));
+    endpoints.push_back(Traits::get_right(interval));
+  }
+  std::sort(endpoints.begin(), endpoints.end(), LessThan<Traits>);
+  *split_point = endpoints[endpoints.size() / 2];
+
+  // Partition into the groups based on the determined split point.
+  for (const interval_type &interval : in) {
+    if (Traits::compare(Traits::get_right(interval), *split_point) < 0) {
+      //                 | split point
+      // |------------|  |
+      //    interval
+      left->push_back(interval);
+    } else if (Traits::compare(Traits::get_left(interval), *split_point) > 0) {
+      //                 | split point
+      //                 |    |------------|
+      //                         interval
+      right->push_back(interval);
+    } else {
+      //                 | split point
+      //                 |
+      //          |------------|
+      //             interval
+      overlapping->push_back(interval);
+    }
+  }
+}
+
+template<class Traits>
+typename IntervalTree<Traits>::node_type *IntervalTree<Traits>::CreateNode(
+  const IntervalVector &intervals) {
+  IntervalVector left, right, overlap;
+  point_type split_point;
+
+  // First partition the input intervals and select a split point
+  Partition(intervals, &split_point, &left, &overlap, &right);
+
+  // Recursively subdivide the intervals which are fully left or fully
+  // right of the split point into subtree nodes.
+  node_type *left_node = !left.empty() ? CreateNode(left) : NULL;
+  node_type *right_node = !right.empty() ? CreateNode(right) : NULL;
+
+  return new node_type(split_point, left_node, overlap, right_node);
+}
+
+namespace interval_tree_internal {
+
+// Node in the interval tree.
+template<typename Traits>
+class ITNode {
+ private:
+  // Import types.
+  typedef std::vector<typename Traits::interval_type> IntervalVector;
+  typedef typename Traits::interval_type interval_type;
+  typedef typename Traits::point_type point_type;
+
+ public:
+  ITNode(point_type split_point,
+         ITNode<Traits> *left,
+         const IntervalVector &overlap,
+         ITNode<Traits> *right);
+  ~ITNode();
+
+  // See IntervalTree::FindContainingPoint(...)
+  template<class QueryPointType>
+  void FindContainingPoint(const QueryPointType &query,
+                           IntervalVector *results) const;
+
+  // See IntervalTree::ForEachIntervalContainingPoints().
+  // We use iterators here since as recursion progresses down the tree, we
+  // process sub-sequences of the original set of query points.
+  template<class Callback, class ItType>
+  void ForEachIntervalContainingPoints(ItType begin_queries,
+                                       ItType end_queries,
+                                       const Callback& cb) const;
+
+  // See IntervalTree::FindIntersectingInterval(...)
+  void FindIntersectingInterval(const interval_type &query,
+                                IntervalVector *results) const;
+
+ private:
+  // Comparators for sorting lists of intervals.
+  static bool SortByAscLeft(const interval_type &a, const interval_type &b);
+  static bool SortByDescRight(const interval_type &a, const interval_type &b);
+
+  // Partition point of this node.
+  point_type split_point_;
+
+  // Those nodes that overlap with split_point_, in ascending order by their left side.
+  IntervalVector overlapping_by_asc_left_;
+
+  // Those nodes that overlap with split_point_, in descending order by their right side.
+  IntervalVector overlapping_by_desc_right_;
+
+  // Tree node for intervals fully left of split_point_, or NULL.
+  ITNode *left_;
+
+  // Tree node for intervals fully right of split_point_, or NULL.
+  ITNode *right_;
+
+  DISALLOW_COPY_AND_ASSIGN(ITNode);
+};
+
+template<class Traits>
+bool ITNode<Traits>::SortByAscLeft(const interval_type &a, const interval_type &b) {
+  return Traits::compare(Traits::get_left(a), Traits::get_left(b)) < 0;
+}
+
+template<class Traits>
+bool ITNode<Traits>::SortByDescRight(const interval_type &a, const interval_type &b) {
+  return Traits::compare(Traits::get_right(a), Traits::get_right(b)) > 0;
+}
+
+template <class Traits>
+ITNode<Traits>::ITNode(typename Traits::point_type split_point,
+                       ITNode<Traits> *left, const IntervalVector &overlap,
+                       ITNode<Traits> *right)
+    : split_point_(std::move(split_point)), left_(left), right_(right) {
+  // Store two copies of the set of intervals which overlap the split point:
+  // 1) Sorted by ascending left boundary
+  overlapping_by_asc_left_.assign(overlap.begin(), overlap.end());
+  std::sort(overlapping_by_asc_left_.begin(), overlapping_by_asc_left_.end(), SortByAscLeft);
+  // 2) Sorted by descending right boundary
+  overlapping_by_desc_right_.assign(overlap.begin(), overlap.end());
+  std::sort(overlapping_by_desc_right_.begin(), overlapping_by_desc_right_.end(), SortByDescRight);
+}
+
+template<class Traits>
+ITNode<Traits>::~ITNode() {
+  if (left_) delete left_;
+  if (right_) delete right_;
+}
+
+template<class Traits>
+template<class Callback, class ItType>
+void ITNode<Traits>::ForEachIntervalContainingPoints(ItType begin_queries,
+                                                     ItType end_queries,
+                                                     const Callback& cb) const {
+  if (begin_queries == end_queries) return;
+
+  typedef decltype(*begin_queries) QueryPointType;
+  const auto& partitioner = [&](const QueryPointType& query_point) {
+    return Traits::compare(query_point, split_point_) < 0;
+  };
+
+  // Partition the query points into those less than the split_point_ and those greater
+  // than or equal to the split_point_. Because the input queries are already sorted, we
+  // can use 'std::partition_point' instead of 'std::partition'.
+  //
+  // The resulting 'partition_point' is the first query point in the second group.
+  //
+  // Complexity: O(log(number of query points))
+  DCHECK(std::is_partitioned(begin_queries, end_queries, partitioner));
+  auto partition_point = std::partition_point(begin_queries, end_queries, partitioner);
+
+  // Recurse left: any query points left of the split point may intersect
+  // with non-overlapping intervals fully-left of our split point.
+  if (left_ != NULL) {
+    left_->ForEachIntervalContainingPoints(begin_queries, partition_point, cb);
+  }
+
+  // Handle the query points < split_point
+  //
+  //      split_point_
+  //         |
+  //   [------]         \
+  //     [-------]       | overlapping_by_asc_left_
+  //       [--------]   /
+  // Q   Q      Q
+  // ^   ^      \___ not handled (right of split_point_)
+  // |   |
+  // \___\___ these points will be handled here
+  //
+
+  // Lower bound of query points still relevant.
+  auto rem_queries = begin_queries;
+  for (const interval_type &interval : overlapping_by_asc_left_) {
+    const auto& interval_left = Traits::get_left(interval);
+    // Find those query points which are right of the left side of the interval.
+    // 'first_match' here is the first query point >= interval_left.
+    // Complexity: O(log(num_queries))
+    //
+    // TODO(todd): The non-batched implementation is O(log(num_intervals) * num_queries)
+    // whereas this loop ends up O(num_intervals * log(num_queries)). So, for
+    // small numbers of queries this is not the fastest way to structure these loops.
+    auto first_match = std::partition_point(
+        rem_queries, partition_point,
+        [&](const QueryPointType& query_point) {
+          return Traits::compare(query_point, interval_left) < 0;
+        });
+    for (auto it = first_match; it != partition_point; ++it) {
+      cb(*it, interval);
+    }
+    // Since the intervals are sorted in ascending-left order, we can start
+    // the search for the next interval at the first match in this interval.
+    // (any query point which was left of the current interval will also be left
+    // of all future intervals).
+    rem_queries = std::move(first_match);
+  }
+
+  // Handle the query points >= split_point
+  //
+  //    split_point_
+  //       |
+  //     [--------]   \
+  //   [-------]       | overlapping_by_desc_right_
+  // [------]         /
+  //   Q   Q      Q
+  //   |    \______\___ these points will be handled here
+  //   |
+  //   \___ not handled (left of split_point_)
+
+  // Upper bound of query points still relevant.
+  rem_queries = end_queries;
+  for (const interval_type &interval : overlapping_by_desc_right_) {
+    const auto& interval_right = Traits::get_right(interval);
+    // Find the first query point which is > the right side of the interval.
+    auto first_non_match = std::partition_point(
+        partition_point, rem_queries,
+        [&](const QueryPointType& query_point) {
+          return Traits::compare(query_point, interval_right) <= 0;
+          });
+    for (auto it = partition_point; it != first_non_match; ++it) {
+      cb(*it, interval);
+    }
+    // Same logic as above: if a query point was fully right of 'interval',
+    // then it will be fully right of all following intervals because they are
+    // sorted by descending-right.
+    rem_queries = std::move(first_non_match);
+  }
+
+  if (right_ != NULL) {
+    while (partition_point != end_queries &&
+           Traits::compare(*partition_point, split_point_) == 0) {
+      ++partition_point;
+    }
+    right_->ForEachIntervalContainingPoints(partition_point, end_queries, cb);
+  }
+}
+
+template<class Traits>
+template<class QueryPointType>
+void ITNode<Traits>::FindContainingPoint(const QueryPointType &query,
+                                         IntervalVector *results) const {
+  int cmp = Traits::compare(query, split_point_);
+  if (cmp < 0) {
+    // None of the intervals in right_ may intersect this.
+    if (left_ != NULL) {
+      left_->FindContainingPoint(query, results);
+    }
+
+    // Any intervals which start before the query point and overlap the split point
+    // must therefore contain the query point.
+    auto p = std::partition_point(
+        overlapping_by_asc_left_.cbegin(), overlapping_by_asc_left_.cend(),
+        [&](const interval_type& interval) {
+          return Traits::compare(Traits::get_left(interval), query) <= 0;
+        });
+    results->insert(results->end(), overlapping_by_asc_left_.cbegin(), p);
+  } else if (cmp > 0) {
+    // None of the intervals in left_ may intersect this.
+    if (right_ != NULL) {
+      right_->FindContainingPoint(query, results);
+    }
+
+    // Any intervals which end after the query point and overlap the split point
+    // must therefore contain the query point.
+    auto p = std::partition_point(
+        overlapping_by_desc_right_.cbegin(), overlapping_by_desc_right_.cend(),
+        [&](const interval_type& interval) {
+          return Traits::compare(Traits::get_right(interval), query) >= 0;
+        });
+    results->insert(results->end(), overlapping_by_desc_right_.cbegin(), p);
+  } else {
+    DCHECK_EQ(cmp, 0);
+    // The query is exactly our split point -- in this case we've already got
+    // the computed list of overlapping intervals.
+    results->insert(results->end(), overlapping_by_asc_left_.begin(),
+                    overlapping_by_asc_left_.end());
+  }
+}
+
+template<class Traits>
+void ITNode<Traits>::FindIntersectingInterval(const interval_type &query,
+                                              IntervalVector *results) const {
+  if (Traits::compare(Traits::get_right(query), split_point_) < 0) {
+    // The interval is fully left of the split point. So, it may not overlap
+    // with any in 'right_'
+    if (left_ != NULL) {
+      left_->FindIntersectingInterval(query, results);
+    }
+
+    // Any intervals whose left edge is <= the query interval's right edge
+    // intersect the query interval. 'std::partition_point' returns the first
+    // such interval which does not meet that criterion, so we insert all
+    // up to that point.
+    auto first_greater = std::partition_point(
+        overlapping_by_asc_left_.cbegin(), overlapping_by_asc_left_.cend(),
+        [&](const interval_type& interval) {
+          return Traits::compare(Traits::get_left(interval), Traits::get_right(query)) <= 0;
+        });
+    results->insert(results->end(), overlapping_by_asc_left_.cbegin(), first_greater);
+  } else if (Traits::compare(Traits::get_left(query), split_point_) > 0) {
+    // The interval is fully right of the split point. So, it may not overlap
+    // with any in 'left_'.
+    if (right_ != NULL) {
+      right_->FindIntersectingInterval(query, results);
+    }
+
+    // Any intervals whose right edge is >= the query interval's left edge
+    // intersect the query interval. 'std::partition_point' returns the first
+    // such interval which does not meet that criterion, so we insert all
+    // up to that point.
+    auto first_lesser = std::partition_point(
+        overlapping_by_desc_right_.cbegin(), overlapping_by_desc_right_.cend(),
+        [&](const interval_type& interval) {
+          return Traits::compare(Traits::get_right(interval), Traits::get_left(query)) >= 0;
+        });
+    results->insert(results->end(), overlapping_by_desc_right_.cbegin(), first_lesser);
+  } else {
+    // The query interval contains the split point. Therefore all other intervals
+    // which also contain the split point are intersecting.
+    results->insert(results->end(), overlapping_by_asc_left_.begin(),
+                    overlapping_by_asc_left_.end());
+
+    // The query interval may _also_ intersect some in either child.
+    if (left_ != NULL) {
+      left_->FindIntersectingInterval(query, results);
+    }
+    if (right_ != NULL) {
+      right_->FindIntersectingInterval(query, results);
+    }
+  }
+}
+
+
+} // namespace interval_tree_internal
+
+} // namespace kudu
+
+#endif


[18/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/interval_tree-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/interval_tree-test.cc b/be/src/kudu/util/interval_tree-test.cc
new file mode 100644
index 0000000..df143d3
--- /dev/null
+++ b/be/src/kudu/util/interval_tree-test.cc
@@ -0,0 +1,353 @@
+// 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.
+
+// All rights reserved.
+
+#include <algorithm>
+#include <cstdlib>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <tuple>  // IWYU pragma: keep
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/interval_tree.h"
+#include "kudu/util/interval_tree-inl.h"
+#include "kudu/util/test_util.h"
+
+using std::pair;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+// Test harness.
+class TestIntervalTree : public KuduTest {
+};
+
+// Simple interval class for integer intervals.
+struct IntInterval {
+  IntInterval(int left, int right, int id = -1)
+      : left(left),
+        right(right),
+        id(id) {
+  }
+
+  bool Intersects(const IntInterval &other) const {
+    if (other.left > right) return false;
+    if (left > other.right) return false;
+    return true;
+  }
+
+  string ToString() const {
+    return strings::Substitute("[$0, $1]($2) ", left, right, id);
+  }
+
+  int left, right, id;
+};
+
+// A wrapper around an int which can be compared with IntTraits::compare()
+// but also can keep a counter of how many times it has been compared. Used
+// for TestBigO below.
+struct CountingQueryPoint {
+  explicit CountingQueryPoint(int v)
+      : val(v),
+        count(new int(0)) {
+  }
+
+  int val;
+  std::shared_ptr<int> count;
+};
+
+// Traits definition for intervals made up of ints on either end.
+struct IntTraits {
+  typedef int point_type;
+  typedef IntInterval interval_type;
+  static point_type get_left(const IntInterval &x) {
+    return x.left;
+  }
+  static point_type get_right(const IntInterval &x) {
+    return x.right;
+  }
+  static int compare(int a, int b) {
+    if (a < b) return -1;
+    if (a > b) return 1;
+    return 0;
+  }
+
+  static int compare(const CountingQueryPoint& q, int b) {
+    (*q.count)++;
+    return compare(q.val, b);
+  }
+  static int compare(int a, const CountingQueryPoint& b) {
+    return -compare(b, a);
+  }
+
+};
+
+// Compare intervals in an arbitrary but consistent way - this is only
+// used for verifying that the two algorithms come up with the same results.
+// It's not necessary to define this to use an interval tree.
+static bool CompareIntervals(const IntInterval &a, const IntInterval &b) {
+  return std::make_tuple(a.left, a.right, a.id) <
+    std::make_tuple(b.left, b.right, b.id);
+}
+
+// Stringify a list of int intervals, for easy test error reporting.
+static string Stringify(const vector<IntInterval> &intervals) {
+  string ret;
+  bool first = true;
+  for (const IntInterval &interval : intervals) {
+    if (!first) {
+      ret.append(",");
+    }
+    ret.append(interval.ToString());
+  }
+  return ret;
+}
+
+// Find any intervals in 'intervals' which contain 'query_point' by brute force.
+static void FindContainingBruteForce(const vector<IntInterval> &intervals,
+                                     int query_point,
+                                     vector<IntInterval> *results) {
+  for (const IntInterval &i : intervals) {
+    if (query_point >= i.left && query_point <= i.right) {
+      results->push_back(i);
+    }
+  }
+}
+
+
+// Find any intervals in 'intervals' which intersect 'query_interval' by brute force.
+static void FindIntersectingBruteForce(const vector<IntInterval> &intervals,
+                                       IntInterval query_interval,
+                                       vector<IntInterval> *results) {
+  for (const IntInterval &i : intervals) {
+    if (query_interval.Intersects(i)) {
+      results->push_back(i);
+    }
+  }
+}
+
+
+// Verify that IntervalTree::FindContainingPoint yields the same results as the naive
+// brute-force O(n) algorithm.
+static void VerifyFindContainingPoint(const vector<IntInterval> all_intervals,
+                                      const IntervalTree<IntTraits> &tree,
+                                      int query_point) {
+  vector<IntInterval> results;
+  tree.FindContainingPoint(query_point, &results);
+  std::sort(results.begin(), results.end(), CompareIntervals);
+
+  vector<IntInterval> brute_force;
+  FindContainingBruteForce(all_intervals, query_point, &brute_force);
+  std::sort(brute_force.begin(), brute_force.end(), CompareIntervals);
+
+  SCOPED_TRACE(Stringify(all_intervals) + StringPrintf(" (q=%d)", query_point));
+  EXPECT_EQ(Stringify(brute_force), Stringify(results));
+}
+
+// Verify that IntervalTree::FindIntersectingInterval yields the same results as the naive
+// brute-force O(n) algorithm.
+static void VerifyFindIntersectingInterval(const vector<IntInterval> all_intervals,
+                                           const IntervalTree<IntTraits> &tree,
+                                           const IntInterval &query_interval) {
+  vector<IntInterval> results;
+  tree.FindIntersectingInterval(query_interval, &results);
+  std::sort(results.begin(), results.end(), CompareIntervals);
+
+  vector<IntInterval> brute_force;
+  FindIntersectingBruteForce(all_intervals, query_interval, &brute_force);
+  std::sort(brute_force.begin(), brute_force.end(), CompareIntervals);
+
+  SCOPED_TRACE(Stringify(all_intervals) +
+               StringPrintf(" (q=[%d,%d])", query_interval.left, query_interval.right));
+  EXPECT_EQ(Stringify(brute_force), Stringify(results));
+}
+
+static vector<IntInterval> CreateRandomIntervals(int n = 100) {
+  vector<IntInterval> intervals;
+  for (int i = 0; i < n; i++) {
+    int l = rand() % 100; // NOLINT(runtime/threadsafe_fn)
+    int r = l + rand() % 20; // NOLINT(runtime/threadsafe_fn)
+    intervals.emplace_back(l, r, i);
+  }
+  return intervals;
+}
+
+TEST_F(TestIntervalTree, TestBasic) {
+  vector<IntInterval> intervals;
+  intervals.emplace_back(1, 2, 1);
+  intervals.emplace_back(3, 4, 2);
+  intervals.emplace_back(1, 4, 3);
+  IntervalTree<IntTraits> t(intervals);
+
+  for (int i = 0; i <= 5; i++) {
+    VerifyFindContainingPoint(intervals, t, i);
+
+    for (int j = i; j <= 5; j++) {
+      VerifyFindIntersectingInterval(intervals, t, IntInterval(i, j, 0));
+    }
+  }
+}
+
+TEST_F(TestIntervalTree, TestRandomized) {
+  SeedRandom();
+
+  // Generate 100 random intervals spanning 0-200 and build an interval tree from them.
+  vector<IntInterval> intervals = CreateRandomIntervals();
+  IntervalTree<IntTraits> t(intervals);
+
+  // Test that we get the correct result on every possible query.
+  for (int i = -1; i < 201; i++) {
+    VerifyFindContainingPoint(intervals, t, i);
+  }
+
+  // Test that we get the correct result for random intervals
+  for (int i = 0; i < 100; i++) {
+    int l = rand() % 100; // NOLINT(runtime/threadsafe_fn)
+    int r = l + rand() % 100; // NOLINT(runtime/threadsafe_fn)
+    VerifyFindIntersectingInterval(intervals, t, IntInterval(l, r));
+  }
+}
+
+TEST_F(TestIntervalTree, TestEmpty) {
+  vector<IntInterval> empty;
+  IntervalTree<IntTraits> t(empty);
+
+  VerifyFindContainingPoint(empty, t, 1);
+  VerifyFindIntersectingInterval(empty, t, IntInterval(1, 2, 0));
+}
+
+TEST_F(TestIntervalTree, TestBigO) {
+#ifndef NDEBUG
+  LOG(WARNING) << "big-O results are not valid if DCHECK is enabled";
+  return;
+#endif
+  SeedRandom();
+
+  LOG(INFO) << "num_int\tnum_q\tresults\tsimple\tbatch";
+  for (int num_intervals = 1; num_intervals < 2000; num_intervals *= 2) {
+    vector<IntInterval> intervals = CreateRandomIntervals(num_intervals);
+    IntervalTree<IntTraits> t(intervals);
+    for (int num_queries = 1; num_queries < 2000; num_queries *= 2) {
+      vector<CountingQueryPoint> queries;
+      for (int i = 0; i < num_queries; i++) {
+        queries.emplace_back(rand() % 100);
+      }
+      std::sort(queries.begin(), queries.end(),
+                [](const CountingQueryPoint& a,
+                   const CountingQueryPoint& b) {
+                  return a.val < b.val;
+                });
+
+      // Test using batch algorithm.
+      int num_results_batch = 0;
+      t.ForEachIntervalContainingPoints(
+          queries,
+          [&](CountingQueryPoint query_point, const IntInterval& interval) {
+            num_results_batch++;
+          });
+      int num_comparisons_batch = 0;
+      for (const auto& q : queries) {
+        num_comparisons_batch += *q.count;
+        *q.count = 0;
+      }
+
+      // Test using one-by-one queries.
+      int num_results_simple = 0;
+      for (auto& q : queries) {
+        vector<IntInterval> intervals;
+        t.FindContainingPoint(q, &intervals);
+        num_results_simple += intervals.size();
+      }
+      int num_comparisons_simple = 0;
+      for (const auto& q : queries) {
+        num_comparisons_simple += *q.count;
+      }
+      ASSERT_EQ(num_results_simple, num_results_batch);
+
+      LOG(INFO) << num_intervals << "\t" << num_queries << "\t" << num_results_simple << "\t"
+                << num_comparisons_simple << "\t" << num_comparisons_batch;
+    }
+  }
+}
+
+TEST_F(TestIntervalTree, TestMultiQuery) {
+  SeedRandom();
+  const int kNumQueries = 1;
+  vector<IntInterval> intervals = CreateRandomIntervals(10);
+  IntervalTree<IntTraits> t(intervals);
+
+  // Generate random queries.
+  vector<int> queries;
+  for (int i = 0; i < kNumQueries; i++) {
+    queries.push_back(rand() % 100);
+  }
+  std::sort(queries.begin(), queries.end());
+
+  vector<pair<string, int>> results_simple;
+  for (int q : queries) {
+    vector<IntInterval> intervals;
+    t.FindContainingPoint(q, &intervals);
+    for (const auto& interval : intervals) {
+      results_simple.emplace_back(interval.ToString(), q);
+    }
+  }
+
+  vector<pair<string, int>> results_batch;
+  t.ForEachIntervalContainingPoints(
+      queries,
+      [&](int query_point, const IntInterval& interval) {
+        results_batch.emplace_back(interval.ToString(), query_point);
+      });
+
+  // Check the property that, when the batch query points are in sorted order,
+  // the results are grouped by interval, and within each interval, sorted by
+  // query point. Each interval may have at most two groups.
+  boost::optional<pair<string, int>> prev = boost::none;
+  std::map<string, int> intervals_seen;
+  for (int i = 0; i < results_batch.size(); i++) {
+    const auto& cur = results_batch[i];
+    // If it's another query point hitting the same interval,
+    // make sure the query points are returned in order.
+    if (prev && prev->first == cur.first) {
+      EXPECT_GE(cur.second, prev->second) << prev->first;
+    } else {
+      // It's the start of a new interval's data. Make sure that we don't
+      // see the same interval twice.
+      EXPECT_LE(++intervals_seen[cur.first], 2)
+          << "Saw more than two groups for interval " << cur.first;
+    }
+    prev = cur;
+  }
+
+  std::sort(results_simple.begin(), results_simple.end());
+  std::sort(results_batch.begin(), results_batch.end());
+  ASSERT_EQ(results_simple, results_batch);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/interval_tree.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/interval_tree.h b/be/src/kudu/util/interval_tree.h
new file mode 100644
index 0000000..a677528
--- /dev/null
+++ b/be/src/kudu/util/interval_tree.h
@@ -0,0 +1,158 @@
+// 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.
+//
+// Implements an Interval Tree. See http://en.wikipedia.org/wiki/Interval_tree
+// or CLRS for a full description of the data structure.
+//
+// Callers of this class should also include interval_tree-inl.h for function
+// definitions.
+#ifndef KUDU_UTIL_INTERVAL_TREE_H
+#define KUDU_UTIL_INTERVAL_TREE_H
+
+#include <glog/logging.h>
+
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+namespace interval_tree_internal {
+template<class Traits>
+class ITNode;
+}
+
+// Implements an Interval Tree.
+//
+// An Interval Tree is a data structure which stores a set of intervals and supports
+// efficient searches to determine which intervals in that set overlap a query
+// point or interval. These operations are O(lg n + k) where 'n' is the number of
+// intervals in the tree and 'k' is the number of results returned for a given query.
+//
+// This particular implementation is a static tree -- intervals may not be added or
+// removed once the tree is instantiated.
+//
+// This class also assumes that all intervals are "closed" intervals -- the intervals
+// are inclusive of their start and end points.
+//
+// The Traits class should have the following members:
+//   Traits::point_type
+//     a typedef for what a "point" in the range is
+//
+//   Traits::interval_type
+//     a typedef for an interval
+//
+//   static point_type get_left(const interval_type &)
+//   static point_type get_right(const interval_type &)
+//     accessors which fetch the left and right bound of the interval, respectively.
+//
+//   static int compare(const point_type &a, const point_type &b)
+//     return < 0 if a < b, 0 if a == b, > 0 if a > b
+//
+// See interval_tree-test.cc for an example Traits class for 'int' ranges.
+template<class Traits>
+class IntervalTree {
+ private:
+  // Import types from the traits class to make code more readable.
+  typedef typename Traits::interval_type interval_type;
+  typedef typename Traits::point_type point_type;
+
+  // And some convenience types.
+  typedef std::vector<interval_type> IntervalVector;
+  typedef interval_tree_internal::ITNode<Traits> node_type;
+
+ public:
+  // Construct an Interval Tree containing the given set of intervals.
+  explicit IntervalTree(const IntervalVector &intervals);
+
+  ~IntervalTree();
+
+  // Find all intervals in the tree which contain the query point.
+  // The resulting intervals are added to the 'results' vector.
+  // The vector is not cleared first.
+  //
+  // NOTE: 'QueryPointType' is usually point_type, but can be any other
+  // type for which there exists the appropriate Traits::Compare(...) method.
+  template<class QueryPointType>
+  void FindContainingPoint(const QueryPointType &query,
+                           IntervalVector *results) const;
+
+  // For each of the query points in the STL container 'queries', find all
+  // intervals in the tree which may contain those points. Calls 'cb(point, interval)'
+  // for each such interval.
+  //
+  // The points in the query container must be comparable to 'point_type'
+  // using Traits::Compare().
+  //
+  // The implementation sequences the calls to 'cb' with the following guarantees:
+  // 1) all of the results corresponding to a given interval will be yielded in at
+  //    most two "groups" of calls (i.e. sub-sequences of calls with the same interval).
+  // 2) within each "group" of calls, the query points will be in ascending order.
+  //
+  // For example, the callback sequence may be:
+  //
+  //  cb(q1, interval_1) -
+  //  cb(q2, interval_1)  | first group of interval_1
+  //  cb(q6, interval_1)  |
+  //  cb(q7, interval_1) -
+  //
+  //  cb(q2, interval_2) -
+  //  cb(q3, interval_2)  | first group of interval_2
+  //  cb(q4, interval_2) -
+  //
+  //  cb(q3, interval_1) -
+  //  cb(q4, interval_1)  | second group of interval_1
+  //  cb(q5, interval_1) -
+  //
+  //  cb(q2, interval_3) -
+  //  cb(q3, interval_3)  | first group of interval_3
+  //  cb(q4, interval_3) -
+  //
+  //  cb(q5, interval_2) -
+  //  cb(q6, interval_2)  | second group of interval_2
+  //  cb(q7, interval_2) -
+  //
+  // REQUIRES: The input points must be pre-sorted or else this will return invalid
+  // results.
+  template<class Callback, class QueryContainer>
+  void ForEachIntervalContainingPoints(const QueryContainer& queries,
+                                       const Callback& cb) const;
+
+  // Find all intervals in the tree which intersect the given interval.
+  // The resulting intervals are added to the 'results' vector.
+  // The vector is not cleared first.
+  void FindIntersectingInterval(const interval_type &query,
+                                IntervalVector *results) const;
+ private:
+  static void Partition(const IntervalVector &in,
+                        point_type *split_point,
+                        IntervalVector *left,
+                        IntervalVector *overlapping,
+                        IntervalVector *right);
+
+  // Create a node containing the given intervals, recursively splitting down the tree.
+  static node_type *CreateNode(const IntervalVector &intervals);
+
+  node_type *root_;
+
+  DISALLOW_COPY_AND_ASSIGN(IntervalTree);
+};
+
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonreader-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonreader-test.cc b/be/src/kudu/util/jsonreader-test.cc
new file mode 100644
index 0000000..9f62c31
--- /dev/null
+++ b/be/src/kudu/util/jsonreader-test.cc
@@ -0,0 +1,193 @@
+// 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 "kudu/util/jsonreader.h"
+
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using rapidjson::Value;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+TEST(JsonReaderTest, Corrupt) {
+  JsonReader r("");
+  Status s = r.Init();
+  ASSERT_TRUE(s.IsCorruption());
+  ASSERT_STR_CONTAINS(
+      s.ToString(), "JSON text is corrupt: Text only contains white space(s)");
+}
+
+TEST(JsonReaderTest, Empty) {
+  JsonReader r("{}");
+  ASSERT_OK(r.Init());
+  JsonReader r2("[]");
+  ASSERT_OK(r2.Init());
+
+  // Not found.
+  ASSERT_TRUE(r.ExtractBool(r.root(), "foo", nullptr).IsNotFound());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "foo", nullptr).IsNotFound());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "foo", nullptr).IsNotFound());
+  ASSERT_TRUE(r.ExtractString(r.root(), "foo", nullptr).IsNotFound());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "foo", nullptr).IsNotFound());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "foo", nullptr).IsNotFound());
+}
+
+TEST(JsonReaderTest, Basic) {
+  JsonReader r("{ \"foo\" : \"bar\" }");
+  ASSERT_OK(r.Init());
+  string foo;
+  ASSERT_OK(r.ExtractString(r.root(), "foo", &foo));
+  ASSERT_EQ("bar", foo);
+
+  // Bad types.
+  ASSERT_TRUE(r.ExtractBool(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "foo", nullptr).IsInvalidArgument());
+}
+
+TEST(JsonReaderTest, LessBasic) {
+  string doc = Substitute(
+      "{ \"small\" : 1, \"big\" : $0, \"null\" : null, \"empty\" : \"\", \"bool\" : true }",
+      kint64max);
+  JsonReader r(doc);
+  ASSERT_OK(r.Init());
+  int32_t small;
+  ASSERT_OK(r.ExtractInt32(r.root(), "small", &small));
+  ASSERT_EQ(1, small);
+  int64_t big;
+  ASSERT_OK(r.ExtractInt64(r.root(), "big", &big));
+  ASSERT_EQ(kint64max, big);
+  string str;
+  ASSERT_OK(r.ExtractString(r.root(), "null", &str));
+  ASSERT_EQ("", str);
+  ASSERT_OK(r.ExtractString(r.root(), "empty", &str));
+  ASSERT_EQ("", str);
+  bool b;
+  ASSERT_OK(r.ExtractBool(r.root(), "bool", &b));
+  ASSERT_TRUE(b);
+
+  // Bad types.
+  ASSERT_TRUE(r.ExtractBool(r.root(), "small", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), "small", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "small", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "small", nullptr).IsInvalidArgument());
+
+  ASSERT_TRUE(r.ExtractBool(r.root(), "big", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "big", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), "big", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "big", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "big", nullptr).IsInvalidArgument());
+
+  ASSERT_TRUE(r.ExtractBool(r.root(), "null", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "null", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "null", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "null", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "null", nullptr).IsInvalidArgument());
+
+  ASSERT_TRUE(r.ExtractBool(r.root(), "empty", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "empty", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "empty", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "empty", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "empty", nullptr).IsInvalidArgument());
+
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "bool", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "bool", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), "bool", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "bool", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "bool", nullptr).IsInvalidArgument());
+}
+
+TEST(JsonReaderTest, Objects) {
+  JsonReader r("{ \"foo\" : { \"1\" : 1 } }");
+  ASSERT_OK(r.Init());
+
+  const Value* foo = nullptr;
+  ASSERT_OK(r.ExtractObject(r.root(), "foo", &foo));
+  ASSERT_TRUE(foo);
+
+  int32_t one;
+  ASSERT_OK(r.ExtractInt32(foo, "1", &one));
+  ASSERT_EQ(1, one);
+
+  // Bad types.
+  ASSERT_TRUE(r.ExtractBool(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObjectArray(r.root(), "foo", nullptr).IsInvalidArgument());
+}
+
+TEST(JsonReaderTest, TopLevelArray) {
+  JsonReader r("[ { \"name\" : \"foo\" }, { \"name\" : \"bar\" } ]");
+  ASSERT_OK(r.Init());
+
+  vector<const Value*> objs;
+  ASSERT_OK(r.ExtractObjectArray(r.root(), nullptr, &objs));
+  ASSERT_EQ(2, objs.size());
+  string name;
+  ASSERT_OK(r.ExtractString(objs[0], "name", &name));
+  ASSERT_EQ("foo", name);
+  ASSERT_OK(r.ExtractString(objs[1], "name", &name));
+  ASSERT_EQ("bar", name);
+
+  // Bad types.
+  ASSERT_TRUE(r.ExtractBool(r.root(), nullptr, nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), nullptr, nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), nullptr, nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), nullptr, nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), nullptr, nullptr).IsInvalidArgument());
+}
+
+TEST(JsonReaderTest, NestedArray) {
+  JsonReader r("{ \"foo\" : [ { \"val\" : 0 }, { \"val\" : 1 }, { \"val\" : 2 } ] }");
+  ASSERT_OK(r.Init());
+
+  vector<const Value*> foo;
+  ASSERT_OK(r.ExtractObjectArray(r.root(), "foo", &foo));
+  ASSERT_EQ(3, foo.size());
+  int i = 0;
+  for (const Value* v : foo) {
+    int32_t number;
+    ASSERT_OK(r.ExtractInt32(v, "val", &number));
+    ASSERT_EQ(i, number);
+    i++;
+  }
+
+  // Bad types.
+  ASSERT_TRUE(r.ExtractBool(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt32(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractInt64(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractString(r.root(), "foo", nullptr).IsInvalidArgument());
+  ASSERT_TRUE(r.ExtractObject(r.root(), "foo", nullptr).IsInvalidArgument());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonreader.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonreader.cc b/be/src/kudu/util/jsonreader.cc
new file mode 100644
index 0000000..acbc869
--- /dev/null
+++ b/be/src/kudu/util/jsonreader.cc
@@ -0,0 +1,141 @@
+// 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 "kudu/util/jsonreader.h"
+
+#include <utility>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+
+using rapidjson::Value;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+JsonReader::JsonReader(string text) : text_(std::move(text)) {}
+
+JsonReader::~JsonReader() {
+}
+
+Status JsonReader::Init() {
+  document_.Parse<0>(text_.c_str());
+  if (document_.HasParseError()) {
+    return Status::Corruption("JSON text is corrupt", document_.GetParseError());
+  }
+  return Status::OK();
+}
+
+Status JsonReader::ExtractBool(const Value* object,
+                               const char* field,
+                               bool* result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsBool())) {
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected bool but got $0",
+        val->GetType()));
+  }
+  *result = val->GetBool();
+  return Status::OK();
+}
+
+Status JsonReader::ExtractInt32(const Value* object,
+                                const char* field,
+                                int32_t* result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsInt())) {
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected int32 but got $0",
+        val->GetType()));
+  }
+  *result = val->GetUint();
+  return Status::OK();
+}
+
+Status JsonReader::ExtractInt64(const Value* object,
+                                const char* field,
+                                int64_t* result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsInt64())) {
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected int64 but got $0",
+        val->GetType()));  }
+  *result = val->GetUint64();
+  return Status::OK();
+}
+
+Status JsonReader::ExtractString(const Value* object,
+                                 const char* field,
+                                 string* result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsString())) {
+    if (val->IsNull()) {
+      *result = "";
+      return Status::OK();
+    }
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected string but got $0",
+        val->GetType()));  }
+  result->assign(val->GetString());
+  return Status::OK();
+}
+
+Status JsonReader::ExtractObject(const Value* object,
+                                 const char* field,
+                                 const Value** result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsObject())) {
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected object but got $0",
+        val->GetType()));  }
+  *result = val;
+  return Status::OK();
+}
+
+Status JsonReader::ExtractObjectArray(const Value* object,
+                                      const char* field,
+                                      vector<const Value*>* result) const {
+  const Value* val;
+  RETURN_NOT_OK(ExtractField(object, field, &val));
+  if (PREDICT_FALSE(!val->IsArray())) {
+    return Status::InvalidArgument(Substitute(
+        "Wrong type during field extraction: expected object array but got $0",
+        val->GetType()));  }
+  for (Value::ConstValueIterator iter = val->Begin(); iter != val->End(); ++iter) {
+    result->push_back(iter);
+  }
+  return Status::OK();
+}
+
+Status JsonReader::ExtractField(const Value* object,
+                                const char* field,
+                                const Value** result) const {
+  if (field && PREDICT_FALSE(!object->HasMember(field))) {
+    return Status::NotFound("Missing field", field);
+  }
+  *result = field ? &(*object)[field] : object;
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonreader.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonreader.h b/be/src/kudu/util/jsonreader.h
new file mode 100644
index 0000000..e389b57
--- /dev/null
+++ b/be/src/kudu/util/jsonreader.h
@@ -0,0 +1,92 @@
+// 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.
+#ifndef KUDU_UTIL_JSONREADER_H_
+#define KUDU_UTIL_JSONREADER_H_
+
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Wraps the JSON parsing functionality of rapidjson::Document.
+//
+// Unlike JsonWriter, this class does not hide rapidjson internals from
+// clients. That's because there's just no easy way to implement object and
+// array parsing otherwise. At most, this class aspires to be a simpler
+// error-handling wrapper for reading and parsing.
+class JsonReader {
+ public:
+  explicit JsonReader(std::string text);
+  ~JsonReader();
+
+  Status Init();
+
+  // Extractor methods.
+  //
+  // If 'field' is not NULL, will look for a field with that name in the
+  // given object, returning Status::NotFound if it cannot be found. If
+  // 'field' is NULL, will try to convert 'object' directly into the
+  // desire type.
+
+  Status ExtractBool(const rapidjson::Value* object,
+                     const char* field,
+                     bool* result) const;
+
+  Status ExtractInt32(const rapidjson::Value* object,
+                      const char* field,
+                      int32_t* result) const;
+
+  Status ExtractInt64(const rapidjson::Value* object,
+                      const char* field,
+                      int64_t* result) const;
+
+  Status ExtractString(const rapidjson::Value* object,
+                       const char* field,
+                       std::string* result) const;
+
+  // 'result' is only valid for as long as JsonReader is alive.
+  Status ExtractObject(const rapidjson::Value* object,
+                       const char* field,
+                       const rapidjson::Value** result) const;
+
+  // 'result' is only valid for as long as JsonReader is alive.
+  Status ExtractObjectArray(const rapidjson::Value* object,
+                            const char* field,
+                            std::vector<const rapidjson::Value*>* result) const;
+
+  const rapidjson::Value* root() const { return &document_; }
+
+ private:
+  Status ExtractField(const rapidjson::Value* object,
+                      const char* field,
+                      const rapidjson::Value** result) const;
+
+  std::string text_;
+  rapidjson::Document document_;
+
+  DISALLOW_COPY_AND_ASSIGN(JsonReader);
+};
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_JSONREADER_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonwriter-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonwriter-test.cc b/be/src/kudu/util/jsonwriter-test.cc
new file mode 100644
index 0000000..6f9b10d
--- /dev/null
+++ b/be/src/kudu/util/jsonwriter-test.cc
@@ -0,0 +1,216 @@
+// 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 <ostream>
+#include <stdint.h>
+#include <string>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/jsonwriter_test.pb.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_util.h"
+
+namespace google { namespace protobuf { class Message; } }
+
+using google::protobuf::Message;
+using jsonwriter_test::TestAllTypes;
+
+namespace kudu {
+
+class TestJsonWriter : public KuduTest {
+ protected:
+  void DoBenchmark(const Message& pb);
+
+  TestAllTypes MakeAllTypesPB() {
+    TestAllTypes pb;
+    pb.set_optional_int32(1);
+    pb.set_optional_int64(2);
+    pb.set_optional_uint32(3);
+    pb.set_optional_uint64(4);
+    pb.set_optional_sint32(5);
+    pb.set_optional_sint64(6);
+    pb.set_optional_fixed32(7);
+    pb.set_optional_fixed64(8);
+    pb.set_optional_sfixed32(9);
+    pb.set_optional_sfixed64(10);
+    pb.set_optional_float(11);
+    pb.set_optional_double(12);
+    pb.set_optional_bool(true);
+    pb.set_optional_string("hello world");
+    pb.set_optional_redacted_string("secret!");
+    pb.set_optional_nested_enum(TestAllTypes::FOO);
+    return pb;
+  }
+
+};
+
+TEST_F(TestJsonWriter, TestPBEmpty) {
+  TestAllTypes pb;
+  ASSERT_EQ("{}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
+}
+
+TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
+  TestAllTypes pb = MakeAllTypesPB();
+
+  ASSERT_EQ("{\n"
+            "    \"optional_int32\": 1,\n"
+            "    \"optional_int64\": 2,\n"
+            "    \"optional_uint32\": 3,\n"
+            "    \"optional_uint64\": 4,\n"
+            "    \"optional_sint32\": 5,\n"
+            "    \"optional_sint64\": 6,\n"
+            "    \"optional_fixed32\": 7,\n"
+            "    \"optional_fixed64\": 8,\n"
+            "    \"optional_sfixed32\": 9,\n"
+            "    \"optional_sfixed64\": 10,\n"
+            "    \"optional_float\": 11,\n"
+            "    \"optional_double\": 12,\n"
+            "    \"optional_bool\": true,\n"
+            "    \"optional_string\": \"hello world\",\n"
+            "    \"optional_redacted_string\": \"<redacted>\",\n"
+            "    \"optional_nested_enum\": \"FOO\"\n"
+            "}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
+  ASSERT_EQ("{"
+            "\"optional_int32\":1,"
+            "\"optional_int64\":2,"
+            "\"optional_uint32\":3,"
+            "\"optional_uint64\":4,"
+            "\"optional_sint32\":5,"
+            "\"optional_sint64\":6,"
+            "\"optional_fixed32\":7,"
+            "\"optional_fixed64\":8,"
+            "\"optional_sfixed32\":9,"
+            "\"optional_sfixed64\":10,"
+            "\"optional_float\":11,"
+            "\"optional_double\":12,"
+            "\"optional_bool\":true,"
+            "\"optional_string\":\"hello world\","
+            "\"optional_redacted_string\":\"<redacted>\","
+            "\"optional_nested_enum\":\"FOO\""
+            "}", JsonWriter::ToJson(pb, JsonWriter::COMPACT));
+
+}
+
+TEST_F(TestJsonWriter, TestPBRepeatedPrimitives) {
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
+  TestAllTypes pb;
+  for (int i = 0; i <= 3; i++) {
+    pb.add_repeated_int32(i);
+    pb.add_repeated_string(strings::Substitute("hi $0", i));
+    pb.add_repeated_redacted_string("secret!");
+    pb.add_repeated_redacted_bytes("secret!");
+  }
+  ASSERT_EQ("{\n"
+            "    \"repeated_int32\": [\n"
+            "        0,\n"
+            "        1,\n"
+            "        2,\n"
+            "        3\n"
+            "    ],\n"
+            "    \"repeated_string\": [\n"
+            "        \"hi 0\",\n"
+            "        \"hi 1\",\n"
+            "        \"hi 2\",\n"
+            "        \"hi 3\"\n"
+            "    ],\n"
+            "    \"repeated_redacted_string\": [\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\"\n"
+            "    ],\n"
+            "    \"repeated_redacted_bytes\": [\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\"\n"
+            "    ]\n"
+            "}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
+  ASSERT_EQ("{\"repeated_int32\":[0,1,2,3],"
+            "\"repeated_string\":[\"hi 0\",\"hi 1\",\"hi 2\",\"hi 3\"],"
+            "\"repeated_redacted_string\":[\"<redacted>\",\"<redacted>\","
+            "\"<redacted>\",\"<redacted>\"],"
+            "\"repeated_redacted_bytes\":[\"<redacted>\",\"<redacted>\","
+            "\"<redacted>\",\"<redacted>\"]}",
+            JsonWriter::ToJson(pb, JsonWriter::COMPACT));
+}
+
+TEST_F(TestJsonWriter, TestPBNestedMessage) {
+  TestAllTypes pb;
+  pb.add_repeated_nested_message()->set_int_field(12345);
+  pb.mutable_optional_nested_message()->set_int_field(54321);
+  ASSERT_EQ("{\n"
+            "    \"optional_nested_message\": {\n"
+            "        \"int_field\": 54321\n"
+            "    },\n"
+            "    \"repeated_nested_message\": [\n"
+            "        {\n"
+            "            \"int_field\": 12345\n"
+            "        }\n"
+            "    ]\n"
+            "}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
+  ASSERT_EQ("{\"optional_nested_message\":{\"int_field\":54321},"
+            "\"repeated_nested_message\":"
+            "[{\"int_field\":12345}]}",
+            JsonWriter::ToJson(pb, JsonWriter::COMPACT));
+}
+
+void TestJsonWriter::DoBenchmark(const Message& pb) {
+  int64_t total_len = 0;
+  Stopwatch sw;
+  sw.start();
+  while (sw.elapsed().wall_seconds() < 5) {
+    std::ostringstream str;
+    JsonWriter jw(&str, JsonWriter::COMPACT);
+    jw.StartArray();
+    for (int i = 0; i < 10000; i++) {
+      jw.Protobuf(pb);
+    }
+    jw.EndArray();
+    total_len += str.str().size();
+  }
+  sw.stop();
+  double mbps = total_len / 1024.0 / 1024.0 / sw.elapsed().user_cpu_seconds();
+  LOG(INFO) << "Throughput: " << mbps << "MB/sec";
+}
+
+TEST_F(TestJsonWriter, BenchmarkAllTypes) {
+  DoBenchmark(MakeAllTypesPB());
+}
+
+TEST_F(TestJsonWriter, BenchmarkNestedMessage) {
+  TestAllTypes pb;
+  pb.add_repeated_nested_message()->set_int_field(12345);
+  pb.mutable_optional_nested_message()->set_int_field(54321);
+  DoBenchmark(pb);
+}
+
+TEST_F(TestJsonWriter, BenchmarkRepeatedInt64) {
+  TestAllTypes pb;
+  for (int i = 0; i < 10000; i++) {
+    pb.add_repeated_int64(i);
+  }
+  DoBenchmark(pb);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonwriter.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonwriter.cc b/be/src/kudu/util/jsonwriter.cc
new file mode 100644
index 0000000..3a5580c
--- /dev/null
+++ b/be/src/kudu/util/jsonwriter.cc
@@ -0,0 +1,352 @@
+// 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 "kudu/util/jsonwriter.h"
+
+#include <new>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
+#include <google/protobuf/message.h>
+#include <rapidjson/writer.h>
+#include <rapidjson/prettywriter.h>
+#include <rapidjson/rapidjson.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/pb_util.pb.h"
+
+using google::protobuf::FieldDescriptor;
+using google::protobuf::Message;
+using google::protobuf::Reflection;
+
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+// Adapter to allow RapidJSON to write directly to a stringstream.
+// Since Squeasel exposes a stringstream as its interface, this is needed to avoid overcopying.
+class UTF8StringStreamBuffer {
+ public:
+  explicit UTF8StringStreamBuffer(std::ostringstream* out);
+  ~UTF8StringStreamBuffer();
+  void Put(rapidjson::UTF8<>::Ch c);
+
+  void Flush();
+
+ private:
+  faststring buf_;
+  std::ostringstream* out_;
+};
+
+// rapidjson doesn't provide any common interface between the PrettyWriter and
+// Writer classes. So, we create our own pure virtual interface here, and then
+// use JsonWriterImpl<T> below to make the two different rapidjson implementations
+// correspond to this subclass.
+class JsonWriterIf {
+ public:
+  virtual void Null() = 0;
+  virtual void Bool(bool b) = 0;
+  virtual void Int(int i) = 0;
+  virtual void Uint(unsigned u) = 0;
+  virtual void Int64(int64_t i64) = 0;
+  virtual void Uint64(uint64_t u64) = 0;
+  virtual void Double(double d) = 0;
+  virtual void String(const char* str, size_t length) = 0;
+  virtual void String(const char* str) = 0;
+  virtual void String(const std::string& str) = 0;
+
+  virtual void StartObject() = 0;
+  virtual void EndObject() = 0;
+  virtual void StartArray() = 0;
+  virtual void EndArray() = 0;
+
+  virtual ~JsonWriterIf() {}
+};
+
+// Adapts the different rapidjson Writer implementations to our virtual
+// interface above.
+template<class T>
+class JsonWriterImpl : public JsonWriterIf {
+ public:
+  explicit JsonWriterImpl(ostringstream* out);
+
+  virtual void Null() OVERRIDE;
+  virtual void Bool(bool b) OVERRIDE;
+  virtual void Int(int i) OVERRIDE;
+  virtual void Uint(unsigned u) OVERRIDE;
+  virtual void Int64(int64_t i64) OVERRIDE;
+  virtual void Uint64(uint64_t u64) OVERRIDE;
+  virtual void Double(double d) OVERRIDE;
+  virtual void String(const char* str, size_t length) OVERRIDE;
+  virtual void String(const char* str) OVERRIDE;
+  virtual void String(const std::string& str) OVERRIDE;
+
+  virtual void StartObject() OVERRIDE;
+  virtual void EndObject() OVERRIDE;
+  virtual void StartArray() OVERRIDE;
+  virtual void EndArray() OVERRIDE;
+
+ private:
+  UTF8StringStreamBuffer stream_;
+  T writer_;
+  DISALLOW_COPY_AND_ASSIGN(JsonWriterImpl);
+};
+
+//
+// JsonWriter
+//
+
+typedef rapidjson::PrettyWriter<UTF8StringStreamBuffer> PrettyWriterClass;
+typedef rapidjson::Writer<UTF8StringStreamBuffer> CompactWriterClass;
+
+JsonWriter::JsonWriter(ostringstream* out, Mode m) {
+  switch (m) {
+    case PRETTY:
+      impl_.reset(new JsonWriterImpl<PrettyWriterClass>(DCHECK_NOTNULL(out)));
+      break;
+    case COMPACT:
+      impl_.reset(new JsonWriterImpl<CompactWriterClass>(DCHECK_NOTNULL(out)));
+      break;
+  }
+}
+JsonWriter::~JsonWriter() {
+}
+
+void JsonWriter::Null() { impl_->Null(); }
+void JsonWriter::Bool(bool b) { impl_->Bool(b); }
+void JsonWriter::Int(int i) { impl_->Int(i); }
+void JsonWriter::Uint(unsigned u) { impl_->Uint(u); }
+void JsonWriter::Int64(int64_t i64) { impl_->Int64(i64); }
+void JsonWriter::Uint64(uint64_t u64) { impl_->Uint64(u64); }
+void JsonWriter::Double(double d) { impl_->Double(d); }
+void JsonWriter::String(const char* str, size_t length) { impl_->String(str, length); }
+void JsonWriter::String(const char* str) { impl_->String(str); }
+void JsonWriter::String(const string& str) { impl_->String(str); }
+void JsonWriter::StartObject() { impl_->StartObject(); }
+void JsonWriter::EndObject() { impl_->EndObject(); }
+void JsonWriter::StartArray() { impl_->StartArray(); }
+void JsonWriter::EndArray() { impl_->EndArray(); }
+
+// Specializations for common primitive metric types.
+template<> void JsonWriter::Value(const bool& val) {
+  Bool(val);
+}
+template<> void JsonWriter::Value(const int32_t& val) {
+  Int(val);
+}
+template<> void JsonWriter::Value(const uint32_t& val) {
+  Uint(val);
+}
+template<> void JsonWriter::Value(const int64_t& val) {
+  Int64(val);
+}
+template<> void JsonWriter::Value(const uint64_t& val) {
+  Uint64(val);
+}
+template<> void JsonWriter::Value(const double& val) {
+  Double(val);
+}
+template<> void JsonWriter::Value(const string& val) {
+  String(val);
+}
+
+#if defined(__APPLE__)
+template<> void JsonWriter::Value(const size_t& val) {
+  Uint64(val);
+}
+#endif
+
+void JsonWriter::Protobuf(const Message& pb) {
+  const Reflection* reflection = pb.GetReflection();
+  vector<const FieldDescriptor*> fields;
+  reflection->ListFields(pb, &fields);
+
+  StartObject();
+  for (const FieldDescriptor* field : fields) {
+    String(field->name());
+    if (field->is_repeated()) {
+      StartArray();
+      int size = reflection->FieldSize(pb, field);
+      for (int i = 0; i < size; i++) {
+        ProtobufRepeatedField(pb, reflection, field, i);
+      }
+      EndArray();
+    } else {
+      ProtobufField(pb, reflection, field);
+    }
+  }
+  EndObject();
+}
+
+void JsonWriter::ProtobufField(const Message& pb, const Reflection* reflection,
+                               const FieldDescriptor* field) {
+  switch (field->cpp_type()) {
+    case FieldDescriptor::CPPTYPE_INT32:
+      Int(reflection->GetInt32(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_INT64:
+      Int64(reflection->GetInt64(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_UINT32:
+      Uint(reflection->GetUInt32(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_UINT64:
+      Uint64(reflection->GetUInt64(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_DOUBLE:
+      Double(reflection->GetDouble(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_FLOAT:
+      Double(reflection->GetFloat(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_BOOL:
+      Bool(reflection->GetBool(pb, field));
+      break;
+    case FieldDescriptor::CPPTYPE_ENUM:
+      String(reflection->GetEnum(pb, field)->name());
+      break;
+    case FieldDescriptor::CPPTYPE_STRING:
+      String(KUDU_MAYBE_REDACT_IF(field->options().GetExtension(REDACT),
+                                  reflection->GetString(pb, field)));
+      break;
+    case FieldDescriptor::CPPTYPE_MESSAGE:
+      Protobuf(reflection->GetMessage(pb, field));
+      break;
+    default:
+      LOG(FATAL) << "Unknown cpp_type: " << field->cpp_type();
+  }
+}
+
+void JsonWriter::ProtobufRepeatedField(const Message& pb, const Reflection* reflection,
+                                       const FieldDescriptor* field, int index) {
+  switch (field->cpp_type()) {
+    case FieldDescriptor::CPPTYPE_INT32:
+      Int(reflection->GetRepeatedInt32(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_INT64:
+      Int64(reflection->GetRepeatedInt64(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_UINT32:
+      Uint(reflection->GetRepeatedUInt32(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_UINT64:
+      Uint64(reflection->GetRepeatedUInt64(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_DOUBLE:
+      Double(reflection->GetRepeatedDouble(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_FLOAT:
+      Double(reflection->GetRepeatedFloat(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_BOOL:
+      Bool(reflection->GetRepeatedBool(pb, field, index));
+      break;
+    case FieldDescriptor::CPPTYPE_ENUM:
+      String(reflection->GetRepeatedEnum(pb, field, index)->name());
+      break;
+    case FieldDescriptor::CPPTYPE_STRING:
+      String(KUDU_MAYBE_REDACT_IF(field->options().GetExtension(REDACT),
+                                  reflection->GetRepeatedString(pb, field, index)));
+      break;
+    case FieldDescriptor::CPPTYPE_MESSAGE:
+      Protobuf(reflection->GetRepeatedMessage(pb, field, index));
+      break;
+    default:
+      LOG(FATAL) << "Unknown cpp_type: " << field->cpp_type();
+  }
+}
+
+string JsonWriter::ToJson(const Message& pb, Mode mode) {
+  ostringstream stream;
+  JsonWriter writer(&stream, mode);
+  writer.Protobuf(pb);
+  return stream.str();
+}
+
+//
+// UTF8StringStreamBuffer
+//
+
+UTF8StringStreamBuffer::UTF8StringStreamBuffer(std::ostringstream* out)
+  : out_(DCHECK_NOTNULL(out)) {
+}
+UTF8StringStreamBuffer::~UTF8StringStreamBuffer() {
+  DCHECK_EQ(buf_.size(), 0) << "Forgot to flush!";
+}
+
+void UTF8StringStreamBuffer::Put(rapidjson::UTF8<>::Ch c) {
+  buf_.push_back(c);
+}
+
+void UTF8StringStreamBuffer::Flush() {
+  out_->write(reinterpret_cast<char*>(buf_.data()), buf_.size());
+  buf_.clear();
+}
+
+//
+// JsonWriterImpl: simply forward to the underlying implementation.
+//
+
+template<class T>
+JsonWriterImpl<T>::JsonWriterImpl(ostringstream* out)
+  : stream_(DCHECK_NOTNULL(out)),
+    writer_(stream_) {
+}
+template<class T>
+void JsonWriterImpl<T>::Null() { writer_.Null(); }
+template<class T>
+void JsonWriterImpl<T>::Bool(bool b) { writer_.Bool(b); }
+template<class T>
+void JsonWriterImpl<T>::Int(int i) { writer_.Int(i); }
+template<class T>
+void JsonWriterImpl<T>::Uint(unsigned u) { writer_.Uint(u); }
+template<class T>
+void JsonWriterImpl<T>::Int64(int64_t i64) { writer_.Int64(i64); }
+template<class T>
+void JsonWriterImpl<T>::Uint64(uint64_t u64) { writer_.Uint64(u64); }
+template<class T>
+void JsonWriterImpl<T>::Double(double d) { writer_.Double(d); }
+template<class T>
+void JsonWriterImpl<T>::String(const char* str, size_t length) { writer_.String(str, length); }
+template<class T>
+void JsonWriterImpl<T>::String(const char* str) { writer_.String(str); }
+template<class T>
+void JsonWriterImpl<T>::String(const string& str) { writer_.String(str.c_str(), str.length()); }
+template<class T>
+void JsonWriterImpl<T>::StartObject() { writer_.StartObject(); }
+template<class T>
+void JsonWriterImpl<T>::EndObject() {
+  writer_.EndObject();
+  stream_.Flush();
+}
+template<class T>
+void JsonWriterImpl<T>::StartArray() { writer_.StartArray(); }
+template<class T>
+void JsonWriterImpl<T>::EndArray() {
+  writer_.EndArray();
+  stream_.Flush();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonwriter.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonwriter.h b/be/src/kudu/util/jsonwriter.h
new file mode 100644
index 0000000..24b4575
--- /dev/null
+++ b/be/src/kudu/util/jsonwriter.h
@@ -0,0 +1,102 @@
+// 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.
+#ifndef KUDU_UTIL_JSONWRITER_H
+#define KUDU_UTIL_JSONWRITER_H
+
+#include <cstddef>
+#include <cstdint>
+#include <iosfwd>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+
+namespace google {
+namespace protobuf {
+class FieldDescriptor;
+class Message;
+class Reflection;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class JsonWriterIf;
+
+// Acts as a pimpl for rapidjson so that not all metrics users must bring in the
+// rapidjson library, which is template-based and therefore hard to forward-declare.
+//
+// This class implements all the methods of rapidjson::JsonWriter, plus an
+// additional convenience method for String(std::string).
+//
+// We take an instance of std::stringstream in the constructor because Mongoose / Squeasel
+// uses std::stringstream for output buffering.
+class JsonWriter {
+ public:
+  enum Mode {
+    // Pretty-print the JSON, with nice indentation, newlines, etc.
+    PRETTY,
+    // Print the JSON as compactly as possible.
+    COMPACT
+  };
+
+  JsonWriter(std::ostringstream* out, Mode mode);
+  ~JsonWriter();
+
+  void Null();
+  void Bool(bool b);
+  void Int(int i);
+  void Uint(unsigned u);
+  void Int64(int64_t i64);
+  void Uint64(uint64_t u64);
+  void Double(double d);
+  void String(const char* str, size_t length);
+  void String(const char* str);
+  void String(const std::string& str);
+
+  // Convert the given protobuf message to JSON.
+  // The output respects redaction for 'string' and 'bytes' fields.
+  void Protobuf(const google::protobuf::Message& message);
+
+  template<typename T>
+  void Value(const T& val);
+
+  void StartObject();
+  void EndObject();
+  void StartArray();
+  void EndArray();
+
+  // Convert the given protobuf to JSON format.
+  static std::string ToJson(const google::protobuf::Message& pb,
+                            Mode mode);
+
+ private:
+  void ProtobufField(const google::protobuf::Message& pb,
+                     const google::protobuf::Reflection* reflection,
+                     const google::protobuf::FieldDescriptor* field);
+  void ProtobufRepeatedField(const google::protobuf::Message& pb,
+                             const google::protobuf::Reflection* reflection,
+                             const google::protobuf::FieldDescriptor* field,
+                             int index);
+
+  std::unique_ptr<JsonWriterIf> impl_;
+  DISALLOW_COPY_AND_ASSIGN(JsonWriter);
+};
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_JSONWRITER_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/jsonwriter_test.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/jsonwriter_test.proto b/be/src/kudu/util/jsonwriter_test.proto
new file mode 100644
index 0000000..b6f0300
--- /dev/null
+++ b/be/src/kudu/util/jsonwriter_test.proto
@@ -0,0 +1,79 @@
+// 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.
+syntax = "proto2";
+package jsonwriter_test;
+
+import "kudu/util/pb_util.proto";
+
+// This proto includes every type of field in both singular and repeated
+// forms. This is mostly copied from 'unittest.proto' in the protobuf source
+// (hence the odd field numbers which skip some).
+message TestAllTypes {
+  message NestedMessage {
+    optional int32 int_field = 1;
+  }
+
+  enum NestedEnum {
+    FOO = 1;
+    BAR = 2;
+    BAZ = 3;
+  }
+
+  // Singular
+  optional    int32 optional_int32    =  1;
+  optional    int64 optional_int64    =  2;
+  optional   uint32 optional_uint32   =  3;
+  optional   uint64 optional_uint64   =  4;
+  optional   sint32 optional_sint32   =  5;
+  optional   sint64 optional_sint64   =  6;
+  optional  fixed32 optional_fixed32  =  7;
+  optional  fixed64 optional_fixed64  =  8;
+  optional sfixed32 optional_sfixed32 =  9;
+  optional sfixed64 optional_sfixed64 = 10;
+  optional    float optional_float    = 11;
+  optional   double optional_double   = 12;
+  optional     bool optional_bool     = 13;
+  optional   string optional_string   = 14;
+  optional   string optional_redacted_string = 15 [ (kudu.REDACT) = true ];
+  optional    bytes optional_bytes    = 16;
+  optional    bytes optional_redacted_bytes = 17 [ (kudu.REDACT) = true ];
+
+  optional NestedMessage optional_nested_message  = 18;
+  optional NestedEnum optional_nested_enum     = 21;
+
+  // Repeated
+  repeated    int32 repeated_int32    = 31;
+  repeated    int64 repeated_int64    = 32;
+  repeated   uint32 repeated_uint32   = 33;
+  repeated   uint64 repeated_uint64   = 34;
+  repeated   sint32 repeated_sint32   = 35;
+  repeated   sint64 repeated_sint64   = 36;
+  repeated  fixed32 repeated_fixed32  = 37;
+  repeated  fixed64 repeated_fixed64  = 38;
+  repeated sfixed32 repeated_sfixed32 = 39;
+  repeated sfixed64 repeated_sfixed64 = 40;
+  repeated    float repeated_float    = 41;
+  repeated   double repeated_double   = 42;
+  repeated     bool repeated_bool     = 43;
+  repeated   string repeated_string   = 44;
+  repeated    bytes repeated_bytes    = 45;
+  repeated   string repeated_redacted_string = 46 [ (kudu.REDACT) = true ];
+  repeated   string repeated_redacted_bytes = 47 [ (kudu.REDACT) = true ];
+
+  repeated NestedMessage repeated_nested_message = 48;
+  repeated NestedEnum repeated_nested_enum = 51;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/kernel_stack_watchdog.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/kernel_stack_watchdog.cc b/be/src/kudu/util/kernel_stack_watchdog.cc
new file mode 100644
index 0000000..27a259c
--- /dev/null
+++ b/be/src/kudu/util/kernel_stack_watchdog.cc
@@ -0,0 +1,256 @@
+// 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 "kudu/util/kernel_stack_watchdog.h"
+
+#include <cstdint>
+#include <cstring>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <boost/bind.hpp>
+#include <glog/logging.h>
+#include <gflags/gflags.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/fault_injection.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/os-util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+DEFINE_int32(hung_task_check_interval_ms, 200,
+             "Number of milliseconds in between checks for hung threads");
+TAG_FLAG(hung_task_check_interval_ms, hidden);
+
+DEFINE_int32(inject_latency_on_kernel_stack_lookup_ms, 0,
+             "Number of milliseconds of latency to inject when reading a thread's "
+             "kernel stack");
+TAG_FLAG(inject_latency_on_kernel_stack_lookup_ms, hidden);
+
+using std::lock_guard;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+__thread KernelStackWatchdog::TLS* KernelStackWatchdog::tls_;
+
+KernelStackWatchdog::KernelStackWatchdog()
+  : log_collector_(nullptr),
+    finish_(1) {
+
+  // During creation of the stack watchdog thread, we need to disable using
+  // the stack watchdog itself. Otherwise, the 'StartThread' function will
+  // try to call back into initializing the stack watchdog, and will self-deadlock.
+  CHECK_OK(Thread::CreateWithFlags(
+      "kernel-watchdog", "kernel-watcher",
+      boost::bind(&KernelStackWatchdog::RunThread, this),
+      Thread::NO_STACK_WATCHDOG,
+      &thread_));
+}
+
+KernelStackWatchdog::~KernelStackWatchdog() {
+  finish_.CountDown();
+  CHECK_OK(ThreadJoiner(thread_.get()).Join());
+}
+
+void KernelStackWatchdog::SaveLogsForTests(bool save_logs) {
+  lock_guard<simple_spinlock> l(log_lock_);
+  if (save_logs) {
+    log_collector_.reset(new std::vector<string>());
+  } else {
+    log_collector_.reset();
+  }
+}
+
+std::vector<string> KernelStackWatchdog::LoggedMessagesForTests() const {
+  lock_guard<simple_spinlock> l(log_lock_);
+  CHECK(log_collector_) << "Must call SaveLogsForTests(true) first";
+  return *log_collector_;
+}
+
+void KernelStackWatchdog::Register(TLS* tls) {
+  int64_t tid = Thread::CurrentThreadId();
+  lock_guard<simple_spinlock> l(tls_lock_);
+  InsertOrDie(&tls_by_tid_, tid, tls);
+}
+
+void KernelStackWatchdog::Unregister() {
+  int64_t tid = Thread::CurrentThreadId();
+
+  std::unique_ptr<TLS> tls(tls_);
+  {
+    std::unique_lock<Mutex> l(unregister_lock_, std::try_to_lock);
+    lock_guard<simple_spinlock> l2(tls_lock_);
+    CHECK(tls_by_tid_.erase(tid));
+    if (!l.owns_lock()) {
+      // The watchdog is in the middle of running and might be accessing
+      // 'tls', so just enqueue it for later deletion. Otherwise it
+      // will go out of scope at the end of this function and get
+      // deleted here.
+      pending_delete_.emplace_back(std::move(tls));
+    }
+  }
+  tls_ = nullptr;
+}
+
+Status GetKernelStack(pid_t p, string* ret) {
+  MAYBE_INJECT_FIXED_LATENCY(FLAGS_inject_latency_on_kernel_stack_lookup_ms);
+  faststring buf;
+  RETURN_NOT_OK(ReadFileToString(Env::Default(), Substitute("/proc/$0/stack", p), &buf));
+  *ret = buf.ToString();
+  return Status::OK();
+}
+
+void KernelStackWatchdog::RunThread() {
+  while (true) {
+    MonoDelta delta = MonoDelta::FromMilliseconds(FLAGS_hung_task_check_interval_ms);
+    if (finish_.WaitFor(delta)) {
+      // Watchdog exiting.
+      break;
+    }
+
+    // Don't send signals while the debugger is running, since it makes it hard to
+    // use.
+    if (IsBeingDebugged()) {
+      continue;
+    }
+
+    // Prevent threads from deleting their TLS objects between the snapshot loop and the sending of
+    // signals. This makes it safe for us to access their TLS.
+    //
+    // NOTE: it's still possible that the thread will have exited in between grabbing its pointer
+    // and sending a signal, but DumpThreadStack() already is safe about not sending a signal
+    // to some other non-Kudu thread.
+    MutexLock l(unregister_lock_);
+
+    // Take the snapshot of the thread information under a short lock.
+    //
+    // 'tls_lock_' prevents new threads from starting, so we don't want to do any lengthy work
+    // (such as gathering stack traces) under this lock.
+    TLSMap tls_map_copy;
+    vector<unique_ptr<TLS>> to_delete;
+    {
+      lock_guard<simple_spinlock> l(tls_lock_);
+      to_delete.swap(pending_delete_);
+      tls_map_copy = tls_by_tid_;
+    }
+    // Actually delete the no-longer-used TLS entries outside of the lock.
+    to_delete.clear();
+
+    MicrosecondsInt64 now = GetMonoTimeMicros();
+    for (const auto& entry : tls_map_copy) {
+      pid_t p = entry.first;
+      TLS::Data* tls = &entry.second->data_;
+      TLS::Data tls_copy;
+      tls->SnapshotCopy(&tls_copy);
+      for (int i = 0; i < tls_copy.depth_; i++) {
+        const TLS::Frame* frame = &tls_copy.frames_[i];
+
+        int paused_ms = (now - frame->start_time_) / 1000;
+        if (paused_ms > frame->threshold_ms_) {
+          string kernel_stack;
+          Status s = GetKernelStack(p, &kernel_stack);
+          if (!s.ok()) {
+            // Can't read the kernel stack of the pid, just ignore it.
+            kernel_stack = "(could not read kernel stack)";
+          }
+
+          string user_stack = DumpThreadStack(p);
+
+          // If the thread exited the frame we're looking at in between when we started
+          // grabbing the stack and now, then our stack isn't correct. We shouldn't log it.
+          //
+          // We just use unprotected reads here since this is a somewhat best-effort
+          // check.
+          if (ANNOTATE_UNPROTECTED_READ(tls->depth_) < tls_copy.depth_ ||
+              ANNOTATE_UNPROTECTED_READ(tls->frames_[i].start_time_) != frame->start_time_) {
+            break;
+          }
+
+          lock_guard<simple_spinlock> l(log_lock_);
+          LOG_STRING(WARNING, log_collector_.get())
+              << "Thread " << p << " stuck at " << frame->status_
+              << " for " << paused_ms << "ms" << ":\n"
+              << "Kernel stack:\n" << kernel_stack << "\n"
+              << "User stack:\n" << user_stack;
+        }
+      }
+    }
+  }
+}
+
+void KernelStackWatchdog::ThreadExiting(void* /* unused */) {
+  KernelStackWatchdog::GetInstance()->Unregister();
+}
+
+void KernelStackWatchdog::CreateAndRegisterTLS() {
+  DCHECK(!tls_);
+  // Disable leak check. LSAN sometimes gets false positives on thread locals.
+  // See: https://github.com/google/sanitizers/issues/757
+  debug::ScopedLeakCheckDisabler d;
+  auto* tls = new TLS();
+  KernelStackWatchdog::GetInstance()->Register(tls);
+  tls_ = tls;
+  kudu::threadlocal::internal::AddDestructor(&ThreadExiting, nullptr);
+}
+
+KernelStackWatchdog::TLS::TLS() {
+  memset(&data_, 0, sizeof(data_));
+}
+
+KernelStackWatchdog::TLS::~TLS() {
+}
+
+// Optimistic concurrency control approach to snapshot the value of another
+// thread's TLS, even though that thread might be changing it.
+//
+// Called by the watchdog thread to see if a target thread is currently in the
+// middle of a watched section.
+void KernelStackWatchdog::TLS::Data::SnapshotCopy(Data* copy) const {
+  while (true) {
+    Atomic32 v_0 = base::subtle::Acquire_Load(&seq_lock_);
+    if (v_0 & 1) {
+      // If the value is odd, then the thread is in the middle of modifying
+      // its TLS, and we have to spin.
+      base::subtle::PauseCPU();
+      continue;
+    }
+    ANNOTATE_IGNORE_READS_BEGIN();
+    memcpy(copy, this, sizeof(*copy));
+    ANNOTATE_IGNORE_READS_END();
+    Atomic32 v_1 = base::subtle::Release_Load(&seq_lock_);
+
+    // If the value hasn't changed since we started the copy, then
+    // we know that the copy was a consistent snapshot.
+    if (v_1 == v_0) break;
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/kernel_stack_watchdog.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/kernel_stack_watchdog.h b/be/src/kudu/util/kernel_stack_watchdog.h
new file mode 100644
index 0000000..6ec7b50
--- /dev/null
+++ b/be/src/kudu/util/kernel_stack_watchdog.h
@@ -0,0 +1,290 @@
+// 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 class defines a singleton thread which manages a map of other thread IDs to
+// watch. Before performing some operation which may stall (eg IO) or which we expect
+// should be short (e.g. a callback on a critical thread that should not block), threads
+// may mark themselves as "watched", with a threshold beyond which they would like
+// warnings to be emitted including their stack trace at that time.
+//
+// In the background, a separate watchdog thread periodically wakes up, and if a thread
+// has been marked longer than its provided threshold, it will dump the stack trace
+// of that thread (both kernel-mode and user-mode stacks).
+//
+// This can be useful for diagnosing I/O stalls coming from the kernel, for example.
+//
+// Users will typically use the macro SCOPED_WATCH_STACK. Example usage:
+//
+//   // We expect the Write() to return in <100ms. If it takes longer than that
+//   // we'll see warnings indicating why it is stalled.
+//   {
+//     SCOPED_WATCH_STACK(100);
+//     file->Write(...);
+//   }
+//
+// If the Write call takes too long, a stack trace will be logged at WARNING level.
+// Note that the threshold time parameter is not a guarantee that a stall will be
+// caught by the watchdog thread. The watchdog only wakes up periodically to look
+// for threads that have been stalled too long. For example, if the threshold is 10ms
+// and the thread blocks for only 20ms, it's quite likely that the watchdog will
+// have missed the event.
+//
+// The SCOPED_WATCH_STACK macro is designed to have minimal overhead: approximately
+// equivalent to a clock_gettime() and a single 'mfence' instruction. Micro-benchmarks
+// measure the cost at about 50ns per call. Thus, it may safely be used in hot code
+// paths.
+//
+// Scopes with SCOPED_WATCH_STACK may be nested, but only up to a hard-coded limited depth
+// (currently 8).
+#ifndef KUDU_UTIL_KERNEL_STACK_WATCHDOG_H
+#define KUDU_UTIL_KERNEL_STACK_WATCHDOG_H
+
+#include <ctime>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/singleton.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/threadlocal.h"
+
+#define SCOPED_WATCH_STACK(threshold_ms) \
+  ScopedWatchKernelStack _stack_watcher(__FILE__ ":" AS_STRING(__LINE__), threshold_ms)
+
+namespace kudu {
+
+class Thread;
+
+// Singleton thread which implements the watchdog.
+class KernelStackWatchdog {
+ public:
+  static KernelStackWatchdog* GetInstance() {
+    return Singleton<KernelStackWatchdog>::get();
+  }
+
+  // Instead of logging through glog, log warning messages into a vector.
+  //
+  // If 'save_logs' is true, will start saving to the vector, and forget any
+  // previously logged messages.
+  // If 'save_logs' is false, disables this functionality.
+  void SaveLogsForTests(bool save_logs);
+
+  // Return any log messages saved since the last call to SaveLogsForTests(true).
+  std::vector<std::string> LoggedMessagesForTests() const;
+
+ private:
+  friend class Singleton<KernelStackWatchdog>;
+  friend class ScopedWatchKernelStack;
+
+  // The thread-local state which captures whether a thread should be watched by
+  // the watchdog. This structure is constructed as a thread-local on first use
+  // and destructed when the thread exits. Upon construction, the TLS structure
+  // registers itself with the WatchDog, and on destruction, unregisters itself.
+  //
+  // See 'seq_lock_' below for details on thread-safe operation.
+  struct TLS {
+    TLS();
+    ~TLS();
+
+    enum Constants {
+      // The maximum nesting depth of SCOPED_WATCH_STACK() macros.
+      kMaxDepth = 8
+    };
+
+    // Because we support nested SCOPED_WATCH_STACK() macros, we need to capture
+    // multiple active frames within the TLS.
+    struct Frame {
+      // The time at which this frame entered the SCOPED_WATCH_STACK section.
+      // We use MicrosecondsInt64 instead of MonoTime because it inlines a bit
+      // better.
+      MicrosecondsInt64 start_time_;
+      // The threshold of time beyond which the watchdog should emit warnings.
+      int threshold_ms_;
+      // A string explaining the state that the thread is in (typically a file:line
+      // string). This is expected to be static storage and is not freed.
+      const char* status_;
+    };
+
+    // The data within the TLS. This is a POD type so that the watchdog can easily
+    // copy data out of a thread's TLS.
+    struct Data {
+      Frame frames_[kMaxDepth];
+      Atomic32 depth_;
+
+      // Counter implementing a simple "sequence lock".
+      //
+      // Before modifying any data inside its TLS, the watched thread increments this value so it is
+      // odd. When the modifications are complete, it increments it again, making it even.
+      //
+      // To read the TLS data from a target thread, the watchdog thread waits for the value
+      // to become even, indicating that no write is in progress. Then, it does a potentially
+      // racy copy of the entire 'Data' structure. Then, it validates the value again.
+      // If it is has not changed, then the snapshot is guaranteed to be consistent.
+      //
+      // We use this type of locking to ensure that the watched thread is as fast as possible,
+      // allowing us to use SCOPED_WATCH_STACK even in hot code paths. In particular,
+      // the watched thread is wait-free, since it doesn't need to loop or retry. In addition, the
+      // memory is only written by that thread, eliminating any cache-line bouncing. The watchdog
+      // thread may have to loop multiple times to see a consistent snapshot, but we're OK delaying
+      // the watchdog arbitrarily since it isn't on any critical path.
+      Atomic32 seq_lock_;
+
+      // Take a consistent snapshot of this data into 'dst'. This may block if the target thread
+      // is currently modifying its TLS.
+      void SnapshotCopy(Data* dst) const;
+    };
+    Data data_;
+  };
+
+  KernelStackWatchdog();
+  ~KernelStackWatchdog();
+
+  // Get or create the TLS for the current thread.
+  static TLS* GetTLS() {
+    if (PREDICT_FALSE(!tls_)) {
+      CreateAndRegisterTLS();
+    }
+    return tls_;
+  }
+
+  // Create a new TLS for the current thread, and register it with the watchdog.
+  // Installs a callback to automatically unregister the thread upon its exit.
+  static void CreateAndRegisterTLS();
+
+  // Callback which is registered to run at thread-exit time by CreateAndRegisterTLS().
+  static void ThreadExiting(void* tls_void);
+
+  // Register a new thread's TLS with the watchdog.
+  // Called by any thread the first time it enters a watched section, when its TLS
+  // is constructed.
+  void Register(TLS* tls);
+
+  // Called when a thread is in the process of exiting, and has a registered TLS
+  // object.
+  void Unregister();
+
+  // The actual watchdog loop that the watchdog thread runs.
+  void RunThread();
+
+  DECLARE_STATIC_THREAD_LOCAL(TLS, tls_);
+
+  typedef std::unordered_map<pid_t, TLS*> TLSMap;
+  TLSMap tls_by_tid_;
+
+  // If a thread exits while the watchdog is in the middle of accessing the TLS
+  // objects, we can't immediately delete the TLS struct. Instead, the thread
+  // enqueues it here for later deletion by the watchdog thread within RunThread().
+  std::vector<std::unique_ptr<TLS>> pending_delete_;
+
+  // If non-NULL, warnings will be emitted into this vector instead of glog.
+  // Used by tests.
+  gscoped_ptr<std::vector<std::string> > log_collector_;
+
+  // Lock protecting log_collector_.
+  mutable simple_spinlock log_lock_;
+
+  // Lock protecting tls_by_tid_ and pending_delete_.
+  mutable simple_spinlock tls_lock_;
+
+  // Lock which prevents threads from unregistering while the watchdog
+  // sends signals.
+  //
+  // This is used to prevent the watchdog from sending a signal to a pid just
+  // after the pid has actually exited and been reused. Sending a signal to
+  // a non-Kudu thread could have unintended consequences.
+  //
+  // When this lock is held concurrently with 'tls_lock_' or 'log_lock_',
+  // this lock must be acquired first.
+  Mutex unregister_lock_;
+
+  // The watchdog thread itself.
+  scoped_refptr<Thread> thread_;
+
+  // Signal to stop the watchdog.
+  CountDownLatch finish_;
+
+  DISALLOW_COPY_AND_ASSIGN(KernelStackWatchdog);
+};
+
+// Scoped object which marks the current thread for watching.
+class ScopedWatchKernelStack {
+ public:
+  // If the current scope is active more than 'threshold_ms' milliseconds, the
+  // watchdog thread will log a warning including the message 'label'. 'label'
+  // is not copied or freed.
+  ScopedWatchKernelStack(const char* label, int threshold_ms) {
+    if (threshold_ms <= 0) return;
+
+    // Rather than just using the lazy GetTLS() method, we'll first try to load
+    // the TLS ourselves. This is usually successful, and avoids us having to inline
+    // the TLS construction path at call sites.
+    KernelStackWatchdog::TLS* tls = KernelStackWatchdog::tls_;
+    if (PREDICT_FALSE(tls == NULL)) {
+      tls = KernelStackWatchdog::GetTLS();
+    }
+    KernelStackWatchdog::TLS::Data* tls_data = &tls->data_;
+
+    // "Acquire" the sequence lock. While the lock value is odd, readers will block.
+    // TODO: technically this barrier is stronger than we need: we are the only writer
+    // to this data, so it's OK to allow loads from within the critical section to
+    // reorder above this next line. All we need is a "StoreStore" barrier (i.e.
+    // prevent any stores in the critical section from getting reordered above the
+    // increment of the counter). However, atomicops.h doesn't provide such a barrier
+    // as of yet, so we'll do the slightly more expensive one for now.
+    base::subtle::Acquire_Store(&tls_data->seq_lock_, tls_data->seq_lock_ + 1);
+
+    KernelStackWatchdog::TLS::Frame* frame = &tls_data->frames_[tls_data->depth_++];
+    DCHECK_LE(tls_data->depth_, KernelStackWatchdog::TLS::kMaxDepth);
+    frame->start_time_ = GetMonoTimeMicros();
+    frame->threshold_ms_ = threshold_ms;
+    frame->status_ = label;
+
+    // "Release" the sequence lock. This resets the lock value to be even, so readers
+    // will proceed.
+    base::subtle::Release_Store(&tls_data->seq_lock_, tls_data->seq_lock_ + 1);
+  }
+
+  ~ScopedWatchKernelStack() {
+    if (!KernelStackWatchdog::tls_) return;
+
+    KernelStackWatchdog::TLS::Data* tls = &KernelStackWatchdog::tls_->data_;
+    int d = tls->depth_;
+    DCHECK_GT(d, 0);
+
+    // We don't bother with a lock/unlock, because the change we're making here is atomic.
+    // If we race with the watchdog, either they'll see the old depth_ or the new depth_,
+    // but in either case the underlying data is perfectly valid.
+    base::subtle::NoBarrier_Store(&tls->depth_, d - 1);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(ScopedWatchKernelStack);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_KERNEL_STACK_WATCHDOG_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/knapsack_solver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/knapsack_solver-test.cc b/be/src/kudu/util/knapsack_solver-test.cc
new file mode 100644
index 0000000..9f717c4
--- /dev/null
+++ b/be/src/kudu/util/knapsack_solver-test.cc
@@ -0,0 +1,172 @@
+// 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 <cstdlib>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/knapsack_solver.h"
+#include "kudu/util/stopwatch.h"  // IWYU pragma: keep
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+class TestKnapsack : public KuduTest {
+};
+
+// A simple test item for use with the knapsack solver.
+// The real code will be solving knapsack over RowSet objects --
+// using simple value/weight pairs in the tests makes it standalone.
+struct TestItem {
+  TestItem(double v, int w)
+    : value(v), weight(w) {
+  }
+
+  double value;
+  int weight;
+};
+
+// A traits class to adapt the knapsack solver to TestItem.
+struct TestItemTraits {
+  typedef TestItem item_type;
+  typedef double value_type;
+  static int get_weight(const TestItem &item) {
+    return item.weight;
+  }
+  static value_type get_value(const TestItem &item) {
+    return item.value;
+  }
+};
+
+// Generate random items into the provided vector.
+static void GenerateRandomItems(int n_items, int max_weight,
+                                vector<TestItem> *out) {
+  for (int i = 0; i < n_items; i++) {
+    double value = 10000.0 / (random() % 10000 + 1);
+    int weight = random() % max_weight;
+    out->push_back(TestItem(value, weight));
+  }
+}
+
+// Join and stringify the given list of ints.
+static string JoinInts(const vector<int> &ints) {
+  string ret;
+  for (int i = 0; i < ints.size(); i++) {
+    if (i > 0) {
+      ret.push_back(',');
+    }
+    ret.append(std::to_string(ints[i]));
+  }
+  return ret;
+}
+
+TEST_F(TestKnapsack, Basics) {
+  KnapsackSolver<TestItemTraits> solver;
+
+  vector<TestItem> in;
+  in.emplace_back(500, 3);
+  in.emplace_back(110, 1);
+  in.emplace_back(125, 1);
+  in.emplace_back(100, 1);
+
+  vector<int> out;
+  double max_val;
+
+  // For 1 weight, pick item 2
+  solver.Solve(in, 1, &out, &max_val);
+  ASSERT_DOUBLE_EQ(125, max_val);
+  ASSERT_EQ("2", JoinInts(out));
+  out.clear();
+
+  // For 2 weight, pick item 1, 2
+  solver.Solve(in, 2, &out, &max_val);
+  ASSERT_DOUBLE_EQ(110 + 125, max_val);
+  ASSERT_EQ("2,1", JoinInts(out));
+  out.clear();
+
+  // For 3 weight, pick item 0
+  solver.Solve(in, 3, &out, &max_val);
+  ASSERT_DOUBLE_EQ(500, max_val);
+  ASSERT_EQ("0", JoinInts(out));
+  out.clear();
+
+  // For 10 weight, pick all.
+  solver.Solve(in, 10, &out, &max_val);
+  ASSERT_DOUBLE_EQ(500 + 110 + 125 + 100, max_val);
+  ASSERT_EQ("3,2,1,0", JoinInts(out));
+  out.clear();
+}
+
+// Test which generates random knapsack instances and verifies
+// that the result satisfies the constraints.
+TEST_F(TestKnapsack, Randomized) {
+  SeedRandom();
+  KnapsackSolver<TestItemTraits> solver;
+
+  const int kNumTrials = AllowSlowTests() ? 200 : 1;
+  const int kMaxWeight = 1000;
+  const int kNumItems = 1000;
+
+  for (int i = 0; i < kNumTrials; i++) {
+    vector<TestItem> in;
+    vector<int> out;
+    GenerateRandomItems(kNumItems, kMaxWeight, &in);
+    double max_val;
+    int max_weight = random() % kMaxWeight;
+    solver.Solve(in, max_weight, &out, &max_val);
+
+    // Verify that the max_val is equal to the sum of the chosen items' values.
+    double sum_val = 0;
+    int sum_weight = 0;
+    for (int i : out) {
+      sum_val += in[i].value;
+      sum_weight += in[i].weight;
+    }
+    ASSERT_NEAR(max_val, sum_val, 0.000001);
+    ASSERT_LE(sum_weight, max_weight);
+  }
+}
+
+#ifdef NDEBUG
+TEST_F(TestKnapsack, Benchmark) {
+  KnapsackSolver<TestItemTraits> solver;
+
+  const int kNumTrials = 1000;
+  const int kMaxWeight = 1000;
+  const int kNumItems = 1000;
+
+  vector<TestItem> in;
+  GenerateRandomItems(kNumItems, kMaxWeight, &in);
+
+  LOG_TIMING(INFO, "benchmark") {
+    vector<int> out;
+    for (int i = 0; i < kNumTrials; i++) {
+      out.clear();
+      double max_val;
+      solver.Solve(in, random() % kMaxWeight, &out, &max_val);
+    }
+  }
+}
+#endif
+
+} // namespace kudu


[32/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/test_certs.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/test_certs.h b/be/src/kudu/security/test/test_certs.h
new file mode 100644
index 0000000..7767cb2
--- /dev/null
+++ b/be/src/kudu/security/test/test_certs.h
@@ -0,0 +1,86 @@
+// 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.
+#pragma once
+
+#include <string>
+
+namespace kudu {
+class Status;
+
+namespace security {
+
+//
+// Set of certificates and private keys used for certificate generation
+// and signing tests (declarations).  See the .cc file for the actual data.
+//
+
+// Valid root CA cerificate (PEM format).
+extern const char kCaCert[];
+// The private key (RSA, 2048 bits) for the certificate above.
+// This is 2048 bit RSA key, in PEM format.
+extern const char kCaPrivateKey[];
+// The public part of the abovementioned private key.
+extern const char kCaPublicKey[];
+
+// Expired root CA certificate (PEM format).
+extern const char kCaExpiredCert[];
+// The private key for the expired CA certificate described above.
+// This is 2048 bit RSA key, in PEM format.
+extern const char kCaExpiredPrivateKey[];
+// The public part of the abovementioned private key.
+extern const char kCaExpiredPublicKey[];
+// Certificate with multiple DNS hostnames in the SAN field.
+extern const char kCertDnsHostnamesInSan[];
+
+extern const char kDataTiny[];
+extern const char kSignatureTinySHA512[];
+
+extern const char kDataShort[];
+extern const char kSignatureShortSHA512[];
+
+extern const char kDataLong[];
+extern const char kSignatureLongSHA512[];
+
+// Creates a matching SSL certificate and unencrypted private key file in 'dir',
+// returning their paths in '*cert_file' and '*key_file'.
+Status CreateTestSSLCertWithPlainKey(const std::string& dir,
+                                     std::string* cert_file,
+                                     std::string* key_file);
+
+// Same as the CreateTestSSLCertWithPlainKey() except that the private key is
+// encrypted with a password that is returned in 'key_password'.
+Status CreateTestSSLCertWithEncryptedKey(const std::string& dir,
+                                         std::string* cert_file,
+                                         std::string* key_file,
+                                         std::string* key_password);
+
+// Same as the CreateTestSSLCertWithPlainKey() except that the 'cert_file' is
+// signed by a CA chain ('ca_cert_file' is a chain of certificates).
+Status CreateTestSSLCertSignedByChain(const std::string& dir,
+                                      std::string* cert_file,
+                                      std::string* key_file,
+                                      std::string* ca_cert_file);
+
+// Same as the CreateTestSSLCertWithPlainKey() except that the 'cert_file' is
+// a chain signed by a root CA ('ca_cert_file' is only the root CA).
+Status CreateTestSSLCertWithChainSignedByRoot(const std::string& dir,
+                                              std::string* cert_file,
+                                              std::string* key_file,
+                                              std::string* ca_cert_file);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/test_pass.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/test_pass.cc b/be/src/kudu/security/test/test_pass.cc
new file mode 100644
index 0000000..f99ab3c
--- /dev/null
+++ b/be/src/kudu/security/test/test_pass.cc
@@ -0,0 +1,40 @@
+// 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 "kudu/security/test/test_pass.h"
+
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+
+using std::string;
+
+namespace kudu {
+namespace security {
+
+Status CreateTestHTPasswd(const string& dir,
+                          string* passwd_file) {
+
+  // In the format of user:realm:digest. Digest is generated based on
+  // user/password pair in kTestAuthString
+  const char *kHTPasswd = "test:mydomain.com:8b6f595afb3c037b7bd79b89d9576d06";
+  *passwd_file = JoinPathSegments(dir, "test.passwd");
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kHTPasswd, *passwd_file));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/test_pass.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/test_pass.h b/be/src/kudu/security/test/test_pass.h
new file mode 100644
index 0000000..5f730fe
--- /dev/null
+++ b/be/src/kudu/security/test/test_pass.h
@@ -0,0 +1,37 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+// Username and password for HTTP authentication, corresponding to
+// .htpasswd created by CreateTestHTPasswd()
+const std::string kTestAuthString = "test:test";
+
+// Creates .htpasswd for HTTP basic authentication in the format
+// of 'user:realm:digest', returning the path in '*passwd_file'.
+Status CreateTestHTPasswd(const std::string &dir,
+                          std::string *passwd_file);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_context.cc b/be/src/kudu/security/tls_context.cc
new file mode 100644
index 0000000..9bf433d
--- /dev/null
+++ b/be/src/kudu/security/tls_context.cc
@@ -0,0 +1,520 @@
+// 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 "kudu/security/tls_context.h"
+
+#include <algorithm>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <boost/algorithm/string/predicate.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/init.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/user.h"
+
+// Hard code OpenSSL flag values from OpenSSL 1.0.1e[1][2] when compiling
+// against OpenSSL 1.0.0 and below. We detect when running against a too-old
+// version of OpenSSL using these definitions at runtime so that Kudu has full
+// functionality when run against a new OpenSSL version, even if it's compiled
+// against an older version.
+//
+// [1]: https://github.com/openssl/openssl/blob/OpenSSL_1_0_1e/ssl/ssl.h#L605-L609
+// [2]: https://github.com/openssl/openssl/blob/OpenSSL_1_0_1e/ssl/tls1.h#L166-L172
+#ifndef SSL_OP_NO_TLSv1
+#define SSL_OP_NO_TLSv1 0x04000000U
+#endif
+#ifndef SSL_OP_NO_TLSv1_1
+#define SSL_OP_NO_TLSv1_1 0x10000000U
+#endif
+#ifndef TLS1_1_VERSION
+#define TLS1_1_VERSION 0x0302
+#endif
+#ifndef TLS1_2_VERSION
+#define TLS1_2_VERSION 0x0303
+#endif
+
+using strings::Substitute;
+using std::string;
+using std::unique_lock;
+using std::vector;
+
+DEFINE_int32(ipki_server_key_size, 2048,
+             "the number of bits for server cert's private key. The server cert "
+             "is used for TLS connections to and from clients and other servers.");
+TAG_FLAG(ipki_server_key_size, experimental);
+
+namespace kudu {
+namespace security {
+
+using ca::CertRequestGenerator;
+
+template<> struct SslTypeTraits<SSL> {
+  static constexpr auto kFreeFunc = &SSL_free;
+};
+template<> struct SslTypeTraits<X509_STORE_CTX> {
+  static constexpr auto kFreeFunc = &X509_STORE_CTX_free;
+};
+
+namespace {
+
+Status CheckMaxSupportedTlsVersion(int tls_version, const char* tls_version_str) {
+  // OpenSSL 1.1 and newer supports all of the TLS versions we care about, so
+  // the below check is only necessary in older versions of OpenSSL.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  auto max_supported_tls_version = SSLv23_method()->version;
+  DCHECK_GE(max_supported_tls_version, TLS1_VERSION);
+
+  if (max_supported_tls_version < tls_version) {
+    return Status::InvalidArgument(
+        Substitute("invalid minimum TLS protocol version (--rpc_tls_min_protocol): "
+                   "this platform does not support $0", tls_version_str));
+  }
+#endif
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+TlsContext::TlsContext()
+    : tls_ciphers_(kudu::security::SecurityDefaults::kDefaultTlsCiphers),
+      tls_min_protocol_(kudu::security::SecurityDefaults::kDefaultTlsMinVersion),
+      lock_(RWMutex::Priority::PREFER_READING),
+      trusted_cert_count_(0),
+      has_cert_(false),
+      is_external_cert_(false) {
+  security::InitializeOpenSSL();
+}
+
+TlsContext::TlsContext(std::string tls_ciphers, std::string tls_min_protocol)
+    : tls_ciphers_(std::move(tls_ciphers)),
+      tls_min_protocol_(std::move(tls_min_protocol)),
+      lock_(RWMutex::Priority::PREFER_READING),
+      trusted_cert_count_(0),
+      has_cert_(false),
+      is_external_cert_(false) {
+  security::InitializeOpenSSL();
+}
+
+Status TlsContext::Init() {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(!ctx_);
+
+  // NOTE: 'SSLv23 method' sounds like it would enable only SSLv2 and SSLv3, but in fact
+  // this is a sort of wildcard which enables all methods (including TLSv1 and later).
+  // We explicitly disable SSLv2 and SSLv3 below so that only TLS methods remain.
+  // See the discussion on https://trac.torproject.org/projects/tor/ticket/11598 for more
+  // info.
+  ctx_ = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (!ctx_) {
+    return Status::RuntimeError("failed to create TLS context", GetOpenSSLErrors());
+  }
+  SSL_CTX_set_mode(ctx_.get(), SSL_MODE_AUTO_RETRY | SSL_MODE_ENABLE_PARTIAL_WRITE);
+
+  // Disable SSLv2 and SSLv3 which are vulnerable to various issues such as POODLE.
+  // We support versions back to TLSv1.0 since OpenSSL on RHEL 6.4 and earlier does not
+  // not support TLSv1.1 or later.
+  //
+  // Disable SSL/TLS compression to free up CPU resources and be less prone
+  // to attacks exploiting the compression feature:
+  //   https://tools.ietf.org/html/rfc7525#section-3.3
+  auto options = SSL_OP_NO_SSLv2 | SSL_OP_NO_SSLv3 | SSL_OP_NO_COMPRESSION;
+
+  if (boost::iequals(tls_min_protocol_, "TLSv1.2")) {
+    RETURN_NOT_OK(CheckMaxSupportedTlsVersion(TLS1_2_VERSION, "TLSv1.2"));
+    options |= SSL_OP_NO_TLSv1 | SSL_OP_NO_TLSv1_1;
+  } else if (boost::iequals(tls_min_protocol_, "TLSv1.1")) {
+    RETURN_NOT_OK(CheckMaxSupportedTlsVersion(TLS1_1_VERSION, "TLSv1.1"));
+    options |= SSL_OP_NO_TLSv1;
+  } else if (!boost::iequals(tls_min_protocol_, "TLSv1")) {
+    return Status::InvalidArgument("unknown value provided for --rpc_tls_min_protocol flag",
+                                   tls_min_protocol_);
+  }
+
+  SSL_CTX_set_options(ctx_.get(), options);
+
+  OPENSSL_RET_NOT_OK(
+      SSL_CTX_set_cipher_list(ctx_.get(), tls_ciphers_.c_str()),
+      "failed to set TLS ciphers");
+
+  // Enable ECDH curves. For OpenSSL 1.1.0 and up, this is done automatically.
+#ifndef OPENSSL_NO_ECDH
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+  // OpenSSL 1.0.1 and below only support setting a single ECDH curve at once.
+  // We choose prime256v1 because it's the first curve listed in the "modern
+  // compatibility" section of the Mozilla Server Side TLS recommendations,
+  // accessed Feb. 2017.
+  c_unique_ptr<EC_KEY> ecdh { EC_KEY_new_by_curve_name(NID_X9_62_prime256v1), &EC_KEY_free };
+  OPENSSL_RET_IF_NULL(ecdh, "failed to create prime256v1 curve");
+  OPENSSL_RET_NOT_OK(SSL_CTX_set_tmp_ecdh(ctx_.get(), ecdh.get()),
+                     "failed to set ECDH curve");
+#elif OPENSSL_VERSION_NUMBER < 0x10100000L
+  // OpenSSL 1.0.2 provides the set_ecdh_auto API which internally figures out
+  // the best curve to use.
+  OPENSSL_RET_NOT_OK(SSL_CTX_set_ecdh_auto(ctx_.get(), 1),
+                     "failed to configure ECDH support");
+#endif
+#endif
+
+  // TODO(KUDU-1926): is it possible to disable client-side renegotiation? it seems there
+  // have been various CVEs related to this feature that we don't need.
+  return Status::OK();
+}
+
+Status TlsContext::VerifyCertChainUnlocked(const Cert& cert) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  X509_STORE* store = SSL_CTX_get_cert_store(ctx_.get());
+  auto store_ctx = ssl_make_unique<X509_STORE_CTX>(X509_STORE_CTX_new());
+
+  OPENSSL_RET_NOT_OK(X509_STORE_CTX_init(store_ctx.get(), store, cert.GetTopOfChainX509(),
+                     cert.GetRawData()), "could not init X509_STORE_CTX");
+  int rc = X509_verify_cert(store_ctx.get());
+  if (rc != 1) {
+    int err = X509_STORE_CTX_get_error(store_ctx.get());
+    if (err == X509_V_ERR_DEPTH_ZERO_SELF_SIGNED_CERT) {
+      // It's OK to provide a self-signed cert.
+      ERR_clear_error(); // in case it left anything on the queue.
+      return Status::OK();
+    }
+
+    // Get the cert that failed to verify.
+    X509* cur_cert = X509_STORE_CTX_get_current_cert(store_ctx.get());
+    string cert_details;
+    if (cur_cert) {
+      cert_details = Substitute(" (error with cert: subject=$0, issuer=$1)",
+                                X509NameToString(X509_get_subject_name(cur_cert)),
+                                X509NameToString(X509_get_issuer_name(cur_cert)));
+    }
+
+    ERR_clear_error(); // in case it left anything on the queue.
+    return Status::RuntimeError(
+        Substitute("could not verify certificate chain$0", cert_details),
+        X509_verify_cert_error_string(err));
+  }
+  return Status::OK();
+}
+
+Status TlsContext::UseCertificateAndKey(const Cert& cert, const PrivateKey& key) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  // Verify that the cert and key match.
+  RETURN_NOT_OK(cert.CheckKeyMatch(key));
+
+  std::unique_lock<RWMutex> lock(lock_);
+
+  // Verify that the appropriate CA certs have been loaded into the context
+  // before we adopt a cert. Otherwise, client connections without the CA cert
+  // available would fail.
+  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
+
+  CHECK(!has_cert_);
+
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_PrivateKey(ctx_.get(), key.GetRawData()),
+                     "failed to use private key");
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_certificate(ctx_.get(), cert.GetTopOfChainX509()),
+                     "failed to use certificate");
+  has_cert_ = true;
+  return Status::OK();
+}
+
+Status TlsContext::AddTrustedCertificate(const Cert& cert) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  VLOG(2) << "Trusting certificate " << cert.SubjectName();
+
+  {
+    // Workaround for a leak in OpenSSL <1.0.1:
+    //
+    // If we start trusting a cert, and its internal public-key field hasn't
+    // yet been populated, then the first time it's used for verification will
+    // populate it. In the case that two threads try to populate it at the same time,
+    // one of the thread's copies will be leaked.
+    //
+    // To avoid triggering the race, we populate the internal public key cache
+    // field up front before adding it to the trust store.
+    //
+    // See OpenSSL commit 33a688e80674aaecfac6d9484ec199daa0ee5b61.
+    PublicKey k;
+    CHECK_OK(cert.GetPublicKey(&k));
+  }
+
+  unique_lock<RWMutex> lock(lock_);
+  auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
+
+  // Iterate through the certificate chain and add each individual certificate to the store.
+  for (int i = 0; i < cert.chain_len(); ++i) {
+    X509* inner_cert = sk_X509_value(cert.GetRawData(), i);
+    int rc = X509_STORE_add_cert(cert_store, inner_cert);
+    if (rc <= 0) {
+      // Ignore the common case of re-adding a cert that is already in the
+      // trust store.
+      auto err = ERR_peek_error();
+      if (ERR_GET_LIB(err) == ERR_LIB_X509 &&
+          ERR_GET_REASON(err) == X509_R_CERT_ALREADY_IN_HASH_TABLE) {
+        ERR_clear_error();
+        return Status::OK();
+      }
+      OPENSSL_RET_NOT_OK(rc, "failed to add trusted certificate");
+    }
+  }
+  trusted_cert_count_ += 1;
+  return Status::OK();
+}
+
+Status TlsContext::DumpTrustedCerts(vector<string>* cert_ders) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  shared_lock<RWMutex> lock(lock_);
+
+  vector<string> ret;
+  auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
+
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+#define STORE_LOCK(CS) CRYPTO_w_lock(CRYPTO_LOCK_X509_STORE)
+#define STORE_UNLOCK(CS) CRYPTO_w_unlock(CRYPTO_LOCK_X509_STORE)
+#define STORE_GET_X509_OBJS(CS) (CS)->objs
+#define X509_OBJ_GET_TYPE(X509_OBJ) (X509_OBJ)->type
+#define X509_OBJ_GET_X509(X509_OBJ) (X509_OBJ)->data.x509
+#else
+#define STORE_LOCK(CS) CHECK_EQ(1, X509_STORE_lock(CS)) << "Could not lock certificate store"
+#define STORE_UNLOCK(CS) CHECK_EQ(1, X509_STORE_unlock(CS)) << "Could not unlock certificate store"
+#define STORE_GET_X509_OBJS(CS) X509_STORE_get0_objects(CS)
+#define X509_OBJ_GET_TYPE(X509_OBJ) X509_OBJECT_get_type(X509_OBJ)
+#define X509_OBJ_GET_X509(X509_OBJ) X509_OBJECT_get0_X509(X509_OBJ)
+#endif
+
+  STORE_LOCK(cert_store);
+  auto unlock = MakeScopedCleanup([&]() {
+      STORE_UNLOCK(cert_store);
+    });
+  auto* objects = STORE_GET_X509_OBJS(cert_store);
+  int num_objects = sk_X509_OBJECT_num(objects);
+  for (int i = 0; i < num_objects; i++) {
+    auto* obj = sk_X509_OBJECT_value(objects, i);
+    if (X509_OBJ_GET_TYPE(obj) != X509_LU_X509) continue;
+    auto* x509 = X509_OBJ_GET_X509(obj);
+    Cert c;
+    c.AdoptAndAddRefX509(x509);
+    string der;
+    RETURN_NOT_OK(c.ToString(&der, DataFormat::DER));
+    ret.emplace_back(std::move(der));
+  }
+
+  cert_ders->swap(ret);
+  return Status::OK();
+}
+
+namespace {
+Status SetCertAttributes(CertRequestGenerator::Config* config) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  RETURN_NOT_OK_PREPEND(GetFQDN(&config->hostname), "could not determine FQDN for CSR");
+
+  // If the server has logged in from a keytab, then we have a 'real' identity,
+  // and our desired CN should match the local username mapped from the Kerberos
+  // principal name. Otherwise, we'll make up a common name based on the hostname.
+  boost::optional<string> principal = GetLoggedInPrincipalFromKeytab();
+  if (!principal) {
+    string uid;
+    RETURN_NOT_OK_PREPEND(GetLoggedInUser(&uid),
+                          "couldn't get local username");
+    config->user_id = uid;
+    return Status::OK();
+  }
+  string uid;
+  RETURN_NOT_OK_PREPEND(security::MapPrincipalToLocalName(*principal, &uid),
+                        "could not get local username for krb5 principal");
+  config->user_id = uid;
+  config->kerberos_principal = *principal;
+  return Status::OK();
+}
+} // anonymous namespace
+
+Status TlsContext::GenerateSelfSignedCertAndKey() {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  // Step 1: generate the private key to be self signed.
+  PrivateKey key;
+  RETURN_NOT_OK_PREPEND(GeneratePrivateKey(FLAGS_ipki_server_key_size,
+                                           &key),
+                                           "failed to generate private key");
+
+  // Step 2: generate a CSR so that the self-signed cert can eventually be
+  // replaced with a CA-signed cert.
+  CertRequestGenerator::Config config;
+  RETURN_NOT_OK(SetCertAttributes(&config));
+  CertRequestGenerator gen(config);
+  RETURN_NOT_OK_PREPEND(gen.Init(), "could not initialize CSR generator");
+  CertSignRequest csr;
+  RETURN_NOT_OK_PREPEND(gen.GenerateRequest(key, &csr), "could not generate CSR");
+
+  // Step 3: generate a self-signed cert that we can use for terminating TLS
+  // connections until we get the CA-signed cert.
+  Cert cert;
+  RETURN_NOT_OK_PREPEND(ca::CertSigner::SelfSignCert(key, config, &cert),
+                        "failed to self-sign cert");
+
+  // Workaround for an OpenSSL memory leak caused by a race in x509v3_cache_extensions.
+  // Upon first use of each certificate, this function gets called to parse various
+  // fields of the certificate. However, it's racey, so if multiple "first calls"
+  // happen concurrently, one call overwrites the cached data from another, causing
+  // a leak. Calling this nonsense X509_check_ca() forces the X509 extensions to
+  // get cached, so we don't hit the race later. 'VerifyCertChain' also has the
+  // effect of triggering the racy codepath.
+  ignore_result(X509_check_ca(cert.GetTopOfChainX509()));
+  ERR_clear_error(); // in case it left anything on the queue.
+
+  // Step 4: Adopt the new key and cert.
+  unique_lock<RWMutex> lock(lock_);
+  CHECK(!has_cert_);
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_PrivateKey(ctx_.get(), key.GetRawData()),
+                     "failed to use private key");
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_certificate(ctx_.get(), cert.GetTopOfChainX509()),
+                     "failed to use certificate");
+  has_cert_ = true;
+  csr_ = std::move(csr);
+  return Status::OK();
+}
+
+boost::optional<CertSignRequest> TlsContext::GetCsrIfNecessary() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  shared_lock<RWMutex> lock(lock_);
+  if (csr_) {
+    return csr_->Clone();
+  }
+  return boost::none;
+}
+
+Status TlsContext::AdoptSignedCert(const Cert& cert) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  unique_lock<RWMutex> lock(lock_);
+
+  if (!csr_) {
+    // A signed cert has already been adopted.
+    return Status::OK();
+  }
+
+  // Verify that the appropriate CA certs have been loaded into the context
+  // before we adopt a cert. Otherwise, client connections without the CA cert
+  // available would fail.
+  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
+
+  PublicKey csr_key;
+  RETURN_NOT_OK(csr_->GetPublicKey(&csr_key));
+  PublicKey cert_key;
+  RETURN_NOT_OK(cert.GetPublicKey(&cert_key));
+  bool equals;
+  RETURN_NOT_OK(csr_key.Equals(cert_key, &equals));
+  if (!equals) {
+    return Status::RuntimeError("certificate public key does not match the CSR public key");
+  }
+
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_certificate(ctx_.get(), cert.GetTopOfChainX509()),
+                     "failed to use certificate");
+
+  // This should never fail since we already compared the cert's public key
+  // against the CSR, but better safe than sorry. If this *does* fail, it
+  // appears to remove the private key from the SSL_CTX, so we are left in a bad
+  // state.
+  OPENSSL_CHECK_OK(SSL_CTX_check_private_key(ctx_.get()))
+    << "certificate does not match the private key";
+
+  csr_ = boost::none;
+
+  return Status::OK();
+}
+
+Status TlsContext::LoadCertificateAndKey(const string& certificate_path,
+                                         const string& key_path) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  Cert c;
+  RETURN_NOT_OK(c.FromFile(certificate_path, DataFormat::PEM));
+  PrivateKey k;
+  RETURN_NOT_OK(k.FromFile(key_path, DataFormat::PEM));
+  is_external_cert_ = true;
+  return UseCertificateAndKey(c, k);
+}
+
+Status TlsContext::LoadCertificateAndPasswordProtectedKey(const string& certificate_path,
+                                                          const string& key_path,
+                                                          const PasswordCallback& password_cb) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  Cert c;
+  RETURN_NOT_OK_PREPEND(c.FromFile(certificate_path, DataFormat::PEM),
+                        "failed to load certificate");
+  PrivateKey k;
+  RETURN_NOT_OK_PREPEND(k.FromFile(key_path, DataFormat::PEM, password_cb),
+                        "failed to load private key file");
+  RETURN_NOT_OK(UseCertificateAndKey(c, k));
+  is_external_cert_ = true;
+  return Status::OK();
+}
+
+Status TlsContext::LoadCertificateAuthority(const string& certificate_path) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (has_cert_) DCHECK(is_external_cert_);
+  Cert c;
+  RETURN_NOT_OK(c.FromFile(certificate_path, DataFormat::PEM));
+  return AddTrustedCertificate(c);
+}
+
+Status TlsContext::InitiateHandshake(TlsHandshakeType handshake_type,
+                                     TlsHandshake* handshake) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ctx_);
+  CHECK(!handshake->ssl_);
+  {
+    shared_lock<RWMutex> lock(lock_);
+    handshake->adopt_ssl(ssl_make_unique(SSL_new(ctx_.get())));
+  }
+  if (!handshake->ssl_) {
+    return Status::RuntimeError("failed to create SSL handle", GetOpenSSLErrors());
+  }
+
+  SSL_set_bio(handshake->ssl(),
+              BIO_new(BIO_s_mem()),
+              BIO_new(BIO_s_mem()));
+
+  switch (handshake_type) {
+    case TlsHandshakeType::SERVER:
+      SSL_set_accept_state(handshake->ssl());
+      break;
+    case TlsHandshakeType::CLIENT:
+      SSL_set_connect_state(handshake->ssl());
+      break;
+  }
+
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_context.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_context.h b/be/src/kudu/security/tls_context.h
new file mode 100644
index 0000000..786ab6e
--- /dev/null
+++ b/be/src/kudu/security/tls_context.h
@@ -0,0 +1,202 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+
+#include "kudu/gutil/port.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/status.h"
+// IWYU pragma: no_include "kudu/security/cert.h"
+
+namespace kudu {
+namespace security {
+
+class Cert;           // IWYU pragma: keep
+class CertSignRequest;// IWYU pragma: keep
+class PrivateKey;
+
+// TlsContext wraps data required by the OpenSSL library for creating and
+// accepting TLS protected channels. A single TlsContext instance should be used
+// per server or client instance.
+//
+// Internally, a 'TlsContext' manages a single keypair which it uses for
+// terminating TLS connections. It also manages a collection of trusted root CA
+// certificates (a trust store), as well as a signed certificate for the
+// keypair.
+//
+// When used on a server, the TlsContext can generate a keypair and a
+// self-signed certificate, and provide a CSR for transititioning to a CA-signed
+// certificate. This allows Kudu servers to start with a self-signed
+// certificate, and later adopt a CA-signed certificate as it becomes available.
+// See GenerateSelfSignedCertAndKey(), GetCsrIfNecessary(), and
+// AdoptSignedCert() for details on how to generate the keypair and self-signed
+// cert, access the CSR, and transtition to a CA-signed cert, repectively.
+//
+// When used in a client or a server, the TlsContext can immediately adopt a
+// private key and CA-signed cert using UseCertificateAndKey(). A TlsContext
+// only manages a single keypair, so if UseCertificateAndKey() is called,
+// GenerateSelfSignedCertAndKey() must not be called, and vice versa.
+//
+// TlsContext may be used with or without a keypair and cert to initiate TLS
+// connections, when mutual TLS authentication is not needed (for example, for
+// token or Kerberos authenticated connections).
+//
+// This class is thread-safe after initialization.
+class TlsContext {
+
+ public:
+
+  TlsContext();
+
+  TlsContext(std::string tls_ciphers, std::string tls_min_protocol);
+
+  ~TlsContext() = default;
+
+  Status Init() WARN_UNUSED_RESULT;
+
+  // Returns true if this TlsContext has been configured with a cert and key for
+  // use with TLS-encrypted connections.
+  bool has_cert() const {
+    shared_lock<RWMutex> lock(lock_);
+    return has_cert_;
+  }
+
+  // Returns true if this TlsContext has been configured with a CA-signed TLS
+  // cert and key for use with TLS-encrypted connections. If this method returns
+  // true, then 'has_trusted_cert' will also return true.
+  bool has_signed_cert() const {
+    shared_lock<RWMutex> lock(lock_);
+    return has_cert_ && !csr_;
+  }
+
+  // Returns true if this TlsContext has at least one certificate in its trust store.
+  bool has_trusted_cert() const {
+    shared_lock<RWMutex> lock(lock_);
+    return trusted_cert_count_ > 0;
+  }
+
+  // Adds 'cert' as a trusted root CA certificate.
+  //
+  // This determines whether other peers are trusted. It also must be called for
+  // any CA certificates that are part of the certificate chain for the cert
+  // passed in to 'UseCertificateAndKey()' or 'AdoptSignedCert()'.
+  //
+  // If this cert has already been marked as trusted, this has no effect.
+  Status AddTrustedCertificate(const Cert& cert) WARN_UNUSED_RESULT;
+
+  // Dump all of the certs that are currently trusted by this context, in DER
+  // form, into 'cert_ders'.
+  Status DumpTrustedCerts(std::vector<std::string>* cert_ders) const WARN_UNUSED_RESULT;
+
+  // Uses 'cert' and 'key' as the cert and key for use with TLS connections.
+  //
+  // Checks that the CA that issued the signature on 'cert' is already trusted
+  // by this context (e.g. by AddTrustedCertificate()).
+  Status UseCertificateAndKey(const Cert& cert, const PrivateKey& key) WARN_UNUSED_RESULT;
+
+  // Generates a self-signed cert and key for use with TLS connections.
+  //
+  // This method should only be used on the server. Once this method is called,
+  // 'GetCsrIfNecessary' can be used to retrieve a CSR for generating a
+  // CA-signed cert for the generated private key, and 'AdoptSignedCert' can be
+  // used to transition to using the CA-signed cert with subsequent TLS
+  // connections.
+  Status GenerateSelfSignedCertAndKey() WARN_UNUSED_RESULT;
+
+  // Returns a new certificate signing request (CSR) in DER format, if this
+  // context's cert is self-signed. If the cert is already signed, returns
+  // boost::none.
+  boost::optional<CertSignRequest> GetCsrIfNecessary() const;
+
+  // Adopts the provided CA-signed certificate for this TLS context.
+  //
+  // The certificate must correspond to a CSR previously returned by
+  // 'GetCsrIfNecessary()'.
+  //
+  // Checks that the CA that issued the signature on 'cert' is already trusted
+  // by this context (e.g. by AddTrustedCertificate()).
+  //
+  // This has no effect if the instance already has a CA-signed cert.
+  Status AdoptSignedCert(const Cert& cert) WARN_UNUSED_RESULT;
+
+  // Convenience functions for loading cert/CA/key from file paths.
+  // -------------------------------------------------------------
+
+  // Load the server certificate and key (PEM encoded).
+  Status LoadCertificateAndKey(const std::string& certificate_path,
+                               const std::string& key_path) WARN_UNUSED_RESULT;
+
+  // Load the server certificate and key (PEM encoded), and use the callback
+  // 'password_cb' to obtain the password that can decrypt the key.
+  Status LoadCertificateAndPasswordProtectedKey(const std::string& certificate_path,
+                                                const std::string& key_path,
+                                                const PasswordCallback& password_cb)
+                                                WARN_UNUSED_RESULT;
+
+  // Load the certificate authority (PEM encoded).
+  Status LoadCertificateAuthority(const std::string& certificate_path) WARN_UNUSED_RESULT;
+
+  // Initiates a new TlsHandshake instance.
+  Status InitiateHandshake(TlsHandshakeType handshake_type,
+                           TlsHandshake* handshake) const WARN_UNUSED_RESULT;
+
+  // Return the number of certs that have been marked as trusted.
+  // Used by tests.
+  int trusted_cert_count_for_tests() const {
+    shared_lock<RWMutex> lock(lock_);
+    return trusted_cert_count_;
+  }
+
+  bool is_external_cert() const { return is_external_cert_; }
+
+ private:
+
+  Status VerifyCertChainUnlocked(const Cert& cert) WARN_UNUSED_RESULT;
+
+  // The cipher suite preferences to use for TLS-secured RPC connections. Uses the OpenSSL
+  // cipher preference list format. See man (1) ciphers for more information.
+  std::string tls_ciphers_;
+
+  // The minimum protocol version to allow when for securing RPC connections with TLS. May be
+  // one of 'TLSv1', 'TLSv1.1', or 'TLSv1.2'.
+  std::string tls_min_protocol_;
+
+  // Protects all members.
+  //
+  // Taken in write mode when any changes are modifying the underlying SSL_CTX
+  // using a mutating method (eg SSL_CTX_use_*) or when changing the value of
+  // any of our own member variables.
+  mutable RWMutex lock_;
+  c_unique_ptr<SSL_CTX> ctx_;
+  int32_t trusted_cert_count_;
+  bool has_cert_;
+  bool is_external_cert_;
+  boost::optional<CertSignRequest> csr_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_handshake-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_handshake-test.cc b/be/src/kudu/security/tls_handshake-test.cc
new file mode 100644
index 0000000..c863d05
--- /dev/null
+++ b/be/src/kudu/security/tls_handshake-test.cc
@@ -0,0 +1,390 @@
+// 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 "kudu/security/tls_handshake.h"
+
+#include <atomic>
+#include <iostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/security-test-util.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+DECLARE_int32(ipki_server_key_size);
+
+namespace kudu {
+namespace security {
+
+using ca::CertSigner;
+
+struct Case {
+  PkiConfig client_pki;
+  TlsVerificationMode client_verification;
+  PkiConfig server_pki;
+  TlsVerificationMode server_verification;
+  Status expected_status;
+};
+
+// Beautifies CLI test output.
+std::ostream& operator<<(std::ostream& o, Case c) {
+  auto verification_mode_name = [] (const TlsVerificationMode& verification_mode) {
+    switch (verification_mode) {
+      case TlsVerificationMode::VERIFY_NONE: return "NONE";
+      case TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST: return "REMOTE_CERT_AND_HOST";
+    }
+    return "unreachable";
+  };
+
+  o << "{client-pki: " << c.client_pki << ", "
+    << "client-verification: " << verification_mode_name(c.client_verification) << ", "
+    << "server-pki: " << c.server_pki << ", "
+    << "server-verification: " << verification_mode_name(c.server_verification) << ", "
+    << "expected-status: " << c.expected_status.ToString() << "}";
+
+  return o;
+}
+
+class TestTlsHandshakeBase : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    ASSERT_OK(client_tls_.Init());
+    ASSERT_OK(server_tls_.Init());
+  }
+
+ protected:
+  // Run a handshake using 'client_tls_' and 'server_tls_'. The client and server
+  // verification modes are set to 'client_verify' and 'server_verify' respectively.
+  Status RunHandshake(TlsVerificationMode client_verify,
+                      TlsVerificationMode server_verify) {
+    TlsHandshake client, server;
+    RETURN_NOT_OK(client_tls_.InitiateHandshake(TlsHandshakeType::CLIENT, &client));
+    RETURN_NOT_OK(server_tls_.InitiateHandshake(TlsHandshakeType::SERVER, &server));
+
+    client.set_verification_mode(client_verify);
+    server.set_verification_mode(server_verify);
+
+    bool client_done = false, server_done = false;
+    string to_client;
+    string to_server;
+    while (!client_done || !server_done) {
+      if (!client_done) {
+        Status s = client.Continue(to_client, &to_server);
+        VLOG(1) << "client->server: " << to_server.size() << " bytes";
+        if (s.ok()) {
+          client_done = true;
+        } else if (!s.IsIncomplete()) {
+          CHECK(s.IsRuntimeError());
+          return s.CloneAndPrepend("client error");
+        }
+      }
+      if (!server_done) {
+        CHECK(!client_done);
+        Status s = server.Continue(to_server, &to_client);
+        VLOG(1) << "server->client: " << to_client.size() << " bytes";
+        if (s.ok()) {
+          server_done = true;
+        } else if (!s.IsIncomplete()) {
+          CHECK(s.IsRuntimeError());
+          return s.CloneAndPrepend("server error");
+        }
+      }
+    }
+    return Status::OK();
+  }
+
+  TlsContext client_tls_;
+  TlsContext server_tls_;
+
+  string cert_path_;
+  string key_path_;
+};
+
+class TestTlsHandshake : public TestTlsHandshakeBase,
+                   public ::testing::WithParamInterface<Case> {};
+
+class TestTlsHandshakeConcurrent : public TestTlsHandshakeBase,
+                   public ::testing::WithParamInterface<int> {};
+
+// Test concurrently running handshakes while changing the certificates on the TLS
+// context. We parameterize across different numbers of threads, because surprisingly,
+// fewer threads seems to trigger issues more easily in some cases.
+INSTANTIATE_TEST_CASE_P(NumThreads, TestTlsHandshakeConcurrent, ::testing::Values(1, 2, 4, 8));
+TEST_P(TestTlsHandshakeConcurrent, TestConcurrentAdoptCert) {
+  const int kNumThreads = GetParam();
+
+  ASSERT_OK(server_tls_.GenerateSelfSignedCertAndKey());
+  std::atomic<bool> done(false);
+  vector<std::thread> handshake_threads;
+  for (int i = 0; i < kNumThreads; i++) {
+    handshake_threads.emplace_back([&]() {
+        while (!done) {
+          RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE);
+        }
+      });
+  }
+  auto c = MakeScopedCleanup([&](){
+      done = true;
+      for (std::thread& t : handshake_threads) {
+        t.join();
+      }
+    });
+
+  SleepFor(MonoDelta::FromMilliseconds(10));
+  {
+    PrivateKey ca_key;
+    Cert ca_cert;
+    ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+    Cert cert;
+    ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(*server_tls_.GetCsrIfNecessary(), &cert));
+    ASSERT_OK(server_tls_.AddTrustedCertificate(ca_cert));
+    ASSERT_OK(server_tls_.AdoptSignedCert(cert));
+  }
+  SleepFor(MonoDelta::FromMilliseconds(10));
+}
+
+TEST_F(TestTlsHandshake, TestHandshakeSequence) {
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  // Both client and server have certs and CA.
+  ASSERT_OK(ConfigureTlsContext(PkiConfig::SIGNED, ca_cert, ca_key, &client_tls_));
+  ASSERT_OK(ConfigureTlsContext(PkiConfig::SIGNED, ca_cert, ca_key, &server_tls_));
+
+  TlsHandshake server;
+  TlsHandshake client;
+  ASSERT_OK(client_tls_.InitiateHandshake(TlsHandshakeType::SERVER, &server));
+  ASSERT_OK(server_tls_.InitiateHandshake(TlsHandshakeType::CLIENT, &client));
+
+  string buf1;
+  string buf2;
+
+  // Client sends Hello
+  ASSERT_TRUE(client.Continue(buf1, &buf2).IsIncomplete());
+  ASSERT_GT(buf2.size(), 0);
+
+  // Server receives client Hello, and sends server Hello
+  ASSERT_TRUE(server.Continue(buf2, &buf1).IsIncomplete());
+  ASSERT_GT(buf1.size(), 0);
+
+  // Client receives server Hello and sends client Finished
+  ASSERT_TRUE(client.Continue(buf1, &buf2).IsIncomplete());
+  ASSERT_GT(buf2.size(), 0);
+
+  // Server receives client Finished and sends server Finished
+  ASSERT_OK(server.Continue(buf2, &buf1));
+  ASSERT_GT(buf1.size(), 0);
+
+  // Client receives server Finished
+  ASSERT_OK(client.Continue(buf1, &buf2));
+  ASSERT_EQ(buf2.size(), 0);
+}
+
+// Tests that the TlsContext can transition from self signed cert to signed
+// cert, and that it rejects invalid certs along the way. We are testing this
+// here instead of in a dedicated TlsContext test because it requires completing
+// handshakes to fully validate.
+TEST_F(TestTlsHandshake, TestTlsContextCertTransition) {
+  ASSERT_FALSE(server_tls_.has_cert());
+  ASSERT_FALSE(server_tls_.has_signed_cert());
+  ASSERT_EQ(boost::none, server_tls_.GetCsrIfNecessary());
+
+  ASSERT_OK(server_tls_.GenerateSelfSignedCertAndKey());
+  ASSERT_TRUE(server_tls_.has_cert());
+  ASSERT_FALSE(server_tls_.has_signed_cert());
+  ASSERT_NE(boost::none, server_tls_.GetCsrIfNecessary());
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE));
+  ASSERT_STR_MATCHES(RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                                  TlsVerificationMode::VERIFY_NONE).ToString(),
+                     "client error:.*certificate verify failed");
+
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  Cert cert;
+  ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(*server_tls_.GetCsrIfNecessary(), &cert));
+
+  // Try to adopt the cert without first trusting the CA.
+  ASSERT_STR_MATCHES(server_tls_.AdoptSignedCert(cert).ToString(),
+                     "could not verify certificate chain");
+
+  // Check that we can still do (unverified) handshakes.
+  ASSERT_TRUE(server_tls_.has_cert());
+  ASSERT_FALSE(server_tls_.has_signed_cert());
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE));
+
+  // Trust the root cert.
+  ASSERT_OK(server_tls_.AddTrustedCertificate(ca_cert));
+
+  // Generate a bogus cert and attempt to adopt it.
+  Cert bogus_cert;
+  {
+    TlsContext bogus_tls;
+    ASSERT_OK(bogus_tls.Init());
+    ASSERT_OK(bogus_tls.GenerateSelfSignedCertAndKey());
+    ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(*bogus_tls.GetCsrIfNecessary(), &bogus_cert));
+  }
+  ASSERT_STR_MATCHES(server_tls_.AdoptSignedCert(bogus_cert).ToString(),
+                     "certificate public key does not match the CSR public key");
+
+  // Check that we can still do (unverified) handshakes.
+  ASSERT_TRUE(server_tls_.has_cert());
+  ASSERT_FALSE(server_tls_.has_signed_cert());
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE));
+
+  // Adopt the legitimate signed cert.
+  ASSERT_OK(server_tls_.AdoptSignedCert(cert));
+
+  // Check that we can do verified handshakes.
+  ASSERT_TRUE(server_tls_.has_cert());
+  ASSERT_TRUE(server_tls_.has_signed_cert());
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE));
+  ASSERT_OK(client_tls_.AddTrustedCertificate(ca_cert));
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                         TlsVerificationMode::VERIFY_NONE));
+}
+
+TEST_P(TestTlsHandshake, TestHandshake) {
+  Case test_case = GetParam();
+
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  ASSERT_OK(ConfigureTlsContext(test_case.client_pki, ca_cert, ca_key, &client_tls_));
+  ASSERT_OK(ConfigureTlsContext(test_case.server_pki, ca_cert, ca_key, &server_tls_));
+
+  Status s = RunHandshake(test_case.client_verification, test_case.server_verification);
+
+  EXPECT_EQ(test_case.expected_status.CodeAsString(), s.CodeAsString());
+  ASSERT_STR_MATCHES(s.ToString(), test_case.expected_status.message().ToString());
+}
+
+INSTANTIATE_TEST_CASE_P(CertCombinations,
+                        TestTlsHandshake,
+                        ::testing::Values(
+
+        // We don't test any cases where the server has no cert or the client
+        // has a self-signed cert, since we don't expect those to occur in
+        // practice.
+
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("server error:.*peer did not return a certificate") },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("server error:.*peer did not return a certificate") },
+        Case { PkiConfig::NONE, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("server error:.*peer did not return a certificate") },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("server error:.*peer did not return a certificate") },
+        Case { PkiConfig::TRUSTED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("server error:.*peer did not return a certificate") },
+
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               // OpenSSL 1.0.0 returns "no certificate returned" for this case,
+               // which appears to be a bug.
+               Status::RuntimeError("server error:.*(certificate verify failed|"
+                                                    "no certificate returned)") },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SELF_SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::RuntimeError("client error:.*certificate verify failed") },
+
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               Status::OK() },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_NONE,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::OK() },
+        Case { PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               PkiConfig::SIGNED, TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+               Status::OK() }
+));
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_handshake.cc b/be/src/kudu/security/tls_handshake.cc
new file mode 100644
index 0000000..5a89592
--- /dev/null
+++ b/be/src/kudu/security/tls_handshake.cc
@@ -0,0 +1,274 @@
+// 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 "kudu/security/tls_handshake.h"
+
+#include <memory>
+#include <string>
+
+#include <openssl/ssl.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/tls_socket.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/trace.h"
+
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+#include "kudu/security/x509_check_host.h"
+#endif // OPENSSL_VERSION_NUMBER
+
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+void TlsHandshake::SetSSLVerify() {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ssl_);
+  CHECK(!has_started_);
+  int ssl_mode = 0;
+  switch (verification_mode_) {
+    case TlsVerificationMode::VERIFY_NONE:
+      ssl_mode = SSL_VERIFY_NONE;
+      break;
+    case TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST:
+      // Server mode: the server sends a client certificate request to the client. The
+      // certificate returned (if any) is checked. If the verification process fails, the TLS/SSL
+      // handshake is immediately terminated with an alert message containing the reason for the
+      // verification failure. The behaviour can be controlled by the additional
+      // SSL_VERIFY_FAIL_IF_NO_PEER_CERT and SSL_VERIFY_CLIENT_ONCE flags.
+
+      // Client mode: the server certificate is verified. If the verification process fails, the
+      // TLS/SSL handshake is immediately terminated with an alert message containing the reason
+      // for the verification failure. If no server certificate is sent, because an anonymous
+      // cipher is used, SSL_VERIFY_PEER is ignored.
+      ssl_mode |= SSL_VERIFY_PEER;
+
+      // Server mode: if the client did not return a certificate, the TLS/SSL handshake is
+      // immediately terminated with a "handshake failure" alert. This flag must be used
+      // together with SSL_VERIFY_PEER.
+      ssl_mode |= SSL_VERIFY_FAIL_IF_NO_PEER_CERT;
+      // Server mode: only request a client certificate on the initial TLS/SSL handshake. Do
+      // not ask for a client certificate again in case of a renegotiation. This flag must be
+      // used together with SSL_VERIFY_PEER.
+      ssl_mode |= SSL_VERIFY_CLIENT_ONCE;
+      break;
+  }
+
+  SSL_set_verify(ssl_.get(), ssl_mode, /* callback = */nullptr);
+}
+
+Status TlsHandshake::Continue(const string& recv, string* send) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (!has_started_) {
+    SetSSLVerify();
+    has_started_ = true;
+  }
+  CHECK(ssl_);
+
+  BIO* rbio = SSL_get_rbio(ssl_.get());
+  int n = BIO_write(rbio, recv.data(), recv.size());
+  DCHECK(n == recv.size() || (n == -1 && recv.empty()));
+  DCHECK_EQ(BIO_ctrl_pending(rbio), recv.size());
+
+  int rc = SSL_do_handshake(ssl_.get());
+  if (rc != 1) {
+    int ssl_err = SSL_get_error(ssl_.get(), rc);
+    // WANT_READ and WANT_WRITE indicate that the handshake is not yet complete.
+    if (ssl_err != SSL_ERROR_WANT_READ && ssl_err != SSL_ERROR_WANT_WRITE) {
+      return Status::RuntimeError("TLS Handshake error", GetSSLErrorDescription(ssl_err));
+    }
+    // In the case that we got SSL_ERROR_WANT_READ or SSL_ERROR_WANT_WRITE,
+    // the OpenSSL implementation guarantees that there is no error entered into
+    // the ERR error queue, so no need to ERR_clear_error() here.
+  }
+
+  BIO* wbio = SSL_get_wbio(ssl_.get());
+  int pending = BIO_ctrl_pending(wbio);
+
+  send->resize(pending);
+  BIO_read(wbio, &(*send)[0], send->size());
+  DCHECK_EQ(BIO_ctrl_pending(wbio), 0);
+
+  if (rc == 1) {
+    // The handshake is done, but in the case of the server, we still need to
+    // send the final response to the client.
+    DCHECK_GE(send->size(), 0);
+    return Status::OK();
+  }
+  return Status::Incomplete("TLS Handshake incomplete");
+}
+
+Status TlsHandshake::Verify(const Socket& socket) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  DCHECK(SSL_is_init_finished(ssl_.get()));
+  CHECK(ssl_);
+
+  if (verification_mode_ == TlsVerificationMode::VERIFY_NONE) {
+    return Status::OK();
+  }
+  DCHECK(verification_mode_ == TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST);
+
+  int rc = SSL_get_verify_result(ssl_.get());
+  if (rc != X509_V_OK) {
+    return Status::NotAuthorized(Substitute("SSL cert verification failed: $0",
+                                            X509_verify_cert_error_string(rc)),
+                                 GetOpenSSLErrors());
+  }
+
+  // Get the peer certificate.
+  X509* cert = remote_cert_.GetTopOfChainX509();
+  if (!cert) {
+    if (SSL_get_verify_mode(ssl_.get()) & SSL_VERIFY_FAIL_IF_NO_PEER_CERT) {
+      return Status::NotAuthorized("Handshake failed: unable to retreive peer certificate");
+    }
+    // No cert, but we weren't requiring one.
+    TRACE("Got no cert from peer, but not required");
+    return Status::OK();
+  }
+
+  // TODO(KUDU-1886): Do hostname verification.
+  /*
+  TRACE("Verifying peer cert");
+
+  // Get the peer's hostname
+  Sockaddr peer_addr;
+  if (!socket.GetPeerAddress(&peer_addr).ok()) {
+    return Status::NotAuthorized(
+        "TLS certificate hostname verification failed: unable to get peer address");
+  }
+  string peer_hostname;
+  RETURN_NOT_OK_PREPEND(peer_addr.LookupHostname(&peer_hostname),
+      "TLS certificate hostname verification failed: unable to lookup peer hostname");
+
+  // Check if the hostname matches with either the Common Name or any of the Subject Alternative
+  // Names of the certificate.
+  int match = X509_check_host(cert,
+                              peer_hostname.c_str(),
+                              peer_hostname.length(),
+                              0,
+                              nullptr);
+  if (match == 0) {
+    return Status::NotAuthorized("TLS certificate hostname verification failed");
+  }
+  if (match < 0) {
+    return Status::RuntimeError("TLS certificate hostname verification error", GetOpenSSLErrors());
+  }
+  DCHECK_EQ(match, 1);
+  */
+  return Status::OK();
+}
+
+Status TlsHandshake::GetCerts() {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  X509* cert = SSL_get_certificate(ssl_.get());
+  if (cert) {
+    // For whatever reason, SSL_get_certificate (unlike SSL_get_peer_certificate)
+    // does not increment the X509's reference count.
+    local_cert_.AdoptAndAddRefX509(cert);
+  }
+
+  cert = SSL_get_peer_certificate(ssl_.get());
+  if (cert) {
+    remote_cert_.AdoptX509(cert);
+  }
+  return Status::OK();
+}
+
+Status TlsHandshake::Finish(unique_ptr<Socket>* socket) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  RETURN_NOT_OK(GetCerts());
+  RETURN_NOT_OK(Verify(**socket));
+
+  int fd = (*socket)->Release();
+
+  // Give the socket to the SSL instance. This will automatically free the
+  // read and write memory BIO instances.
+  int ret = SSL_set_fd(ssl_.get(), fd);
+  if (ret != 1) {
+    return Status::RuntimeError("TLS handshake error", GetOpenSSLErrors());
+  }
+
+  // Transfer the SSL instance to the socket.
+  socket->reset(new TlsSocket(fd, std::move(ssl_)));
+
+  return Status::OK();
+}
+
+Status TlsHandshake::FinishNoWrap(const Socket& socket) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  RETURN_NOT_OK(GetCerts());
+  return Verify(socket);
+}
+
+Status TlsHandshake::GetLocalCert(Cert* cert) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (!local_cert_.GetRawData()) {
+    return Status::RuntimeError("no local certificate");
+  }
+  cert->AdoptAndAddRefRawData(local_cert_.GetRawData());
+  return Status::OK();
+}
+
+Status TlsHandshake::GetRemoteCert(Cert* cert) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (!remote_cert_.GetRawData()) {
+    return Status::RuntimeError("no remote certificate");
+  }
+  cert->AdoptAndAddRefRawData(remote_cert_.GetRawData());
+  return Status::OK();
+}
+
+string TlsHandshake::GetCipherSuite() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(has_started_);
+  return SSL_get_cipher_name(ssl_.get());
+}
+
+string TlsHandshake::GetProtocol() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(has_started_);
+  return SSL_get_version(ssl_.get());
+}
+
+string TlsHandshake::GetCipherDescription() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(has_started_);
+  const SSL_CIPHER* cipher = SSL_get_current_cipher(ssl_.get());
+  if (!cipher) {
+    return "NONE";
+  }
+  char buf[512];
+  const char* description = SSL_CIPHER_description(cipher, buf, sizeof(buf));
+  if (!description) {
+    return "NONE";
+  }
+  string ret(description);
+  StripTrailingNewline(&ret);
+  StripDupCharacters(&ret, ' ', 0);
+  return ret;
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_handshake.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_handshake.h b/be/src/kudu/security/tls_handshake.h
new file mode 100644
index 0000000..56020c4
--- /dev/null
+++ b/be/src/kudu/security/tls_handshake.h
@@ -0,0 +1,171 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Socket;
+
+namespace security {
+
+enum class TlsHandshakeType {
+  // The local endpoint is the TLS client (initiator).
+  CLIENT,
+  // The local endpoint is the TLS server (acceptor).
+  SERVER,
+};
+
+// Mode for performing verification of the remote peer's identity during a handshake.
+enum class TlsVerificationMode {
+  // SERVER:
+  //    No certificate will be requested from the client, and no verification
+  //    will be done.
+  // CLIENT:
+  //    The server's certificate will be obtained but no verification will be done.
+  //    (the server still requires a certificate, even if it is self-signed).
+  VERIFY_NONE,
+
+  // BOTH:
+  // The remote peer is required to have a signed certificate. The certificate will
+  // be verified in two ways:
+  //  1) The certificate must be signed by a trusted CA (or chain of CAs).
+  //  2) Second, the hostname of the remote peer (as determined by reverse DNS of the
+  //    socket address) must match the common name or one of the Subject Alternative
+  //    Names stored in the certificate.
+  VERIFY_REMOTE_CERT_AND_HOST
+};
+
+// TlsHandshake manages an ongoing TLS handshake between a client and server.
+//
+// TlsHandshake instances are default constructed, but must be initialized
+// before use using TlsContext::InitiateHandshake.
+class TlsHandshake {
+ public:
+
+   TlsHandshake() = default;
+   ~TlsHandshake() = default;
+
+  // Set the verification mode for this handshake. The default verification mode
+  // is VERIFY_REMOTE_CERT_AND_HOST.
+  //
+  // This must be called before the first call to Continue().
+  void set_verification_mode(TlsVerificationMode mode) {
+    DCHECK(!has_started_);
+    verification_mode_ = mode;
+  }
+
+  // Continue or start a new handshake.
+  //
+  // 'recv' should contain the input buffer from the remote end, or an empty
+  // string when the handshake is new.
+  //
+  // 'send' should contain the output buffer which must be sent to the remote
+  // end.
+  //
+  // Returns Status::OK when the handshake is complete, however the 'send'
+  // buffer may contain a message which must still be transmitted to the remote
+  // end. If the send buffer is empty after this call and the return is
+  // Status::OK, the socket should immediately be wrapped in the TLS channel
+  // using 'Finish'. If the send buffer is not empty, the message should be sent
+  // to the remote end, and then the socket should be wrapped using 'Finish'.
+  //
+  // Returns Status::Incomplete when the handshake must continue for another
+  // round of messages.
+  //
+  // Returns any other status code on error.
+  Status Continue(const std::string& recv, std::string* send) WARN_UNUSED_RESULT;
+
+  // Finishes the handshake, wrapping the provided socket in the negotiated TLS
+  // channel. This 'TlsHandshake' instance should not be used again after
+  // calling this.
+  Status Finish(std::unique_ptr<Socket>* socket) WARN_UNUSED_RESULT;
+
+  // Finish the handshake, using the provided socket to verify the remote peer,
+  // but without wrapping the socket.
+  Status FinishNoWrap(const Socket& socket) WARN_UNUSED_RESULT;
+
+  // Retrieve the local certificate. This will return an error status if there
+  // is no local certificate.
+  //
+  // May only be called after 'Finish' or 'FinishNoWrap'.
+  Status GetLocalCert(Cert* cert) const WARN_UNUSED_RESULT;
+
+  // Retrieve the remote peer's certificate. This will return an error status if
+  // there is no remote certificate.
+  //
+  // May only be called after 'Finish' or 'FinishNoWrap'.
+  Status GetRemoteCert(Cert* cert) const WARN_UNUSED_RESULT;
+
+  // Retrieve the negotiated cipher suite. Only valid to call after the
+  // handshake is complete and before 'Finish()'.
+  std::string GetCipherSuite() const;
+
+  // Retrieve the negotiated TLS protocol version. Only valid to call after the
+  // handshake is complete and before 'Finish()'.
+  std::string GetProtocol() const;
+
+  // Retrive the description of the negotiated cipher.
+  // Only valid to call after the handshake is complete and before 'Finish()'.
+  std::string GetCipherDescription() const;
+
+ private:
+  friend class TlsContext;
+
+  bool has_started_ = false;
+  TlsVerificationMode verification_mode_ = TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST;
+
+  // Set the verification mode on the underlying SSL object.
+  void SetSSLVerify();
+
+  // Set the SSL to use during the handshake. Called once by
+  // TlsContext::InitiateHandshake before starting the handshake processes.
+  void adopt_ssl(c_unique_ptr<SSL> ssl) {
+    CHECK(!ssl_);
+    ssl_ = std::move(ssl);
+  }
+
+  SSL* ssl() {
+    return ssl_.get();
+  }
+
+  // Populates local_cert_ and remote_cert_.
+  Status GetCerts() WARN_UNUSED_RESULT;
+
+  // Verifies that the handshake is valid for the provided socket.
+  Status Verify(const Socket& socket) const WARN_UNUSED_RESULT;
+
+  // Owned SSL handle.
+  c_unique_ptr<SSL> ssl_;
+
+  Cert local_cert_;
+  Cert remote_cert_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_socket-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_socket-test.cc b/be/src/kudu/security/tls_socket-test.cc
new file mode 100644
index 0000000..001a206
--- /dev/null
+++ b/be/src/kudu/security/tls_socket-test.cc
@@ -0,0 +1,366 @@
+// 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 "kudu/security/tls_handshake.h"
+
+#include <algorithm>
+#include <pthread.h>
+#include <sched.h>
+#include <sys/uio.h>
+
+#include <atomic>
+#include <csignal>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <iostream>
+#include <memory>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+const MonoDelta kTimeout = MonoDelta::FromSeconds(10);
+
+// Size is big enough to not fit into output socket buffer of default size
+// (controlled by setsockopt() with SO_SNDBUF).
+constexpr size_t kEchoChunkSize = 32 * 1024 * 1024;
+
+class TlsSocketTest : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+    ASSERT_OK(client_tls_.Init());
+  }
+
+ protected:
+  void ConnectClient(const Sockaddr& addr, unique_ptr<Socket>* sock);
+  TlsContext client_tls_;
+};
+
+Status DoNegotiationSide(Socket* sock, TlsHandshake* tls, const char* side) {
+  tls->set_verification_mode(TlsVerificationMode::VERIFY_NONE);
+
+  bool done = false;
+  string received;
+  while (!done) {
+    string to_send;
+    Status s = tls->Continue(received, &to_send);
+    if (s.ok()) {
+      done = true;
+    } else if (!s.IsIncomplete()) {
+      RETURN_NOT_OK_PREPEND(s, "unexpected tls error");
+    }
+    if (!to_send.empty()) {
+      size_t nwritten;
+      auto deadline = MonoTime::Now() + MonoDelta::FromSeconds(10);
+      RETURN_NOT_OK_PREPEND(sock->BlockingWrite(
+          reinterpret_cast<const uint8_t*>(to_send.data()),
+          to_send.size(), &nwritten, deadline),
+                            "error sending");
+    }
+
+    if (!done) {
+      uint8_t buf[1024];
+      int32_t n = 0;
+      RETURN_NOT_OK_PREPEND(sock->Recv(buf, arraysize(buf), &n),
+                            "error receiving");
+      received = string(reinterpret_cast<char*>(&buf[0]), n);
+    }
+  }
+  LOG(INFO) << side << ": negotiation complete";
+  return Status::OK();
+}
+
+void TlsSocketTest::ConnectClient(const Sockaddr& addr, unique_ptr<Socket>* sock) {
+  unique_ptr<Socket> client_sock(new Socket());
+  ASSERT_OK(client_sock->Init(0));
+  ASSERT_OK(client_sock->Connect(addr));
+
+  TlsHandshake client;
+  ASSERT_OK(client_tls_.InitiateHandshake(TlsHandshakeType::CLIENT, &client));
+  ASSERT_OK(DoNegotiationSide(client_sock.get(), &client, "client"));
+  ASSERT_OK(client.Finish(&client_sock));
+  *sock = std::move(client_sock);
+}
+
+class EchoServer {
+ public:
+  EchoServer()
+      : pthread_sync_(1) {
+  }
+  ~EchoServer() {
+    Stop();
+    Join();
+  }
+
+  void Start() {
+    ASSERT_OK(server_tls_.Init());
+    ASSERT_OK(server_tls_.GenerateSelfSignedCertAndKey());
+    ASSERT_OK(listen_addr_.ParseString("127.0.0.1", 0));
+    ASSERT_OK(listener_.Init(0));
+    ASSERT_OK(listener_.BindAndListen(listen_addr_, /*listen_queue_size=*/10));
+    ASSERT_OK(listener_.GetSocketAddress(&listen_addr_));
+
+    thread_ = thread([&] {
+        pthread_ = pthread_self();
+        pthread_sync_.CountDown();
+        unique_ptr<Socket> sock(new Socket());
+        Sockaddr remote;
+        CHECK_OK(listener_.Accept(sock.get(), &remote, /*flags=*/0));
+
+        TlsHandshake server;
+        CHECK_OK(server_tls_.InitiateHandshake(TlsHandshakeType::SERVER, &server));
+        CHECK_OK(DoNegotiationSide(sock.get(), &server, "server"));
+        CHECK_OK(server.Finish(&sock));
+
+        CHECK_OK(sock->SetRecvTimeout(kTimeout));
+        unique_ptr<uint8_t[]> buf(new uint8_t[kEchoChunkSize]);
+        // An "echo" loop for kEchoChunkSize byte buffers.
+        while (!stop_) {
+          size_t n;
+          Status s = sock->BlockingRecv(buf.get(), kEchoChunkSize, &n, MonoTime::Now() + kTimeout);
+          if (!s.ok()) {
+            CHECK(stop_) << "unexpected error reading: " << s.ToString();
+          }
+
+          LOG(INFO) << "server echoing " << n << " bytes";
+          size_t written;
+          s = sock->BlockingWrite(buf.get(), n, &written, MonoTime::Now() + kTimeout);
+          if (!s.ok()) {
+            CHECK(stop_) << "unexpected error writing: " << s.ToString();
+          }
+          if (slow_read_) {
+            SleepFor(MonoDelta::FromMilliseconds(10));
+          }
+        }
+
+        CHECK_OK(listener_.Close());
+      });
+  }
+
+  void EnableSlowRead() {
+    slow_read_ = true;
+  }
+
+  const Sockaddr& listen_addr() const {
+    return listen_addr_;
+  }
+
+  bool stopped() const {
+    return stop_;
+  }
+
+  void Stop() {
+    stop_ = true;
+  }
+  void Join() {
+    thread_.join();
+  }
+
+  const pthread_t& pthread() {
+    pthread_sync_.Wait();
+    return pthread_;
+  }
+
+ private:
+  TlsContext server_tls_;
+  Socket listener_;
+  Sockaddr listen_addr_;
+  thread thread_;
+  pthread_t pthread_;
+  CountDownLatch pthread_sync_;
+  std::atomic<bool> stop_ { false };
+
+  bool slow_read_ = false;
+};
+
+void handler(int /* signal */) {}
+
+TEST_F(TlsSocketTest, TestRecvFailure) {
+    EchoServer server;
+    server.Start();
+    unique_ptr<Socket> client_sock;
+    NO_FATALS(ConnectClient(server.listen_addr(), &client_sock));
+    unique_ptr<uint8_t[]> buf(new uint8_t[kEchoChunkSize]);
+
+    SleepFor(MonoDelta::FromMilliseconds(100));
+    server.Stop();
+
+    size_t nwritten;
+    ASSERT_OK(client_sock->BlockingWrite(buf.get(), kEchoChunkSize, &nwritten,
+        MonoTime::Now() + kTimeout));
+    size_t nread;
+
+    ASSERT_OK(client_sock->BlockingRecv(buf.get(), kEchoChunkSize, &nread,
+        MonoTime::Now() + kTimeout));
+
+    Status s = client_sock->BlockingRecv(buf.get(), kEchoChunkSize, &nread,
+        MonoTime::Now() + kTimeout);
+
+    ASSERT_TRUE(!s.ok());
+    ASSERT_TRUE(s.IsNetworkError());
+    ASSERT_STR_MATCHES(s.message().ToString(), "BlockingRecv error: failed to read from "
+                                               "TLS socket \\(remote: 127.0.0.1:[0-9]+\\): ");
+}
+
+// Test for failures to handle EINTR during TLS connection
+// negotiation and data send/receive.
+TEST_F(TlsSocketTest, TestTlsSocketInterrupted) {
+  // Set up a no-op signal handler for SIGUSR2.
+  struct sigaction sa, sa_old;
+  memset(&sa, 0, sizeof(sa));
+  sa.sa_handler = &handler;
+  sigaction(SIGUSR2, &sa, &sa_old);
+  SCOPED_CLEANUP({ sigaction(SIGUSR2, &sa_old, nullptr); });
+
+  EchoServer server;
+  NO_FATALS(server.Start());
+
+  // Start a thread to send signals to the server thread.
+  thread killer([&]() {
+      while (!server.stopped()) {
+        PCHECK(pthread_kill(server.pthread(), SIGUSR2) == 0);
+        SleepFor(MonoDelta::FromMicroseconds(rand() % 10));
+      }
+    });
+  SCOPED_CLEANUP({ killer.join(); });
+
+  unique_ptr<Socket> client_sock;
+  NO_FATALS(ConnectClient(server.listen_addr(), &client_sock));
+
+  unique_ptr<uint8_t[]> buf(new uint8_t[kEchoChunkSize]);
+  for (int i = 0; i < 10; i++) {
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    size_t nwritten;
+    ASSERT_OK(client_sock->BlockingWrite(buf.get(), kEchoChunkSize, &nwritten,
+        MonoTime::Now() + kTimeout));
+    size_t n;
+    ASSERT_OK(client_sock->BlockingRecv(buf.get(), kEchoChunkSize, &n,
+        MonoTime::Now() + kTimeout));
+  }
+  server.Stop();
+  ASSERT_OK(client_sock->Close());
+  LOG(INFO) << "client done";
+}
+
+// Return an iovec containing the same data as the buffer 'buf' with the length 'len',
+// but split into random-sized chunks. The chunks are sized randomly between 1 and
+// 'max_chunk_size' bytes.
+vector<struct iovec> ChunkIOVec(Random* rng, uint8_t* buf, int len, int max_chunk_size) {
+  vector<struct iovec> ret;
+  uint8_t* p = buf;
+  int rem = len;
+  while (rem > 0) {
+    int len = rng->Uniform(max_chunk_size) + 1;
+    len = std::min(len, rem);
+    ret.push_back({p, static_cast<size_t>(len)});
+    p += len;
+    rem -= len;
+  }
+  return ret;
+}
+
+// Regression test for KUDU-2218, a bug in which Writev would improperly handle
+// partial writes in non-blocking mode.
+TEST_F(TlsSocketTest, TestNonBlockingWritev) {
+  Random rng(GetRandomSeed32());
+
+  EchoServer server;
+  server.EnableSlowRead();
+  NO_FATALS(server.Start());
+
+  unique_ptr<Socket> client_sock;
+  NO_FATALS(ConnectClient(server.listen_addr(), &client_sock));
+
+  unique_ptr<uint8_t[]> buf(new uint8_t[kEchoChunkSize]);
+  unique_ptr<uint8_t[]> rbuf(new uint8_t[kEchoChunkSize]);
+  RandomString(buf.get(), kEchoChunkSize, &rng);
+
+  for (int i = 0; i < 10; i++) {
+    ASSERT_OK(client_sock->SetNonBlocking(true));
+
+    // Prepare an IOV with the input data split into a bunch of randomly-sized
+    // chunks.
+    vector<struct iovec> iov = ChunkIOVec(&rng, buf.get(), kEchoChunkSize, 1024 * 1024);
+
+    // Loop calling writev until the iov is exhausted
+    int rem = kEchoChunkSize;
+    while (rem > 0) {
+      CHECK(!iov.empty()) << rem;
+      int64_t n;
+      Status s = client_sock->Writev(&iov[0], iov.size(), &n);
+      if (Socket::IsTemporarySocketError(s.posix_code())) {
+        sched_yield();
+        continue;
+      }
+      ASSERT_OK(s);
+      ASSERT_LE(n, rem);
+      rem -= n;
+      ASSERT_GE(n, 0);
+      while (n > 0) {
+        if (n < iov[0].iov_len) {
+          iov[0].iov_len -= n;
+          iov[0].iov_base = reinterpret_cast<uint8_t*>(iov[0].iov_base) + n;
+          n = 0;
+        } else {
+          n -= iov[0].iov_len;
+          iov.erase(iov.begin());
+        }
+      }
+    }
+    LOG(INFO) << "client waiting";
+
+    size_t n;
+    ASSERT_OK(client_sock->SetNonBlocking(false));
+    ASSERT_OK(client_sock->BlockingRecv(rbuf.get(), kEchoChunkSize, &n,
+        MonoTime::Now() + kTimeout));
+    LOG(INFO) << "client got response";
+
+    ASSERT_EQ(0, memcmp(buf.get(), rbuf.get(), kEchoChunkSize));
+  }
+
+  server.Stop();
+  ASSERT_OK(client_sock->Close());
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_socket.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_socket.cc b/be/src/kudu/security/tls_socket.cc
new file mode 100644
index 0000000..355f04b
--- /dev/null
+++ b/be/src/kudu/security/tls_socket.cc
@@ -0,0 +1,185 @@
+// 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 "kudu/security/tls_socket.h"
+
+#include <sys/uio.h>
+
+#include <cerrno>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+#include <openssl/err.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+
+namespace kudu {
+namespace security {
+
+TlsSocket::TlsSocket(int fd, c_unique_ptr<SSL> ssl)
+    : Socket(fd),
+      ssl_(std::move(ssl)) {
+}
+
+TlsSocket::~TlsSocket() {
+  ignore_result(Close());
+}
+
+Status TlsSocket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
+  CHECK(ssl_);
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+
+  *nwritten = 0;
+  if (PREDICT_FALSE(amt == 0)) {
+    // Writing an empty buffer is a no-op. This happens occasionally, eg in the
+    // case where the response has an empty sidecar. We have to special case
+    // it, because SSL_write can return '0' to indicate certain types of errors.
+    return Status::OK();
+  }
+
+  errno = 0;
+  int32_t bytes_written = SSL_write(ssl_.get(), buf, amt);
+  int save_errno = errno;
+  if (bytes_written <= 0) {
+    auto error_code = SSL_get_error(ssl_.get(), bytes_written);
+    if (error_code == SSL_ERROR_WANT_WRITE) {
+      if (save_errno != 0) {
+        return Status::NetworkError("SSL_write error",
+                                    ErrnoToString(save_errno), save_errno);
+      }
+      // Socket not ready to write yet.
+      return Status::OK();
+    }
+    return Status::NetworkError("failed to write to TLS socket",
+                                GetSSLErrorDescription(error_code));
+  }
+  *nwritten = bytes_written;
+  return Status::OK();
+}
+
+Status TlsSocket::Writev(const struct ::iovec *iov, int iov_len, int64_t *nwritten) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ssl_);
+  *nwritten = 0;
+  // Allows packets to be aggresively be accumulated before sending.
+  RETURN_NOT_OK(SetTcpCork(1));
+  Status write_status = Status::OK();
+  for (int i = 0; i < iov_len; ++i) {
+    int32_t frame_size = iov[i].iov_len;
+    int32_t bytes_written;
+    // Don't return before unsetting TCP_CORK.
+    write_status = Write(static_cast<uint8_t*>(iov[i].iov_base), frame_size, &bytes_written);
+    if (!write_status.ok()) break;
+
+    // nwritten should have the correct amount written.
+    *nwritten += bytes_written;
+    if (bytes_written < frame_size) break;
+  }
+  RETURN_NOT_OK(SetTcpCork(0));
+  // If we did manage to write something, but not everything, due to a temporary socket
+  // error, then we should still return an OK status indicating a successful _partial_
+  // write.
+  if (*nwritten > 0 && Socket::IsTemporarySocketError(write_status.posix_code())) {
+    return Status::OK();
+  }
+  return write_status;
+}
+
+Status TlsSocket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+
+  CHECK(ssl_);
+  errno = 0;
+  int32_t bytes_read = SSL_read(ssl_.get(), buf, amt);
+  int save_errno = errno;
+  if (bytes_read <= 0) {
+    Sockaddr remote;
+    Socket::GetPeerAddress(&remote);
+    std::string kErrString = strings::Substitute("failed to read from TLS socket (remote: $0)",
+                                                 remote.ToString());
+
+    if (bytes_read == 0 && SSL_get_shutdown(ssl_.get()) == SSL_RECEIVED_SHUTDOWN) {
+      return Status::NetworkError(kErrString, ErrnoToString(ESHUTDOWN), ESHUTDOWN);
+    }
+    auto error_code = SSL_get_error(ssl_.get(), bytes_read);
+    if (error_code == SSL_ERROR_WANT_READ) {
+      if (save_errno != 0) {
+        return Status::NetworkError("SSL_read error from " + remote.ToString(),
+                                    ErrnoToString(save_errno), save_errno);
+      }
+      // Nothing available to read yet.
+      *nread = 0;
+      return Status::OK();
+    }
+    if (error_code == SSL_ERROR_SYSCALL && ERR_peek_error() == 0) {
+      // From the OpenSSL docs:
+      //   Some I/O error occurred.  The OpenSSL error queue may contain more
+      //   information on the error.  If the error queue is empty (i.e.
+      //   ERR_get_error() returns 0), ret can be used to find out more about
+      //   the error: If ret == 0, an EOF was observed that violates the pro-
+      //   tocol.  If ret == -1, the underlying BIO reported an I/O error (for
+      //   socket I/O on Unix systems, consult errno for details).
+      if (bytes_read == 0) {
+        // "EOF was observed that violates the protocol" (eg the other end disconnected)
+        return Status::NetworkError(kErrString, ErrnoToString(ECONNRESET), ECONNRESET);
+      }
+      if (bytes_read == -1 && save_errno != 0) {
+        return Status::NetworkError(kErrString, ErrnoToString(save_errno), save_errno);
+      }
+      return Status::NetworkError(kErrString, "unknown ERROR_SYSCALL");
+    }
+    return Status::NetworkError(kErrString, GetSSLErrorDescription(error_code));
+  }
+  *nread = bytes_read;
+  return Status::OK();
+}
+
+Status TlsSocket::Close() {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  errno = 0;
+
+  if (!ssl_) {
+    // Socket is already closed.
+    return Status::OK();
+  }
+
+  // Start the TLS shutdown processes. We don't care about waiting for the
+  // response, since the underlying socket will not be reused.
+  int32_t ret = SSL_shutdown(ssl_.get());
+  Status ssl_shutdown;
+  if (ret >= 0) {
+    ssl_shutdown = Status::OK();
+  } else {
+    auto error_code = SSL_get_error(ssl_.get(), ret);
+    ssl_shutdown = Status::NetworkError("TlsSocket::Close", GetSSLErrorDescription(error_code));
+  }
+
+  ssl_.reset();
+
+  // Close the underlying socket.
+  RETURN_NOT_OK(Socket::Close());
+  return ssl_shutdown;
+}
+
+} // namespace security
+} // namespace kudu


[02/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/trace-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/trace-test.cc b/be/src/kudu/util/trace-test.cc
new file mode 100644
index 0000000..ed0b577
--- /dev/null
+++ b/be/src/kudu/util/trace-test.cc
@@ -0,0 +1,891 @@
+// 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 <cctype>
+#include <cstdint>
+#include <cstring>
+#include <map>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <rapidjson/document.h>
+#include <rapidjson/rapidjson.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/debug/trace_event_impl.h"
+#include "kudu/util/debug/trace_event_synthetic_delay.h"
+#include "kudu/util/debug/trace_logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace_metrics.h"
+#include "kudu/util/trace.h"
+
+using kudu::debug::TraceLog;
+using kudu::debug::TraceResultBuffer;
+using kudu::debug::CategoryFilter;
+using rapidjson::Document;
+using rapidjson::Value;
+using std::string;
+using std::thread;
+using std::vector;
+
+namespace kudu {
+
+class TraceTest : public KuduTest {
+};
+
+// Replace all digits in 's' with the character 'X'.
+static string XOutDigits(const string& s) {
+  string ret;
+  ret.reserve(s.size());
+  for (char c : s) {
+    if (isdigit(c)) {
+      ret.push_back('X');
+    } else {
+      ret.push_back(c);
+    }
+  }
+  return ret;
+}
+
+TEST_F(TraceTest, TestBasic) {
+  scoped_refptr<Trace> t(new Trace);
+  TRACE_TO(t, "hello $0, $1", "world", 12345);
+  TRACE_TO(t, "goodbye $0, $1", "cruel world", 54321);
+
+  string result = XOutDigits(t->DumpToString(Trace::NO_FLAGS));
+  ASSERT_EQ("XXXX XX:XX:XX.XXXXXX trace-test.cc:XX] hello world, XXXXX\n"
+            "XXXX XX:XX:XX.XXXXXX trace-test.cc:XX] goodbye cruel world, XXXXX\n",
+            result);
+}
+
+TEST_F(TraceTest, TestAttach) {
+  scoped_refptr<Trace> traceA(new Trace);
+  scoped_refptr<Trace> traceB(new Trace);
+  {
+    ADOPT_TRACE(traceA.get());
+    EXPECT_EQ(traceA.get(), Trace::CurrentTrace());
+    {
+      ADOPT_TRACE(traceB.get());
+      EXPECT_EQ(traceB.get(), Trace::CurrentTrace());
+      TRACE("hello from traceB");
+    }
+    EXPECT_EQ(traceA.get(), Trace::CurrentTrace());
+    TRACE("hello from traceA");
+  }
+  EXPECT_TRUE(Trace::CurrentTrace() == nullptr);
+  TRACE("this goes nowhere");
+
+  EXPECT_EQ("XXXX XX:XX:XX.XXXXXX trace-test.cc:XXX] hello from traceA\n",
+            XOutDigits(traceA->DumpToString(Trace::NO_FLAGS)));
+  EXPECT_EQ("XXXX XX:XX:XX.XXXXXX trace-test.cc:XXX] hello from traceB\n",
+            XOutDigits(traceB->DumpToString(Trace::NO_FLAGS)));
+}
+
+TEST_F(TraceTest, TestChildTrace) {
+  scoped_refptr<Trace> traceA(new Trace);
+  scoped_refptr<Trace> traceB(new Trace);
+  ADOPT_TRACE(traceA.get());
+  traceA->AddChildTrace("child", traceB.get());
+  TRACE("hello from traceA");
+  {
+    ADOPT_TRACE(traceB.get());
+    TRACE("hello from traceB");
+  }
+  EXPECT_EQ("XXXX XX:XX:XX.XXXXXX trace-test.cc:XXX] hello from traceA\n"
+            "Related trace 'child':\n"
+            "XXXX XX:XX:XX.XXXXXX trace-test.cc:XXX] hello from traceB\n",
+            XOutDigits(traceA->DumpToString(Trace::NO_FLAGS)));
+}
+
+static void GenerateTraceEvents(int thread_id,
+                                int num_events) {
+  for (int i = 0; i < num_events; i++) {
+    TRACE_EVENT1("test", "foo", "thread_id", thread_id);
+  }
+}
+
+// Parse the dumped trace data and return the number of events
+// found within, including only those with the "test" category.
+int ParseAndReturnEventCount(const string& trace_json) {
+  Document d;
+  d.Parse<0>(trace_json.c_str());
+  CHECK(d.IsObject()) << "bad json: " << trace_json;
+  const Value& events_json = d["traceEvents"];
+  CHECK(events_json.IsArray()) << "bad json: " << trace_json;
+
+  // Count how many of our events were seen. We have to filter out
+  // the metadata events.
+  int seen_real_events = 0;
+  for (int i = 0; i < events_json.Size(); i++) {
+    if (events_json[i]["cat"].GetString() == string("test")) {
+      seen_real_events++;
+    }
+  }
+
+  return seen_real_events;
+}
+
+TEST_F(TraceTest, TestChromeTracing) {
+  const int kNumThreads = 4;
+  const int kEventsPerThread = AllowSlowTests() ? 1000000 : 10000;
+
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::RECORD_CONTINUOUSLY);
+
+  vector<scoped_refptr<Thread> > threads(kNumThreads);
+
+  Stopwatch s;
+  s.start();
+  for (int i = 0; i < kNumThreads; i++) {
+    CHECK_OK(Thread::Create("test", "gen-traces", &GenerateTraceEvents, i, kEventsPerThread,
+                            &threads[i]));
+  }
+
+  for (int i = 0; i < kNumThreads; i++) {
+    threads[i]->Join();
+  }
+  tl->SetDisabled();
+
+  int total_events = kNumThreads * kEventsPerThread;
+  double elapsed = s.elapsed().wall_seconds();
+
+  LOG(INFO) << "Trace performance: " << static_cast<int>(total_events / elapsed) << " traces/sec";
+
+  string trace_json = TraceResultBuffer::FlushTraceLogToString();
+
+  // Verify that the JSON contains events. It won't have exactly
+  // kEventsPerThread * kNumThreads because the trace buffer isn't large enough
+  // for that.
+  ASSERT_GE(ParseAndReturnEventCount(trace_json), 100);
+}
+
+// Test that, if a thread exits before filling a full trace buffer, we still
+// see its results. This is a regression test for a bug in the earlier integration
+// of Chromium tracing into Kudu.
+TEST_F(TraceTest, TestTraceFromExitedThread) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::RECORD_CONTINUOUSLY);
+
+  // Generate 10 trace events in a separate thread.
+  int kNumEvents = 10;
+  scoped_refptr<Thread> t;
+  CHECK_OK(Thread::Create("test", "gen-traces", &GenerateTraceEvents, 1, kNumEvents,
+                          &t));
+  t->Join();
+  tl->SetDisabled();
+  string trace_json = TraceResultBuffer::FlushTraceLogToString();
+  LOG(INFO) << trace_json;
+
+  // Verify that the buffer contains 10 trace events
+  ASSERT_EQ(10, ParseAndReturnEventCount(trace_json));
+}
+
+static void GenerateWideSpan() {
+  TRACE_EVENT0("test", "GenerateWideSpan");
+  for (int i = 0; i < 1000; i++) {
+    TRACE_EVENT0("test", "InnerLoop");
+  }
+}
+
+// Test creating a trace event which contains many other trace events.
+// This ensures that we can go back and update a TraceEvent which fell in
+// a different trace chunk.
+TEST_F(TraceTest, TestWideSpan) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::RECORD_CONTINUOUSLY);
+
+  scoped_refptr<Thread> t;
+  CHECK_OK(Thread::Create("test", "gen-traces", &GenerateWideSpan, &t));
+  t->Join();
+  tl->SetDisabled();
+
+  string trace_json = TraceResultBuffer::FlushTraceLogToString();
+  ASSERT_EQ(1001, ParseAndReturnEventCount(trace_json));
+}
+
+// Regression test for KUDU-753: faulty JSON escaping when dealing with
+// single quote characters.
+TEST_F(TraceTest, TestJsonEncodingString) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::RECORD_CONTINUOUSLY);
+  {
+    TRACE_EVENT1("test", "test", "arg", "this is a test with \"'\"' and characters\nand new lines");
+  }
+  tl->SetDisabled();
+  string trace_json = TraceResultBuffer::FlushTraceLogToString();
+  ASSERT_EQ(1, ParseAndReturnEventCount(trace_json));
+}
+
+// Generate trace events continuously until 'latch' fires.
+// Increment *num_events_generated for each event generated.
+void GenerateTracesUntilLatch(AtomicInt<int64_t>* num_events_generated,
+                              CountDownLatch* latch) {
+  while (latch->count()) {
+    {
+      // This goes in its own scope so that the event is fully generated (with
+      // both its START and END times) before we do the counter increment below.
+      TRACE_EVENT0("test", "GenerateTracesUntilLatch");
+    }
+    num_events_generated->Increment();
+  }
+}
+
+// Test starting and stopping tracing while a thread is running.
+// This is a regression test for bugs in earlier versions of the imported
+// trace code.
+TEST_F(TraceTest, TestStartAndStopCollection) {
+  TraceLog* tl = TraceLog::GetInstance();
+
+  CountDownLatch latch(1);
+  AtomicInt<int64_t> num_events_generated(0);
+  scoped_refptr<Thread> t;
+  CHECK_OK(Thread::Create("test", "gen-traces", &GenerateTracesUntilLatch,
+                          &num_events_generated, &latch, &t));
+
+  const int num_flushes = AllowSlowTests() ? 50 : 3;
+  for (int i = 0; i < num_flushes; i++) {
+    tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                   TraceLog::RECORDING_MODE,
+                   TraceLog::RECORD_CONTINUOUSLY);
+
+    const int64_t num_events_before = num_events_generated.Load();
+    SleepFor(MonoDelta::FromMilliseconds(10));
+    const int64_t num_events_after = num_events_generated.Load();
+    tl->SetDisabled();
+
+    string trace_json = TraceResultBuffer::FlushTraceLogToString();
+    // We might under-count the number of events, since we only measure the sleep,
+    // and tracing is enabled before and disabled after we start counting.
+    // We might also over-count by at most 1, because we could enable tracing
+    // right in between creating a trace event and incrementing the counter.
+    // But, we should never over-count by more than 1.
+    int expected_events_lowerbound = num_events_after - num_events_before - 1;
+    int captured_events = ParseAndReturnEventCount(trace_json);
+    ASSERT_GE(captured_events, expected_events_lowerbound);
+  }
+
+  latch.CountDown();
+  t->Join();
+}
+
+TEST_F(TraceTest, TestChromeSampling) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 static_cast<TraceLog::Options>(TraceLog::RECORD_CONTINUOUSLY |
+                                                TraceLog::ENABLE_SAMPLING));
+
+  for (int i = 0; i < 100; i++) {
+    switch (i % 3) {
+      case 0:
+        TRACE_EVENT_SET_SAMPLING_STATE("test", "state-0");
+        break;
+      case 1:
+        TRACE_EVENT_SET_SAMPLING_STATE("test", "state-1");
+        break;
+      case 2:
+        TRACE_EVENT_SET_SAMPLING_STATE("test", "state-2");
+        break;
+    }
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+  tl->SetDisabled();
+  string trace_json = TraceResultBuffer::FlushTraceLogToString();
+  ASSERT_GT(ParseAndReturnEventCount(trace_json), 0);
+}
+
+class TraceEventCallbackTest : public KuduTest {
+ public:
+  virtual void SetUp() OVERRIDE {
+    KuduTest::SetUp();
+    ASSERT_EQ(nullptr, s_instance);
+    s_instance = this;
+  }
+  virtual void TearDown() OVERRIDE {
+    TraceLog::GetInstance()->SetDisabled();
+
+    // Flush the buffer so that one test doesn't end up leaving any
+    // extra results for the next test.
+    TraceResultBuffer::FlushTraceLogToString();
+
+    ASSERT_TRUE(!!s_instance);
+    s_instance = nullptr;
+    KuduTest::TearDown();
+
+  }
+
+ protected:
+  void EndTraceAndFlush() {
+    TraceLog::GetInstance()->SetDisabled();
+    string trace_json = TraceResultBuffer::FlushTraceLogToString();
+    trace_doc_.Parse<0>(trace_json.c_str());
+    LOG(INFO) << trace_json;
+    ASSERT_TRUE(trace_doc_.IsObject());
+    trace_parsed_ = trace_doc_["traceEvents"];
+    ASSERT_TRUE(trace_parsed_.IsArray());
+  }
+
+  void DropTracedMetadataRecords() {
+    // NB: rapidjson has move-semantics, like auto_ptr.
+    Value old_trace_parsed;
+    old_trace_parsed = trace_parsed_;
+    trace_parsed_.SetArray();
+    size_t old_trace_parsed_size = old_trace_parsed.Size();
+
+    for (size_t i = 0; i < old_trace_parsed_size; i++) {
+      Value value;
+      value = old_trace_parsed[i];
+      if (value.GetType() != rapidjson::kObjectType) {
+        trace_parsed_.PushBack(value, trace_doc_.GetAllocator());
+        continue;
+      }
+      string tmp;
+      if (value.HasMember("ph") && strcmp(value["ph"].GetString(), "M") == 0) {
+        continue;
+      }
+
+      trace_parsed_.PushBack(value, trace_doc_.GetAllocator());
+    }
+  }
+
+  // Search through the given array for any dictionary which has a key
+  // or value which has 'string_to_match' as a substring.
+  // Returns the matching dictionary, or NULL.
+  static const Value* FindTraceEntry(
+    const Value& trace_parsed,
+    const char* string_to_match) {
+    // Scan all items
+    size_t trace_parsed_count = trace_parsed.Size();
+    for (size_t i = 0; i < trace_parsed_count; i++) {
+      const Value& value = trace_parsed[i];
+      if (value.GetType() != rapidjson::kObjectType) {
+        continue;
+      }
+
+      for (Value::ConstMemberIterator it = value.MemberBegin();
+           it != value.MemberEnd();
+           ++it) {
+        if (it->name.IsString() && strstr(it->name.GetString(), string_to_match) != nullptr) {
+          return &value;
+        }
+        if (it->value.IsString() && strstr(it->value.GetString(), string_to_match) != nullptr) {
+          return &value;
+        }
+      }
+    }
+    return nullptr;
+  }
+
+  // For TraceEventCallbackAndRecordingX tests.
+  void VerifyCallbackAndRecordedEvents(size_t expected_callback_count,
+                                       size_t expected_recorded_count) {
+    // Callback events.
+    EXPECT_EQ(expected_callback_count, collected_events_names_.size());
+    for (size_t i = 0; i < collected_events_names_.size(); ++i) {
+      EXPECT_EQ("callback", collected_events_categories_[i]);
+      EXPECT_EQ("yes", collected_events_names_[i]);
+    }
+
+    // Recorded events.
+    EXPECT_EQ(expected_recorded_count, trace_parsed_.Size());
+    EXPECT_TRUE(FindTraceEntry(trace_parsed_, "recording"));
+    EXPECT_FALSE(FindTraceEntry(trace_parsed_, "callback"));
+    EXPECT_TRUE(FindTraceEntry(trace_parsed_, "yes"));
+    EXPECT_FALSE(FindTraceEntry(trace_parsed_, "no"));
+  }
+
+  void VerifyCollectedEvent(size_t i,
+                            unsigned phase,
+                            const string& category,
+                            const string& name) {
+    EXPECT_EQ(phase, collected_events_phases_[i]);
+    EXPECT_EQ(category, collected_events_categories_[i]);
+    EXPECT_EQ(name, collected_events_names_[i]);
+  }
+
+  Document trace_doc_;
+  Value trace_parsed_;
+
+  vector<string> collected_events_categories_;
+  vector<string> collected_events_names_;
+  vector<unsigned char> collected_events_phases_;
+  vector<MicrosecondsInt64> collected_events_timestamps_;
+
+  static TraceEventCallbackTest* s_instance;
+  static void Callback(MicrosecondsInt64 timestamp,
+                       char phase,
+                       const unsigned char* category_group_enabled,
+                       const char* name,
+                       uint64_t id,
+                       int num_args,
+                       const char* const arg_names[],
+                       const unsigned char arg_types[],
+                       const uint64_t arg_values[],
+                       unsigned char flags) {
+    s_instance->collected_events_phases_.push_back(phase);
+    s_instance->collected_events_categories_.emplace_back(
+        TraceLog::GetCategoryGroupName(category_group_enabled));
+    s_instance->collected_events_names_.emplace_back(name);
+    s_instance->collected_events_timestamps_.push_back(timestamp);
+  }
+};
+
+TraceEventCallbackTest* TraceEventCallbackTest::s_instance;
+
+TEST_F(TraceEventCallbackTest, TraceEventCallback) {
+  TRACE_EVENT_INSTANT0("all", "before enable", TRACE_EVENT_SCOPE_THREAD);
+  TraceLog::GetInstance()->SetEventCallbackEnabled(
+      CategoryFilter("*"), Callback);
+  TRACE_EVENT_INSTANT0("all", "event1", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("all", "event2", TRACE_EVENT_SCOPE_GLOBAL);
+  {
+    TRACE_EVENT0("all", "duration");
+    TRACE_EVENT_INSTANT0("all", "event3", TRACE_EVENT_SCOPE_GLOBAL);
+  }
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  TRACE_EVENT_INSTANT0("all", "after callback removed",
+                       TRACE_EVENT_SCOPE_GLOBAL);
+  ASSERT_EQ(5u, collected_events_names_.size());
+  EXPECT_EQ("event1", collected_events_names_[0]);
+  EXPECT_EQ(TRACE_EVENT_PHASE_INSTANT, collected_events_phases_[0]);
+  EXPECT_EQ("event2", collected_events_names_[1]);
+  EXPECT_EQ(TRACE_EVENT_PHASE_INSTANT, collected_events_phases_[1]);
+  EXPECT_EQ("duration", collected_events_names_[2]);
+  EXPECT_EQ(TRACE_EVENT_PHASE_BEGIN, collected_events_phases_[2]);
+  EXPECT_EQ("event3", collected_events_names_[3]);
+  EXPECT_EQ(TRACE_EVENT_PHASE_INSTANT, collected_events_phases_[3]);
+  EXPECT_EQ("duration", collected_events_names_[4]);
+  EXPECT_EQ(TRACE_EVENT_PHASE_END, collected_events_phases_[4]);
+  for (size_t i = 1; i < collected_events_timestamps_.size(); i++) {
+    EXPECT_LE(collected_events_timestamps_[i - 1],
+              collected_events_timestamps_[i]);
+  }
+}
+
+TEST_F(TraceEventCallbackTest, TraceEventCallbackWhileFull) {
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter("*"),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_UNTIL_FULL);
+  do {
+    TRACE_EVENT_INSTANT0("all", "badger badger", TRACE_EVENT_SCOPE_GLOBAL);
+  } while (!TraceLog::GetInstance()->BufferIsFull());
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("*"),
+                                                   Callback);
+  TRACE_EVENT_INSTANT0("all", "a snake", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  ASSERT_EQ(1u, collected_events_names_.size());
+  EXPECT_EQ("a snake", collected_events_names_[0]);
+}
+
+// 1: Enable callback, enable recording, disable callback, disable recording.
+TEST_F(TraceEventCallbackTest, TraceEventCallbackAndRecording1) {
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("callback"),
+                                                   Callback);
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter("recording"),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_UNTIL_FULL);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  EndTraceAndFlush();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+
+  DropTracedMetadataRecords();
+  ASSERT_NO_FATAL_FAILURE();
+  VerifyCallbackAndRecordedEvents(2, 2);
+}
+
+// 2: Enable callback, enable recording, disable recording, disable callback.
+TEST_F(TraceEventCallbackTest, TraceEventCallbackAndRecording2) {
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("callback"),
+                                                   Callback);
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter("recording"),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_UNTIL_FULL);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  EndTraceAndFlush();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+
+  DropTracedMetadataRecords();
+  VerifyCallbackAndRecordedEvents(3, 1);
+}
+
+// 3: Enable recording, enable callback, disable callback, disable recording.
+TEST_F(TraceEventCallbackTest, TraceEventCallbackAndRecording3) {
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter("recording"),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_UNTIL_FULL);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("callback"),
+                                                   Callback);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  EndTraceAndFlush();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+
+  DropTracedMetadataRecords();
+  VerifyCallbackAndRecordedEvents(1, 3);
+}
+
+// 4: Enable recording, enable callback, disable recording, disable callback.
+TEST_F(TraceEventCallbackTest, TraceEventCallbackAndRecording4) {
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter("recording"),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_UNTIL_FULL);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("callback"),
+                                                   Callback);
+  TRACE_EVENT_INSTANT0("recording", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  EndTraceAndFlush();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "yes", TRACE_EVENT_SCOPE_GLOBAL);
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+  TRACE_EVENT_INSTANT0("recording", "no", TRACE_EVENT_SCOPE_GLOBAL);
+  TRACE_EVENT_INSTANT0("callback", "no", TRACE_EVENT_SCOPE_GLOBAL);
+
+  DropTracedMetadataRecords();
+  VerifyCallbackAndRecordedEvents(2, 2);
+}
+
+TEST_F(TraceEventCallbackTest, TraceEventCallbackAndRecordingDuration) {
+  TraceLog::GetInstance()->SetEventCallbackEnabled(CategoryFilter("*"),
+                                                   Callback);
+  {
+    TRACE_EVENT0("callback", "duration1");
+    TraceLog::GetInstance()->SetEnabled(
+        CategoryFilter("*"),
+        TraceLog::RECORDING_MODE,
+        TraceLog::RECORD_UNTIL_FULL);
+    TRACE_EVENT0("callback", "duration2");
+    EndTraceAndFlush();
+    TRACE_EVENT0("callback", "duration3");
+  }
+  TraceLog::GetInstance()->SetEventCallbackDisabled();
+
+  ASSERT_EQ(6u, collected_events_names_.size());
+  VerifyCollectedEvent(0, TRACE_EVENT_PHASE_BEGIN, "callback", "duration1");
+  VerifyCollectedEvent(1, TRACE_EVENT_PHASE_BEGIN, "callback", "duration2");
+  VerifyCollectedEvent(2, TRACE_EVENT_PHASE_BEGIN, "callback", "duration3");
+  VerifyCollectedEvent(3, TRACE_EVENT_PHASE_END, "callback", "duration3");
+  VerifyCollectedEvent(4, TRACE_EVENT_PHASE_END, "callback", "duration2");
+  VerifyCollectedEvent(5, TRACE_EVENT_PHASE_END, "callback", "duration1");
+}
+
+////////////////////////////////////////////////////////////
+// Tests for synthetic delay
+// (from chromium-base/debug/trace_event_synthetic_delay_unittest.cc)
+////////////////////////////////////////////////////////////
+
+namespace {
+
+const int kTargetDurationMs = 100;
+// Allow some leeway in timings to make it possible to run these tests with a
+// wall clock time source too.
+const int kShortDurationMs = 10;
+
+}  // namespace
+
+namespace debug {
+
+class TraceEventSyntheticDelayTest : public KuduTest,
+                                     public TraceEventSyntheticDelayClock {
+ public:
+  TraceEventSyntheticDelayTest() {
+    now_ = MonoTime::Min();
+  }
+
+  virtual ~TraceEventSyntheticDelayTest() {
+    ResetTraceEventSyntheticDelays();
+  }
+
+  // TraceEventSyntheticDelayClock implementation.
+  virtual MonoTime Now() OVERRIDE {
+    AdvanceTime(MonoDelta::FromMilliseconds(kShortDurationMs / 10));
+    return now_;
+  }
+
+  TraceEventSyntheticDelay* ConfigureDelay(const char* name) {
+    TraceEventSyntheticDelay* delay = TraceEventSyntheticDelay::Lookup(name);
+    delay->SetClock(this);
+    delay->SetTargetDuration(
+      MonoDelta::FromMilliseconds(kTargetDurationMs));
+    return delay;
+  }
+
+  void AdvanceTime(MonoDelta delta) { now_ += delta; }
+
+  int TestFunction() {
+    MonoTime start = Now();
+    { TRACE_EVENT_SYNTHETIC_DELAY("test.Delay"); }
+    MonoTime end = Now();
+    return (end - start).ToMilliseconds();
+  }
+
+  int AsyncTestFunctionBegin() {
+    MonoTime start = Now();
+    { TRACE_EVENT_SYNTHETIC_DELAY_BEGIN("test.AsyncDelay"); }
+    MonoTime end = Now();
+    return (end - start).ToMilliseconds();
+  }
+
+  int AsyncTestFunctionEnd() {
+    MonoTime start = Now();
+    { TRACE_EVENT_SYNTHETIC_DELAY_END("test.AsyncDelay"); }
+    MonoTime end = Now();
+    return (end - start).ToMilliseconds();
+  }
+
+ private:
+  MonoTime now_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceEventSyntheticDelayTest);
+};
+
+TEST_F(TraceEventSyntheticDelayTest, StaticDelay) {
+  TraceEventSyntheticDelay* delay = ConfigureDelay("test.Delay");
+  delay->SetMode(TraceEventSyntheticDelay::STATIC);
+  EXPECT_GE(TestFunction(), kTargetDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, OneShotDelay) {
+  TraceEventSyntheticDelay* delay = ConfigureDelay("test.Delay");
+  delay->SetMode(TraceEventSyntheticDelay::ONE_SHOT);
+  EXPECT_GE(TestFunction(), kTargetDurationMs);
+  EXPECT_LT(TestFunction(), kShortDurationMs);
+
+  delay->SetTargetDuration(
+      MonoDelta::FromMilliseconds(kTargetDurationMs));
+  EXPECT_GE(TestFunction(), kTargetDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AlternatingDelay) {
+  TraceEventSyntheticDelay* delay = ConfigureDelay("test.Delay");
+  delay->SetMode(TraceEventSyntheticDelay::ALTERNATING);
+  EXPECT_GE(TestFunction(), kTargetDurationMs);
+  EXPECT_LT(TestFunction(), kShortDurationMs);
+  EXPECT_GE(TestFunction(), kTargetDurationMs);
+  EXPECT_LT(TestFunction(), kShortDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AsyncDelay) {
+  ConfigureDelay("test.AsyncDelay");
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  EXPECT_GE(AsyncTestFunctionEnd(), kTargetDurationMs / 2);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AsyncDelayExceeded) {
+  ConfigureDelay("test.AsyncDelay");
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  AdvanceTime(MonoDelta::FromMilliseconds(kTargetDurationMs));
+  EXPECT_LT(AsyncTestFunctionEnd(), kShortDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AsyncDelayNoActivation) {
+  ConfigureDelay("test.AsyncDelay");
+  EXPECT_LT(AsyncTestFunctionEnd(), kShortDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AsyncDelayNested) {
+  ConfigureDelay("test.AsyncDelay");
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  EXPECT_LT(AsyncTestFunctionEnd(), kShortDurationMs);
+  EXPECT_GE(AsyncTestFunctionEnd(), kTargetDurationMs / 2);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, AsyncDelayUnbalanced) {
+  ConfigureDelay("test.AsyncDelay");
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  EXPECT_GE(AsyncTestFunctionEnd(), kTargetDurationMs / 2);
+  EXPECT_LT(AsyncTestFunctionEnd(), kShortDurationMs);
+
+  EXPECT_LT(AsyncTestFunctionBegin(), kShortDurationMs);
+  EXPECT_GE(AsyncTestFunctionEnd(), kTargetDurationMs / 2);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, ResetDelays) {
+  ConfigureDelay("test.Delay");
+  ResetTraceEventSyntheticDelays();
+  EXPECT_LT(TestFunction(), kShortDurationMs);
+}
+
+TEST_F(TraceEventSyntheticDelayTest, BeginParallel) {
+  TraceEventSyntheticDelay* delay = ConfigureDelay("test.AsyncDelay");
+  MonoTime end_times[2];
+  MonoTime start_time = Now();
+
+  delay->BeginParallel(&end_times[0]);
+  EXPECT_FALSE(!end_times[0].Initialized());
+
+  delay->BeginParallel(&end_times[1]);
+  EXPECT_FALSE(!end_times[1].Initialized());
+
+  delay->EndParallel(end_times[0]);
+  EXPECT_GE((Now() - start_time).ToMilliseconds(), kTargetDurationMs);
+
+  start_time = Now();
+  delay->EndParallel(end_times[1]);
+  EXPECT_LT((Now() - start_time).ToMilliseconds(), kShortDurationMs);
+}
+
+TEST_F(TraceTest, TestVLogTrace) {
+  for (FLAGS_v = 0; FLAGS_v <= 1; FLAGS_v++) {
+    TraceLog* tl = TraceLog::GetInstance();
+    tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                   TraceLog::RECORDING_MODE,
+                   TraceLog::RECORD_CONTINUOUSLY);
+    VLOG_AND_TRACE("test", 1) << "hello world";
+    tl->SetDisabled();
+    string trace_json = TraceResultBuffer::FlushTraceLogToString();
+    ASSERT_STR_CONTAINS(trace_json, "hello world");
+    ASSERT_STR_CONTAINS(trace_json, "trace-test.cc");
+  }
+}
+
+namespace {
+string FunctionWithSideEffect(bool* b) {
+  *b = true;
+  return "function-result";
+}
+} // anonymous namespace
+
+// Test that, if tracing is not enabled, a VLOG_AND_TRACE doesn't evaluate its
+// arguments.
+TEST_F(TraceTest, TestVLogTraceLazyEvaluation) {
+  FLAGS_v = 0;
+  bool function_run = false;
+  VLOG_AND_TRACE("test", 1) << FunctionWithSideEffect(&function_run);
+  ASSERT_FALSE(function_run);
+
+  // If we enable verbose logging, we should run the side effect even though
+  // trace logging is disabled.
+  FLAGS_v = 1;
+  VLOG_AND_TRACE("test", 1) << FunctionWithSideEffect(&function_run);
+  ASSERT_TRUE(function_run);
+}
+
+TEST_F(TraceTest, TestVLogAndEchoToConsole) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::ECHO_TO_CONSOLE);
+  FLAGS_v = 1;
+  VLOG_AND_TRACE("test", 1) << "hello world";
+  tl->SetDisabled();
+}
+
+TEST_F(TraceTest, TestTraceMetrics) {
+  scoped_refptr<Trace> trace(new Trace);
+  trace->metrics()->Increment("foo", 10);
+  trace->metrics()->Increment("bar", 10);
+  for (int i = 0; i < 1000; i++) {
+    trace->metrics()->Increment("baz", i);
+  }
+  EXPECT_EQ("{\"bar\":10,\"baz\":499500,\"foo\":10}",
+            trace->MetricsAsJSON());
+
+  {
+    ADOPT_TRACE(trace.get());
+    TRACE_COUNTER_SCOPE_LATENCY_US("test_scope_us");
+    SleepFor(MonoDelta::FromMilliseconds(100));
+  }
+  auto m = trace->metrics()->Get();
+  EXPECT_GE(m["test_scope_us"], 80 * 1000);
+}
+
+// Regression test for KUDU-2075: using tracing from vanilla threads
+// should work fine, even if some pthread_self identifiers have been
+// reused.
+TEST_F(TraceTest, TestTraceFromVanillaThreads) {
+  TraceLog::GetInstance()->SetEnabled(
+      CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+      TraceLog::RECORDING_MODE,
+      TraceLog::RECORD_CONTINUOUSLY);
+  SCOPED_CLEANUP({ TraceLog::GetInstance()->SetDisabled(); });
+
+  // Do several passes to make it more likely that the thread identifiers
+  // will get reused.
+  for (int pass = 0; pass < 10; pass++) {
+    vector<thread> threads;
+    for (int i = 0; i < 100; i++) {
+      threads.emplace_back([i] {
+          GenerateTraceEvents(i, 1);
+        });
+    }
+    for (auto& t : threads) {
+      t.join();
+    }
+  }
+}
+} // namespace debug
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/trace.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/trace.cc b/be/src/kudu/util/trace.cc
new file mode 100644
index 0000000..ac56660
--- /dev/null
+++ b/be/src/kudu/util/trace.cc
@@ -0,0 +1,259 @@
+// 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 "kudu/util/trace.h"
+
+#include <cstdint>
+#include <cstring>
+#include <iomanip>
+#include <iostream>
+#include <map>
+#include <mutex>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/memory/arena.h"
+
+using std::pair;
+using std::string;
+using std::vector;
+using strings::internal::SubstituteArg;
+
+namespace kudu {
+
+__thread Trace* Trace::threadlocal_trace_;
+
+Trace::Trace()
+    : arena_(new ThreadSafeArena(1024)),
+      entries_head_(nullptr),
+      entries_tail_(nullptr) {
+  // We expect small allocations from our Arena so no need to have
+  // a large arena component. Small allocations are more likely to
+  // come out of thread cache and be fast.
+  arena_->SetMaxBufferSize(4096);
+}
+
+Trace::~Trace() {
+}
+
+// Struct which precedes each entry in the trace.
+struct TraceEntry {
+  MicrosecondsInt64 timestamp_micros;
+
+  // The source file and line number which generated the trace message.
+  const char* file_path;
+  int line_number;
+
+  uint32_t message_len;
+  TraceEntry* next;
+
+  // The actual trace message follows the entry header.
+  char* message() {
+    return reinterpret_cast<char*>(this) + sizeof(*this);
+  }
+};
+
+// Get the part of filepath after the last path separator.
+// (Doesn't modify filepath, contrary to basename() in libgen.h.)
+// Borrowed from glog.
+static const char* const_basename(const char* filepath) {
+  const char* base = strrchr(filepath, '/');
+#ifdef OS_WINDOWS  // Look for either path separator in Windows
+  if (!base)
+    base = strrchr(filepath, '\\');
+#endif
+  return base ? (base+1) : filepath;
+}
+
+
+void Trace::SubstituteAndTrace(const char* file_path,
+                               int line_number,
+                               StringPiece format,
+                               const SubstituteArg& arg0, const SubstituteArg& arg1,
+                               const SubstituteArg& arg2, const SubstituteArg& arg3,
+                               const SubstituteArg& arg4, const SubstituteArg& arg5,
+                               const SubstituteArg& arg6, const SubstituteArg& arg7,
+                               const SubstituteArg& arg8, const SubstituteArg& arg9) {
+  const SubstituteArg* const args_array[] = {
+    &arg0, &arg1, &arg2, &arg3, &arg4, &arg5, &arg6, &arg7, &arg8, &arg9, nullptr
+  };
+
+  int msg_len = strings::internal::SubstitutedSize(format, args_array);
+  TraceEntry* entry = NewEntry(msg_len, file_path, line_number);
+  SubstituteToBuffer(format, args_array, entry->message());
+  AddEntry(entry);
+}
+
+TraceEntry* Trace::NewEntry(int msg_len, const char* file_path, int line_number) {
+  int size = sizeof(TraceEntry) + msg_len;
+  uint8_t* dst = reinterpret_cast<uint8_t*>(arena_->AllocateBytes(size));
+  TraceEntry* entry = reinterpret_cast<TraceEntry*>(dst);
+  entry->timestamp_micros = GetCurrentTimeMicros();
+  entry->message_len = msg_len;
+  entry->file_path = file_path;
+  entry->line_number = line_number;
+  return entry;
+}
+
+void Trace::AddEntry(TraceEntry* entry) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  entry->next = nullptr;
+
+  if (entries_tail_ != nullptr) {
+    entries_tail_->next = entry;
+  } else {
+    DCHECK(entries_head_ == nullptr);
+    entries_head_ = entry;
+  }
+  entries_tail_ = entry;
+}
+
+void Trace::Dump(std::ostream* out, int flags) const {
+  // Gather a copy of the list of entries under the lock. This is fast
+  // enough that we aren't worried about stalling concurrent tracers
+  // (whereas doing the logging itself while holding the lock might be
+  // too slow, if the output stream is a file, for example).
+  vector<TraceEntry*> entries;
+  vector<pair<StringPiece, scoped_refptr<Trace>>> child_traces;
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    for (TraceEntry* cur = entries_head_;
+         cur != nullptr;
+         cur = cur->next) {
+      entries.push_back(cur);
+    }
+
+    child_traces = child_traces_;
+  }
+
+  // Save original flags.
+  std::ios::fmtflags save_flags(out->flags());
+
+  int64_t prev_usecs = 0;
+  for (TraceEntry* e : entries) {
+    // Log format borrowed from glog/logging.cc
+    int64_t usecs_since_prev = 0;
+    if (prev_usecs != 0) {
+      usecs_since_prev = e->timestamp_micros - prev_usecs;
+    }
+    prev_usecs = e->timestamp_micros;
+
+    using std::setw;
+    *out << FormatTimestampForLog(e->timestamp_micros);
+    *out << ' ';
+    if (flags & INCLUDE_TIME_DELTAS) {
+      out->fill(' ');
+      *out << "(+" << setw(6) << usecs_since_prev << "us) ";
+    }
+    *out << const_basename(e->file_path) << ':' << e->line_number
+         << "] ";
+    out->write(reinterpret_cast<char*>(e) + sizeof(TraceEntry),
+               e->message_len);
+    *out << std::endl;
+  }
+
+  for (const auto& entry : child_traces) {
+    const auto& t = entry.second;
+    *out << "Related trace '" << entry.first << "':" << std::endl;
+    *out << t->DumpToString(flags & (~INCLUDE_METRICS));
+  }
+
+  if (flags & INCLUDE_METRICS) {
+    *out << "Metrics: " << MetricsAsJSON();
+  }
+
+  // Restore stream flags.
+  out->flags(save_flags);
+}
+
+string Trace::DumpToString(int flags) const {
+  std::ostringstream s;
+  Dump(&s, flags);
+  return s.str();
+}
+
+string Trace::MetricsAsJSON() const {
+  std::ostringstream s;
+  JsonWriter jw(&s, JsonWriter::COMPACT);
+  MetricsToJSON(&jw);
+  return s.str();
+}
+
+void Trace::MetricsToJSON(JsonWriter* jw) const {
+  // Convert into a map with 'std::string' keys instead of 'const char*'
+  // keys, so that the results are in a consistent (sorted) order.
+  std::map<string, int64_t> counters;
+  for (const auto& entry : metrics_.Get()) {
+    counters[entry.first] = entry.second;
+  }
+
+  jw->StartObject();
+  for (const auto& e : counters) {
+    jw->String(e.first);
+    jw->Int64(e.second);
+  }
+  vector<pair<StringPiece, scoped_refptr<Trace>>> child_traces;
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    child_traces = child_traces_;
+  }
+
+  if (!child_traces.empty()) {
+    jw->String("child_traces");
+    jw->StartArray();
+
+    for (const auto& e : child_traces) {
+      jw->StartArray();
+      jw->String(e.first.data(), e.first.size());
+      e.second->MetricsToJSON(jw);
+      jw->EndArray();
+    }
+    jw->EndArray();
+  }
+  jw->EndObject();
+}
+
+void Trace::DumpCurrentTrace() {
+  Trace* t = CurrentTrace();
+  if (t == nullptr) {
+    LOG(INFO) << "No trace is currently active.";
+    return;
+  }
+  t->Dump(&std::cerr, true);
+}
+
+void Trace::AddChildTrace(StringPiece label, Trace* child_trace) {
+  CHECK(arena_->RelocateStringPiece(label, &label));
+
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<Trace> ptr(child_trace);
+  child_traces_.emplace_back(label, ptr);
+}
+
+std::vector<std::pair<StringPiece, scoped_refptr<Trace>>> Trace::ChildTraces() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return child_traces_;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/trace.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/trace.h b/be/src/kudu/util/trace.h
new file mode 100644
index 0000000..1c29fa9
--- /dev/null
+++ b/be/src/kudu/util/trace.h
@@ -0,0 +1,292 @@
+// 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.
+#ifndef KUDU_UTIL_TRACE_H
+#define KUDU_UTIL_TRACE_H
+
+#include <iosfwd>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/threading/thread_collision_warner.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/trace_metrics.h"
+
+namespace kudu {
+class Trace;
+}
+
+// Adopt a Trace on the current thread for the duration of the current
+// scope. The old current Trace is restored when the scope is exited.
+//
+// 't' should be a Trace* pointer.
+#define ADOPT_TRACE(t) kudu::ScopedAdoptTrace _adopt_trace(t);
+
+// Issue a trace message, if tracing is enabled in the current thread.
+// See Trace::SubstituteAndTrace for arguments.
+// Example:
+//  TRACE("Acquired timestamp $0", timestamp);
+#define TRACE(format, substitutions...) \
+  do { \
+    kudu::Trace* _trace = Trace::CurrentTrace(); \
+    if (_trace) { \
+      _trace->SubstituteAndTrace(__FILE__, __LINE__, (format),  \
+        ##substitutions); \
+    } \
+  } while (0);
+
+// Like the above, but takes the trace pointer as an explicit argument.
+#define TRACE_TO(trace, format, substitutions...) \
+  (trace)->SubstituteAndTrace(__FILE__, __LINE__, (format), ##substitutions)
+
+// Increment a counter associated with the current trace.
+//
+// Each trace contains a map of counters which can be used to keep
+// request-specific statistics. It is significantly faster to increment
+// a trace counter compared to logging a message. Additionally, having
+// slightly more structured information makes it easier to aggregate
+// and show information back to operators.
+//
+// NOTE: the 'counter_name' MUST be a string which stays alive forever.
+// Typically, this is a compile-time constant. If something other than
+// a constant is required, use TraceMetric::InternName() in order to
+// create a string which will last for the process lifetime. Of course,
+// these strings will never be cleaned up, so it's important to use this
+// judiciously.
+//
+// If no trace is active, this does nothing and does not evaluate its
+// parameters.
+#define TRACE_COUNTER_INCREMENT(counter_name, val) \
+  do { \
+    kudu::Trace* _trace = Trace::CurrentTrace(); \
+    if (_trace) { \
+      _trace->metrics()->Increment(counter_name, val); \
+    } \
+  } while (0);
+
+// Increment a counter for the amount of wall time spent in the current
+// scope. For example:
+//
+//  void DoFoo() {
+//    TRACE_COUNTER_SCOPE_LATENCY_US("foo_us");
+//    ... do expensive Foo thing
+//  }
+//
+//  will result in a trace metric indicating the number of microseconds spent
+//  in invocations of DoFoo().
+#define TRACE_COUNTER_SCOPE_LATENCY_US(counter_name) \
+  ::kudu::ScopedTraceLatencyCounter _scoped_latency(counter_name)
+
+// Construct a constant C string counter name which acts as a sort of
+// coarse-grained histogram for trace metrics.
+#define BUCKETED_COUNTER_NAME(prefix, duration_us)      \
+  [=]() -> const char* {                                \
+    if (duration_us >= 100 * 1000) {                    \
+      return prefix "_gt_100_ms";                       \
+    } else if (duration_us >= 10 * 1000) {              \
+      return prefix "_10-100_ms";                       \
+    } else if (duration_us >= 1000) {                   \
+      return prefix "_1-10_ms";                         \
+    } else {                                            \
+      return prefix "_lt_1ms";                          \
+    }                                                   \
+  }();
+
+namespace kudu {
+
+class JsonWriter;
+class ThreadSafeArena;
+struct TraceEntry;
+
+// A trace for a request or other process. This supports collecting trace entries
+// from a number of threads, and later dumping the results to a stream.
+//
+// Callers should generally not add trace messages directly using the public
+// methods of this class. Rather, the TRACE(...) macros defined above should
+// be used such that file/line numbers are automatically included, etc.
+//
+// This class is thread-safe.
+class Trace : public RefCountedThreadSafe<Trace> {
+ public:
+  Trace();
+
+  // Logs a message into the trace buffer.
+  //
+  // See strings::Substitute for details.
+  //
+  // N.B.: the file path passed here is not copied, so should be a static
+  // constant (eg __FILE__).
+  void SubstituteAndTrace(const char* filepath, int line_number,
+                          StringPiece format,
+                          const strings::internal::SubstituteArg& arg0 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg1 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg2 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg3 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg4 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg5 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg6 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg7 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg8 =
+                            strings::internal::SubstituteArg::kNoArg,
+                          const strings::internal::SubstituteArg& arg9 =
+                            strings::internal::SubstituteArg::kNoArg);
+
+  // Dump the trace buffer to the given output stream.
+  //
+  enum {
+    NO_FLAGS = 0,
+
+    // If set, calculate and print the difference between successive trace messages.
+    INCLUDE_TIME_DELTAS = 1 << 0,
+    // If set, include a 'Metrics' line showing any attached trace metrics.
+    INCLUDE_METRICS =     1 << 1,
+
+    INCLUDE_ALL = INCLUDE_TIME_DELTAS | INCLUDE_METRICS
+  };
+  void Dump(std::ostream* out, int flags) const;
+
+  // Dump the trace buffer as a string.
+  std::string DumpToString(int flags = INCLUDE_ALL) const;
+
+  std::string MetricsAsJSON() const;
+
+  // Attaches the given trace which will get appended at the end when Dumping.
+  //
+  // The 'label' does not necessarily have to be unique, and is used to identify
+  // the child trace when dumped. The contents of the StringPiece are copied
+  // into this trace's arena.
+  void AddChildTrace(StringPiece label, Trace* child_trace);
+
+  // Return a copy of the current set of related "child" traces.
+  std::vector<std::pair<StringPiece, scoped_refptr<Trace>>> ChildTraces() const;
+
+  // Return the current trace attached to this thread, if there is one.
+  static Trace* CurrentTrace() {
+    return threadlocal_trace_;
+  }
+
+  // Simple function to dump the current trace to stderr, if one is
+  // available. This is meant for usage when debugging in gdb via
+  // 'call kudu::Trace::DumpCurrentTrace();'.
+  static void DumpCurrentTrace();
+
+  TraceMetrics* metrics() {
+    return &metrics_;
+  }
+  const TraceMetrics& metrics() const {
+    return metrics_;
+  }
+
+ private:
+  friend class ScopedAdoptTrace;
+  friend class RefCountedThreadSafe<Trace>;
+  ~Trace();
+
+  // The current trace for this thread. Threads should only set this using
+  // using ScopedAdoptTrace, which handles reference counting the underlying
+  // object.
+  static __thread Trace* threadlocal_trace_;
+
+  // Allocate a new entry from the arena, with enough space to hold a
+  // message of length 'len'.
+  TraceEntry* NewEntry(int len, const char* file_path, int line_number);
+
+  // Add the entry to the linked list of entries.
+  void AddEntry(TraceEntry* entry);
+
+  void MetricsToJSON(JsonWriter* jw) const;
+
+  gscoped_ptr<ThreadSafeArena> arena_;
+
+  // Lock protecting the entries linked list.
+  mutable simple_spinlock lock_;
+  // The head of the linked list of entries (allocated inside arena_)
+  TraceEntry* entries_head_;
+  // The tail of the linked list of entries (allocated inside arena_)
+  TraceEntry* entries_tail_;
+
+  std::vector<std::pair<StringPiece, scoped_refptr<Trace>>> child_traces_;
+
+  TraceMetrics metrics_;
+
+  DISALLOW_COPY_AND_ASSIGN(Trace);
+};
+
+// Adopt a Trace object into the current thread for the duration
+// of this object.
+// This should only be used on the stack (and thus created and destroyed
+// on the same thread)
+class ScopedAdoptTrace {
+ public:
+  explicit ScopedAdoptTrace(Trace* t) :
+    old_trace_(Trace::threadlocal_trace_) {
+    Trace::threadlocal_trace_ = t;
+    if (t) {
+      t->AddRef();
+    }
+    DFAKE_SCOPED_LOCK_THREAD_LOCKED(ctor_dtor_);
+  }
+
+  ~ScopedAdoptTrace() {
+    if (Trace::threadlocal_trace_) {
+      Trace::threadlocal_trace_->Release();
+    }
+    Trace::threadlocal_trace_ = old_trace_;
+    DFAKE_SCOPED_LOCK_THREAD_LOCKED(ctor_dtor_);
+  }
+
+ private:
+  DFAKE_MUTEX(ctor_dtor_);
+  Trace* old_trace_;
+
+  DISALLOW_COPY_AND_ASSIGN(ScopedAdoptTrace);
+};
+
+// Implementation for TRACE_COUNTER_SCOPE_LATENCY_US(...) macro above.
+class ScopedTraceLatencyCounter {
+ public:
+  explicit ScopedTraceLatencyCounter(const char* counter)
+      : counter_(counter),
+        start_time_(GetCurrentTimeMicros()) {
+  }
+
+  ~ScopedTraceLatencyCounter() {
+    TRACE_COUNTER_INCREMENT(counter_, GetCurrentTimeMicros() - start_time_);
+  }
+
+ private:
+  const char* const counter_;
+  MicrosecondsInt64 start_time_;
+  DISALLOW_COPY_AND_ASSIGN(ScopedTraceLatencyCounter);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_TRACE_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/trace_metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/trace_metrics.cc b/be/src/kudu/util/trace_metrics.cc
new file mode 100644
index 0000000..565a6e8
--- /dev/null
+++ b/be/src/kudu/util/trace_metrics.cc
@@ -0,0 +1,74 @@
+// 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 "kudu/util/trace_metrics.h"
+
+#include <algorithm>
+#include <cctype>
+#include <cstring>
+#include <map>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+#include <glog/stl_logging.h>
+
+#include "kudu/util/debug/leakcheck_disabler.h"
+
+using std::string;
+
+namespace kudu {
+
+// Make glog's STL-compatible operators visible inside this namespace.
+using ::operator<<;
+
+namespace {
+
+static simple_spinlock g_intern_map_lock;
+typedef std::map<string, const char*> InternMap;
+static InternMap* g_intern_map;
+
+} // anonymous namespace
+
+const char* TraceMetrics::InternName(const string& name) {
+  DCHECK(std::all_of(name.begin(), name.end(), [] (char c) { return isprint(c); } ))
+      << "not printable: " << name;
+
+  debug::ScopedLeakCheckDisabler no_leakcheck;
+  std::lock_guard<simple_spinlock> l(g_intern_map_lock);
+  if (g_intern_map == nullptr) {
+    g_intern_map = new InternMap();
+  }
+
+  InternMap::iterator it = g_intern_map->find(name);
+  if (it != g_intern_map->end()) {
+    return it->second;
+  }
+
+  const char* dup = strdup(name.c_str());
+  (*g_intern_map)[name] = dup;
+
+  // We don't expect this map to grow large.
+  DCHECK_LT(g_intern_map->size(), 100) <<
+      "Too many interned strings: " << *g_intern_map;
+
+  return dup;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/trace_metrics.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/trace_metrics.h b/be/src/kudu/util/trace_metrics.h
new file mode 100644
index 0000000..8c460bd
--- /dev/null
+++ b/be/src/kudu/util/trace_metrics.h
@@ -0,0 +1,89 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/util/locks.h"
+
+namespace kudu {
+
+// A simple map of constant string names to integer counters.
+//
+// Typically, the TRACE_COUNTER_INCREMENT(...) macro defined in
+// trace.h is used to increment a counter within this map.
+//
+// This currently is just a thin wrapper around a spinlocked map,
+// but if it becomes noticeable in the CPU profile, various optimizations
+// are plausible.
+class TraceMetrics {
+ public:
+  TraceMetrics() {}
+  ~TraceMetrics() {}
+
+  // Internalize the given string by duplicating it into a process-wide
+  // pool. If this string has already been interned, returns a pointer
+  // to a previous instance. Otherwise, copies it into the pool.
+  //
+  // The resulting strings are purposefully leaked, so this should only
+  // be used in cases where the number of unique strings that will be
+  // passed is relatively low (i.e. not user-specified).
+  //
+  // Because 'name' is exposed back to operators, it must be a printable
+  // ASCII string.
+  static const char* InternName(const std::string& name);
+
+  // Increment the given counter.
+  void Increment(const char* name, int64_t amount);
+
+  // Return a copy of the current counter map.
+  std::map<const char*, int64_t> Get() const;
+
+  // Return metric's current value.
+  //
+  // NOTE: the 'name' MUST be the same const char* which is used for
+  // insertion. This is because we do pointer-wise comparison internally.
+  int64_t GetMetric(const char* name) const;
+
+ private:
+  mutable simple_spinlock lock_;
+  std::map<const char*, int64_t> counters_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceMetrics);
+};
+
+inline void TraceMetrics::Increment(const char* name, int64_t amount) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  counters_[name] += amount;
+}
+
+inline std::map<const char*, int64_t> TraceMetrics::Get() const {
+  std::unique_lock<simple_spinlock> l(lock_);
+  return counters_;
+}
+
+inline int64_t TraceMetrics::GetMetric(const char* name) const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return FindWithDefault(counters_, name, 0);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/url-coding-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/url-coding-test.cc b/be/src/kudu/util/url-coding-test.cc
new file mode 100644
index 0000000..3892772
--- /dev/null
+++ b/be/src/kudu/util/url-coding-test.cc
@@ -0,0 +1,112 @@
+// 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 <cstring>
+#include <cstdint>
+#include <iostream>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/url-coding.h"
+
+using namespace std; // NOLINT(*)
+
+namespace kudu {
+
+// Tests encoding/decoding of input.  If expected_encoded is non-empty, the
+// encoded string is validated against it.
+void TestUrl(const string& input, const string& expected_encoded, bool hive_compat) {
+  string intermediate;
+  UrlEncode(input, &intermediate, hive_compat);
+  string output;
+  if (!expected_encoded.empty()) {
+    EXPECT_EQ(expected_encoded, intermediate);
+  }
+  EXPECT_TRUE(UrlDecode(intermediate, &output, hive_compat));
+  EXPECT_EQ(input, output);
+
+  // Convert string to vector and try that also
+  vector<uint8_t> input_vector;
+  input_vector.resize(input.size());
+  if (!input.empty()) {
+    memcpy(&input_vector[0], input.c_str(), input.size());
+  }
+  string intermediate2;
+  UrlEncode(input_vector, &intermediate2, hive_compat);
+  EXPECT_EQ(intermediate, intermediate2);
+}
+
+void TestBase64(const string& input, const string& expected_encoded) {
+  string intermediate;
+  Base64Encode(input, &intermediate);
+  string output;
+  if (!expected_encoded.empty()) {
+    EXPECT_EQ(intermediate, expected_encoded);
+  }
+  EXPECT_TRUE(Base64Decode(intermediate, &output));
+  EXPECT_EQ(input, output);
+
+  // Convert string to vector and try that also
+  vector<uint8_t> input_vector;
+  input_vector.resize(input.size());
+  memcpy(&input_vector[0], input.c_str(), input.size());
+  string intermediate2;
+  Base64Encode(input_vector, &intermediate2);
+  EXPECT_EQ(intermediate, intermediate2);
+}
+
+// Test URL encoding. Check that the values that are put in are the
+// same that come out.
+TEST(UrlCodingTest, Basic) {
+  string input = "ABCDEFGHIJKLMNOPQRSTUWXYZ1234567890~!@#$%^&*()<>?,./:\";'{}|[]\\_+-=";
+  TestUrl(input, "", false);
+  TestUrl(input, "", true);
+}
+
+TEST(UrlCodingTest, HiveExceptions) {
+  TestUrl(" +", " +", true);
+}
+
+TEST(UrlCodingTest, BlankString) {
+  TestUrl("", "", false);
+  TestUrl("", "", true);
+}
+
+TEST(UrlCodingTest, PathSeparators) {
+  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", false);
+  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", true);
+}
+
+TEST(Base64Test, Basic) {
+  TestBase64("a", "YQ==");
+  TestBase64("ab", "YWI=");
+  TestBase64("abc", "YWJj");
+  TestBase64("abcd", "YWJjZA==");
+  TestBase64("abcde", "YWJjZGU=");
+  TestBase64("abcdef", "YWJjZGVm");
+}
+
+TEST(HtmlEscapingTest, Basic) {
+  string before = "<html><body>&amp";
+  ostringstream after;
+  EscapeForHtml(before, &after);
+  EXPECT_EQ(after.str(), "&lt;html&gt;&lt;body&gt;&amp;amp");
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/url-coding.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/url-coding.cc b/be/src/kudu/util/url-coding.cc
new file mode 100644
index 0000000..81a2994
--- /dev/null
+++ b/be/src/kudu/util/url-coding.cc
@@ -0,0 +1,208 @@
+// 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 "kudu/util/url-coding.h"
+
+#include <algorithm>
+#include <cctype>
+#include <cstddef>
+#include <exception>
+#include <iterator>
+#include <sstream>
+
+#include <boost/algorithm/string/classification.hpp>
+#include <boost/archive/iterators/base64_from_binary.hpp>
+#include <boost/archive/iterators/binary_from_base64.hpp>
+#include <boost/archive/iterators/transform_width.hpp>
+#include <boost/iterator/iterator_facade.hpp>
+#include <boost/function.hpp>
+#include <glog/logging.h>
+
+using std::string;
+using std::vector;
+using namespace boost::archive::iterators; // NOLINT(*)
+
+namespace kudu {
+
+// Hive selectively encodes characters. This is the whitelist of
+// characters it will encode.
+// See common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+// in the Hive source code for the source of this list.
+static boost::function<bool (char)> HiveShouldEscape = boost::is_any_of("\"#%\\*/:=?\u00FF"); // NOLINT(*)
+
+// It is more convenient to maintain the complement of the set of
+// characters to escape when not in Hive-compat mode.
+static boost::function<bool (char)> ShouldNotEscape = boost::is_any_of("-_.~"); // NOLINT(*)
+
+static inline void UrlEncode(const char* in, int in_len, string* out, bool hive_compat) {
+  (*out).reserve(in_len);
+  std::ostringstream ss;
+  for (int i = 0; i < in_len; ++i) {
+    const char ch = in[i];
+    // Escape the character iff a) we are in Hive-compat mode and the
+    // character is in the Hive whitelist or b) we are not in
+    // Hive-compat mode, and the character is not alphanumeric or one
+    // of the four commonly excluded characters.
+    if ((hive_compat && HiveShouldEscape(ch)) ||
+        (!hive_compat && !(isalnum(ch) || ShouldNotEscape(ch)))) {
+      ss << '%' << std::uppercase << std::hex << static_cast<uint32_t>(ch);
+    } else {
+      ss << ch;
+    }
+  }
+
+  (*out) = ss.str();
+}
+
+void UrlEncode(const vector<uint8_t>& in, string* out, bool hive_compat) {
+  if (in.empty()) {
+    *out = "";
+  } else {
+    UrlEncode(reinterpret_cast<const char*>(&in[0]), in.size(), out, hive_compat);
+  }
+}
+
+void UrlEncode(const string& in, string* out, bool hive_compat) {
+  UrlEncode(in.c_str(), in.size(), out, hive_compat);
+}
+
+string UrlEncodeToString(const std::string& in, bool hive_compat) {
+  string ret;
+  UrlEncode(in, &ret, hive_compat);
+  return ret;
+}
+
+// Adapted from
+// http://www.boost.org/doc/libs/1_40_0/doc/html/boost_asio/
+//   example/http/server3/request_handler.cpp
+// See http://www.boost.org/LICENSE_1_0.txt for license for this method.
+bool UrlDecode(const string& in, string* out, bool hive_compat) {
+  out->clear();
+  out->reserve(in.size());
+  for (size_t i = 0; i < in.size(); ++i) {
+    if (in[i] == '%') {
+      if (i + 3 <= in.size()) {
+        int value = 0;
+        std::istringstream is(in.substr(i + 1, 2));
+        if (is >> std::hex >> value) {
+          (*out) += static_cast<char>(value);
+          i += 2;
+        } else {
+          return false;
+        }
+      } else {
+        return false;
+      }
+    } else if (!hive_compat && in[i] == '+') { // Hive does not encode ' ' as '+'
+      (*out) += ' ';
+    } else {
+      (*out) += in[i];
+    }
+  }
+  return true;
+}
+
+static inline void Base64Encode(const char* in, int in_len, std::ostringstream* out) {
+  typedef base64_from_binary<transform_width<const char*, 6, 8> > base64_encode;
+  // Base64 encodes 8 byte chars as 6 bit values.
+  std::ostringstream::pos_type len_before = out->tellp();
+  copy(base64_encode(in), base64_encode(in + in_len), std::ostream_iterator<char>(*out));
+  int bytes_written = out->tellp() - len_before;
+  // Pad with = to make it valid base64 encoded string
+  int num_pad = bytes_written % 4;
+  if (num_pad != 0) {
+    num_pad = 4 - num_pad;
+    for (int i = 0; i < num_pad; ++i) {
+      (*out) << "=";
+    }
+  }
+  DCHECK_EQ(out->str().size() % 4, 0);
+}
+
+void Base64Encode(const vector<uint8_t>& in, string* out) {
+  if (in.empty()) {
+    *out = "";
+  } else {
+    std::ostringstream ss;
+    Base64Encode(in, &ss);
+    *out = ss.str();
+  }
+}
+
+void Base64Encode(const vector<uint8_t>& in, std::ostringstream* out) {
+  if (!in.empty()) {
+    // Boost does not like non-null terminated strings
+    string tmp(reinterpret_cast<const char*>(&in[0]), in.size());
+    Base64Encode(tmp.c_str(), tmp.size(), out);
+  }
+}
+
+void Base64Encode(const string& in, string* out) {
+  std::ostringstream ss;
+  Base64Encode(in.c_str(), in.size(), &ss);
+  *out = ss.str();
+}
+
+void Base64Encode(const string& in, std::ostringstream* out) {
+  Base64Encode(in.c_str(), in.size(), out);
+}
+
+bool Base64Decode(const string& in, string* out) {
+  typedef transform_width<binary_from_base64<string::const_iterator>, 8, 6> base64_decode;
+  string tmp = in;
+  // Replace padding with base64 encoded NULL
+  replace(tmp.begin(), tmp.end(), '=', 'A');
+  try {
+    *out = string(base64_decode(tmp.begin()), base64_decode(tmp.end()));
+  } catch(std::exception& e) {
+    return false;
+  }
+
+  // Remove trailing '\0' that were added as padding.  Since \0 is special,
+  // the boost functions get confused so do this manually.
+  int num_padded_chars = 0;
+  for (int i = out->size() - 1; i >= 0; --i) {
+    if ((*out)[i] != '\0') break;
+    ++num_padded_chars;
+  }
+  out->resize(out->size() - num_padded_chars);
+  return true;
+}
+
+void EscapeForHtml(const string& in, std::ostringstream* out) {
+  DCHECK(out != nullptr);
+  for (const char& c : in) {
+    switch (c) {
+      case '<': (*out) << "&lt;";
+                break;
+      case '>': (*out) << "&gt;";
+                break;
+      case '&': (*out) << "&amp;";
+                break;
+      default: (*out) << c;
+    }
+  }
+}
+
+std::string EscapeForHtmlToString(const std::string& in) {
+  std::ostringstream str;
+  EscapeForHtml(in, &str);
+  return str.str();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/url-coding.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/url-coding.h b/be/src/kudu/util/url-coding.h
new file mode 100644
index 0000000..3f667aa
--- /dev/null
+++ b/be/src/kudu/util/url-coding.h
@@ -0,0 +1,69 @@
+// 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.
+#ifndef UTIL_URL_CODING_H
+#define UTIL_URL_CODING_H
+
+#include <cstdint>
+#include <iosfwd>
+#include <string>
+#include <vector>
+
+namespace kudu {
+
+// Utility method to URL-encode a string (that is, replace special
+// characters with %<hex value in ascii>).
+// The optional parameter hive_compat controls whether we mimic Hive's
+// behaviour when encoding a string, which is only to encode certain
+// characters (excluding, e.g., ' ')
+void UrlEncode(const std::string& in, std::string* out, bool hive_compat = false);
+void UrlEncode(const std::vector<uint8_t>& in, std::string* out,
+    bool hive_compat = false);
+std::string UrlEncodeToString(const std::string& in, bool hive_compat = false);
+
+// Utility method to decode a string that was URL-encoded. Returns
+// true unless the string could not be correctly decoded.
+// The optional parameter hive_compat controls whether or not we treat
+// the strings as encoded by Hive, which means selectively ignoring
+// certain characters like ' '.
+bool UrlDecode(const std::string& in, std::string* out, bool hive_compat = false);
+
+// Utility method to encode input as base-64 encoded.  This is not
+// very performant (multiple string copies) and should not be used
+// in a hot path.
+void Base64Encode(const std::vector<uint8_t>& in, std::string* out);
+void Base64Encode(const std::vector<uint8_t>& in, std::ostringstream* out);
+void Base64Encode(const std::string& in, std::string* out);
+void Base64Encode(const std::string& in, std::ostringstream* out);
+
+// Utility method to decode base64 encoded strings.  Also not extremely
+// performant.
+// Returns true unless the string could not be correctly decoded.
+bool Base64Decode(const std::string& in, std::string* out);
+
+// Replaces &, < and > with &amp;, &lt; and &gt; respectively. This is
+// not the full set of required encodings, but one that should be
+// added to on a case-by-case basis. Slow, since it necessarily
+// inspects each character in turn, and copies them all to *out; use
+// judiciously.
+void EscapeForHtml(const std::string& in, std::ostringstream* out);
+
+// Same as above, but returns a string.
+std::string EscapeForHtmlToString(const std::string& in);
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/user-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/user-test.cc b/be/src/kudu/util/user-test.cc
new file mode 100644
index 0000000..35785d0
--- /dev/null
+++ b/be/src/kudu/util/user-test.cc
@@ -0,0 +1,44 @@
+// 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 <string>
+#include <ostream>
+
+#include <gtest/gtest.h>
+#include <glog/logging.h>
+
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/user.h"
+
+namespace kudu {
+
+using std::string;
+
+class TestUser : public KuduTest {
+};
+
+// Validate that the current username is non-empty.
+TEST_F(TestUser, TestNonEmpty) {
+  string username;
+  ASSERT_TRUE(username.empty());
+  ASSERT_OK(GetLoggedInUser(&username));
+  ASSERT_FALSE(username.empty());
+  LOG(INFO) << "Name of the current user is: " << username;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/user.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/user.cc b/be/src/kudu/util/user.cc
new file mode 100644
index 0000000..f44e040
--- /dev/null
+++ b/be/src/kudu/util/user.cc
@@ -0,0 +1,90 @@
+// 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 "kudu/util/user.h"
+
+#include <pwd.h>
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdlib>
+#include <mutex>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+namespace {
+
+Status DoGetLoggedInUser(string* user_name) {
+  DCHECK(user_name != nullptr);
+
+  struct passwd pwd;
+  struct passwd *result;
+
+  // Get the system-defined limit for usernames. If the value was indeterminate,
+  // use a constant that should be more than enough, per the man page.
+  int64_t retval = sysconf(_SC_GETPW_R_SIZE_MAX);
+  size_t bufsize = retval > 0 ? retval : 16384;
+
+  gscoped_ptr<char[], FreeDeleter> buf(static_cast<char *>(malloc(bufsize)));
+  if (buf.get() == nullptr) {
+    return Status::RuntimeError("malloc failed", ErrnoToString(errno), errno);
+  }
+
+  int ret = getpwuid_r(getuid(), &pwd, buf.get(), bufsize, &result);
+  if (result == nullptr) {
+    if (ret == 0) {
+      return Status::NotFound("Current logged-in user not found! This is an unexpected error.");
+    } else {
+      // Errno in ret
+      return Status::RuntimeError("Error calling getpwuid_r()", ErrnoToString(ret), ret);
+    }
+  }
+  *user_name = pwd.pw_name;
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+Status GetLoggedInUser(string* user_name) {
+  static std::once_flag once;
+  static string* once_user_name;
+  static Status* once_status;
+  std::call_once(once, [](){
+      string u;
+      Status s = DoGetLoggedInUser(&u);
+      debug::ScopedLeakCheckDisabler ignore_leaks;
+      once_status = new Status(std::move(s));
+      once_user_name = new string(std::move(u));
+    });
+
+  RETURN_NOT_OK(*once_status);
+  *user_name = *once_user_name;
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/user.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/user.h b/be/src/kudu/util/user.h
new file mode 100644
index 0000000..6839a81
--- /dev/null
+++ b/be/src/kudu/util/user.h
@@ -0,0 +1,32 @@
+// 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.
+#ifndef KUDU_UTIL_USER_H
+#define KUDU_UTIL_USER_H
+
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Get current logged-in user with getpwuid_r().
+// user name is written to user_name.
+Status GetLoggedInUser(std::string* user_name);
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_USER_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_info.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_info.cc b/be/src/kudu/util/version_info.cc
new file mode 100644
index 0000000..1dfcdec
--- /dev/null
+++ b/be/src/kudu/util/version_info.cc
@@ -0,0 +1,84 @@
+// 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 "kudu/util/version_info.h"
+
+#include <cstring>
+#include <string>
+
+#include "kudu/generated/version_defines.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/version_info.pb.h"
+
+using std::string;
+
+namespace kudu {
+
+string VersionInfo::GetGitHash() {
+  string ret = KUDU_GIT_HASH;
+  if (!KUDU_BUILD_CLEAN_REPO) {
+    ret += "-dirty";
+  }
+  return ret;
+}
+
+string VersionInfo::GetShortVersionInfo() {
+  return KUDU_VERSION_STRING;
+}
+
+string VersionInfo::GetVersionInfo() {
+  return strings::Substitute("kudu $0 (rev $1)",
+                             KUDU_VERSION_STRING,
+                             GetGitHash());
+}
+
+string VersionInfo::GetAllVersionInfo() {
+  string ret = strings::Substitute(
+      "kudu $0\n"
+      "revision $1\n"
+      "build type $2\n"
+      "built by $3 at $4 on $5",
+      KUDU_VERSION_STRING,
+      GetGitHash(),
+      KUDU_BUILD_TYPE,
+      KUDU_BUILD_USERNAME,
+      KUDU_BUILD_TIMESTAMP,
+      KUDU_BUILD_HOSTNAME);
+  if (strlen(KUDU_BUILD_ID) > 0) {
+    strings::SubstituteAndAppend(&ret, "\nbuild id $0", KUDU_BUILD_ID);
+  }
+#ifdef ADDRESS_SANITIZER
+  ret += "\nASAN enabled";
+#endif
+#ifdef THREAD_SANITIZER
+  ret += "\nTSAN enabled";
+#endif
+  return ret;
+}
+
+void VersionInfo::GetVersionInfoPB(VersionInfoPB* pb) {
+  pb->set_git_hash(KUDU_GIT_HASH);
+  pb->set_build_hostname(KUDU_BUILD_HOSTNAME);
+  pb->set_build_timestamp(KUDU_BUILD_TIMESTAMP);
+  pb->set_build_username(KUDU_BUILD_USERNAME);
+  pb->set_build_clean_repo(KUDU_BUILD_CLEAN_REPO);
+  pb->set_build_id(KUDU_BUILD_ID);
+  pb->set_build_type(KUDU_BUILD_TYPE);
+  pb->set_version_string(KUDU_VERSION_STRING);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_info.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_info.h b/be/src/kudu/util/version_info.h
new file mode 100644
index 0000000..e19830d
--- /dev/null
+++ b/be/src/kudu/util/version_info.h
@@ -0,0 +1,51 @@
+// 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.
+#ifndef KUDU_UTIL_VERSION_INFO_H
+#define KUDU_UTIL_VERSION_INFO_H
+
+#include <string>
+
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+class VersionInfoPB;
+
+// Static functions related to fetching information about the current build.
+class VersionInfo {
+ public:
+  // Get a short version string ("1.2.3" or "1.9.3-SNAPSHOT").
+  static std::string GetShortVersionInfo();
+
+  // Get a version string ("kudu 1.2.3 (rev abcdef...)").
+  static std::string GetVersionInfo();
+
+  // Get a multi-line string including version info, build time, etc.
+  static std::string GetAllVersionInfo();
+
+  // Set the version info in 'pb'.
+  static void GetVersionInfoPB(VersionInfoPB* pb);
+ private:
+  // Get the git hash for this build. If the working directory was dirty when
+  // Kudu was built, also appends "-dirty".
+  static std::string GetGitHash();
+
+  DISALLOW_IMPLICIT_CONSTRUCTORS(VersionInfo);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_VERSION_INFO_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_info.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_info.proto b/be/src/kudu/util/version_info.proto
new file mode 100644
index 0000000..ca82f12
--- /dev/null
+++ b/be/src/kudu/util/version_info.proto
@@ -0,0 +1,32 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+option java_package = "org.apache.kudu";
+
+// Information about the build environment, configuration, etc.
+message VersionInfoPB {
+  optional string git_hash = 1;
+  optional string build_hostname = 2;
+  optional string build_timestamp = 3;
+  optional string build_username = 4;
+  optional bool build_clean_repo = 5;
+  optional string build_id = 6;
+  optional string build_type = 7;
+  optional string version_string = 8;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/version_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/version_util-test.cc b/be/src/kudu/util/version_util-test.cc
new file mode 100644
index 0000000..54e8e76
--- /dev/null
+++ b/be/src/kudu/util/version_util-test.cc
@@ -0,0 +1,66 @@
+// 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 "kudu/util/version_util.h"
+
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+class VersionUtilTest : public KuduTest {};
+
+TEST_F(VersionUtilTest, TestVersion) {
+  const vector<Version> good_test_cases = {
+    { "0.0.0", 0, 0, 0, "" },
+    { "1.0.0", 1, 0, 0, "" },
+    { "1.1.0", 1, 1, 0, "" },
+    { "1.1.1", 1, 1, 1, "" },
+    { "1.10.100-1000", 1, 10, 100, "1000" },
+    { "1.2.3-SNAPSHOT", 1, 2, 3, "SNAPSHOT" },
+  };
+
+  Version v;
+  for (const auto& test_case : good_test_cases) {
+    ASSERT_OK(ParseVersion(test_case.raw_version, &v));
+    EXPECT_EQ(test_case, v);
+  }
+
+  const vector<string> bad_test_cases = {
+    "",
+    "foo",
+    "foo.1.0",
+    "1.bar.0",
+    "1.0.foo",
+    "1.0foo.bar",
+    "foo5-1.4.3",
+  };
+
+  for (const auto& test_case : bad_test_cases) {
+    ASSERT_TRUE(ParseVersion(test_case, &v).IsInvalidArgument());
+  }
+}
+
+} // namespace kudu


[05/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/subprocess.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/subprocess.cc b/be/src/kudu/util/subprocess.cc
new file mode 100644
index 0000000..d68cb7f
--- /dev/null
+++ b/be/src/kudu/util/subprocess.cc
@@ -0,0 +1,815 @@
+// 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 "kudu/util/subprocess.h"
+
+#include <dirent.h>
+#include <fcntl.h>
+#include <signal.h>
+#if defined(__linux__)
+#include <sys/prctl.h>
+#endif
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <ev++.h>
+#include <glog/logging.h>
+#include <glog/stl_logging.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/signal.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+
+using std::map;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Split;
+using strings::Substitute;
+using strings::SubstituteAndAppend;
+
+namespace kudu {
+
+// Make glog's STL-compatible operators visible inside this namespace.
+using ::operator<<;
+
+namespace {
+
+static double kProcessWaitTimeoutSeconds = 5.0;
+
+static const char* kProcSelfFd =
+#if defined(__APPLE__)
+  "/dev/fd";
+#else
+  "/proc/self/fd";
+#endif // defined(__APPLE__)
+
+#if defined(__linux__)
+#define READDIR readdir64
+#define DIRENT dirent64
+#else
+#define READDIR readdir
+#define DIRENT dirent
+#endif
+
+// Since opendir() calls malloc(), this must be called before fork().
+// This function is not async-signal-safe.
+Status OpenProcFdDir(DIR** dir) {
+  *dir = opendir(kProcSelfFd);
+  if (PREDICT_FALSE(dir == nullptr)) {
+    return Status::IOError(Substitute("opendir(\"$0\") failed", kProcSelfFd),
+                           ErrnoToString(errno), errno);
+  }
+  return Status::OK();
+}
+
+// Close the directory stream opened by OpenProcFdDir().
+// This function is not async-signal-safe.
+void CloseProcFdDir(DIR* dir) {
+  if (PREDICT_FALSE(closedir(dir) == -1)) {
+    LOG(WARNING) << "Unable to close fd dir: "
+                 << Status::IOError(Substitute("closedir(\"$0\") failed", kProcSelfFd),
+                                    ErrnoToString(errno), errno).ToString();
+  }
+}
+
+// Close all open file descriptors other than stdin, stderr, stdout.
+// Expects a directory stream created by OpenProdFdDir() as a parameter.
+// This function is called after fork() and must not call malloc().
+// The rule of thumb is to only call async-signal-safe functions in such cases
+// if at all possible.
+void CloseNonStandardFDs(DIR* fd_dir) {
+  // This is implemented by iterating over the open file descriptors
+  // rather than using sysconf(SC_OPEN_MAX) -- the latter is error prone
+  // since it may not represent the highest open fd if the fd soft limit
+  // has changed since the process started. This should also be faster
+  // since iterating over all possible fds is likely to cause 64k+ syscalls
+  // in typical configurations.
+  //
+  // Note also that this doesn't use any of the Env utility functions, to
+  // make it as lean and mean as possible -- this runs in the subprocess
+  // after a fork, so there's some possibility that various global locks
+  // inside malloc() might be held, so allocating memory is a no-no.
+  PCHECK(fd_dir != nullptr);
+  int dir_fd = dirfd(fd_dir);
+
+  struct DIRENT* ent;
+  // readdir64() is not reentrant (it uses a static buffer) and it also
+  // locks fd_dir->lock, so it must not be called in a multi-threaded
+  // environment and is certainly not async-signal-safe.
+  // However, it appears to be safe to call right after fork(), since only one
+  // thread exists in the child process at that time. It also does not call
+  // malloc() or free(). We could use readdir64_r() instead, but all that
+  // buys us is reentrancy, and not async-signal-safety, due to the use of
+  // dir->lock, so seems not worth the added complexity in lifecycle & plumbing.
+  while ((ent = READDIR(fd_dir)) != nullptr) {
+    uint32_t fd;
+    if (!safe_strtou32(ent->d_name, &fd)) continue;
+    if (!(fd == STDIN_FILENO  ||
+          fd == STDOUT_FILENO ||
+          fd == STDERR_FILENO ||
+          fd == dir_fd))  {
+      int ret;
+      RETRY_ON_EINTR(ret, close(fd));
+    }
+  }
+}
+
+void RedirectToDevNull(int fd) {
+  // We must not close stderr or stdout, because then when a new file
+  // descriptor is opened, it might reuse the closed file descriptor's number
+  // (we always allocate the lowest available file descriptor number).
+  //
+  // Instead, we open /dev/null as a new file descriptor, then use dup2() to
+  // atomically close 'fd' and reuse its file descriptor number as an open file
+  // handle to /dev/null.
+  //
+  // It is expected that the file descriptor allocated when opening /dev/null
+  // will be closed when the child process closes all of its "non-standard"
+  // file descriptors later on.
+  int dev_null;
+  RETRY_ON_EINTR(dev_null, open("/dev/null", O_WRONLY));
+  if (dev_null < 0) {
+    PLOG(WARNING) << "failed to open /dev/null";
+  } else {
+    int ret;
+    RETRY_ON_EINTR(ret, dup2(dev_null, fd));
+    PCHECK(ret);
+  }
+}
+
+// Stateful libev watcher to help ReadFdsFully().
+class ReadFdsFullyHelper {
+ public:
+  ReadFdsFullyHelper(string progname, ev::dynamic_loop* loop, int fd)
+      : progname_(std::move(progname)) {
+    // Bind the watcher to the provided loop, to this functor, and to the
+    // readable fd.
+    watcher_.set(*loop);
+    watcher_.set(this);
+    watcher_.set(fd, ev::READ);
+
+    // The watcher will now be polled when its loop is run.
+    watcher_.start();
+  }
+
+  void operator() (ev::io &w, int revents) {
+    DCHECK_EQ(ev::READ, revents);
+
+    char buf[1024];
+    ssize_t n;
+    RETRY_ON_EINTR(n, read(w.fd, buf, arraysize(buf)));
+    if (n == 0) {
+      // EOF, stop watching.
+      w.stop();
+    } else if (n < 0) {
+      // A fatal error. Store it and stop watching.
+      status_ = Status::IOError("IO error reading from " + progname_,
+                                ErrnoToString(errno), errno);
+      w.stop();
+    } else {
+      // Add our bytes and keep watching.
+      output_.append(buf, n);
+    }
+  }
+
+  const Status& status() const { return status_; }
+  const string& output() const { return output_; }
+
+ private:
+  const string progname_;
+
+  ev::io watcher_;
+  string output_;
+  Status status_;
+};
+
+// Reads from all descriptors in 'fds' until EOF on all of them. If any read
+// yields an error, it is returned. Otherwise, 'out' contains the bytes read
+// for each fd, in the same order as was in 'fds'.
+Status ReadFdsFully(const string& progname,
+                    const vector<int>& fds,
+                    vector<string>* out) {
+  ev::dynamic_loop loop;
+
+  // Set up a watcher for each fd.
+  vector<unique_ptr<ReadFdsFullyHelper>> helpers;
+  for (int fd : fds) {
+    helpers.emplace_back(new ReadFdsFullyHelper(progname, &loop, fd));
+  }
+
+  // This will read until all fds return EOF.
+  loop.run();
+
+  // Check for failures.
+  for (const auto& h : helpers) {
+    if (!h->status().ok()) {
+      return h->status();
+    }
+  }
+
+  // No failures; write the output to the caller.
+  for (const auto& h : helpers) {
+    out->push_back(h->output());
+  }
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+Subprocess::Subprocess(vector<string> argv, int sig_on_destruct)
+    : program_(argv[0]),
+      argv_(std::move(argv)),
+      state_(kNotStarted),
+      child_pid_(-1),
+      fd_state_(),
+      child_fds_(),
+      sig_on_destruct_(sig_on_destruct) {
+  // By convention, the first argument in argv is the base name of the program.
+  argv_[0] = BaseName(argv_[0]);
+
+  fd_state_[STDIN_FILENO]   = PIPED;
+  fd_state_[STDOUT_FILENO]  = SHARED;
+  fd_state_[STDERR_FILENO]  = SHARED;
+  child_fds_[STDIN_FILENO]  = -1;
+  child_fds_[STDOUT_FILENO] = -1;
+  child_fds_[STDERR_FILENO] = -1;
+}
+
+Subprocess::~Subprocess() {
+  if (state_ == kRunning) {
+    LOG(WARNING) << Substitute(
+        "Child process $0 ($1) was orphaned. Sending signal $2...",
+        child_pid_, JoinStrings(argv_, " "), sig_on_destruct_);
+    WARN_NOT_OK(KillAndWait(sig_on_destruct_),
+                Substitute("Failed to KillAndWait() with signal $0",
+                           sig_on_destruct_));
+  }
+
+  for (int i = 0; i < 3; ++i) {
+    if (fd_state_[i] == PIPED && child_fds_[i] >= 0) {
+      int ret;
+      RETRY_ON_EINTR(ret, close(child_fds_[i]));
+    }
+  }
+}
+
+#if defined(__APPLE__)
+static int pipe2(int pipefd[2], int flags) {
+  DCHECK_EQ(O_CLOEXEC, flags);
+
+  int new_fds[2];
+  if (pipe(new_fds) == -1) {
+    return -1;
+  }
+  if (fcntl(new_fds[0], F_SETFD, O_CLOEXEC) == -1) {
+    int ret;
+    RETRY_ON_EINTR(ret, close(new_fds[0]));
+    RETRY_ON_EINTR(ret, close(new_fds[1]));
+    return -1;
+  }
+  if (fcntl(new_fds[1], F_SETFD, O_CLOEXEC) == -1) {
+    int ret;
+    RETRY_ON_EINTR(ret, close(new_fds[0]));
+    RETRY_ON_EINTR(ret, close(new_fds[1]));
+    return -1;
+  }
+  pipefd[0] = new_fds[0];
+  pipefd[1] = new_fds[1];
+  return 0;
+}
+#endif
+
+Status Subprocess::Start() {
+  VLOG(2) << "Invoking command: " << argv_;
+  if (state_ != kNotStarted) {
+    const string err_str = Substitute("$0: illegal sub-process state", state_);
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+  if (argv_.empty()) {
+    return Status::InvalidArgument("argv must have at least one elem");
+  }
+
+  // We explicitly set SIGPIPE to SIG_IGN here because we are using UNIX pipes.
+  IgnoreSigPipe();
+
+  vector<char*> argv_ptrs;
+  for (const string& arg : argv_) {
+    argv_ptrs.push_back(const_cast<char*>(arg.c_str()));
+  }
+  argv_ptrs.push_back(nullptr);
+
+  // Pipe from caller process to child's stdin
+  // [0] = stdin for child, [1] = how parent writes to it
+  int child_stdin[2] = {-1, -1};
+  if (fd_state_[STDIN_FILENO] == PIPED) {
+    PCHECK(pipe2(child_stdin, O_CLOEXEC) == 0);
+  }
+  // Pipe from child's stdout back to caller process
+  // [0] = how parent reads from child's stdout, [1] = how child writes to it
+  int child_stdout[2] = {-1, -1};
+  if (fd_state_[STDOUT_FILENO] == PIPED) {
+    PCHECK(pipe2(child_stdout, O_CLOEXEC) == 0);
+  }
+  // Pipe from child's stderr back to caller process
+  // [0] = how parent reads from child's stderr, [1] = how child writes to it
+  int child_stderr[2] = {-1, -1};
+  if (fd_state_[STDERR_FILENO] == PIPED) {
+    PCHECK(pipe2(child_stderr, O_CLOEXEC) == 0);
+  }
+  // The synchronization pipe: this trick is to make sure the parent returns
+  // control only after the child process has invoked execvp().
+  int sync_pipe[2];
+  PCHECK(pipe2(sync_pipe, O_CLOEXEC) == 0);
+
+  DIR* fd_dir = nullptr;
+  RETURN_NOT_OK_PREPEND(OpenProcFdDir(&fd_dir), "Unable to open fd dir");
+  unique_ptr<DIR, std::function<void(DIR*)>> fd_dir_closer(fd_dir,
+                                                           CloseProcFdDir);
+  int ret;
+  RETRY_ON_EINTR(ret, fork());
+  if (ret == -1) {
+    return Status::RuntimeError("Unable to fork", ErrnoToString(errno), errno);
+  }
+  if (ret == 0) { // We are the child
+    // Send the child a SIGTERM when the parent dies. This is done as early
+    // as possible in the child's life to prevent any orphaning whatsoever
+    // (e.g. from KUDU-402).
+#if defined(__linux__)
+    // TODO: prctl(PR_SET_PDEATHSIG) is Linux-specific, look into portable ways
+    // to prevent orphans when parent is killed.
+    prctl(PR_SET_PDEATHSIG, SIGKILL);
+#endif
+
+    // stdin
+    if (fd_state_[STDIN_FILENO] == PIPED) {
+      int dup2_ret;
+      RETRY_ON_EINTR(dup2_ret, dup2(child_stdin[0], STDIN_FILENO));
+      PCHECK(dup2_ret == STDIN_FILENO);
+    } else {
+      DCHECK_EQ(SHARED, fd_state_[STDIN_FILENO]);
+    }
+
+    // stdout
+    switch (fd_state_[STDOUT_FILENO]) {
+      case PIPED: {
+        int dup2_ret;
+        RETRY_ON_EINTR(dup2_ret, dup2(child_stdout[1], STDOUT_FILENO));
+        PCHECK(dup2_ret == STDOUT_FILENO);
+        break;
+      }
+      case DISABLED: {
+        RedirectToDevNull(STDOUT_FILENO);
+        break;
+      }
+      default:
+        DCHECK_EQ(SHARED, fd_state_[STDOUT_FILENO]);
+        break;
+    }
+
+    // stderr
+    switch (fd_state_[STDERR_FILENO]) {
+      case PIPED: {
+        int dup2_ret;
+        RETRY_ON_EINTR(dup2_ret, dup2(child_stderr[1], STDERR_FILENO));
+        PCHECK(dup2_ret == STDERR_FILENO);
+        break;
+      }
+      case DISABLED: {
+        RedirectToDevNull(STDERR_FILENO);
+        break;
+      }
+      default:
+        DCHECK_EQ(SHARED, fd_state_[STDERR_FILENO]);
+        break;
+    }
+
+    // Close the read side of the sync pipe;
+    // the write side should be closed upon execvp().
+    int close_ret;
+    RETRY_ON_EINTR(close_ret, close(sync_pipe[0]));
+    PCHECK(close_ret == 0);
+
+    CloseNonStandardFDs(fd_dir);
+
+    // Ensure we are not ignoring or blocking signals in the child process.
+    ResetAllSignalMasksToUnblocked();
+
+    // Reset the disposition of SIGPIPE to SIG_DFL because we routinely set its
+    // disposition to SIG_IGN via IgnoreSigPipe(). At the time of writing, we
+    // don't explicitly ignore any other signals in Kudu.
+    ResetSigPipeHandlerToDefault();
+
+    // Set the current working directory of the subprocess.
+    if (!cwd_.empty()) {
+      PCHECK(chdir(cwd_.c_str()) == 0);
+    }
+
+    // Set the environment for the subprocess. This is more portable than
+    // using execvpe(), which doesn't exist on OS X. We rely on the 'p'
+    // variant of exec to do $PATH searching if the executable specified
+    // by the caller isn't an absolute path.
+    for (const auto& env : env_) {
+      ignore_result(setenv(env.first.c_str(), env.second.c_str(), 1 /* overwrite */));
+    }
+
+    execvp(program_.c_str(), &argv_ptrs[0]);
+    int err = errno;
+    PLOG(ERROR) << "Couldn't exec " << program_;
+    _exit(err);
+  } else {
+    // We are the parent
+    child_pid_ = ret;
+    // Close child's side of the pipes
+    int close_ret;
+    if (fd_state_[STDIN_FILENO]  == PIPED) RETRY_ON_EINTR(close_ret, close(child_stdin[0]));
+    if (fd_state_[STDOUT_FILENO] == PIPED) RETRY_ON_EINTR(close_ret, close(child_stdout[1]));
+    if (fd_state_[STDERR_FILENO] == PIPED) RETRY_ON_EINTR(close_ret, close(child_stderr[1]));
+    // Keep parent's side of the pipes
+    child_fds_[STDIN_FILENO]  = child_stdin[1];
+    child_fds_[STDOUT_FILENO] = child_stdout[0];
+    child_fds_[STDERR_FILENO] = child_stderr[0];
+
+    // Wait for the child process to invoke execvp(). The trick involves
+    // a pipe with O_CLOEXEC option for its descriptors. The parent process
+    // performs blocking read from the pipe while the write side of the pipe
+    // is kept open by the child (it does not write any data, though). The write
+    // side of the pipe is closed when the child invokes execvp(). At that
+    // point, the parent should receive EOF, i.e. read() should return 0.
+    {
+      // Close the write side of the sync pipe. It's crucial to make sure
+      // it succeeds otherwise the blocking read() below might wait forever
+      // even if the child process has closed the pipe.
+      RETRY_ON_EINTR(close_ret, close(sync_pipe[1]));
+      PCHECK(close_ret == 0);
+      while (true) {
+        uint8_t buf;
+        int err = 0;
+        int rc;
+        RETRY_ON_EINTR(rc, read(sync_pipe[0], &buf, 1));
+        if (rc == -1) {
+          err = errno;
+        }
+        RETRY_ON_EINTR(close_ret, close(sync_pipe[0]));
+        PCHECK(close_ret == 0);
+        if (rc == 0) {
+          // That's OK -- expecting EOF from the other side of the pipe.
+          break;
+        } else if (rc == -1) {
+          // Other errors besides EINTR are not expected.
+          return Status::RuntimeError("Unexpected error from the sync pipe",
+                                      ErrnoToString(err), err);
+        }
+        // No data is expected from the sync pipe.
+        LOG(FATAL) << Substitute("$0: unexpected data from the sync pipe", rc);
+      }
+    }
+  }
+
+  state_ = kRunning;
+  return Status::OK();
+}
+
+Status Subprocess::Wait(int* wait_status) {
+  return DoWait(wait_status, BLOCKING);
+}
+
+Status Subprocess::WaitNoBlock(int* wait_status) {
+  return DoWait(wait_status, NON_BLOCKING);
+}
+
+Status Subprocess::GetProcfsState(int pid, ProcfsState* state) {
+  faststring data;
+  string filename = Substitute("/proc/$0/stat", pid);
+  RETURN_NOT_OK(ReadFileToString(Env::Default(), filename, &data));
+
+  // The part of /proc/<pid>/stat that's relevant for us looks like this:
+  //
+  //   "16009 (subprocess-test) R ..."
+  //
+  // The first number is the PID, the string in the parens in the command, and
+  // the single letter afterwards is the process' state.
+  //
+  // To extract the state, we scan backwards looking for the last ')', then
+  // increment past it and the separating space. This is safer than scanning
+  // forward as it properly handles commands containing parens.
+  string data_str = data.ToString();
+  const char* end_parens = strrchr(data_str.c_str(), ')');
+  if (end_parens == nullptr) {
+    return Status::RuntimeError(Substitute("unexpected layout in $0", filename));
+  }
+  char proc_state = end_parens[2];
+
+  switch (proc_state) {
+    case 'T':
+      *state = ProcfsState::PAUSED;
+      break;
+    default:
+      *state = ProcfsState::RUNNING;
+      break;
+  }
+  return Status::OK();
+}
+
+Status Subprocess::Kill(int signal) {
+  if (state_ != kRunning) {
+    const string err_str = "Sub-process is not running";
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+  if (kill(child_pid_, signal) != 0) {
+    return Status::RuntimeError("Unable to kill",
+                                ErrnoToString(errno),
+                                errno);
+  }
+
+  // Signal delivery is often asynchronous. For some signals, we try to wait
+  // for the process to actually change state, using /proc/<pid>/stat as a
+  // guide. This is best-effort.
+  ProcfsState desired_state;
+  switch (signal) {
+    case SIGSTOP:
+      desired_state = ProcfsState::PAUSED;
+      break;
+    case SIGCONT:
+      desired_state = ProcfsState::RUNNING;
+      break;
+    default:
+      return Status::OK();
+  }
+  Stopwatch sw;
+  sw.start();
+  do {
+    ProcfsState current_state;
+    if (!GetProcfsState(child_pid_, &current_state).ok()) {
+      // There was some error parsing /proc/<pid>/stat (or perhaps it doesn't
+      // exist on this platform).
+      return Status::OK();
+    }
+    if (current_state == desired_state) {
+      return Status::OK();
+    }
+    SleepFor(MonoDelta::FromMilliseconds(10));
+  } while (sw.elapsed().wall_seconds() < kProcessWaitTimeoutSeconds);
+  return Status::OK();
+}
+
+Status Subprocess::KillAndWait(int signal) {
+  string procname = Substitute("$0 (pid $1)", argv0(), pid());
+
+  // This is a fatal error because all errors in Kill() are signal-independent,
+  // so Kill(SIGKILL) is just as likely to fail if this did.
+  RETURN_NOT_OK_PREPEND(
+      Kill(signal), Substitute("Failed to send signal $0 to $1",
+                               signal, procname));
+  if (signal == SIGKILL) {
+    RETURN_NOT_OK_PREPEND(
+        Wait(), Substitute("Failed to wait on $0", procname));
+  } else {
+    Status s;
+    Stopwatch sw;
+    sw.start();
+    do {
+      s = WaitNoBlock();
+      if (s.ok()) {
+        break;
+      } else if (!s.IsTimedOut()) {
+        // An unexpected error in WaitNoBlock() is likely to manifest repeatedly,
+        // so there's no point in retrying this.
+        RETURN_NOT_OK_PREPEND(
+            s, Substitute("Unexpected failure while waiting on $0", procname));
+      }
+      SleepFor(MonoDelta::FromMilliseconds(10));
+    } while (sw.elapsed().wall_seconds() < kProcessWaitTimeoutSeconds);
+    if (s.IsTimedOut()) {
+      return KillAndWait(SIGKILL);
+    }
+  }
+  return Status::OK();
+}
+
+Status Subprocess::GetExitStatus(int* exit_status, string* info_str) const {
+  if (state_ != kExited) {
+    const string err_str = "Sub-process termination hasn't yet been detected";
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+  string info;
+  int status;
+  if (WIFEXITED(wait_status_)) {
+    status = WEXITSTATUS(wait_status_);
+    if (status == 0) {
+      info = Substitute("$0: process successfully exited", program_);
+    } else {
+      info = Substitute("$0: process exited with non-zero status $1",
+                        program_, status);
+    }
+  } else if (WIFSIGNALED(wait_status_)) {
+    // Using signal number as exit status.
+    status = WTERMSIG(wait_status_);
+    info = Substitute("$0: process exited on signal $1", program_, status);
+#if defined(WCOREDUMP)
+    if (WCOREDUMP(wait_status_)) {
+      SubstituteAndAppend(&info, " (core dumped)");
+    }
+#endif
+  } else {
+    status = -1;
+    info = Substitute("$0: process reported unexpected wait status $1",
+                      program_, wait_status_);
+    LOG(DFATAL) << info;
+  }
+  if (exit_status) {
+    *exit_status = status;
+  }
+  if (info_str) {
+    *info_str = info;
+  }
+  return Status::OK();
+}
+
+Status Subprocess::Call(const string& arg_str) {
+  vector<string> argv = Split(arg_str, " ");
+  return Call(argv, "", nullptr, nullptr);
+}
+
+Status Subprocess::Call(const vector<string>& argv,
+                        const string& stdin_in,
+                        string* stdout_out,
+                        string* stderr_out) {
+  Subprocess p(argv);
+
+  if (stdout_out) {
+    p.ShareParentStdout(false);
+  }
+  if (stderr_out) {
+    p.ShareParentStderr(false);
+  }
+  RETURN_NOT_OK_PREPEND(p.Start(),
+                        "Unable to fork " + argv[0]);
+
+  if (!stdin_in.empty()) {
+    ssize_t written;
+    RETRY_ON_EINTR(written, write(p.to_child_stdin_fd(), stdin_in.data(), stdin_in.size()));
+    if (written < stdin_in.size()) {
+      return Status::IOError("Unable to write to child process stdin",
+                             ErrnoToString(errno), errno);
+    }
+  }
+
+  int err;
+  RETRY_ON_EINTR(err, close(p.ReleaseChildStdinFd()));
+  if (PREDICT_FALSE(err != 0)) {
+    return Status::IOError("Unable to close child process stdin", ErrnoToString(errno), errno);
+  }
+
+  vector<int> fds;
+  if (stdout_out) {
+    fds.push_back(p.from_child_stdout_fd());
+  }
+  if (stderr_out) {
+    fds.push_back(p.from_child_stderr_fd());
+  }
+  vector<string> outv;
+  RETURN_NOT_OK(ReadFdsFully(argv[0], fds, &outv));
+
+  // Given that ReadFdsFully captures the strings in the order in which we
+  // had installed 'fds' above, it can be assured that we can receive
+  // as many strings as there were 'fds' in the vector and in that order.
+  CHECK_EQ(outv.size(), fds.size());
+  if (stdout_out) {
+    *stdout_out = std::move(outv.front());
+  }
+  if (stderr_out) {
+    *stderr_out = std::move(outv.back());
+  }
+
+  RETURN_NOT_OK_PREPEND(p.Wait(), "Unable to wait() for " + argv[0]);
+  int exit_status;
+  string exit_info_str;
+  RETURN_NOT_OK(p.GetExitStatus(&exit_status, &exit_info_str));
+  if (exit_status != 0) {
+    return Status::RuntimeError(exit_info_str);
+  }
+  return Status::OK();
+}
+
+pid_t Subprocess::pid() const {
+  CHECK_EQ(state_, kRunning);
+  return child_pid_;
+}
+
+Status Subprocess::DoWait(int* wait_status, WaitMode mode) {
+  if (state_ == kExited) {
+    if (wait_status) {
+      *wait_status = wait_status_;
+    }
+    return Status::OK();
+  }
+  if (state_ != kRunning) {
+    const string err_str = Substitute("$0: illegal sub-process state", state_);
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+
+  const int options = (mode == NON_BLOCKING) ? WNOHANG : 0;
+  int status;
+  int rc;
+  RETRY_ON_EINTR(rc, waitpid(child_pid_, &status, options));
+  if (rc == -1) {
+    return Status::RuntimeError("Unable to wait on child",
+                                ErrnoToString(errno), errno);
+  }
+  if (mode == NON_BLOCKING && rc == 0) {
+    return Status::TimedOut("");
+  }
+  CHECK_EQ(rc, child_pid_);
+  CHECK(WIFEXITED(status) || WIFSIGNALED(status));
+
+  child_pid_ = -1;
+  wait_status_ = status;
+  state_ = kExited;
+  if (wait_status) {
+    *wait_status = status;
+  }
+  return Status::OK();
+}
+
+void Subprocess::SetEnvVars(map<string, string> env) {
+  CHECK_EQ(state_, kNotStarted);
+  env_ = std::move(env);
+}
+
+void Subprocess::SetCurrentDir(string cwd) {
+  CHECK_EQ(state_, kNotStarted);
+  cwd_ = std::move(cwd);
+}
+
+void Subprocess::SetFdShared(int stdfd, bool share) {
+  CHECK_EQ(state_, kNotStarted);
+  fd_state_[stdfd] = share ? SHARED : PIPED;
+}
+
+void Subprocess::DisableStderr() {
+  CHECK_EQ(state_, kNotStarted);
+  fd_state_[STDERR_FILENO] = DISABLED;
+}
+
+void Subprocess::DisableStdout() {
+  CHECK_EQ(state_, kNotStarted);
+  fd_state_[STDOUT_FILENO] = DISABLED;
+}
+
+int Subprocess::CheckAndOffer(int stdfd) const {
+  CHECK_EQ(state_, kRunning);
+  CHECK_EQ(fd_state_[stdfd], PIPED);
+  return child_fds_[stdfd];
+}
+
+int Subprocess::ReleaseChildFd(int stdfd) {
+  CHECK_EQ(state_, kRunning);
+  CHECK_GE(child_fds_[stdfd], 0);
+  CHECK_EQ(fd_state_[stdfd], PIPED);
+  int ret = child_fds_[stdfd];
+  child_fds_[stdfd] = -1;
+  return ret;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/subprocess.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/subprocess.h b/be/src/kudu/util/subprocess.h
new file mode 100644
index 0000000..4d33c8f
--- /dev/null
+++ b/be/src/kudu/util/subprocess.h
@@ -0,0 +1,219 @@
+// 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.
+#ifndef KUDU_UTIL_SUBPROCESS_H
+#define KUDU_UTIL_SUBPROCESS_H
+
+#include <signal.h>
+#include <unistd.h>
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Wrapper around a spawned subprocess.
+//
+// program will be treated as an absolute path unless it begins with a dot or a
+// slash.
+//
+// This takes care of creating pipes to/from the subprocess and offers
+// basic functionality to wait on it or send signals.
+// By default, child process only has stdin captured and separate from the parent.
+// The stdout/stderr streams are shared with the parent by default.
+//
+// The process may only be started and waited on/killed once.
+//
+// Optionally, user may change parent/child stream sharing. Also, a user may disable
+// a subprocess stream. A user cannot do both.
+//
+// Note that, when the Subprocess object is destructed, the child process
+// will be forcibly SIGKILLed to avoid orphaning processes.
+class Subprocess {
+ public:
+  // Constructs a new Subprocess that will execute 'argv' on Start().
+  //
+  // If the process isn't explicitly killed, 'sig_on_destroy' will be delivered
+  // to it when the Subprocess goes out of scope.
+  explicit Subprocess(std::vector<std::string> argv, int sig_on_destruct = SIGKILL);
+  ~Subprocess();
+
+  // Disables subprocess stream output. Is mutually exclusive with stream sharing.
+  //
+  // Must be called before subprocess starts.
+  void DisableStderr();
+  void DisableStdout();
+
+  // Configures the subprocess to share the parent's stream. Is mutually
+  // exclusive with stream disabling.
+  //
+  // Must be called before subprocess starts.
+  void ShareParentStdin(bool  share = true) { SetFdShared(STDIN_FILENO,  share); }
+  void ShareParentStdout(bool share = true) { SetFdShared(STDOUT_FILENO, share); }
+  void ShareParentStderr(bool share = true) { SetFdShared(STDERR_FILENO, share); }
+
+  // Add environment variables to be set before executing the subprocess.
+  //
+  // These environment variables are merged into the existing environment
+  // of the parent process. In other words, there is no need to prime this
+  // map with the current environment; instead, just specify any variables
+  // that should be overridden.
+  //
+  // Repeated calls to this function replace earlier calls.
+  void SetEnvVars(std::map<std::string, std::string> env);
+
+  // Set the initial current working directory of the subprocess.
+  //
+  // Must be set before starting the subprocess.
+  void SetCurrentDir(std::string cwd);
+
+  // Start the subprocess. Can only be called once.
+  //
+  // This returns a bad Status if the fork() fails. However,
+  // note that if the executable path was incorrect such that
+  // exec() fails, this will still return Status::OK. You must
+  // use Wait() to check for failure.
+  Status Start() WARN_UNUSED_RESULT;
+
+  // Wait for the subprocess to exit. The return value is the same as
+  // that of the waitpid() syscall. Only call after starting.
+  //
+  // NOTE: unlike the standard wait(2) call, this may be called multiple
+  // times. If the process has exited, it will repeatedly return the same
+  // exit code.
+  Status Wait(int* wait_status = nullptr) WARN_UNUSED_RESULT;
+
+  // Like the above, but does not block. This returns Status::TimedOut
+  // immediately if the child has not exited. Otherwise returns Status::OK
+  // and sets *ret. Only call after starting.
+  //
+  // NOTE: unlike the standard wait(2) call, this may be called multiple
+  // times. If the process has exited, it will repeatedly return the same
+  // exit code.
+  Status WaitNoBlock(int* wait_status = nullptr) WARN_UNUSED_RESULT;
+
+  // Send a signal to the subprocess.
+  // Note that this does not reap the process -- you must still Wait()
+  // in order to reap it. Only call after starting.
+  Status Kill(int signal) WARN_UNUSED_RESULT;
+
+  // Sends a signal to the subprocess and waits for it to exit.
+  //
+  // If the signal is not SIGKILL and the process doesn't appear to be exiting,
+  // retries with SIGKILL.
+  Status KillAndWait(int signal);
+
+  // Retrieve exit status of the process awaited by Wait() and/or WaitNoBlock()
+  // methods. Must be called only after calling Wait()/WaitNoBlock().
+  Status GetExitStatus(int* exit_status, std::string* info_str = nullptr) const
+      WARN_UNUSED_RESULT;
+
+  // Helper method that creates a Subprocess, issues a Start() then a Wait().
+  // Expects a blank-separated list of arguments, with the first being the
+  // full path to the executable.
+  // The returned Status will only be OK if all steps were successful and
+  // the return code was 0.
+  static Status Call(const std::string& arg_str) WARN_UNUSED_RESULT;
+
+  // Same as above, but accepts a vector that includes the path to the
+  // executable as argv[0] and the arguments to the program in argv[1..n].
+  //
+  // Writes the value of 'stdin_in' to the subprocess' stdin. The length of
+  // 'stdin_in' should be limited to 64kib.
+  //
+  // Also collects the output from the child process stdout and stderr into
+  // 'stdout_out' and 'stderr_out' respectively.
+  static Status Call(const std::vector<std::string>& argv,
+                     const std::string& stdin_in = "",
+                     std::string* stdout_out = nullptr,
+                     std::string* stderr_out = nullptr) WARN_UNUSED_RESULT;
+
+  // Return the pipe fd to the child's standard stream.
+  // Stream should not be disabled or shared.
+  int to_child_stdin_fd()    const { return CheckAndOffer(STDIN_FILENO); }
+  int from_child_stdout_fd() const { return CheckAndOffer(STDOUT_FILENO); }
+  int from_child_stderr_fd() const { return CheckAndOffer(STDERR_FILENO); }
+
+  // Release control of the file descriptor for the child's stream, only if piped.
+  // Writes to this FD show up on stdin in the subprocess
+  int ReleaseChildStdinFd()  { return ReleaseChildFd(STDIN_FILENO ); }
+  // Reads from this FD come from stdout of the subprocess
+  int ReleaseChildStdoutFd() { return ReleaseChildFd(STDOUT_FILENO); }
+  // Reads from this FD come from stderr of the subprocess
+  int ReleaseChildStderrFd() { return ReleaseChildFd(STDERR_FILENO); }
+
+  pid_t pid() const;
+  const std::string& argv0() const { return argv_[0]; }
+
+ private:
+  FRIEND_TEST(SubprocessTest, TestGetProcfsState);
+
+  enum State {
+    kNotStarted,
+    kRunning,
+    kExited
+  };
+  enum StreamMode {SHARED, DISABLED, PIPED};
+  enum WaitMode {BLOCKING, NON_BLOCKING};
+
+  // Process state according to /proc/<pid>/stat.
+  enum class ProcfsState {
+    // "T  Stopped (on a signal) or (before Linux 2.6.33) trace stopped"
+    PAUSED,
+
+    // Every other process state.
+    RUNNING,
+  };
+
+  // Extracts the process state for /proc/<pid>/stat.
+  //
+  // Returns an error if /proc/</pid>/stat doesn't exist or if parsing failed.
+  static Status GetProcfsState(int pid, ProcfsState* state);
+
+  Status DoWait(int* wait_status, WaitMode mode) WARN_UNUSED_RESULT;
+  void SetFdShared(int stdfd, bool share);
+  int CheckAndOffer(int stdfd) const;
+  int ReleaseChildFd(int stdfd);
+
+  std::string program_;
+  std::vector<std::string> argv_;
+  std::map<std::string, std::string> env_;
+  State state_;
+  int child_pid_;
+  enum StreamMode fd_state_[3];
+  int child_fds_[3];
+  std::string cwd_;
+
+  // The cached wait status if Wait()/WaitNoBlock() has been called.
+  // Only valid if state_ == kExited.
+  int wait_status_;
+
+  // Custom signal to deliver when the subprocess goes out of scope, provided
+  // the process hasn't already been killed.
+  int sig_on_destruct_;
+
+  DISALLOW_COPY_AND_ASSIGN(Subprocess);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_SUBPROCESS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_graph.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_graph.cc b/be/src/kudu/util/test_graph.cc
new file mode 100644
index 0000000..59f4d30
--- /dev/null
+++ b/be/src/kudu/util/test_graph.cc
@@ -0,0 +1,121 @@
+// 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 "kudu/util/test_graph.h"
+
+#include <mutex>
+#include <ostream>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+using std::shared_ptr;
+using std::string;
+
+namespace kudu {
+
+void TimeSeries::AddValue(double val) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  val_ += val;
+}
+
+void TimeSeries::SetValue(double val) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  val_ = val;
+}
+
+double TimeSeries::value() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return val_;
+}
+
+TimeSeriesCollector::~TimeSeriesCollector() {
+  if (started_) {
+    StopDumperThread();
+  }
+}
+
+shared_ptr<TimeSeries> TimeSeriesCollector::GetTimeSeries(const string &key) {
+  MutexLock l(series_lock_);
+  SeriesMap::const_iterator it = series_map_.find(key);
+  if (it == series_map_.end()) {
+    shared_ptr<TimeSeries> ts(new TimeSeries());
+    series_map_[key] = ts;
+    return ts;
+  } else {
+    return (*it).second;
+  }
+}
+
+void TimeSeriesCollector::StartDumperThread() {
+  LOG(INFO) << "Starting metrics dumper";
+  CHECK(!started_);
+  exit_latch_.Reset(1);
+  started_ = true;
+  CHECK_OK(kudu::Thread::Create("time series", "dumper",
+      &TimeSeriesCollector::DumperThread, this, &dumper_thread_));
+}
+
+void TimeSeriesCollector::StopDumperThread() {
+  CHECK(started_);
+  exit_latch_.CountDown();
+  CHECK_OK(ThreadJoiner(dumper_thread_.get()).Join());
+  started_ = false;
+}
+
+void TimeSeriesCollector::DumperThread() {
+  CHECK(started_);
+  WallTime start_time = WallTime_Now();
+
+  faststring metrics_str;
+  while (true) {
+    metrics_str.clear();
+    metrics_str.append("metrics: ");
+    BuildMetricsString(WallTime_Now() - start_time, &metrics_str);
+    LOG(INFO) << metrics_str.ToString();
+
+    // Sleep until next dump time, or return if we should exit
+    if (exit_latch_.WaitFor(MonoDelta::FromMilliseconds(250))) {
+      return;
+    }
+  }
+}
+
+void TimeSeriesCollector::BuildMetricsString(
+  WallTime time_since_start, faststring *dst_buf) const {
+  MutexLock l(series_lock_);
+
+  dst_buf->append(StringPrintf("{ \"scope\": \"%s\", \"time\": %.3f",
+                               scope_.c_str(), time_since_start));
+
+  for (SeriesMap::const_reference entry : series_map_) {
+    dst_buf->append(StringPrintf(", \"%s\": %.3f",
+                                 entry.first.c_str(),  entry.second->value()));
+  }
+  dst_buf->append("}");
+}
+
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_graph.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_graph.h b/be/src/kudu/util/test_graph.h
new file mode 100644
index 0000000..41df430
--- /dev/null
+++ b/be/src/kudu/util/test_graph.h
@@ -0,0 +1,90 @@
+// 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.
+#ifndef KUDU_TEST_GRAPH_COLLECTOR_H
+#define KUDU_TEST_GRAPH_COLLECTOR_H
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/mutex.h"
+
+namespace kudu {
+
+class Thread;
+class faststring;
+
+class TimeSeries {
+ public:
+  void AddValue(double val);
+  void SetValue(double val);
+
+  double value() const;
+
+ private:
+  friend class TimeSeriesCollector;
+
+  DISALLOW_COPY_AND_ASSIGN(TimeSeries);
+
+  TimeSeries() :
+    val_(0)
+  {}
+
+  mutable simple_spinlock lock_;
+  double val_;
+};
+
+class TimeSeriesCollector {
+ public:
+  explicit TimeSeriesCollector(std::string scope)
+      : scope_(std::move(scope)), exit_latch_(0), started_(false) {}
+
+  ~TimeSeriesCollector();
+
+  std::shared_ptr<TimeSeries> GetTimeSeries(const std::string &key);
+  void StartDumperThread();
+  void StopDumperThread();
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(TimeSeriesCollector);
+
+  void DumperThread();
+  void BuildMetricsString(WallTime time_since_start, faststring *dst_buf) const;
+
+  std::string scope_;
+
+  typedef std::unordered_map<std::string, std::shared_ptr<TimeSeries> > SeriesMap;
+  SeriesMap series_map_;
+  mutable Mutex series_lock_;
+
+  scoped_refptr<kudu::Thread> dumper_thread_;
+
+  // Latch used to stop the dumper_thread_. When the thread is started,
+  // this is set to 1, and when the thread should exit, it is counted down.
+  CountDownLatch exit_latch_;
+
+  bool started_;
+};
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_macros.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_macros.h b/be/src/kudu/util/test_macros.h
new file mode 100644
index 0000000..63cae5a
--- /dev/null
+++ b/be/src/kudu/util/test_macros.h
@@ -0,0 +1,123 @@
+// 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.
+#ifndef KUDU_UTIL_TEST_MACROS_H
+#define KUDU_UTIL_TEST_MACROS_H
+
+#include <gmock/gmock.h>
+#include <string>
+
+// ASSERT_NO_FATAL_FAILURE is just too long to type.
+#define NO_FATALS(expr) \
+  ASSERT_NO_FATAL_FAILURE(expr)
+
+// Detect fatals in the surrounding scope. NO_FATALS() only checks for fatals
+// in the expression passed to it.
+#define NO_PENDING_FATALS() \
+  if (testing::Test::HasFatalFailure()) { return; }
+
+#define ASSERT_OK(status) do { \
+  const Status& _s = status;        \
+  if (_s.ok()) { \
+    SUCCEED(); \
+  } else { \
+    FAIL() << "Bad status: " << _s.ToString();  \
+  } \
+} while (0);
+
+#define EXPECT_OK(status) do { \
+  const Status& _s = status; \
+  if (_s.ok()) { \
+    SUCCEED(); \
+  } else { \
+    ADD_FAILURE() << "Bad status: " << _s.ToString();  \
+  } \
+} while (0);
+
+// Like the above, but doesn't record successful
+// tests.
+#define ASSERT_OK_FAST(status) do { \
+  const Status& _s = status; \
+  if (!_s.ok()) { \
+    FAIL() << "Bad status: " << _s.ToString(); \
+  } \
+} while (0);
+
+// Substring matches.
+#define ASSERT_STR_CONTAINS(str, substr) \
+  ASSERT_THAT(str, testing::HasSubstr(substr))
+
+#define ASSERT_STR_NOT_CONTAINS(str, substr) \
+  ASSERT_THAT(str, testing::Not(testing::HasSubstr(substr)))
+
+// Substring regular expressions in extended regex (POSIX) syntax.
+#define ASSERT_STR_MATCHES(str, pattern) \
+  ASSERT_THAT(str, testing::ContainsRegex(pattern))
+
+#define ASSERT_STR_NOT_MATCHES(str, pattern) \
+  ASSERT_THAT(str, testing::Not(testing::ContainsRegex(pattern)))
+
+// Batched substring regular expressions in extended regex (POSIX) syntax.
+//
+// All strings must match the pattern.
+#define ASSERT_STRINGS_ALL_MATCH(strings, pattern) do { \
+  const auto& _strings = (strings); \
+  const auto& _pattern = (pattern); \
+  int _str_idx = 0; \
+  for (const auto& str : _strings) { \
+    ASSERT_STR_MATCHES(str, _pattern) \
+        << "string " << _str_idx << ": pattern " << _pattern \
+        << " does not match string " << str; \
+    _str_idx++; \
+  } \
+} while (0)
+
+// Batched substring regular expressions in extended regex (POSIX) syntax.
+//
+// At least one string must match the pattern.
+#define ASSERT_STRINGS_ANY_MATCH(strings, pattern) do { \
+  const auto& _strings = (strings); \
+  const auto& _pattern = (pattern); \
+  bool matched = false; \
+  for (const auto& str : _strings) { \
+    if (testing::internal::RE::PartialMatch(str, testing::internal::RE(_pattern))) { \
+      matched = true; \
+      break; \
+    } \
+  } \
+  ASSERT_TRUE(matched) \
+      << "not one string matched pattern " << _pattern; \
+} while (0)
+
+#define ASSERT_FILE_EXISTS(env, path) do { \
+  const std::string& _s = path; \
+  ASSERT_TRUE(env->FileExists(_s)) \
+    << "Expected file to exist: " << _s; \
+} while (0);
+
+#define ASSERT_FILE_NOT_EXISTS(env, path) do { \
+  const std::string& _s = path; \
+  ASSERT_FALSE(env->FileExists(_s)) \
+    << "Expected file not to exist: " << _s; \
+} while (0);
+
+#define CURRENT_TEST_NAME() \
+  ::testing::UnitTest::GetInstance()->current_test_info()->name()
+
+#define CURRENT_TEST_CASE_NAME() \
+  ::testing::UnitTest::GetInstance()->current_test_info()->test_case_name()
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_main.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_main.cc b/be/src/kudu/util/test_main.cc
new file mode 100644
index 0000000..c75e5ae
--- /dev/null
+++ b/be/src/kudu/util/test_main.cc
@@ -0,0 +1,109 @@
+// 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 <ostream>
+#include <thread>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/minidump.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/pstack_watcher.h"
+#include "kudu/util/signal.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_util.h"
+
+DEFINE_int32(test_timeout_after, 0,
+             "Maximum total seconds allowed for all unit tests in the suite. Default: disabled");
+
+DEFINE_int32(stress_cpu_threads, 0,
+             "Number of threads to start that burn CPU in an attempt to "
+             "stimulate race conditions");
+
+namespace kudu {
+
+// Start thread that kills the process if --test_timeout_after is exceeded before
+// the tests complete.
+static void CreateAndStartTimeoutThread() {
+  if (FLAGS_test_timeout_after == 0) return;
+
+  // KUDU-1995: if running death tests using EXPECT_EXIT()/ASSERT_EXIT(), LSAN
+  // reports leaks in CreateAndStartTimeoutThread(). Adding a couple of scoped
+  // leak check disablers as a workaround since right now it's not clear what
+  // is going on exactly: LSAN does not report those leaks for tests which run
+  // ASSERT_DEATH(). This does not seem harmful or hiding any potential leaks
+  // since it's scoped and targeted only for this utility thread.
+  debug::ScopedLeakCheckDisabler disabler;
+  std::thread([=](){
+      debug::ScopedLeakCheckDisabler disabler;
+      SleepFor(MonoDelta::FromSeconds(FLAGS_test_timeout_after));
+      // Dump a pstack to stdout.
+      WARN_NOT_OK(PstackWatcher::DumpStacks(), "Unable to print pstack");
+
+      // ...and abort.
+      LOG(FATAL) << "Maximum unit test time exceeded (" << FLAGS_test_timeout_after << " sec)";
+    }).detach();
+}
+} // namespace kudu
+
+
+static void StartStressThreads() {
+  for (int i = 0; i < FLAGS_stress_cpu_threads; i++) {
+    std::thread([]{
+        while (true) {
+          // Do something which won't be optimized out.
+          base::subtle::MemoryBarrier();
+        }
+      }).detach();
+  }
+}
+
+int main(int argc, char **argv) {
+  google::InstallFailureSignalHandler();
+
+  // We don't use InitGoogleLoggingSafe() because gtest initializes glog, so we
+  // need to block SIGUSR1 explicitly in order to test minidump generation.
+  CHECK_OK(kudu::BlockSigUSR1());
+
+  // Ignore SIGPIPE for all tests so that threads writing to TLS
+  // sockets do not crash when writing to a closed socket. See KUDU-1910.
+  kudu::IgnoreSigPipe();
+
+  // InitGoogleTest() must precede ParseCommandLineFlags(), as the former
+  // removes gtest-related flags from argv that would trip up the latter.
+  ::testing::InitGoogleTest(&argc, argv);
+  kudu::ParseCommandLineFlags(&argc, &argv, true);
+
+  // Create the test-timeout timer.
+  kudu::CreateAndStartTimeoutThread();
+
+  StartStressThreads();
+
+  // This is called by the KuduTest setup method, but in case we have
+  // any tests that don't inherit from KuduTest, it's helpful to
+  // cover our bases and call it here too.
+  kudu::KuduTest::OverrideKrb5Environment();
+
+  int ret = RUN_ALL_TESTS();
+
+  return ret;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_util.cc b/be/src/kudu/util/test_util.cc
new file mode 100644
index 0000000..c960441
--- /dev/null
+++ b/be/src/kudu/util/test_util.cc
@@ -0,0 +1,446 @@
+// 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 "kudu/util/test_util.h"
+
+#include <errno.h>
+#include <limits.h>
+#include <unistd.h>
+
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#ifdef __APPLE__
+#include <fcntl.h>
+#include <sys/param.h> // for MAXPATHLEN
+#endif
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest-spi.h>
+
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strcat.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/spinlock_profiling.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+
+DEFINE_string(test_leave_files, "on_failure",
+              "Whether to leave test files around after the test run. "
+              " Valid values are 'always', 'on_failure', or 'never'");
+
+DEFINE_int32(test_random_seed, 0, "Random seed to use for randomized tests");
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+const char* kInvalidPath = "/dev/invalid-path-for-kudu-tests";
+static const char* const kSlowTestsEnvVariable = "KUDU_ALLOW_SLOW_TESTS";
+
+static const uint64_t kTestBeganAtMicros = Env::Default()->NowMicros();
+
+// Global which production code can check to see if it is running
+// in a GTest environment (assuming the test binary links in this module,
+// which is typically a good assumption).
+//
+// This can be checked using the 'IsGTest()' function from test_util_prod.cc.
+bool g_is_gtest = true;
+
+///////////////////////////////////////////////////
+// KuduTest
+///////////////////////////////////////////////////
+
+KuduTest::KuduTest()
+  : env_(Env::Default()),
+    flag_saver_(new google::FlagSaver()),
+    test_dir_(GetTestDataDirectory()) {
+  std::map<const char*, const char*> flags_for_tests = {
+    // Disabling fsync() speeds up tests dramatically, and it's safe to do as no
+    // tests rely on cutting power to a machine or equivalent.
+    {"never_fsync", "true"},
+    // Disable redaction.
+    {"redact", "none"},
+    // Reduce default RSA key length for faster tests. We are using strong/high
+    // TLS v1.2 cipher suites, so minimum possible for TLS-related RSA keys is
+    // 768 bits. However, for the external mini cluster we use 1024 bits because
+    // Java default security policies require at least 1024 bits for RSA keys
+    // used in certificates. For uniformity, here 1024 RSA bit keys are used
+    // as well. As for the TSK keys, 512 bits is the minimum since the SHA256
+    // digest is used for token signing/verification.
+    {"ipki_server_key_size", "1024"},
+    {"ipki_ca_key_size", "1024"},
+    {"tsk_num_rsa_bits", "512"},
+  };
+  for (const auto& e : flags_for_tests) {
+    // We don't check for errors here, because we have some default flags that
+    // only apply to certain tests.
+    google::SetCommandLineOptionWithMode(e.first, e.second, google::SET_FLAGS_DEFAULT);
+  }
+  // If the TEST_TMPDIR variable has been set, then glog will automatically use that
+  // as its default log directory. We would prefer that the default log directory
+  // instead be the test-case-specific subdirectory.
+  FLAGS_log_dir = GetTestDataDirectory();
+}
+
+KuduTest::~KuduTest() {
+  // Reset the flags first to prevent them from affecting test directory cleanup.
+  flag_saver_.reset();
+
+  // Clean up the test directory in the destructor instead of a TearDown
+  // method. This is better because it ensures that the child-class
+  // dtor runs first -- so, if the child class is using a minicluster, etc,
+  // we will shut that down before we remove files underneath.
+  if (FLAGS_test_leave_files == "always") {
+    LOG(INFO) << "-----------------------------------------------";
+    LOG(INFO) << "--test_leave_files specified, leaving files in " << test_dir_;
+  } else if (FLAGS_test_leave_files == "on_failure" && HasFatalFailure()) {
+    LOG(INFO) << "-----------------------------------------------";
+    LOG(INFO) << "Had fatal failures, leaving test files at " << test_dir_;
+  } else {
+    VLOG(1) << "Cleaning up temporary test files...";
+    WARN_NOT_OK(env_->DeleteRecursively(test_dir_),
+                "Couldn't remove test files");
+  }
+}
+
+void KuduTest::SetUp() {
+  InitSpinLockContentionProfiling();
+  OverrideKrb5Environment();
+}
+
+string KuduTest::GetTestPath(const string& relative_path) const {
+  return JoinPathSegments(test_dir_, relative_path);
+}
+
+void KuduTest::OverrideKrb5Environment() {
+  // Set these variables to paths that definitely do not exist and
+  // couldn't be accidentally created.
+  //
+  // Note that if we were to set these to /dev/null, we end up triggering a leak in krb5
+  // when it tries to read an empty file as a ticket cache, whereas non-existent files
+  // don't have this issue. See MIT krb5 bug #8509.
+  //
+  // NOTE: we don't simply *unset* the variables, because then we'd still pick up
+  // the user's /etc/krb5.conf and other default locations.
+  setenv("KRB5_CONFIG", kInvalidPath, 1);
+  setenv("KRB5_KTNAME", kInvalidPath, 1);
+  setenv("KRB5CCNAME", kInvalidPath, 1);
+}
+
+///////////////////////////////////////////////////
+// Test utility functions
+///////////////////////////////////////////////////
+
+bool AllowSlowTests() {
+  char *e = getenv(kSlowTestsEnvVariable);
+  if ((e == nullptr) ||
+      (strlen(e) == 0) ||
+      (strcasecmp(e, "false") == 0) ||
+      (strcasecmp(e, "0") == 0) ||
+      (strcasecmp(e, "no") == 0)) {
+    return false;
+  }
+  if ((strcasecmp(e, "true") == 0) ||
+      (strcasecmp(e, "1") == 0) ||
+      (strcasecmp(e, "yes") == 0)) {
+    return true;
+  }
+  LOG(FATAL) << "Unrecognized value for " << kSlowTestsEnvVariable << ": " << e;
+  return false;
+}
+
+void OverrideFlagForSlowTests(const std::string& flag_name,
+                              const std::string& new_value) {
+  // Ensure that the flag is valid.
+  google::GetCommandLineFlagInfoOrDie(flag_name.c_str());
+
+  // If we're not running slow tests, don't override it.
+  if (!AllowSlowTests()) {
+    return;
+  }
+  google::SetCommandLineOptionWithMode(flag_name.c_str(), new_value.c_str(),
+                                       google::SET_FLAG_IF_DEFAULT);
+}
+
+int SeedRandom() {
+  int seed;
+  // Initialize random seed
+  if (FLAGS_test_random_seed == 0) {
+    // Not specified by user
+    seed = static_cast<int>(GetCurrentTimeMicros());
+  } else {
+    seed = FLAGS_test_random_seed;
+  }
+  LOG(INFO) << "Using random seed: " << seed;
+  srand(seed);
+  return seed;
+}
+
+string GetTestDataDirectory() {
+  const ::testing::TestInfo* const test_info =
+    ::testing::UnitTest::GetInstance()->current_test_info();
+  CHECK(test_info) << "Must be running in a gtest unit test to call this function";
+  string dir;
+  CHECK_OK(Env::Default()->GetTestDirectory(&dir));
+
+  // The directory name includes some strings for specific reasons:
+  // - program name: identifies the directory to the test invoker
+  // - timestamp and pid: disambiguates with prior runs of the same test
+  //
+  // e.g. "env-test.TestEnv.TestReadFully.1409169025392361-23600"
+  //
+  // If the test is sharded, the shard index is also included so that the test
+  // invoker can more easily identify all directories belonging to each shard.
+  string shard_index_infix;
+  const char* shard_index = getenv("GTEST_SHARD_INDEX");
+  if (shard_index && shard_index[0] != '\0') {
+    shard_index_infix = Substitute("$0.", shard_index);
+  }
+  dir += Substitute("/$0.$1$2.$3.$4-$5",
+    StringReplace(google::ProgramInvocationShortName(), "/", "_", true),
+    shard_index_infix,
+    StringReplace(test_info->test_case_name(), "/", "_", true),
+    StringReplace(test_info->name(), "/", "_", true),
+    kTestBeganAtMicros,
+    getpid());
+  Status s = Env::Default()->CreateDir(dir);
+  CHECK(s.IsAlreadyPresent() || s.ok())
+    << "Could not create directory " << dir << ": " << s.ToString();
+  if (s.ok()) {
+    string metadata;
+
+    StrAppend(&metadata, Substitute("PID=$0\n", getpid()));
+
+    StrAppend(&metadata, Substitute("PPID=$0\n", getppid()));
+
+    char* jenkins_build_id = getenv("BUILD_ID");
+    if (jenkins_build_id) {
+      StrAppend(&metadata, Substitute("BUILD_ID=$0\n", jenkins_build_id));
+    }
+
+    CHECK_OK(WriteStringToFile(Env::Default(), metadata,
+                               Substitute("$0/test_metadata", dir)));
+  }
+  return dir;
+}
+
+string GetTestExecutableDirectory() {
+  string exec;
+  CHECK_OK(Env::Default()->GetExecutablePath(&exec));
+  return DirName(exec);
+}
+
+void AssertEventually(const std::function<void(void)>& f,
+                      const MonoDelta& timeout,
+                      AssertBackoff backoff) {
+  const MonoTime deadline = MonoTime::Now() + timeout;
+  {
+    // Disable --gtest_break_on_failure, or else the assertion failures
+    // inside our attempts will cause the test to SEGV even though we
+    // would like to retry.
+    bool old_break_on_failure = testing::FLAGS_gtest_break_on_failure;
+    auto c = MakeScopedCleanup([old_break_on_failure]() {
+      testing::FLAGS_gtest_break_on_failure = old_break_on_failure;
+    });
+    testing::FLAGS_gtest_break_on_failure = false;
+
+    for (int attempts = 0; MonoTime::Now() < deadline; attempts++) {
+      // Capture any assertion failures within this scope (i.e. from their function)
+      // into 'results'
+      testing::TestPartResultArray results;
+      testing::ScopedFakeTestPartResultReporter reporter(
+          testing::ScopedFakeTestPartResultReporter::INTERCEPT_ONLY_CURRENT_THREAD,
+          &results);
+      f();
+
+      // Determine whether their function produced any new test failure results.
+      bool has_failures = false;
+      for (int i = 0; i < results.size(); i++) {
+        has_failures |= results.GetTestPartResult(i).failed();
+      }
+      if (!has_failures) {
+        return;
+      }
+
+      // If they had failures, sleep and try again.
+      int sleep_ms;
+      switch (backoff) {
+        case AssertBackoff::EXPONENTIAL:
+          sleep_ms = (attempts < 10) ? (1 << attempts) : 1000;
+          break;
+        case AssertBackoff::NONE:
+          sleep_ms = 1;
+          break;
+        default:
+          LOG(FATAL) << "Unknown backoff type";
+      }
+      SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
+    }
+  }
+
+  // If we ran out of time looping, run their function one more time
+  // without capturing its assertions. This way the assertions will
+  // propagate back out to the normal test reporter. Of course it's
+  // possible that it will pass on this last attempt, but that's OK
+  // too, since we aren't trying to be that strict about the deadline.
+  f();
+  if (testing::Test::HasFatalFailure()) {
+    ADD_FAILURE() << "Timed out waiting for assertion to pass.";
+  }
+}
+
+int CountOpenFds(Env* env, const string& path_pattern) {
+  static const char* kProcSelfFd =
+#if defined(__APPLE__)
+    "/dev/fd";
+#else
+    "/proc/self/fd";
+#endif // defined(__APPLE__)
+  faststring path_buf;
+  vector<string> children;
+  CHECK_OK(env->GetChildren(kProcSelfFd, &children));
+  int num_fds = 0;
+  for (const auto& c : children) {
+    // Skip '.' and '..'.
+    if (c == "." || c == "..") {
+      continue;
+    }
+    int32_t fd;
+    CHECK(safe_strto32(c, &fd)) << "Unexpected file in fd list: " << c;
+#ifdef __APPLE__
+    path_buf.resize(MAXPATHLEN);
+    if (fcntl(fd, F_GETPATH, path_buf.data()) != 0) {
+      if (errno == EBADF) {
+        // The file was closed while we were looping. This is likely the
+        // actual file descriptor used for opening /proc/fd itself.
+        continue;
+      }
+      PLOG(FATAL) << "Unknown error in fcntl(F_GETPATH): " << fd;
+    }
+    char* buf_data = reinterpret_cast<char*>(path_buf.data());
+    path_buf.resize(strlen(buf_data));
+#else
+    path_buf.resize(PATH_MAX);
+    char* buf_data = reinterpret_cast<char*>(path_buf.data());
+    auto proc_file = JoinPathSegments(kProcSelfFd, c);
+    int path_len = readlink(proc_file.c_str(), buf_data, path_buf.size());
+    if (path_len < 0) {
+      if (errno == ENOENT) {
+        // The file was closed while we were looping. This is likely the
+        // actual file descriptor used for opening /proc/fd itself.
+        continue;
+      }
+      PLOG(FATAL) << "Unknown error in readlink: " << proc_file;
+    }
+    path_buf.resize(path_len);
+#endif
+    if (!MatchPattern(path_buf.ToString(), path_pattern)) {
+      continue;
+    }
+    num_fds++;
+  }
+
+  return num_fds;
+}
+
+namespace {
+Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeout) {
+  // In general, processes do not expose the port they bind to, and
+  // reimplementing lsof involves parsing a lot of files in /proc/. So,
+  // requiring lsof for tests and parsing its output seems more
+  // straight-forward. We call lsof in a loop since it typically takes a long
+  // time for it to initialize and bind a port.
+
+  string lsof;
+  RETURN_NOT_OK(FindExecutable("lsof", {"/sbin", "/usr/sbin"}, &lsof));
+
+  const vector<string> cmd = {
+    lsof, "-wbnP", "-Ffn",
+    "-p", std::to_string(pid),
+    "-a", "-i", kind
+  };
+
+  MonoTime deadline = MonoTime::Now() + timeout;
+  string lsof_out;
+
+  for (int64_t i = 1; ; i++) {
+    lsof_out.clear();
+    Status s = Subprocess::Call(cmd, "", &lsof_out);
+
+    if (s.ok()) {
+      StripTrailingNewline(&lsof_out);
+      break;
+    }
+    if (deadline < MonoTime::Now()) {
+      return s;
+    }
+
+    SleepFor(MonoDelta::FromMilliseconds(i * 10));
+  }
+
+  // The '-Ffn' flag gets lsof to output something like:
+  //   p19730
+  //   f123
+  //   n*:41254
+  // The first line is the pid. We ignore it.
+  // The second line is the file descriptor number. We ignore it.
+  // The third line has the bind address and port.
+  // Subsequent lines show active connections.
+  vector<string> lines = strings::Split(lsof_out, "\n");
+  int32_t p = -1;
+  if (lines.size() < 3 ||
+      lines[2].substr(0, 3) != "n*:" ||
+      !safe_strto32(lines[2].substr(3), &p) ||
+      p <= 0) {
+    return Status::RuntimeError("unexpected lsof output", lsof_out);
+  }
+  CHECK(p > 0 && p < std::numeric_limits<uint16_t>::max()) << "parsed invalid port: " << p;
+  VLOG(1) << "Determined bound port: " << p;
+  *port = p;
+  return Status::OK();
+}
+} // anonymous namespace
+
+Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
+  return WaitForBind(pid, port, "4TCP", timeout);
+}
+
+Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
+  return WaitForBind(pid, port, "4UDP", timeout);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_util.h b/be/src/kudu/util/test_util.h
new file mode 100644
index 0000000..8090fbc
--- /dev/null
+++ b/be/src/kudu/util/test_util.h
@@ -0,0 +1,146 @@
+// 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.
+//
+// Base test class, with various utility functions.
+#ifndef KUDU_UTIL_TEST_UTIL_H
+#define KUDU_UTIL_TEST_UTIL_H
+
+#include <sys/types.h>
+
+#include <cstdint>
+#include <functional>
+#include <memory>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/monotime.h"
+
+#define ASSERT_EVENTUALLY(expr) do { \
+  AssertEventually(expr); \
+  NO_PENDING_FATALS(); \
+} while (0)
+
+namespace google {
+class FlagSaver;
+} // namespace google
+
+namespace kudu {
+
+class Env;
+class Status;
+
+extern const char* kInvalidPath;
+
+class KuduTest : public ::testing::Test {
+ public:
+  KuduTest();
+
+  virtual ~KuduTest();
+
+  virtual void SetUp() OVERRIDE;
+
+  // Tests assume that they run with no outside-provided kerberos credentials, and if the
+  // user happened to have some credentials available they might fail due to being already
+  // kinitted to a different realm, etc. This function overrides the relevant environment
+  // variables so that we don't pick up the user's credentials.
+  static void OverrideKrb5Environment();
+
+ protected:
+  // Returns absolute path based on a unit test-specific work directory, given
+  // a relative path. Useful for writing test files that should be deleted after
+  // the test ends.
+  std::string GetTestPath(const std::string& relative_path) const;
+
+  Env* env_;
+
+  // Reset flags on every test. Allocated on the heap so it can be destroyed
+  // (and the flags reset) before test_dir_ is deleted.
+  std::unique_ptr<google::FlagSaver> flag_saver_;
+
+  std::string test_dir_;
+};
+
+// Returns true if slow tests are runtime-enabled.
+bool AllowSlowTests();
+
+// Override the given gflag to the new value, only in the case that
+// slow tests are enabled and the user hasn't otherwise overridden
+// it on the command line.
+// Example usage:
+//
+// OverrideFlagForSlowTests(
+//     "client_inserts_per_thread",
+//     strings::Substitute("$0", FLAGS_client_inserts_per_thread * 100));
+//
+void OverrideFlagForSlowTests(const std::string& flag_name,
+                              const std::string& new_value);
+
+// Call srand() with a random seed based on the current time, reporting
+// that seed to the logs. The time-based seed may be overridden by passing
+// --test_random_seed= from the CLI in order to reproduce a failed randomized
+// test. Returns the seed.
+int SeedRandom();
+
+// Return a per-test directory in which to store test data. Guaranteed to
+// return the same directory every time for a given unit test.
+//
+// May only be called from within a gtest unit test. Prefer KuduTest::test_dir_
+// if a KuduTest instance is available.
+std::string GetTestDataDirectory();
+
+// Return the directory which contains the test's executable.
+std::string GetTestExecutableDirectory();
+
+// Wait until 'f()' succeeds without adding any GTest 'fatal failures'.
+// For example:
+//
+//   AssertEventually([]() {
+//     ASSERT_GT(ReadValueOfMetric(), 10);
+//   });
+//
+// The function is run in a loop with optional back-off.
+//
+// To check whether AssertEventually() eventually succeeded, call
+// NO_PENDING_FATALS() afterward, or use ASSERT_EVENTUALLY() which performs
+// this check automatically.
+enum class AssertBackoff {
+  // Use exponential back-off while looping, capped at one second.
+  EXPONENTIAL,
+
+  // Sleep for a millisecond while looping.
+  NONE,
+};
+void AssertEventually(const std::function<void(void)>& f,
+                      const MonoDelta& timeout = MonoDelta::FromSeconds(30),
+                      AssertBackoff backoff = AssertBackoff::EXPONENTIAL);
+
+// Count the number of open file descriptors in use by this process.
+// 'path_pattern' is a glob-style pattern. Only paths that match this
+// pattern are included. Note that '*' in this pattern is recursive
+// unlike the usual behavior of path globs.
+int CountOpenFds(Env* env, const std::string& path_pattern);
+
+// Waits for the subprocess to bind to any listening TCP port, and returns the port.
+Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
+
+// Waits for the subprocess to bind to any listening UDP port, and returns the port.
+Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_util_prod.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_util_prod.cc b/be/src/kudu/util/test_util_prod.cc
new file mode 100644
index 0000000..5523bac
--- /dev/null
+++ b/be/src/kudu/util/test_util_prod.cc
@@ -0,0 +1,28 @@
+// 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 "kudu/util/test_util_prod.h"
+
+#include <dlfcn.h>
+
+namespace kudu {
+
+bool IsGTest() {
+  return dlsym(RTLD_DEFAULT, "_ZN4kudu10g_is_gtestE") != nullptr;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/test_util_prod.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/test_util_prod.h b/be/src/kudu/util/test_util_prod.h
new file mode 100644
index 0000000..8b7ea61
--- /dev/null
+++ b/be/src/kudu/util/test_util_prod.h
@@ -0,0 +1,32 @@
+// 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.
+
+// Test-related utility methods that can be called from non-test
+// code. This module is part of the 'util' module and is built into
+// all binaries, not just tests, whereas 'test_util.cc' is linked
+// only into test binaries.
+
+#pragma once
+
+namespace kudu {
+
+// Return true if the current binary is a gtest. More specifically,
+// returns true if the 'test_util.cc' module has been linked in
+// (either dynamically or statically) to the running process.
+bool IsGTest();
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/thread-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/thread-test.cc b/be/src/kudu/util/thread-test.cc
new file mode 100644
index 0000000..d3ee733
--- /dev/null
+++ b/be/src/kudu/util/thread-test.cc
@@ -0,0 +1,160 @@
+// 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 "kudu/util/thread.h"
+
+#include <sys/types.h>
+#include <unistd.h>
+
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread_restrictions.h"
+
+using std::string;
+
+namespace kudu {
+
+class ThreadTest : public KuduTest {};
+
+// Join with a thread and emit warnings while waiting to join.
+// This has to be manually verified.
+TEST_F(ThreadTest, TestJoinAndWarn) {
+  if (!AllowSlowTests()) {
+    LOG(INFO) << "Skipping test in quick test mode, since this sleeps";
+    return;
+  }
+
+  scoped_refptr<Thread> holder;
+  ASSERT_OK(Thread::Create("test", "sleeper thread", usleep, 1000*1000, &holder));
+  ASSERT_OK(ThreadJoiner(holder.get())
+                   .warn_after_ms(10)
+                   .warn_every_ms(100)
+                   .Join());
+}
+
+TEST_F(ThreadTest, TestFailedJoin) {
+  if (!AllowSlowTests()) {
+    LOG(INFO) << "Skipping test in quick test mode, since this sleeps";
+    return;
+  }
+
+  scoped_refptr<Thread> holder;
+  ASSERT_OK(Thread::Create("test", "sleeper thread", usleep, 1000*1000, &holder));
+  Status s = ThreadJoiner(holder.get())
+    .give_up_after_ms(50)
+    .Join();
+  ASSERT_STR_CONTAINS(s.ToString(), "Timed out after 50ms joining on sleeper thread");
+}
+
+static void TryJoinOnSelf() {
+  Status s = ThreadJoiner(Thread::current_thread()).Join();
+  // Use CHECK instead of ASSERT because gtest isn't thread-safe.
+  CHECK(s.IsInvalidArgument());
+}
+
+// Try to join on the thread that is currently running.
+TEST_F(ThreadTest, TestJoinOnSelf) {
+  scoped_refptr<Thread> holder;
+  ASSERT_OK(Thread::Create("test", "test", TryJoinOnSelf, &holder));
+  holder->Join();
+  // Actual assertion is done by the thread spawned above.
+}
+
+TEST_F(ThreadTest, TestDoubleJoinIsNoOp) {
+  scoped_refptr<Thread> holder;
+  ASSERT_OK(Thread::Create("test", "sleeper thread", usleep, 0, &holder));
+  ThreadJoiner joiner(holder.get());
+  ASSERT_OK(joiner.Join());
+  ASSERT_OK(joiner.Join());
+}
+
+TEST_F(ThreadTest, ThreadStartBenchmark) {
+  std::vector<scoped_refptr<Thread>> threads(1000);
+  LOG_TIMING(INFO, "starting threads") {
+    for (auto& t : threads) {
+      ASSERT_OK(Thread::Create("test", "TestCallOnExit", usleep, 0, &t));
+    }
+  }
+  LOG_TIMING(INFO, "waiting for all threads to publish TIDs") {
+    for (auto& t : threads) {
+      t->tid();
+    }
+  }
+
+  for (auto& t : threads) {
+    t->Join();
+  }
+}
+
+// The following tests only run in debug mode, since thread restrictions are no-ops
+// in release builds.
+#ifndef NDEBUG
+TEST_F(ThreadTest, TestThreadRestrictions_IO) {
+  // Default should be to allow IO
+  ThreadRestrictions::AssertIOAllowed();
+
+  ThreadRestrictions::SetIOAllowed(false);
+  {
+    ThreadRestrictions::ScopedAllowIO allow_io;
+    ASSERT_TRUE(Env::Default()->FileExists("/"));
+  }
+  ThreadRestrictions::SetIOAllowed(true);
+
+  // Disallow IO - doing IO should crash the process.
+  ASSERT_DEATH({
+      ThreadRestrictions::SetIOAllowed(false);
+      ignore_result(Env::Default()->FileExists("/"));
+    },
+    "Function marked as IO-only was called from a thread that disallows IO");
+}
+
+TEST_F(ThreadTest, TestThreadRestrictions_Waiting) {
+  // Default should be to allow IO
+  ThreadRestrictions::AssertWaitAllowed();
+
+  ThreadRestrictions::SetWaitAllowed(false);
+  {
+    ThreadRestrictions::ScopedAllowWait allow_wait;
+    CountDownLatch l(0);
+    l.Wait();
+  }
+  ThreadRestrictions::SetWaitAllowed(true);
+
+  // Disallow waiting - blocking on a latch should crash the process.
+  ASSERT_DEATH({
+      ThreadRestrictions::SetWaitAllowed(false);
+      CountDownLatch l(0);
+      l.Wait();
+    },
+    "Waiting is not allowed to be used on this thread");
+}
+#endif // NDEBUG
+
+} // namespace kudu


[16/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/maintenance_manager.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/maintenance_manager.cc b/be/src/kudu/util/maintenance_manager.cc
new file mode 100644
index 0000000..9a42464
--- /dev/null
+++ b/be/src/kudu/util/maintenance_manager.cc
@@ -0,0 +1,550 @@
+// 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 "kudu/util/maintenance_manager.h"
+
+#include <cinttypes>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <sstream>
+#include <string>
+#include <type_traits>
+#include <utility>
+
+#include <boost/bind.hpp>
+#include <gflags/gflags.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/debug/trace_logging.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/maintenance_manager.pb.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/process_memory.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/trace.h"
+
+using std::pair;
+using std::string;
+using strings::Substitute;
+
+DEFINE_int32(maintenance_manager_num_threads, 1,
+             "Size of the maintenance manager thread pool. "
+             "For spinning disks, the number of threads should "
+             "not be above the number of devices.");
+TAG_FLAG(maintenance_manager_num_threads, stable);
+
+DEFINE_int32(maintenance_manager_polling_interval_ms, 250,
+       "Polling interval for the maintenance manager scheduler, "
+       "in milliseconds.");
+TAG_FLAG(maintenance_manager_polling_interval_ms, hidden);
+
+DEFINE_int32(maintenance_manager_history_size, 8,
+       "Number of completed operations the manager is keeping track of.");
+TAG_FLAG(maintenance_manager_history_size, hidden);
+
+DEFINE_bool(enable_maintenance_manager, true,
+       "Enable the maintenance manager, runs compaction and tablet cleaning tasks.");
+TAG_FLAG(enable_maintenance_manager, unsafe);
+
+DEFINE_int64(log_target_replay_size_mb, 1024,
+             "The target maximum size of logs to be replayed at startup. If a tablet "
+             "has in-memory operations that are causing more than this size of logs "
+             "to be retained, then the maintenance manager will prioritize flushing "
+             "these operations to disk.");
+TAG_FLAG(log_target_replay_size_mb, experimental);
+
+DEFINE_int64(data_gc_min_size_mb, 0,
+             "The (exclusive) minimum number of megabytes of ancient data on "
+             "disk, per tablet, needed to prioritize deletion of that data.");
+TAG_FLAG(data_gc_min_size_mb, experimental);
+
+DEFINE_double(data_gc_prioritization_prob, 0.5,
+             "The probability that we will prioritize data GC over performance "
+             "improvement operations. If set to 1.0, we will always prefer to "
+             "delete old data before running performance improvement operations "
+             "such as delta compaction.");
+TAG_FLAG(data_gc_prioritization_prob, experimental);
+
+namespace kudu {
+
+MaintenanceOpStats::MaintenanceOpStats() {
+  Clear();
+}
+
+void MaintenanceOpStats::Clear() {
+  valid_ = false;
+  runnable_ = false;
+  ram_anchored_ = 0;
+  logs_retained_bytes_ = 0;
+  data_retained_bytes_ = 0;
+  perf_improvement_ = 0;
+  last_modified_ = MonoTime();
+}
+
+MaintenanceOp::MaintenanceOp(std::string name, IOUsage io_usage)
+    : name_(std::move(name)),
+      running_(0),
+      cancel_(false),
+      io_usage_(io_usage) {
+}
+
+MaintenanceOp::~MaintenanceOp() {
+  CHECK(!manager_.get()) << "You must unregister the " << name_
+         << " Op before destroying it.";
+}
+
+void MaintenanceOp::Unregister() {
+  CHECK(manager_.get()) << "Op " << name_ << " was never registered.";
+  manager_->UnregisterOp(this);
+}
+
+MaintenanceManagerStatusPB_OpInstancePB OpInstance::DumpToPB() const {
+  MaintenanceManagerStatusPB_OpInstancePB pb;
+  pb.set_thread_id(thread_id);
+  pb.set_name(name);
+  if (duration.Initialized()) {
+    pb.set_duration_millis(duration.ToMilliseconds());
+  }
+  MonoDelta delta(MonoTime::Now() - start_mono_time);
+  pb.set_millis_since_start(delta.ToMilliseconds());
+  return pb;
+}
+
+const MaintenanceManager::Options MaintenanceManager::kDefaultOptions = {
+  .num_threads = 0,
+  .polling_interval_ms = 0,
+  .history_size = 0,
+};
+
+MaintenanceManager::MaintenanceManager(const Options& options,
+                                       std::string server_uuid)
+  : server_uuid_(std::move(server_uuid)),
+    num_threads_(options.num_threads <= 0 ?
+                 FLAGS_maintenance_manager_num_threads : options.num_threads),
+    cond_(&lock_),
+    shutdown_(false),
+    polling_interval_ms_(options.polling_interval_ms <= 0 ?
+          FLAGS_maintenance_manager_polling_interval_ms :
+          options.polling_interval_ms),
+    running_ops_(0),
+    completed_ops_count_(0),
+    rand_(GetRandomSeed32()),
+    memory_pressure_func_(&process_memory::UnderMemoryPressure) {
+  CHECK_OK(ThreadPoolBuilder("MaintenanceMgr").set_min_threads(num_threads_)
+               .set_max_threads(num_threads_).Build(&thread_pool_));
+  uint32_t history_size = options.history_size == 0 ?
+                          FLAGS_maintenance_manager_history_size :
+                          options.history_size;
+  completed_ops_.resize(history_size);
+}
+
+MaintenanceManager::~MaintenanceManager() {
+  Shutdown();
+}
+
+Status MaintenanceManager::Start() {
+  CHECK(!monitor_thread_);
+  RETURN_NOT_OK(Thread::Create("maintenance", "maintenance_scheduler",
+      boost::bind(&MaintenanceManager::RunSchedulerThread, this),
+      &monitor_thread_));
+  return Status::OK();
+}
+
+void MaintenanceManager::Shutdown() {
+  {
+    std::lock_guard<Mutex> guard(lock_);
+    if (shutdown_) {
+      return;
+    }
+    shutdown_ = true;
+    cond_.Broadcast();
+  }
+  if (monitor_thread_.get()) {
+    CHECK_OK(ThreadJoiner(monitor_thread_.get()).Join());
+    monitor_thread_.reset();
+    // Wait for all the running and queued tasks before shutting down. Otherwise,
+    // Shutdown() can remove a queued task silently. We count on eventually running the
+    // queued tasks to decrement their "running" count, which is incremented at the time
+    // they are enqueued.
+    thread_pool_->Wait();
+    thread_pool_->Shutdown();
+  }
+}
+
+void MaintenanceManager::RegisterOp(MaintenanceOp* op) {
+  CHECK(op);
+  std::lock_guard<Mutex> guard(lock_);
+  CHECK(!op->manager_) << "Tried to register " << op->name()
+          << ", but it was already registered.";
+  pair<OpMapTy::iterator, bool> val
+    (ops_.insert(OpMapTy::value_type(op, MaintenanceOpStats())));
+  CHECK(val.second)
+      << "Tried to register " << op->name()
+      << ", but it already exists in ops_.";
+  op->manager_ = shared_from_this();
+  op->cond_.reset(new ConditionVariable(&lock_));
+  VLOG_AND_TRACE("maintenance", 1) << LogPrefix() << "Registered " << op->name();
+}
+
+void MaintenanceManager::UnregisterOp(MaintenanceOp* op) {
+  {
+    std::lock_guard<Mutex> guard(lock_);
+    CHECK(op->manager_.get() == this) << "Tried to unregister " << op->name()
+          << ", but it is not currently registered with this maintenance manager.";
+    auto iter = ops_.find(op);
+    CHECK(iter != ops_.end()) << "Tried to unregister " << op->name()
+        << ", but it was never registered";
+    // While the op is running, wait for it to be finished.
+    if (iter->first->running_ > 0) {
+      VLOG_AND_TRACE("maintenance", 1) << LogPrefix() << "Waiting for op " << op->name()
+                                       << " to finish so we can unregister it.";
+    }
+    op->CancelAndDisable();
+    while (iter->first->running_ > 0) {
+      op->cond_->Wait();
+      iter = ops_.find(op);
+      CHECK(iter != ops_.end()) << "Tried to unregister " << op->name()
+          << ", but another thread unregistered it while we were "
+          << "waiting for it to complete";
+    }
+    ops_.erase(iter);
+  }
+  LOG_WITH_PREFIX(INFO) << "Unregistered op " << op->name();
+  op->cond_.reset();
+  // Remove the op's shared_ptr reference to us.  This might 'delete this'.
+  op->manager_.reset();
+}
+
+bool MaintenanceManager::disabled_for_tests() const {
+  return !ANNOTATE_UNPROTECTED_READ(FLAGS_enable_maintenance_manager);
+}
+
+void MaintenanceManager::RunSchedulerThread() {
+  if (!FLAGS_enable_maintenance_manager) {
+    LOG(INFO) << "Maintenance manager is disabled. Stopping thread.";
+    return;
+  }
+
+  MonoDelta polling_interval = MonoDelta::FromMilliseconds(polling_interval_ms_);
+
+  std::unique_lock<Mutex> guard(lock_);
+
+  // Set to true if the scheduler runs and finds that there is no work to do.
+  bool prev_iter_found_no_work = false;
+
+  while (true) {
+    // We'll keep sleeping if:
+    //    1) there are no free threads available to perform a maintenance op.
+    // or 2) we just tried to schedule an op but found nothing to run.
+    // However, if it's time to shut down, we want to do so immediately.
+    while ((running_ops_ >= num_threads_ || prev_iter_found_no_work || disabled_for_tests()) &&
+           !shutdown_) {
+      cond_.WaitFor(polling_interval);
+      prev_iter_found_no_work = false;
+    }
+    if (shutdown_) {
+      VLOG_AND_TRACE("maintenance", 1) << LogPrefix() << "Shutting down maintenance manager.";
+      return;
+    }
+
+    // Find the best op.
+    pair<MaintenanceOp*, string> op_and_note = FindBestOp();
+    auto* op = op_and_note.first;
+    const auto& note = op_and_note.second;
+
+    // If we found no work to do, then we should sleep before trying again to schedule.
+    // Otherwise, we can go right into trying to find the next op.
+    prev_iter_found_no_work = (op == nullptr);
+    if (!op) {
+      VLOG_AND_TRACE("maintenance", 2) << LogPrefix()
+                                       << "No maintenance operations look worth doing.";
+      continue;
+    }
+
+    // Prepare the maintenance operation.
+    op->running_++;
+    running_ops_++;
+    guard.unlock();
+    bool ready = op->Prepare();
+    guard.lock();
+    if (!ready) {
+      LOG_WITH_PREFIX(INFO) << "Prepare failed for " << op->name()
+                            << ".  Re-running scheduler.";
+      op->running_--;
+      running_ops_--;
+      op->cond_->Signal();
+      continue;
+    }
+
+    LOG_AND_TRACE("maintenance", INFO) << LogPrefix() << "Scheduling "
+                                       << op->name() << ": " << note;
+    // Run the maintenance operation.
+    Status s = thread_pool_->SubmitFunc(boost::bind(
+        &MaintenanceManager::LaunchOp, this, op));
+    CHECK(s.ok());
+  }
+}
+
+// Finding the best operation goes through four filters:
+// - If there's an Op that we can run quickly that frees log retention, we run it.
+// - If we've hit the overall process memory limit (note: this includes memory that the Ops cannot
+//   free), we run the Op with the highest RAM usage.
+// - If there are Ops that are retaining logs past our target replay size, we run the one that has
+//   the highest retention (and if many qualify, then we run the one that also frees up the
+//   most RAM).
+// - Finally, if there's nothing else that we really need to do, we run the Op that will improve
+//   performance the most.
+//
+// The reason it's done this way is that we want to prioritize limiting the amount of resources we
+// hold on to. Low IO Ops go first since we can quickly run them, then we can look at memory usage.
+// Reversing those can starve the low IO Ops when the system is under intense memory pressure.
+//
+// In the third priority we're at a point where nothing's urgent and there's nothing we can run
+// quickly.
+// TODO We currently optimize for freeing log retention but we could consider having some sort of
+// sliding priority between log retention and RAM usage. For example, is an Op that frees
+// 128MB of log retention and 12MB of RAM always better than an op that frees 12MB of log retention
+// and 128MB of RAM? Maybe a more holistic approach would be better.
+pair<MaintenanceOp*, string> MaintenanceManager::FindBestOp() {
+  TRACE_EVENT0("maintenance", "MaintenanceManager::FindBestOp");
+
+  size_t free_threads = num_threads_ - running_ops_;
+  if (free_threads == 0) {
+    return {nullptr, "no free threads"};
+  }
+
+  int64_t low_io_most_logs_retained_bytes = 0;
+  MaintenanceOp* low_io_most_logs_retained_bytes_op = nullptr;
+
+  uint64_t most_mem_anchored = 0;
+  MaintenanceOp* most_mem_anchored_op = nullptr;
+
+  int64_t most_logs_retained_bytes = 0;
+  int64_t most_logs_retained_bytes_ram_anchored = 0;
+  MaintenanceOp* most_logs_retained_bytes_op = nullptr;
+
+  int64_t most_data_retained_bytes = 0;
+  MaintenanceOp* most_data_retained_bytes_op = nullptr;
+
+  double best_perf_improvement = 0;
+  MaintenanceOp* best_perf_improvement_op = nullptr;
+  for (OpMapTy::value_type &val : ops_) {
+    MaintenanceOp* op(val.first);
+    MaintenanceOpStats& stats(val.second);
+    VLOG_WITH_PREFIX(3) << "Considering MM op " << op->name();
+    // Update op stats.
+    stats.Clear();
+    op->UpdateStats(&stats);
+    if (op->cancelled() || !stats.valid() || !stats.runnable()) {
+      continue;
+    }
+    if (stats.logs_retained_bytes() > low_io_most_logs_retained_bytes &&
+        op->io_usage() == MaintenanceOp::LOW_IO_USAGE) {
+      low_io_most_logs_retained_bytes_op = op;
+      low_io_most_logs_retained_bytes = stats.logs_retained_bytes();
+      VLOG_AND_TRACE("maintenance", 2) << LogPrefix() << "Op " << op->name() << " can free "
+                                       << stats.logs_retained_bytes() << " bytes of logs";
+    }
+
+    if (stats.ram_anchored() > most_mem_anchored) {
+      most_mem_anchored_op = op;
+      most_mem_anchored = stats.ram_anchored();
+    }
+    // We prioritize ops that can free more logs, but when it's the same we pick the one that
+    // also frees up the most memory.
+    if (stats.logs_retained_bytes() > 0 &&
+        (stats.logs_retained_bytes() > most_logs_retained_bytes ||
+            (stats.logs_retained_bytes() == most_logs_retained_bytes &&
+                stats.ram_anchored() > most_logs_retained_bytes_ram_anchored))) {
+      most_logs_retained_bytes_op = op;
+      most_logs_retained_bytes = stats.logs_retained_bytes();
+      most_logs_retained_bytes_ram_anchored = stats.ram_anchored();
+    }
+
+    if (stats.data_retained_bytes() > most_data_retained_bytes) {
+      most_data_retained_bytes_op = op;
+      most_data_retained_bytes = stats.data_retained_bytes();
+      VLOG_AND_TRACE("maintenance", 2) << LogPrefix() << "Op " << op->name() << " can free "
+                                       << stats.data_retained_bytes() << " bytes of data";
+    }
+
+    if ((!best_perf_improvement_op) ||
+        (stats.perf_improvement() > best_perf_improvement)) {
+      best_perf_improvement_op = op;
+      best_perf_improvement = stats.perf_improvement();
+    }
+  }
+
+  // Look at ops that we can run quickly that free up log retention.
+  if (low_io_most_logs_retained_bytes_op) {
+    if (low_io_most_logs_retained_bytes > 0) {
+      string notes = Substitute("free $0 bytes of WAL", low_io_most_logs_retained_bytes);
+      return {low_io_most_logs_retained_bytes_op, std::move(notes)};
+    }
+  }
+
+  // Look at free memory. If it is dangerously low, we must select something
+  // that frees memory-- the op with the most anchored memory.
+  double capacity_pct;
+  if (memory_pressure_func_(&capacity_pct)) {
+    if (!most_mem_anchored_op) {
+      std::string msg = StringPrintf("System under memory pressure "
+          "(%.2f%% of limit used). However, there are no ops currently "
+          "runnable which would free memory.", capacity_pct);
+      LOG_WITH_PREFIX(INFO) << msg;
+      return {nullptr, msg};
+    }
+    string note = StringPrintf("under memory pressure (%.2f%% used, "
+                               "can flush %" PRIu64 " bytes)",
+                               capacity_pct, most_mem_anchored);
+    return {most_mem_anchored_op, std::move(note)};
+  }
+
+  if (most_logs_retained_bytes_op &&
+      most_logs_retained_bytes / 1024 / 1024 >= FLAGS_log_target_replay_size_mb) {
+    string note = Substitute("$0 bytes log retention", most_logs_retained_bytes);
+    return {most_logs_retained_bytes_op, std::move(note)};
+  }
+
+  // Look at ops that we can run quickly that free up data on disk.
+  if (most_data_retained_bytes_op &&
+      most_data_retained_bytes > FLAGS_data_gc_min_size_mb * 1024 * 1024) {
+    if (!best_perf_improvement_op || best_perf_improvement <= 0 ||
+        rand_.NextDoubleFraction() <= FLAGS_data_gc_prioritization_prob) {
+      string note = Substitute("$0 bytes on disk", most_data_retained_bytes);
+      return {most_data_retained_bytes_op, std::move(note)};
+    }
+    VLOG(1) << "Skipping data GC due to prioritizing perf improvement";
+  }
+
+  if (best_perf_improvement_op && best_perf_improvement > 0) {
+    string note = StringPrintf("perf score=%.6f", best_perf_improvement);
+    return {best_perf_improvement_op, std::move(note)};
+  }
+  return {nullptr, "no ops with positive improvement"};
+}
+
+void MaintenanceManager::LaunchOp(MaintenanceOp* op) {
+  int64_t thread_id = Thread::CurrentThreadId();
+  OpInstance op_instance;
+  op_instance.thread_id = thread_id;
+  op_instance.name = op->name();
+  op_instance.start_mono_time = MonoTime::Now();
+  op->RunningGauge()->Increment();
+  {
+    std::lock_guard<Mutex> lock(running_instances_lock_);
+    InsertOrDie(&running_instances_, thread_id, &op_instance);
+  }
+
+  SCOPED_CLEANUP({
+    op->RunningGauge()->Decrement();
+
+    std::lock_guard<Mutex> l(lock_);
+    {
+      std::lock_guard<Mutex> lock(running_instances_lock_);
+      running_instances_.erase(thread_id);
+    }
+    op_instance.duration = MonoTime::Now() - op_instance.start_mono_time;
+    completed_ops_[completed_ops_count_ % completed_ops_.size()] = op_instance;
+    completed_ops_count_++;
+
+    op->DurationHistogram()->Increment(op_instance.duration.ToMilliseconds());
+
+    running_ops_--;
+    op->running_--;
+    op->cond_->Signal();
+    cond_.Signal(); // wake up scheduler
+  });
+
+  scoped_refptr<Trace> trace(new Trace);
+  Stopwatch sw;
+  sw.start();
+  {
+    ADOPT_TRACE(trace.get());
+    TRACE_EVENT1("maintenance", "MaintenanceManager::LaunchOp",
+                 "name", op->name());
+    op->Perform();
+    sw.stop();
+  }
+  LOG_WITH_PREFIX(INFO) << op->name() << " complete. "
+                        << "Timing: " << sw.elapsed().ToString()
+                        << " Metrics: " << trace->MetricsAsJSON();
+}
+
+void MaintenanceManager::GetMaintenanceManagerStatusDump(MaintenanceManagerStatusPB* out_pb) {
+  DCHECK(out_pb != nullptr);
+  std::lock_guard<Mutex> guard(lock_);
+  pair<MaintenanceOp*, string> best_op_and_why = FindBestOp();
+  auto* best_op = best_op_and_why.first;
+
+  for (MaintenanceManager::OpMapTy::value_type& val : ops_) {
+    MaintenanceManagerStatusPB_MaintenanceOpPB* op_pb = out_pb->add_registered_operations();
+    MaintenanceOp* op(val.first);
+    MaintenanceOpStats& stat(val.second);
+    op_pb->set_name(op->name());
+    op_pb->set_running(op->running());
+    if (stat.valid()) {
+      op_pb->set_runnable(stat.runnable());
+      op_pb->set_ram_anchored_bytes(stat.ram_anchored());
+      op_pb->set_logs_retained_bytes(stat.logs_retained_bytes());
+      op_pb->set_perf_improvement(stat.perf_improvement());
+    } else {
+      op_pb->set_runnable(false);
+      op_pb->set_ram_anchored_bytes(0);
+      op_pb->set_logs_retained_bytes(0);
+      op_pb->set_perf_improvement(0.0);
+    }
+
+    if (best_op == op) {
+      out_pb->mutable_best_op()->CopyFrom(*op_pb);
+    }
+  }
+
+  {
+    std::lock_guard<Mutex> lock(running_instances_lock_);
+    for (const auto& running_instance : running_instances_) {
+      *out_pb->add_running_operations() = running_instance.second->DumpToPB();
+    }
+  }
+
+  for (int n = 1; n <= completed_ops_.size(); n++) {
+    int i = completed_ops_count_ - n;
+    if (i < 0) break;
+    const auto& completed_op = completed_ops_[i % completed_ops_.size()];
+
+    if (!completed_op.name.empty()) {
+      *out_pb->add_completed_operations() = completed_op.DumpToPB();
+    }
+  }
+}
+
+std::string MaintenanceManager::LogPrefix() const {
+  return Substitute("P $0: ", server_uuid_);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/maintenance_manager.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/maintenance_manager.h b/be/src/kudu/util/maintenance_manager.h
new file mode 100644
index 0000000..7d20c8a
--- /dev/null
+++ b/be/src/kudu/util/maintenance_manager.h
@@ -0,0 +1,361 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/random.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+template<class T>
+class AtomicGauge;
+class Histogram;
+class MaintenanceManager;
+class MaintenanceManagerStatusPB;
+class MaintenanceManagerStatusPB_OpInstancePB;
+class Thread;
+class ThreadPool;
+
+class MaintenanceOpStats {
+ public:
+  MaintenanceOpStats();
+
+  // Zero all stats. They are invalid until the first setter is called.
+  void Clear();
+
+  bool runnable() const {
+    DCHECK(valid_);
+    return runnable_;
+  }
+
+  void set_runnable(bool runnable) {
+    UpdateLastModified();
+    runnable_ = runnable;
+  }
+
+  uint64_t ram_anchored() const {
+    DCHECK(valid_);
+    return ram_anchored_;
+  }
+
+  void set_ram_anchored(uint64_t ram_anchored) {
+    UpdateLastModified();
+    ram_anchored_ = ram_anchored;
+  }
+
+  int64_t logs_retained_bytes() const {
+    DCHECK(valid_);
+    return logs_retained_bytes_;
+  }
+
+  void set_logs_retained_bytes(int64_t logs_retained_bytes) {
+    UpdateLastModified();
+    logs_retained_bytes_ = logs_retained_bytes;
+  }
+
+  int64_t data_retained_bytes() const {
+    DCHECK(valid_);
+    return data_retained_bytes_;
+  }
+
+  void set_data_retained_bytes(int64_t data_retained_bytes) {
+    UpdateLastModified();
+    data_retained_bytes_ = data_retained_bytes;
+  }
+
+  double perf_improvement() const {
+    DCHECK(valid_);
+    return perf_improvement_;
+  }
+
+  void set_perf_improvement(double perf_improvement) {
+    UpdateLastModified();
+    perf_improvement_ = perf_improvement;
+  }
+
+  const MonoTime& last_modified() const {
+    DCHECK(valid_);
+    return last_modified_;
+  }
+
+  bool valid() const {
+    return valid_;
+  }
+
+ private:
+  void UpdateLastModified() {
+    valid_ = true;
+    last_modified_ = MonoTime::Now();
+  }
+
+  // Important: Update Clear() when adding fields to this class.
+
+  // True if these stats are valid.
+  bool valid_;
+
+  // True if this op can be run now.
+  bool runnable_;
+
+  // The approximate amount of memory that not doing this operation keeps
+  // around.  This number is used to decide when to start freeing memory, so it
+  // should be fairly accurate.  May be 0.
+  uint64_t ram_anchored_;
+
+  // Approximate amount of disk space in WAL files that would be freed if this
+  // operation ran. May be 0.
+  int64_t logs_retained_bytes_;
+
+  // Approximate amount of disk space in data blocks that would be freed if
+  // this operation ran. May be 0.
+  int64_t data_retained_bytes_;
+
+  // The estimated performance improvement-- how good it is to do this on some
+  // absolute scale (yet TBD).
+  double perf_improvement_;
+
+  // The last time that the stats were modified.
+  MonoTime last_modified_;
+};
+
+// Represents an instance of a maintenance operation.
+struct OpInstance {
+  // Id of thread the instance ran on.
+  int64_t thread_id;
+  // Name of operation.
+  std::string name;
+  // Time the operation took to run. Value is unitialized if instance is still running.
+  MonoDelta duration;
+  MonoTime start_mono_time;
+
+  MaintenanceManagerStatusPB_OpInstancePB DumpToPB() const;
+};
+
+// MaintenanceOp objects represent background operations that the
+// MaintenanceManager can schedule.  Once a MaintenanceOp is registered, the
+// manager will periodically poll it for statistics.  The registrant is
+// responsible for managing the memory associated with the MaintenanceOp object.
+// Op objects should be unregistered before being de-allocated.
+class MaintenanceOp {
+ public:
+  friend class MaintenanceManager;
+
+  // General indicator of how much IO the Op will use.
+  enum IOUsage {
+    LOW_IO_USAGE, // Low impact operations like removing a file, updating metadata.
+    HIGH_IO_USAGE // Everything else.
+  };
+
+  explicit MaintenanceOp(std::string name, IOUsage io_usage);
+  virtual ~MaintenanceOp();
+
+  // Unregister this op, if it is currently registered.
+  void Unregister();
+
+  // Update the op statistics.  This will be called every scheduling period
+  // (about a few times a second), so it should not be too expensive.  It's
+  // possible for the returned statistics to be invalid; the caller should
+  // call MaintenanceOpStats::valid() before using them.  This will be run
+  // under the MaintenanceManager lock.
+  virtual void UpdateStats(MaintenanceOpStats* stats) = 0;
+
+  // Prepare to perform the operation.  This will be run without holding the
+  // maintenance manager lock.  It should be short, since it is run from the
+  // context of the maintenance op scheduler thread rather than a worker thread.
+  // If this returns false, we will abort the operation.
+  virtual bool Prepare() = 0;
+
+  // Perform the operation.  This will be run without holding the maintenance
+  // manager lock, and may take a long time.
+  virtual void Perform() = 0;
+
+  // Returns the histogram for this op that tracks duration. Cannot be NULL.
+  virtual scoped_refptr<Histogram> DurationHistogram() const = 0;
+
+  // Returns the gauge for this op that tracks when this op is running. Cannot be NULL.
+  virtual scoped_refptr<AtomicGauge<uint32_t> > RunningGauge() const = 0;
+
+  uint32_t running() { return running_; }
+
+  std::string name() const { return name_; }
+
+  IOUsage io_usage() const { return io_usage_; }
+
+  // Return true if the operation has been cancelled due to Unregister() pending.
+  bool cancelled() const {
+    return cancel_.Load();
+  }
+
+  // Cancel this operation, which prevents new instances of it from being scheduled
+  // regardless of whether the statistics indicate it is runnable. Instances may also
+  // optionally poll 'cancelled()' on a periodic basis to know if they should abort a
+  // lengthy operation in the middle of Perform().
+  void CancelAndDisable() {
+    cancel_.Store(true);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(MaintenanceOp);
+
+  // The name of the operation.  Op names must be unique.
+  const std::string name_;
+
+  // The number of times that this op is currently running.
+  uint32_t running_;
+
+  // Set when we are trying to unregister the maintenance operation.
+  // Ongoing operations could read this boolean and cancel themselves.
+  // New operations will not be scheduled when this boolean is set.
+  AtomicBool cancel_;
+
+  // Condition variable which the UnregisterOp function can wait on.
+  //
+  // Note: 'cond_' is used with the MaintenanceManager's mutex. As such,
+  // it only exists when the op is registered.
+  gscoped_ptr<ConditionVariable> cond_;
+
+  // The MaintenanceManager with which this op is registered, or null
+  // if it is not registered.
+  std::shared_ptr<MaintenanceManager> manager_;
+
+  IOUsage io_usage_;
+};
+
+struct MaintenanceOpComparator {
+  bool operator() (const MaintenanceOp* lhs,
+                   const MaintenanceOp* rhs) const {
+    return lhs->name().compare(rhs->name()) < 0;
+  }
+};
+
+// The MaintenanceManager manages the scheduling of background operations such
+// as flushes or compactions.  It runs these operations in the background, in a
+// thread pool.  It uses information provided in MaintenanceOpStats objects to
+// decide which operations, if any, to run.
+class MaintenanceManager : public std::enable_shared_from_this<MaintenanceManager> {
+ public:
+  struct Options {
+    int32_t num_threads;
+    int32_t polling_interval_ms;
+    uint32_t history_size;
+  };
+
+  MaintenanceManager(const Options& options, std::string server_uuid);
+  ~MaintenanceManager();
+
+  // Start running the maintenance manager.
+  // Must be called at most once.
+  Status Start();
+  void Shutdown();
+
+  // Register an op with the manager.
+  void RegisterOp(MaintenanceOp* op);
+
+  // Unregister an op with the manager.
+  // If the Op is currently running, it will not be interrupted.  However, this
+  // function will block until the Op is finished.
+  void UnregisterOp(MaintenanceOp* op);
+
+  void GetMaintenanceManagerStatusDump(MaintenanceManagerStatusPB* out_pb);
+
+  void set_memory_pressure_func_for_tests(std::function<bool(double*)> f) {
+    std::lock_guard<Mutex> guard(lock_);
+    memory_pressure_func_ = std::move(f);
+  }
+
+  static const Options kDefaultOptions;
+
+ private:
+  FRIEND_TEST(MaintenanceManagerTest, TestLogRetentionPrioritization);
+  typedef std::map<MaintenanceOp*, MaintenanceOpStats,
+          MaintenanceOpComparator> OpMapTy;
+
+  // Return true if tests have currently disabled the maintenance
+  // manager by way of changing the gflags at runtime.
+  bool disabled_for_tests() const;
+
+  void RunSchedulerThread();
+
+  // Find the best op, or null if there is nothing we want to run.
+  //
+  // Returns the op, as well as a string explanation of why that op was chosen,
+  // suitable for logging.
+  std::pair<MaintenanceOp*, std::string> FindBestOp();
+
+  void LaunchOp(MaintenanceOp* op);
+
+  std::string LogPrefix() const;
+
+  const std::string server_uuid_;
+  const int32_t num_threads_;
+  OpMapTy ops_; // registered operations
+  Mutex lock_;
+  scoped_refptr<kudu::Thread> monitor_thread_;
+  gscoped_ptr<ThreadPool> thread_pool_;
+  ConditionVariable cond_;
+  bool shutdown_;
+  int32_t polling_interval_ms_;
+  uint64_t running_ops_;
+  // Vector used as a circular buffer for recently completed ops. Elements need to be added at
+  // the completed_ops_count_ % the vector's size and then the count needs to be incremented.
+  std::vector<OpInstance> completed_ops_;
+  int64_t completed_ops_count_;
+  Random rand_;
+
+  // Function which should return true if the server is under global memory pressure.
+  // This is indirected for testing purposes.
+  std::function<bool(double*)> memory_pressure_func_;
+
+  // Running instances lock.
+  //
+  // This is separate of lock_ so that worker threads don't need to take the
+  // global MM lock when beginning operations. When taking both
+  // running_instances_lock_ and lock_, lock_ must be acquired first.
+  Mutex running_instances_lock_;
+
+  // Maps thread ids to instances of an op that they're running. Instances should be added
+  // right before MaintenanceOp::Perform() is called, and should be removed right after
+  // MaintenanceOp::Perform() completes. Any thread that adds an instance to this map
+  // owns that instance, and the instance should exist until the same thread removes it.
+  //
+  // Protected by running_instances_lock_;
+  std::unordered_map<int64_t, OpInstance*> running_instances_;
+
+  DISALLOW_COPY_AND_ASSIGN(MaintenanceManager);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/maintenance_manager.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/maintenance_manager.proto b/be/src/kudu/util/maintenance_manager.proto
new file mode 100644
index 0000000..b6b1203
--- /dev/null
+++ b/be/src/kudu/util/maintenance_manager.proto
@@ -0,0 +1,54 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+option java_package = "org.apache.kudu";
+
+// Used to present the maintenance manager's internal state.
+message MaintenanceManagerStatusPB {
+  message MaintenanceOpPB {
+    required string name = 1;
+    // Number of times this operation is currently running.
+    required uint32 running = 2;
+    required bool runnable = 3;
+    required uint64 ram_anchored_bytes = 4;
+    required int64 logs_retained_bytes = 5;
+    required double perf_improvement = 6;
+  }
+
+  message OpInstancePB {
+    required int64 thread_id = 1;
+    required string name = 2;
+    // How long the op took to run. Only present if the instance completed.
+    optional int32 duration_millis = 3;
+    // Number of milliseconds since this operation started.
+    required int32 millis_since_start = 4;
+  }
+
+  // The next operation that would run.
+  optional MaintenanceOpPB best_op = 1;
+
+  // List of all the operations.
+  repeated MaintenanceOpPB registered_operations = 2;
+
+  // This list isn't in order of anything. Can contain the same operation multiple times.
+  repeated OpInstancePB running_operations = 3;
+
+  // This list isn't in order of anything. Can contain the same operation multiple times.
+  repeated OpInstancePB completed_operations = 4;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/make_shared.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/make_shared.h b/be/src/kudu/util/make_shared.h
new file mode 100644
index 0000000..649cae7
--- /dev/null
+++ b/be/src/kudu/util/make_shared.h
@@ -0,0 +1,64 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <utility>
+
+// It isn't possible to use 'std::make_shared' on a class with private or protected
+// constructors. Using friends as a workaround worked in some earlier libc++/libstdcxx
+// versions, but in the latest versions there are some static_asserts that seem to defeat
+// this trickery. So, instead, we rely on the "curiously recurring template pattern" (CRTP)
+// to inject a static 'make_shared' function inside the class.
+//
+// See https://stackoverflow.com/questions/8147027/how-do-i-call-stdmake-shared-on-a-class-with-only-protected-or-private-const
+// for some details.
+//
+// Usage:
+//
+//  class MyClass : public enable_make_shared<MyClass> {
+//   public:
+//     ...
+//
+//   protected:
+//    // The constructor must be protected rather than private.
+//    MyClass(Foo arg1, Bar arg2) {
+//    }
+//
+//  }
+//
+//    shared_ptr<MyClass> foo = MyClass::make_shared(arg1, arg2);
+template<class T>
+class enable_make_shared { // NOLINT
+ public:
+
+  // Define a static make_shared member which constructs the public subclass
+  // and casts it back to the desired class.
+  template<typename... Arg>
+  static std::shared_ptr<T> make_shared(Arg&&... args) {
+    // Define a struct subclass with a public constructor which will be accessible
+    // from make_shared.
+    struct make_shared_enabler : public T { // NOLINT
+      explicit make_shared_enabler(Arg&&... args) : T(std::forward<Arg>(args)...) {
+      }
+    };
+
+    return ::std::make_shared<make_shared_enabler>(
+        ::std::forward<Arg>(args)...);
+  }
+};

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/malloc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/malloc.cc b/be/src/kudu/util/malloc.cc
new file mode 100644
index 0000000..3fec2db
--- /dev/null
+++ b/be/src/kudu/util/malloc.cc
@@ -0,0 +1,35 @@
+// 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 "kudu/util/malloc.h"
+
+#if defined(__linux__)
+#include <malloc.h>
+#else
+#include <malloc/malloc.h>
+#endif // defined(__linux__)
+
+namespace kudu {
+
+int64_t kudu_malloc_usable_size(const void* obj) {
+#if defined(__linux__)
+  return malloc_usable_size(const_cast<void*>(obj));
+#else
+  return malloc_size(obj);
+#endif // defined(__linux__)
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/malloc.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/malloc.h b/be/src/kudu/util/malloc.h
new file mode 100644
index 0000000..e8a27c5
--- /dev/null
+++ b/be/src/kudu/util/malloc.h
@@ -0,0 +1,32 @@
+// 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.
+#ifndef KUDU_UTIL_MALLOC_H
+#define KUDU_UTIL_MALLOC_H
+
+#include <stdint.h>
+
+namespace kudu {
+
+// Simple wrapper for malloc_usable_size().
+//
+// Really just centralizes the const_cast, as this function is often called
+// on const pointers (i.e. "this" in a const method).
+int64_t kudu_malloc_usable_size(const void* obj);
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_MALLOC_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/map-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/map-util-test.cc b/be/src/kudu/util/map-util-test.cc
new file mode 100644
index 0000000..3aa9448
--- /dev/null
+++ b/be/src/kudu/util/map-util-test.cc
@@ -0,0 +1,116 @@
+// 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 unit test belongs in gutil, but it depends on test_main which is
+// part of util.
+#include "kudu/gutil/map-util.h"
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <gtest/gtest.h>
+
+using std::map;
+using std::string;
+using std::shared_ptr;
+using std::unique_ptr;
+
+namespace kudu {
+
+TEST(FloorTest, TestMapUtil) {
+  map<int, int> my_map;
+
+  ASSERT_EQ(nullptr, FindFloorOrNull(my_map, 5));
+
+  my_map[5] = 5;
+  ASSERT_EQ(5, *FindFloorOrNull(my_map, 6));
+  ASSERT_EQ(5, *FindFloorOrNull(my_map, 5));
+  ASSERT_EQ(nullptr, FindFloorOrNull(my_map, 4));
+
+  my_map[1] = 1;
+  ASSERT_EQ(5, *FindFloorOrNull(my_map, 6));
+  ASSERT_EQ(5, *FindFloorOrNull(my_map, 5));
+  ASSERT_EQ(1, *FindFloorOrNull(my_map, 4));
+  ASSERT_EQ(1, *FindFloorOrNull(my_map, 1));
+  ASSERT_EQ(nullptr, FindFloorOrNull(my_map, 0));
+}
+
+TEST(ComputeIfAbsentTest, TestComputeIfAbsent) {
+  map<string, string> my_map;
+  auto result = ComputeIfAbsent(&my_map, "key", []{ return "hello_world"; });
+  ASSERT_EQ(*result, "hello_world");
+  auto result2 = ComputeIfAbsent(&my_map, "key", [] { return "hello_world2"; });
+  ASSERT_EQ(*result2, "hello_world");
+}
+
+TEST(ComputeIfAbsentTest, TestComputeIfAbsentAndReturnAbsense) {
+  map<string, string> my_map;
+  auto result = ComputeIfAbsentReturnAbsense(&my_map, "key", []{ return "hello_world"; });
+  ASSERT_TRUE(result.second);
+  ASSERT_EQ(*result.first, "hello_world");
+  auto result2 = ComputeIfAbsentReturnAbsense(&my_map, "key", [] { return "hello_world2"; });
+  ASSERT_FALSE(result2.second);
+  ASSERT_EQ(*result2.first, "hello_world");
+}
+
+TEST(FindPointeeOrNullTest, TestFindPointeeOrNull) {
+  map<string, unique_ptr<string>> my_map;
+  auto iter = my_map.emplace("key", unique_ptr<string>(new string("hello_world")));
+  ASSERT_TRUE(iter.second);
+  string* value = FindPointeeOrNull(my_map, "key");
+  ASSERT_TRUE(value != nullptr);
+  ASSERT_EQ(*value, "hello_world");
+  my_map.erase(iter.first);
+  value = FindPointeeOrNull(my_map, "key");
+  ASSERT_TRUE(value == nullptr);
+}
+
+TEST(EraseKeyReturnValuePtrTest, TestRawAndSmartSmartPointers) {
+  map<string, unique_ptr<string>> my_map;
+  unique_ptr<string> value = EraseKeyReturnValuePtr(&my_map, "key");
+  ASSERT_TRUE(value.get() == nullptr);
+  my_map.emplace("key", unique_ptr<string>(new string("hello_world")));
+  value = EraseKeyReturnValuePtr(&my_map, "key");
+  ASSERT_EQ(*value, "hello_world");
+  value.reset();
+  value = EraseKeyReturnValuePtr(&my_map, "key");
+  ASSERT_TRUE(value.get() == nullptr);
+  map<string, shared_ptr<string>> my_map2;
+  shared_ptr<string> value2 = EraseKeyReturnValuePtr(&my_map2, "key");
+  ASSERT_TRUE(value2.get() == nullptr);
+  my_map2.emplace("key", std::make_shared<string>("hello_world"));
+  value2 = EraseKeyReturnValuePtr(&my_map2, "key");
+  ASSERT_EQ(*value2, "hello_world");
+  map<string, string*> my_map_raw;
+  my_map_raw.emplace("key", new string("hello_world"));
+  value.reset(EraseKeyReturnValuePtr(&my_map_raw, "key"));
+  ASSERT_EQ(*value, "hello_world");
+}
+
+TEST(EmplaceTest, TestEmplace) {
+  // Map with move-only value type.
+  map<string, unique_ptr<string>> my_map;
+  unique_ptr<string> val(new string("foo"));
+  ASSERT_TRUE(EmplaceIfNotPresent(&my_map, "k", std::move(val)));
+  ASSERT_TRUE(ContainsKey(my_map, "k"));
+  ASSERT_FALSE(EmplaceIfNotPresent(&my_map, "k", nullptr))
+      << "Should return false for already-present";
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mem_tracker-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mem_tracker-test.cc b/be/src/kudu/util/mem_tracker-test.cc
new file mode 100644
index 0000000..dbadd09
--- /dev/null
+++ b/be/src/kudu/util/mem_tracker-test.cc
@@ -0,0 +1,285 @@
+// 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 "kudu/util/mem_tracker.h"
+
+#include <atomic>
+#include <functional>
+#include <memory>
+#include <string>
+#include <system_error>
+#include <thread>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+using std::equal_to;
+using std::hash;
+using std::pair;
+using std::shared_ptr;
+using std::string;
+using std::unordered_map;
+using std::vector;
+using strings::Substitute;
+
+TEST(MemTrackerTest, SingleTrackerNoLimit) {
+  shared_ptr<MemTracker> t = MemTracker::CreateTracker(-1, "t");
+  EXPECT_FALSE(t->has_limit());
+  t->Consume(10);
+  EXPECT_EQ(t->consumption(), 10);
+  t->Consume(10);
+  EXPECT_EQ(t->consumption(), 20);
+  t->Release(15);
+  EXPECT_EQ(t->consumption(), 5);
+  EXPECT_FALSE(t->LimitExceeded());
+  t->Release(5);
+  EXPECT_EQ(t->consumption(), 0);
+}
+
+TEST(MemTrackerTest, SingleTrackerWithLimit) {
+  shared_ptr<MemTracker> t = MemTracker::CreateTracker(11, "t");
+  EXPECT_TRUE(t->has_limit());
+  t->Consume(10);
+  EXPECT_EQ(t->consumption(), 10);
+  EXPECT_FALSE(t->LimitExceeded());
+  t->Consume(10);
+  EXPECT_EQ(t->consumption(), 20);
+  EXPECT_TRUE(t->LimitExceeded());
+  t->Release(15);
+  EXPECT_EQ(t->consumption(), 5);
+  EXPECT_FALSE(t->LimitExceeded());
+  t->Release(5);
+}
+
+TEST(MemTrackerTest, TrackerHierarchy) {
+  shared_ptr<MemTracker> p = MemTracker::CreateTracker(100, "p");
+  shared_ptr<MemTracker> c1 = MemTracker::CreateTracker(80, "c1", p);
+  shared_ptr<MemTracker> c2 = MemTracker::CreateTracker(50, "c2", p);
+
+  // everything below limits
+  c1->Consume(60);
+  EXPECT_EQ(c1->consumption(), 60);
+  EXPECT_FALSE(c1->LimitExceeded());
+  EXPECT_FALSE(c1->AnyLimitExceeded());
+  EXPECT_EQ(c2->consumption(), 0);
+  EXPECT_FALSE(c2->LimitExceeded());
+  EXPECT_FALSE(c2->AnyLimitExceeded());
+  EXPECT_EQ(p->consumption(), 60);
+  EXPECT_FALSE(p->LimitExceeded());
+  EXPECT_FALSE(p->AnyLimitExceeded());
+
+  // p goes over limit
+  c2->Consume(50);
+  EXPECT_EQ(c1->consumption(), 60);
+  EXPECT_FALSE(c1->LimitExceeded());
+  EXPECT_TRUE(c1->AnyLimitExceeded());
+  EXPECT_EQ(c2->consumption(), 50);
+  EXPECT_FALSE(c2->LimitExceeded());
+  EXPECT_TRUE(c2->AnyLimitExceeded());
+  EXPECT_EQ(p->consumption(), 110);
+  EXPECT_TRUE(p->LimitExceeded());
+
+  // c2 goes over limit, p drops below limit
+  c1->Release(20);
+  c2->Consume(10);
+  EXPECT_EQ(c1->consumption(), 40);
+  EXPECT_FALSE(c1->LimitExceeded());
+  EXPECT_FALSE(c1->AnyLimitExceeded());
+  EXPECT_EQ(c2->consumption(), 60);
+  EXPECT_TRUE(c2->LimitExceeded());
+  EXPECT_TRUE(c2->AnyLimitExceeded());
+  EXPECT_EQ(p->consumption(), 100);
+  EXPECT_FALSE(p->LimitExceeded());
+  c1->Release(40);
+  c2->Release(60);
+}
+
+class GcFunctionHelper {
+ public:
+  static const int kNumReleaseBytes = 1;
+
+  explicit GcFunctionHelper(MemTracker* tracker) : tracker_(tracker) { }
+
+  void GcFunc() { tracker_->Release(kNumReleaseBytes); }
+
+ private:
+  MemTracker* tracker_;
+};
+
+TEST(MemTrackerTest, STLContainerAllocator) {
+  shared_ptr<MemTracker> t = MemTracker::CreateTracker(-1, "t");
+  MemTrackerAllocator<int> vec_alloc(t);
+  MemTrackerAllocator<pair<const int, int>> map_alloc(t);
+
+  // Simple test: use the allocator in a vector.
+  {
+    vector<int, MemTrackerAllocator<int> > v(vec_alloc);
+    ASSERT_EQ(0, t->consumption());
+    v.reserve(5);
+    ASSERT_EQ(5 * sizeof(int), t->consumption());
+    v.reserve(10);
+    ASSERT_EQ(10 * sizeof(int), t->consumption());
+  }
+  ASSERT_EQ(0, t->consumption());
+
+  // Complex test: use it in an unordered_map, where it must be rebound in
+  // order to allocate the map's buckets.
+  {
+    unordered_map<int, int, hash<int>, equal_to<int>, MemTrackerAllocator<pair<const int, int>>> um(
+        10,
+        hash<int>(),
+        equal_to<int>(),
+        map_alloc);
+
+    // Don't care about the value (it depends on map internals).
+    ASSERT_GT(t->consumption(), 0);
+  }
+  ASSERT_EQ(0, t->consumption());
+}
+
+TEST(MemTrackerTest, FindFunctionsTakeOwnership) {
+  // In each test, ToString() would crash if the MemTracker is destroyed when
+  // 'm' goes out of scope.
+
+  shared_ptr<MemTracker> ref;
+  {
+    shared_ptr<MemTracker> m = MemTracker::CreateTracker(-1, "test");
+    ASSERT_TRUE(MemTracker::FindTracker(m->id(), &ref));
+  }
+  LOG(INFO) << ref->ToString();
+  ref.reset();
+
+  {
+    shared_ptr<MemTracker> m = MemTracker::CreateTracker(-1, "test");
+    ref = MemTracker::FindOrCreateGlobalTracker(-1, m->id());
+  }
+  LOG(INFO) << ref->ToString();
+  ref.reset();
+
+  vector<shared_ptr<MemTracker> > refs;
+  {
+    shared_ptr<MemTracker> m = MemTracker::CreateTracker(-1, "test");
+    MemTracker::ListTrackers(&refs);
+  }
+  for (const shared_ptr<MemTracker>& r : refs) {
+    LOG(INFO) << r->ToString();
+  }
+  refs.clear();
+}
+
+TEST(MemTrackerTest, ScopedTrackedConsumption) {
+  shared_ptr<MemTracker> m = MemTracker::CreateTracker(-1, "test");
+  ASSERT_EQ(0, m->consumption());
+  {
+    ScopedTrackedConsumption consumption(m, 1);
+    ASSERT_EQ(1, m->consumption());
+
+    consumption.Reset(3);
+    ASSERT_EQ(3, m->consumption());
+  }
+  ASSERT_EQ(0, m->consumption());
+}
+
+TEST(MemTrackerTest, CollisionDetection) {
+  shared_ptr<MemTracker> p = MemTracker::CreateTracker(-1, "parent");
+  shared_ptr<MemTracker> c = MemTracker::CreateTracker(-1, "child", p);
+  vector<shared_ptr<MemTracker>> all;
+
+  // Three trackers: root, parent, and child.
+  MemTracker::ListTrackers(&all);
+  ASSERT_EQ(3, all.size());
+
+  // Now only two because the child has been destroyed.
+  c.reset();
+  MemTracker::ListTrackers(&all);
+  ASSERT_EQ(2, all.size());
+  shared_ptr<MemTracker> not_found;
+  ASSERT_FALSE(MemTracker::FindTracker("child", &not_found, p));
+
+  // Let's duplicate the parent. It's not recommended, but it's allowed.
+  shared_ptr<MemTracker> p2 = MemTracker::CreateTracker(-1, "parent");
+  ASSERT_EQ(p->ToString(), p2->ToString());
+
+  // Only when we do a Find() operation do we crash.
+#ifndef NDEBUG
+  const string kDeathMsg = "Multiple memtrackers with same id";
+  EXPECT_DEATH({
+    shared_ptr<MemTracker> found;
+    MemTracker::FindTracker("parent", &found);
+  }, kDeathMsg);
+  EXPECT_DEATH({
+    MemTracker::FindOrCreateGlobalTracker(-1, "parent");
+  }, kDeathMsg);
+#endif
+}
+
+TEST(MemTrackerTest, TestMultiThreadedRegisterAndDestroy) {
+  std::atomic<bool> done(false);
+  vector<std::thread> threads;
+  for (int i = 0; i < 10; i++) {
+    threads.emplace_back([&done]{
+        while (!done.load()) {
+          shared_ptr<MemTracker> t = MemTracker::FindOrCreateGlobalTracker(
+              1000, "foo");
+        }
+      });
+  }
+
+  SleepFor(MonoDelta::FromSeconds(AllowSlowTests() ? 5 : 1));
+  done.store(true);
+  for (auto& t : threads) {
+    t.join();
+  }
+}
+
+TEST(MemTrackerTest, TestMultiThreadedCreateFind) {
+  shared_ptr<MemTracker> p = MemTracker::CreateTracker(-1, "p");
+  shared_ptr<MemTracker> c1 = MemTracker::CreateTracker(-1, "c1", p);
+  std::atomic<bool> done(false);
+  vector<std::thread> threads;
+  threads.emplace_back([&]{
+    while (!done.load()) {
+      shared_ptr<MemTracker> c1_copy;
+      CHECK(MemTracker::FindTracker(c1->id(), &c1_copy, p));
+    }
+  });
+  for (int i = 0; i < 5; i++) {
+    threads.emplace_back([&, i]{
+      while (!done.load()) {
+        shared_ptr<MemTracker> c2 =
+            MemTracker::CreateTracker(-1, Substitute("ci-$0", i), p);
+      }
+    });
+  }
+
+  SleepFor(MonoDelta::FromMilliseconds(500));
+  done.store(true);
+  for (auto& t : threads) {
+    t.join();
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mem_tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mem_tracker.cc b/be/src/kudu/util/mem_tracker.cc
new file mode 100644
index 0000000..f7294d1
--- /dev/null
+++ b/be/src/kudu/util/mem_tracker.cc
@@ -0,0 +1,296 @@
+// 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 "kudu/util/mem_tracker.h"
+
+#include <algorithm>
+#include <cstddef>
+#include <deque>
+#include <limits>
+#include <list>
+#include <memory>
+#include <ostream>
+
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/process_memory.h"
+
+namespace kudu {
+
+// NOTE: this class has been adapted from Impala, so the code style varies
+// somewhat from kudu.
+
+using std::deque;
+using std::list;
+using std::shared_ptr;
+using std::string;
+using std::vector;
+using std::weak_ptr;
+
+using strings::Substitute;
+
+// The ancestor for all trackers. Every tracker is visible from the root down.
+static shared_ptr<MemTracker> root_tracker;
+static GoogleOnceType root_tracker_once = GOOGLE_ONCE_INIT;
+
+void MemTracker::CreateRootTracker() {
+  root_tracker.reset(new MemTracker(-1, "root", shared_ptr<MemTracker>()));
+  root_tracker->Init();
+}
+
+shared_ptr<MemTracker> MemTracker::CreateTracker(int64_t byte_limit,
+                                                 const string& id,
+                                                 shared_ptr<MemTracker> parent) {
+  shared_ptr<MemTracker> real_parent;
+  if (parent) {
+    real_parent = std::move(parent);
+  } else {
+    real_parent = GetRootTracker();
+  }
+  shared_ptr<MemTracker> tracker(new MemTracker(byte_limit, id, real_parent));
+  real_parent->AddChildTracker(tracker);
+  tracker->Init();
+
+  return tracker;
+}
+
+MemTracker::MemTracker(int64_t byte_limit, const string& id, shared_ptr<MemTracker> parent)
+    : limit_(byte_limit),
+      id_(id),
+      descr_(Substitute("memory consumption for $0", id)),
+      parent_(std::move(parent)),
+      consumption_(0) {
+  VLOG(1) << "Creating tracker " << ToString();
+}
+
+MemTracker::~MemTracker() {
+  VLOG(1) << "Destroying tracker " << ToString();
+  if (parent_) {
+    DCHECK(consumption() == 0) << "Memory tracker " << ToString()
+        << " has unreleased consumption " << consumption();
+    parent_->Release(consumption());
+
+    MutexLock l(parent_->child_trackers_lock_);
+    if (child_tracker_it_ != parent_->child_trackers_.end()) {
+      parent_->child_trackers_.erase(child_tracker_it_);
+      child_tracker_it_ = parent_->child_trackers_.end();
+    }
+  }
+}
+
+string MemTracker::ToString() const {
+  string s;
+  const MemTracker* tracker = this;
+  while (tracker) {
+    if (s != "") {
+      s += "->";
+    }
+    s += tracker->id();
+    tracker = tracker->parent_.get();
+  }
+  return s;
+}
+
+bool MemTracker::FindTracker(const string& id,
+                             shared_ptr<MemTracker>* tracker,
+                             const shared_ptr<MemTracker>& parent) {
+  return FindTrackerInternal(id, tracker, parent ? parent : GetRootTracker());
+}
+
+bool MemTracker::FindTrackerInternal(const string& id,
+                                     shared_ptr<MemTracker>* tracker,
+                                     const shared_ptr<MemTracker>& parent) {
+  DCHECK(parent != NULL);
+
+  list<weak_ptr<MemTracker>> children;
+  {
+    MutexLock l(parent->child_trackers_lock_);
+    children = parent->child_trackers_;
+  }
+
+  // Search for the matching child without holding the parent's lock.
+  //
+  // If the lock were held while searching, it'd be possible for 'child' to be
+  // the last live ref to a tracker, which would lead to a recursive
+  // acquisition of the parent lock during the 'child' destructor call.
+  vector<shared_ptr<MemTracker>> found;
+  for (const auto& child_weak : children) {
+    shared_ptr<MemTracker> child = child_weak.lock();
+    if (child && child->id() == id) {
+      found.emplace_back(std::move(child));
+    }
+  }
+  if (PREDICT_TRUE(found.size() == 1)) {
+    *tracker = found[0];
+    return true;
+  } else if (found.size() > 1) {
+    LOG(DFATAL) <<
+        Substitute("Multiple memtrackers with same id ($0) found on parent $1",
+                   id, parent->ToString());
+    *tracker = found[0];
+    return true;
+  }
+  return false;
+}
+
+shared_ptr<MemTracker> MemTracker::FindOrCreateGlobalTracker(
+    int64_t byte_limit,
+    const string& id) {
+  // The calls below comprise a critical section, but we can't use the root
+  // tracker's child_trackers_lock_ to synchronize it as the lock must be
+  // released during FindTrackerInternal(). Since this function creates
+  // globally-visible MemTrackers which are the exception rather than the rule,
+  // it's reasonable to synchronize their creation on a singleton lock.
+  static Mutex find_or_create_lock;
+  MutexLock l(find_or_create_lock);
+
+  shared_ptr<MemTracker> found;
+  if (FindTrackerInternal(id, &found, GetRootTracker())) {
+    return found;
+  }
+  return CreateTracker(byte_limit, id, GetRootTracker());
+}
+
+void MemTracker::ListTrackers(vector<shared_ptr<MemTracker>>* trackers) {
+  trackers->clear();
+  deque<shared_ptr<MemTracker> > to_process;
+  to_process.push_front(GetRootTracker());
+  while (!to_process.empty()) {
+    shared_ptr<MemTracker> t = to_process.back();
+    to_process.pop_back();
+
+    trackers->push_back(t);
+    {
+      MutexLock l(t->child_trackers_lock_);
+      for (const auto& child_weak : t->child_trackers_) {
+        shared_ptr<MemTracker> child = child_weak.lock();
+        if (child) {
+          to_process.emplace_back(std::move(child));
+        }
+      }
+    }
+  }
+}
+
+void MemTracker::Consume(int64_t bytes) {
+  if (bytes < 0) {
+    Release(-bytes);
+    return;
+  }
+
+  if (bytes == 0) {
+    return;
+  }
+  for (auto& tracker : all_trackers_) {
+    tracker->consumption_.IncrementBy(bytes);
+  }
+}
+
+bool MemTracker::TryConsume(int64_t bytes) {
+  if (bytes <= 0) {
+    Release(-bytes);
+    return true;
+  }
+
+  int i = 0;
+  // Walk the tracker tree top-down, consuming memory from each in turn.
+  for (i = all_trackers_.size() - 1; i >= 0; --i) {
+    MemTracker *tracker = all_trackers_[i];
+    if (tracker->limit_ < 0) {
+      tracker->consumption_.IncrementBy(bytes);
+    } else {
+      if (!tracker->consumption_.TryIncrementBy(bytes, tracker->limit_)) {
+        break;
+      }
+    }
+  }
+  // Everyone succeeded, return.
+  if (i == -1) {
+    return true;
+  }
+
+  // Someone failed, roll back the ones that succeeded.
+  // TODO(todd): this doesn't roll it back completely since the max values for
+  // the updated trackers aren't decremented. The max values are only used
+  // for error reporting so this is probably okay. Rolling those back is
+  // pretty hard; we'd need something like 2PC.
+  for (int j = all_trackers_.size() - 1; j > i; --j) {
+    all_trackers_[j]->consumption_.IncrementBy(-bytes);
+  }
+  return false;
+}
+
+void MemTracker::Release(int64_t bytes) {
+  if (bytes < 0) {
+    Consume(-bytes);
+    return;
+  }
+
+  if (bytes == 0) {
+    return;
+  }
+
+  for (auto& tracker : all_trackers_) {
+    tracker->consumption_.IncrementBy(-bytes);
+  }
+  process_memory::MaybeGCAfterRelease(bytes);
+}
+
+bool MemTracker::AnyLimitExceeded() {
+  for (const auto& tracker : limit_trackers_) {
+    if (tracker->LimitExceeded()) {
+      return true;
+    }
+  }
+  return false;
+}
+
+int64_t MemTracker::SpareCapacity() const {
+  int64_t result = std::numeric_limits<int64_t>::max();
+  for (const auto& tracker : limit_trackers_) {
+    int64_t mem_left = tracker->limit() - tracker->consumption();
+    result = std::min(result, mem_left);
+  }
+  return result;
+}
+
+
+void MemTracker::Init() {
+  // populate all_trackers_ and limit_trackers_
+  MemTracker* tracker = this;
+  while (tracker) {
+    all_trackers_.push_back(tracker);
+    if (tracker->has_limit()) limit_trackers_.push_back(tracker);
+    tracker = tracker->parent_.get();
+  }
+  DCHECK_GT(all_trackers_.size(), 0);
+  DCHECK_EQ(all_trackers_[0], this);
+}
+
+void MemTracker::AddChildTracker(const shared_ptr<MemTracker>& tracker) {
+  MutexLock l(child_trackers_lock_);
+  tracker->child_tracker_it_ = child_trackers_.insert(child_trackers_.end(), tracker);
+}
+
+shared_ptr<MemTracker> MemTracker::GetRootTracker() {
+  GoogleOnceInit(&root_tracker_once, &MemTracker::CreateRootTracker);
+  return root_tracker;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mem_tracker.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mem_tracker.h b/be/src/kudu/util/mem_tracker.h
new file mode 100644
index 0000000..14db374
--- /dev/null
+++ b/be/src/kudu/util/mem_tracker.h
@@ -0,0 +1,272 @@
+// 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.
+#ifndef KUDU_UTIL_MEM_TRACKER_H
+#define KUDU_UTIL_MEM_TRACKER_H
+
+#include <cstdint>
+#include <list>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/util/high_water_mark.h"
+#include "kudu/util/mutex.h"
+
+namespace kudu {
+
+// A MemTracker tracks memory consumption; it contains an optional limit and is
+// arranged into a tree structure such that the consumption tracked by a
+// MemTracker is also tracked by its ancestors.
+//
+// The MemTracker hierarchy is rooted in a single static MemTracker.
+// The root MemTracker always exists, and it is the common
+// ancestor to all MemTrackers. All operations that discover MemTrackers begin
+// at the root and work their way down the tree, while operations that deal
+// with adjusting memory consumption begin at a particular MemTracker and work
+// their way up the tree to the root. All MemTrackers (except the root) must
+// have a parent. As a rule, all children belonging to a parent should have
+// unique ids, but this is only enforced during a Find() operation to allow for
+// transient duplicates (e.g. the web UI grabbing very short-lived references
+// to all MemTrackers while rendering a web page). This also means id
+// uniqueness only exists where it's actually needed.
+//
+// When a MemTracker begins its life, it has a strong reference to its parent
+// and the parent has a weak reference to it. Both remain for the lifetime of
+// the MemTracker.
+//
+// Memory consumption is tracked via calls to Consume()/Release(), either to
+// the tracker itself or to one of its descendants.
+//
+// This class is thread-safe.
+class MemTracker : public std::enable_shared_from_this<MemTracker> {
+ public:
+  ~MemTracker();
+
+  // Creates and adds the tracker to the tree so that it can be retrieved with
+  // FindTracker/FindOrCreateTracker.
+  //
+  // byte_limit < 0 means no limit; 'id' is a used as a label to uniquely identify
+  // the MemTracker for the below Find...() calls as well as the web UI.
+  //
+  // Use the two-argument form if there is no parent.
+  static std::shared_ptr<MemTracker> CreateTracker(
+      int64_t byte_limit,
+      const std::string& id,
+      std::shared_ptr<MemTracker> parent = std::shared_ptr<MemTracker>());
+
+  // If a tracker with the specified 'id' and 'parent' exists in the tree, sets
+  // 'tracker' to reference that instance. Returns false if no such tracker
+  // exists.
+  //
+  // Use the two-argument form if there is no parent.
+  //
+  // Note: this function will enforce that 'id' is unique amongst the children
+  // of 'parent'.
+  static bool FindTracker(
+      const std::string& id,
+      std::shared_ptr<MemTracker>* tracker,
+      const std::shared_ptr<MemTracker>& parent = std::shared_ptr<MemTracker>());
+
+  // If a global tracker with the specified 'id' exists in the tree, returns a
+  // shared_ptr to that instance. Otherwise, creates a new MemTracker with the
+  // specified byte_limit and id, parented to the root MemTracker.
+  //
+  // Note: this function will enforce that 'id' is unique amongst the children
+  // of the root MemTracker.
+  static std::shared_ptr<MemTracker> FindOrCreateGlobalTracker(
+      int64_t byte_limit, const std::string& id);
+
+  // Returns a list of all the valid trackers.
+  static void ListTrackers(std::vector<std::shared_ptr<MemTracker> >* trackers);
+
+  // Gets a shared_ptr to the "root" tracker, creating it if necessary.
+  static std::shared_ptr<MemTracker> GetRootTracker();
+
+  // Increases consumption of this tracker and its ancestors by 'bytes'.
+  void Consume(int64_t bytes);
+
+  // Increases consumption of this tracker and its ancestors by 'bytes' only if
+  // they can all consume 'bytes'. If this brings any of them over, none of them
+  // are updated.
+  // Returns true if the try succeeded.
+  bool TryConsume(int64_t bytes);
+
+  // Decreases consumption of this tracker and its ancestors by 'bytes'.
+  //
+  // This will also cause the process to periodically trigger tcmalloc "ReleaseMemory"
+  // to ensure that memory is released to the OS.
+  void Release(int64_t bytes);
+
+  // Returns true if a valid limit of this tracker or one of its ancestors is
+  // exceeded.
+  bool AnyLimitExceeded();
+
+  // If this tracker has a limit, checks the limit and attempts to free up some memory if
+  // the limit is exceeded by calling any added GC functions. Returns true if the limit is
+  // exceeded after calling the GC functions. Returns false if there is no limit.
+  bool LimitExceeded() {
+    return limit_ >= 0 && limit_ < consumption();
+  }
+
+  // Returns the maximum consumption that can be made without exceeding the limit on
+  // this tracker or any of its parents. Returns int64_t::max() if there are no
+  // limits and a negative value if any limit is already exceeded.
+  int64_t SpareCapacity() const;
+
+
+  int64_t limit() const { return limit_; }
+  bool has_limit() const { return limit_ >= 0; }
+  const std::string& id() const { return id_; }
+
+  // Returns the memory consumed in bytes.
+  int64_t consumption() const {
+    return consumption_.current_value();
+  }
+
+  int64_t peak_consumption() const { return consumption_.max_value(); }
+
+  // Retrieve the parent tracker, or NULL If one is not set.
+  std::shared_ptr<MemTracker> parent() const { return parent_; }
+
+  // Returns a textual representation of the tracker that is likely (but not
+  // guaranteed) to be globally unique.
+  std::string ToString() const;
+
+ private:
+  // byte_limit < 0 means no limit
+  // 'id' is the label for LogUsage() and web UI.
+  MemTracker(int64_t byte_limit, const std::string& id, std::shared_ptr<MemTracker> parent);
+
+  // Further initializes the tracker.
+  void Init();
+
+  // Adds tracker to child_trackers_.
+  void AddChildTracker(const std::shared_ptr<MemTracker>& tracker);
+
+  // Variant of FindTracker() that must be called with a non-NULL parent.
+  static bool FindTrackerInternal(
+      const std::string& id,
+      std::shared_ptr<MemTracker>* tracker,
+      const std::shared_ptr<MemTracker>& parent);
+
+  // Creates the root tracker.
+  static void CreateRootTracker();
+
+  int64_t limit_;
+  const std::string id_;
+  const std::string descr_;
+  std::shared_ptr<MemTracker> parent_;
+
+  HighWaterMark consumption_;
+
+  // this tracker plus all of its ancestors
+  std::vector<MemTracker*> all_trackers_;
+  // all_trackers_ with valid limits
+  std::vector<MemTracker*> limit_trackers_;
+
+  // All the child trackers of this tracker. Used for error reporting and
+  // listing only (i.e. updating the consumption of a parent tracker does not
+  // update that of its children).
+  mutable Mutex child_trackers_lock_;
+  std::list<std::weak_ptr<MemTracker>> child_trackers_;
+
+  // Iterator into parent_->child_trackers_ for this object. Stored to have O(1)
+  // remove.
+  std::list<std::weak_ptr<MemTracker>>::iterator child_tracker_it_;
+};
+
+// An std::allocator that manipulates a MemTracker during allocation
+// and deallocation.
+template<typename T, typename Alloc = std::allocator<T> >
+class MemTrackerAllocator : public Alloc {
+ public:
+  typedef typename Alloc::pointer pointer;
+  typedef typename Alloc::const_pointer const_pointer;
+  typedef typename Alloc::size_type size_type;
+
+  explicit MemTrackerAllocator(std::shared_ptr<MemTracker> mem_tracker)
+      : mem_tracker_(std::move(mem_tracker)) {}
+
+  // This constructor is used for rebinding.
+  template <typename U>
+  MemTrackerAllocator(const MemTrackerAllocator<U>& allocator)
+      : Alloc(allocator),
+        mem_tracker_(allocator.mem_tracker()) {
+  }
+
+  ~MemTrackerAllocator() {
+  }
+
+  pointer allocate(size_type n, const_pointer hint = 0) {
+    // Ideally we'd use TryConsume() here to enforce the tracker's limit.
+    // However, that means throwing bad_alloc if the limit is exceeded, and
+    // it's not clear that the rest of Kudu can handle that.
+    mem_tracker_->Consume(n * sizeof(T));
+    return Alloc::allocate(n, hint);
+  }
+
+  void deallocate(pointer p, size_type n) {
+    Alloc::deallocate(p, n);
+    mem_tracker_->Release(n * sizeof(T));
+  }
+
+  // This allows an allocator<T> to be used for a different type.
+  template <class U>
+  struct rebind {
+    typedef MemTrackerAllocator<U, typename Alloc::template rebind<U>::other> other;
+  };
+
+  const std::shared_ptr<MemTracker>& mem_tracker() const { return mem_tracker_; }
+
+ private:
+  std::shared_ptr<MemTracker> mem_tracker_;
+};
+
+// Convenience class that adds memory consumption to a tracker when declared,
+// releasing it when the end of scope is reached.
+class ScopedTrackedConsumption {
+ public:
+  ScopedTrackedConsumption(std::shared_ptr<MemTracker> tracker,
+                           int64_t to_consume)
+      : tracker_(std::move(tracker)), consumption_(to_consume) {
+    DCHECK(tracker_);
+    tracker_->Consume(consumption_);
+  }
+
+  void Reset(int64_t new_consumption) {
+    // Consume(-x) is the same as Release(x).
+    tracker_->Consume(new_consumption - consumption_);
+    consumption_ = new_consumption;
+  }
+
+  ~ScopedTrackedConsumption() {
+    tracker_->Release(consumption_);
+  }
+
+  int64_t consumption() const { return consumption_; }
+
+ private:
+  std::shared_ptr<MemTracker> tracker_;
+  int64_t consumption_;
+};
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_MEM_TRACKER_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memcmpable_varint-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memcmpable_varint-test.cc b/be/src/kudu/util/memcmpable_varint-test.cc
new file mode 100644
index 0000000..fcbe25d
--- /dev/null
+++ b/be/src/kudu/util/memcmpable_varint-test.cc
@@ -0,0 +1,220 @@
+// 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 <cstdint>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/faststring.h"
+#include "kudu/util/hexdump.h"
+#include "kudu/util/memcmpable_varint.h"
+#include "kudu/util/random.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/stopwatch.h"  // IWYU pragma: keep
+#include "kudu/util/test_util.h"
+
+// Add operator<< to print pairs, used in a test below.
+// This has to be done in the 'std' namespace due to the way that
+// template resolution works.
+namespace std {
+template<typename T1, typename T2>
+ostream &operator <<(ostream &os, const pair<T1, T2> &pair) {
+  return os << "(" << pair.first << ", " << pair.second << ")";
+}
+}
+
+using std::make_pair;
+using std::pair;
+using std::vector;
+
+namespace kudu {
+
+class TestMemcmpableVarint : public KuduTest {
+ protected:
+  TestMemcmpableVarint() : random_(SeedRandom()) {}
+
+  // Random number generator that generates different length integers
+  // with equal probability -- i.e it is equally as likely to generate
+  // a number with 8 bits as it is to generate one with 64 bits.
+  // This is useful for testing varint implementations, where a uniform
+  // random is skewed towards generating longer integers.
+  uint64_t Rand64WithRandomBitLength() {
+    return random_.Next64() >> random_.Uniform(64);
+  }
+
+  Random random_;
+};
+
+static void DoRoundTripTest(uint64_t to_encode) {
+  static faststring buf;
+  buf.clear();
+  PutMemcmpableVarint64(&buf, to_encode);
+
+  uint64_t decoded;
+  Slice slice(buf);
+  bool success = GetMemcmpableVarint64(&slice, &decoded);
+  ASSERT_TRUE(success);
+  ASSERT_EQ(to_encode, decoded);
+  ASSERT_TRUE(slice.empty());
+}
+
+
+TEST_F(TestMemcmpableVarint, TestRoundTrip) {
+  // Test the first 100K integers
+  // (exercises the special cases for <= 67823 in the code)
+  for (int i = 0; i < 100000; i++) {
+    DoRoundTripTest(i);
+  }
+
+  // Test a bunch of random integers (which are likely to be many bytes)
+  for (int i = 0; i < 100000; i++) {
+    DoRoundTripTest(random_.Next64());
+  }
+}
+
+
+// Test that a composite key can be made up of multiple memcmpable
+// varints strung together, and that the resulting key compares the
+// same as the original pair of integers (i.e left-to-right).
+TEST_F(TestMemcmpableVarint, TestCompositeKeys) {
+  faststring buf1;
+  faststring buf2;
+
+  const int n_trials = 1000;
+
+  for (int i = 0; i < n_trials; i++) {
+    buf1.clear();
+    buf2.clear();
+
+    pair<uint64_t, uint64_t> p1 =
+        make_pair(Rand64WithRandomBitLength(), Rand64WithRandomBitLength());
+    PutMemcmpableVarint64(&buf1, p1.first);
+    PutMemcmpableVarint64(&buf1, p1.second);
+
+    pair<uint64_t, uint64_t> p2 =
+        make_pair(Rand64WithRandomBitLength(), Rand64WithRandomBitLength());
+    PutMemcmpableVarint64(&buf2, p2.first);
+    PutMemcmpableVarint64(&buf2, p2.second);
+
+    SCOPED_TRACE(testing::Message() << p1 << "\n" << HexDump(Slice(buf1))
+                 << "  vs\n" << p2 << "\n" << HexDump(Slice(buf2)));
+    if (p1 < p2) {
+      ASSERT_LT(Slice(buf1).compare(Slice(buf2)), 0);
+    } else if (p1 > p2) {
+      ASSERT_GT(Slice(buf1).compare(Slice(buf2)), 0);
+    } else {
+      ASSERT_EQ(Slice(buf1).compare(Slice(buf2)), 0);
+    }
+  }
+}
+
+// Similar to the above test, but instead of being randomized, specifically
+// tests "interesting" values -- i.e values around the boundaries of where
+// the encoding changes its number of bytes.
+TEST_F(TestMemcmpableVarint, TestInterestingCompositeKeys) {
+  const vector<uint64_t> interesting_values = {
+    0, 1, 240, // 1 byte
+    241, 2000, 2287, // 2 bytes
+    2288, 40000, 67823, // 3 bytes
+    67824, 1ULL << 23, (1ULL << 24) - 1, // 4 bytes
+    1ULL << 24, 1ULL << 30, (1ULL << 32) - 1, // 5 bytes
+  };
+
+  faststring buf1;
+  faststring buf2;
+
+  for (uint64_t v1 : interesting_values) {
+    for (uint64_t v2 : interesting_values) {
+      buf1.clear();
+      pair<uint64_t, uint64_t> p1 = make_pair(v1, v2);
+      PutMemcmpableVarint64(&buf1, p1.first);
+      PutMemcmpableVarint64(&buf1, p1.second);
+
+      for (uint64_t v3 : interesting_values) {
+        for (uint64_t v4 : interesting_values) {
+          buf2.clear();
+          pair<uint64_t, uint64_t> p2 = make_pair(v3, v4);
+          PutMemcmpableVarint64(&buf2, p2.first);
+          PutMemcmpableVarint64(&buf2, p2.second);
+
+          SCOPED_TRACE(testing::Message() << p1 << "\n" << HexDump(Slice(buf1))
+                       << "  vs\n" << p2 << "\n" << HexDump(Slice(buf2)));
+          if (p1 < p2) {
+            ASSERT_LT(Slice(buf1).compare(Slice(buf2)), 0);
+          } else if (p1 > p2) {
+            ASSERT_GT(Slice(buf1).compare(Slice(buf2)), 0);
+          } else {
+            ASSERT_EQ(Slice(buf1).compare(Slice(buf2)), 0);
+          }
+        }
+      }
+    }
+  }
+}
+
+////////////////////////////////////////////////////////////
+// Benchmarks
+////////////////////////////////////////////////////////////
+
+#ifdef NDEBUG
+TEST_F(TestMemcmpableVarint, BenchmarkEncode) {
+  faststring buf;
+
+  int sum_sizes = 0; // need to do something with results to force evaluation
+
+  LOG_TIMING(INFO, "Encoding integers") {
+    for (int trial = 0; trial < 100; trial++) {
+      for (uint64_t i = 0; i < 1000000; i++) {
+        buf.clear();
+        PutMemcmpableVarint64(&buf, i);
+        sum_sizes += buf.size();
+      }
+    }
+  }
+  ASSERT_GT(sum_sizes, 1); // use 'sum_sizes' to avoid optimizing it out.
+}
+
+TEST_F(TestMemcmpableVarint, BenchmarkDecode) {
+  faststring buf;
+
+  // Encode 1M integers into the buffer
+  for (uint64_t i = 0; i < 1000000; i++) {
+    PutMemcmpableVarint64(&buf, i);
+  }
+
+  // Decode the whole buffer 100 times.
+  LOG_TIMING(INFO, "Decoding integers") {
+    uint64_t sum_vals = 0;
+    for (int trial = 0; trial < 100; trial++) {
+      Slice s(buf);
+      while (!s.empty()) {
+        uint64_t decoded;
+        CHECK(GetMemcmpableVarint64(&s, &decoded));
+        sum_vals += decoded;
+      }
+    }
+    ASSERT_GT(sum_vals, 1); // use 'sum_vals' to avoid optimizing it out.
+  }
+}
+
+#endif
+
+} // namespace kudu


[11/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/sockaddr.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/sockaddr.cc b/be/src/kudu/util/net/sockaddr.cc
new file mode 100644
index 0000000..60905b2
--- /dev/null
+++ b/be/src/kudu/util/net/sockaddr.cc
@@ -0,0 +1,136 @@
+// 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 "kudu/util/net/sockaddr.h"
+
+#include <arpa/inet.h>
+#include <netdb.h>
+#include <sys/socket.h>
+
+#include <cerrno>
+#include <cstring>
+#include <string>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/hash/builtin_type_hash.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/stopwatch.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+///
+/// Sockaddr
+///
+Sockaddr::Sockaddr() {
+  memset(&addr_, 0, sizeof(addr_));
+  addr_.sin_family = AF_INET;
+  addr_.sin_addr.s_addr = INADDR_ANY;
+}
+
+Sockaddr::Sockaddr(const struct sockaddr_in& addr) {
+  memcpy(&addr_, &addr, sizeof(struct sockaddr_in));
+}
+
+Status Sockaddr::ParseString(const std::string& s, uint16_t default_port) {
+  HostPort hp;
+  RETURN_NOT_OK(hp.ParseString(s, default_port));
+
+  if (inet_pton(AF_INET, hp.host().c_str(), &addr_.sin_addr) != 1) {
+    return Status::InvalidArgument("Invalid IP address", hp.host());
+  }
+  set_port(hp.port());
+  return Status::OK();
+}
+
+Sockaddr& Sockaddr::operator=(const struct sockaddr_in &addr) {
+  memcpy(&addr_, &addr, sizeof(struct sockaddr_in));
+  return *this;
+}
+
+bool Sockaddr::operator==(const Sockaddr& other) const {
+  return memcmp(&other.addr_, &addr_, sizeof(addr_)) == 0;
+}
+
+bool Sockaddr::operator<(const Sockaddr &rhs) const {
+  return addr_.sin_addr.s_addr < rhs.addr_.sin_addr.s_addr;
+}
+
+uint32_t Sockaddr::HashCode() const {
+  uint32_t hash = Hash32NumWithSeed(addr_.sin_addr.s_addr, 0);
+  hash = Hash32NumWithSeed(addr_.sin_port, hash);
+  return hash;
+}
+
+void Sockaddr::set_port(int port) {
+  addr_.sin_port = htons(port);
+}
+
+int Sockaddr::port() const {
+  return ntohs(addr_.sin_port);
+}
+
+std::string Sockaddr::host() const {
+  char str[INET_ADDRSTRLEN];
+  ::inet_ntop(AF_INET, &addr_.sin_addr, str, INET_ADDRSTRLEN);
+  return str;
+}
+
+const struct sockaddr_in& Sockaddr::addr() const {
+  return addr_;
+}
+
+std::string Sockaddr::ToString() const {
+  return Substitute("$0:$1", host(), port());
+}
+
+bool Sockaddr::IsWildcard() const {
+  return addr_.sin_addr.s_addr == 0;
+}
+
+bool Sockaddr::IsAnyLocalAddress() const {
+  return (NetworkByteOrder::FromHost32(addr_.sin_addr.s_addr) >> 24) == 127;
+}
+
+Status Sockaddr::LookupHostname(string* hostname) const {
+  char host[NI_MAXHOST];
+  int flags = 0;
+
+  int rc;
+  LOG_SLOW_EXECUTION(WARNING, 200,
+                     Substitute("DNS reverse-lookup for $0", ToString())) {
+    rc = getnameinfo((struct sockaddr *) &addr_, sizeof(sockaddr_in),
+                     host, NI_MAXHOST,
+                     nullptr, 0, flags);
+  }
+  if (PREDICT_FALSE(rc != 0)) {
+    if (rc == EAI_SYSTEM) {
+      int errno_saved = errno;
+      return Status::NetworkError(Substitute("getnameinfo: $0", gai_strerror(rc)),
+                                  strerror(errno_saved), errno_saved);
+    }
+    return Status::NetworkError("getnameinfo", gai_strerror(rc), rc);
+  }
+  *hostname = host;
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/sockaddr.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/sockaddr.h b/be/src/kudu/util/net/sockaddr.h
new file mode 100644
index 0000000..dffd151
--- /dev/null
+++ b/be/src/kudu/util/net/sockaddr.h
@@ -0,0 +1,94 @@
+// 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.
+#ifndef KUDU_UTIL_NET_SOCKADDR_H
+#define KUDU_UTIL_NET_SOCKADDR_H
+
+#include <netinet/in.h>
+
+#include <cstdint>
+#include <functional>
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+///
+/// Represents a sockaddr.
+///
+/// Currently only IPv4 is implemented.  When IPv6 and UNIX domain are
+/// implemented, this should become an abstract base class and those should be
+/// multiple implementations.
+///
+class Sockaddr {
+ public:
+  Sockaddr();
+  explicit Sockaddr(const struct sockaddr_in &addr);
+
+  // Parse a string IP address of the form "A.B.C.D:port", storing the result
+  // in this Sockaddr object. If no ':port' is specified, uses 'default_port'.
+  // Note that this function will not handle resolving hostnames.
+  //
+  // Returns a bad Status if the input is malformed.
+  Status ParseString(const std::string& s, uint16_t default_port);
+
+  Sockaddr& operator=(const struct sockaddr_in &addr);
+
+  bool operator==(const Sockaddr& other) const;
+
+  // Compare the endpoints of two sockaddrs.
+  // The port number is ignored in this comparison.
+  bool operator<(const Sockaddr &rhs) const;
+
+  uint32_t HashCode() const;
+
+  // Returns the dotted-decimal string '1.2.3.4' of the host component of this address.
+  std::string host() const;
+
+  void set_port(int port);
+  int port() const;
+  const struct sockaddr_in& addr() const;
+
+  // Returns the stringified address in '1.2.3.4:<port>' format.
+  std::string ToString() const;
+
+  // Returns true if the address is 0.0.0.0
+  bool IsWildcard() const;
+
+  // Returns true if the address is 127.*.*.*
+  bool IsAnyLocalAddress() const;
+
+  // Does reverse DNS lookup of the address and stores it in hostname.
+  Status LookupHostname(std::string* hostname) const;
+
+  // the default auto-generated copy constructor is fine here
+ private:
+  struct sockaddr_in addr_;
+};
+
+} // namespace kudu
+
+// Specialize std::hash for Sockaddr
+namespace std {
+template<>
+struct hash<kudu::Sockaddr> {
+  int operator()(const kudu::Sockaddr& addr) const {
+    return addr.HashCode();
+  }
+};
+} // namespace std
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/socket-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/socket-test.cc b/be/src/kudu/util/net/socket-test.cc
new file mode 100644
index 0000000..8ecea4e
--- /dev/null
+++ b/be/src/kudu/util/net/socket-test.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/util/net/socket.h"
+
+#include <thread>
+
+#include <cstdint>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <memory>
+#include <stddef.h>
+#include <string>
+
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+
+namespace kudu {
+
+constexpr size_t kEchoChunkSize = 32 * 1024 * 1024;
+
+class SocketTest : public KuduTest {
+ protected:
+  void DoTest(bool accept, const std::string &message) {
+    Sockaddr address;
+    address.ParseString("0.0.0.0", 0);
+    Socket listener_;
+
+    CHECK_OK(listener_.Init(0));
+    CHECK_OK(listener_.BindAndListen(address, 0));
+    Sockaddr listen_address;
+    CHECK_OK(listener_.GetSocketAddress(&listen_address));
+
+    std::thread t([&]{
+      if (accept) {
+        Sockaddr new_addr;
+        Socket sock;
+        CHECK_OK(listener_.Accept(&sock, &new_addr, 0));
+        CHECK_OK(sock.Close());
+      } else {
+        SleepFor(MonoDelta::FromMilliseconds(200));
+        CHECK_OK(listener_.Close());
+      }
+    });
+
+    Socket client;
+    ASSERT_OK(client.Init(0));
+    ASSERT_OK(client.Connect(listen_address));
+    CHECK_OK(client.SetRecvTimeout(MonoDelta::FromMilliseconds(100)));
+
+    int n;
+    std::unique_ptr<uint8_t[]> buf(new uint8_t[kEchoChunkSize]);
+    Status s = client.Recv(buf.get(), kEchoChunkSize, &n);
+
+    ASSERT_TRUE(!s.ok());
+    ASSERT_TRUE(s.IsNetworkError());
+    ASSERT_STR_MATCHES(s.message().ToString(), message);
+
+    t.join();
+  }
+};
+
+TEST_F(SocketTest, TestRecvReset) {
+  DoTest(false, "recv error from 127.0.0.1:[0-9]+: Resource temporarily unavailable");
+}
+
+TEST_F(SocketTest, TestRecvEOF) {
+  DoTest(true, "recv got EOF from 127.0.0.1:[0-9]+");
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/socket.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/socket.cc b/be/src/kudu/util/net/socket.cc
new file mode 100644
index 0000000..cc14702
--- /dev/null
+++ b/be/src/kudu/util/net/socket.cc
@@ -0,0 +1,590 @@
+// 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 "kudu/util/net/socket.h"
+
+#include <fcntl.h>
+#include <netinet/in.h>
+#include <netinet/tcp.h>
+#include <sys/socket.h>
+#include <sys/time.h>
+#include <unistd.h>
+
+#include <cerrno>
+#include <cinttypes>
+#include <cstring>
+#include <limits>
+#include <ostream>
+#include <string>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/slice.h"
+
+DEFINE_string(local_ip_for_outbound_sockets, "",
+              "IP to bind to when making outgoing socket connections. "
+              "This must be an IP address of the form A.B.C.D, not a hostname. "
+              "Advanced parameter, subject to change.");
+TAG_FLAG(local_ip_for_outbound_sockets, experimental);
+
+DEFINE_bool(socket_inject_short_recvs, false,
+            "Inject short recv() responses which return less data than "
+            "requested");
+TAG_FLAG(socket_inject_short_recvs, hidden);
+TAG_FLAG(socket_inject_short_recvs, unsafe);
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+Socket::Socket()
+  : fd_(-1) {
+}
+
+Socket::Socket(int fd)
+  : fd_(fd) {
+}
+
+void Socket::Reset(int fd) {
+  ignore_result(Close());
+  fd_ = fd;
+}
+
+int Socket::Release() {
+  int fd = fd_;
+  fd_ = -1;
+  return fd;
+}
+
+Socket::~Socket() {
+  ignore_result(Close());
+}
+
+Status Socket::Close() {
+  if (fd_ < 0) {
+    return Status::OK();
+  }
+  int fd = fd_;
+  int ret;
+  RETRY_ON_EINTR(ret, ::close(fd));
+  if (ret < 0) {
+    int err = errno;
+    return Status::NetworkError("close error", ErrnoToString(err), err);
+  }
+  fd_ = -1;
+  return Status::OK();
+}
+
+Status Socket::Shutdown(bool shut_read, bool shut_write) {
+  DCHECK_GE(fd_, 0);
+  int flags = 0;
+  if (shut_read && shut_write) {
+    flags |= SHUT_RDWR;
+  } else if (shut_read) {
+    flags |= SHUT_RD;
+  } else if (shut_write) {
+    flags |= SHUT_WR;
+  }
+  if (::shutdown(fd_, flags) < 0) {
+    int err = errno;
+    return Status::NetworkError("shutdown error", ErrnoToString(err), err);
+  }
+  return Status::OK();
+}
+
+int Socket::GetFd() const {
+  return fd_;
+}
+
+bool Socket::IsTemporarySocketError(int err) {
+  return ((err == EAGAIN) || (err == EWOULDBLOCK) || (err == EINTR));
+}
+
+#if defined(__linux__)
+
+Status Socket::Init(int flags) {
+  int nonblocking_flag = (flags & FLAG_NONBLOCKING) ? SOCK_NONBLOCK : 0;
+  Reset(::socket(AF_INET, SOCK_STREAM | SOCK_CLOEXEC | nonblocking_flag, 0));
+  if (fd_ < 0) {
+    int err = errno;
+    return Status::NetworkError("error opening socket", ErrnoToString(err), err);
+  }
+
+  return Status::OK();
+}
+
+#else
+
+Status Socket::Init(int flags) {
+  Reset(::socket(AF_INET, SOCK_STREAM, 0));
+  if (fd_ < 0) {
+    int err = errno;
+    return Status::NetworkError("error opening socket", ErrnoToString(err), err);
+  }
+  RETURN_NOT_OK(SetNonBlocking(flags & FLAG_NONBLOCKING));
+  RETURN_NOT_OK(SetCloseOnExec());
+
+  // Disable SIGPIPE.
+  int set = 1;
+  RETURN_NOT_OK_PREPEND(SetSockOpt(SOL_SOCKET, SO_NOSIGPIPE, set),
+                        "failed to set SO_NOSIGPIPE");
+  return Status::OK();
+}
+
+#endif // defined(__linux__)
+
+Status Socket::SetNoDelay(bool enabled) {
+  int flag = enabled ? 1 : 0;
+  RETURN_NOT_OK_PREPEND(SetSockOpt(IPPROTO_TCP, TCP_NODELAY, flag),
+                        "failed to set TCP_NODELAY");
+  return Status::OK();
+}
+
+Status Socket::SetTcpCork(bool enabled) {
+#if defined(__linux__)
+  int flag = enabled ? 1 : 0;
+  RETURN_NOT_OK_PREPEND(SetSockOpt(IPPROTO_TCP, TCP_CORK, flag),
+                        "failed to set TCP_CORK");
+#endif // defined(__linux__)
+  // TODO(unknown): Use TCP_NOPUSH for OSX if perf becomes an issue.
+  return Status::OK();
+}
+
+Status Socket::SetNonBlocking(bool enabled) {
+  int curflags = ::fcntl(fd_, F_GETFL, 0);
+  if (curflags == -1) {
+    int err = errno;
+    return Status::NetworkError(
+        StringPrintf("Failed to get file status flags on fd %d", fd_),
+        ErrnoToString(err), err);
+  }
+  int newflags = (enabled) ? (curflags | O_NONBLOCK) : (curflags & ~O_NONBLOCK);
+  if (::fcntl(fd_, F_SETFL, newflags) == -1) {
+    int err = errno;
+    if (enabled) {
+      return Status::NetworkError(
+          StringPrintf("Failed to set O_NONBLOCK on fd %d", fd_),
+          ErrnoToString(err), err);
+    } else {
+      return Status::NetworkError(
+          StringPrintf("Failed to clear O_NONBLOCK on fd %d", fd_),
+          ErrnoToString(err), err);
+    }
+  }
+  return Status::OK();
+}
+
+Status Socket::IsNonBlocking(bool* is_nonblock) const {
+  int curflags = ::fcntl(fd_, F_GETFL, 0);
+  if (curflags == -1) {
+    int err = errno;
+    return Status::NetworkError(
+        StringPrintf("Failed to get file status flags on fd %d", fd_),
+        ErrnoToString(err), err);
+  }
+  *is_nonblock = ((curflags & O_NONBLOCK) != 0);
+  return Status::OK();
+}
+
+Status Socket::SetCloseOnExec() {
+  int curflags = fcntl(fd_, F_GETFD, 0);
+  if (curflags == -1) {
+    int err = errno;
+    Reset(-1);
+    return Status::NetworkError("fcntl(F_GETFD) error", ErrnoToString(err), err);
+  }
+  if (fcntl(fd_, F_SETFD, curflags | FD_CLOEXEC) == -1) {
+    int err = errno;
+    Reset(-1);
+    return Status::NetworkError("fcntl(F_SETFD) error", ErrnoToString(err), err);
+  }
+  return Status::OK();
+}
+
+Status Socket::SetSendTimeout(const MonoDelta& timeout) {
+  return SetTimeout(SO_SNDTIMEO, "SO_SNDTIMEO", timeout);
+}
+
+Status Socket::SetRecvTimeout(const MonoDelta& timeout) {
+  return SetTimeout(SO_RCVTIMEO, "SO_RCVTIMEO", timeout);
+}
+
+Status Socket::SetReuseAddr(bool flag) {
+  int int_flag = flag ? 1 : 0;
+  RETURN_NOT_OK_PREPEND(SetSockOpt(SOL_SOCKET, SO_REUSEADDR, int_flag),
+                        "failed to set SO_REUSEADDR");
+  return Status::OK();
+}
+
+Status Socket::SetReusePort(bool flag) {
+  int int_flag = flag ? 1 : 0;
+  RETURN_NOT_OK_PREPEND(SetSockOpt(SOL_SOCKET, SO_REUSEPORT, int_flag),
+                        "failed to set SO_REUSEPORT");
+  return Status::OK();
+}
+
+Status Socket::BindAndListen(const Sockaddr &sockaddr,
+                             int listen_queue_size) {
+  RETURN_NOT_OK(SetReuseAddr(true));
+  RETURN_NOT_OK(Bind(sockaddr));
+  RETURN_NOT_OK(Listen(listen_queue_size));
+  return Status::OK();
+}
+
+Status Socket::Listen(int listen_queue_size) {
+  if (listen(fd_, listen_queue_size)) {
+    int err = errno;
+    return Status::NetworkError("listen() error", ErrnoToString(err));
+  }
+  return Status::OK();
+}
+
+Status Socket::GetSocketAddress(Sockaddr *cur_addr) const {
+  struct sockaddr_in sin;
+  socklen_t len = sizeof(sin);
+  DCHECK_GE(fd_, 0);
+  if (::getsockname(fd_, reinterpret_cast<struct sockaddr*>(&sin), &len) == -1) {
+    int err = errno;
+    return Status::NetworkError("getsockname error", ErrnoToString(err), err);
+  }
+  *cur_addr = sin;
+  return Status::OK();
+}
+
+Status Socket::GetPeerAddress(Sockaddr *cur_addr) const {
+  struct sockaddr_in sin;
+  socklen_t len = sizeof(sin);
+  DCHECK_GE(fd_, 0);
+  if (::getpeername(fd_, reinterpret_cast<struct sockaddr*>(&sin), &len) == -1) {
+    int err = errno;
+    return Status::NetworkError("getpeername error", ErrnoToString(err), err);
+  }
+  *cur_addr = sin;
+  return Status::OK();
+}
+
+bool Socket::IsLoopbackConnection() const {
+  Sockaddr local, remote;
+  if (!GetSocketAddress(&local).ok()) return false;
+  if (!GetPeerAddress(&remote).ok()) return false;
+
+  // Compare without comparing ports.
+  local.set_port(0);
+  remote.set_port(0);
+  return local == remote;
+}
+
+Status Socket::Bind(const Sockaddr& bind_addr) {
+  struct sockaddr_in addr = bind_addr.addr();
+
+  DCHECK_GE(fd_, 0);
+  if (PREDICT_FALSE(::bind(fd_, (struct sockaddr*) &addr, sizeof(addr)))) {
+    int err = errno;
+    Status s = Status::NetworkError(
+        strings::Substitute("error binding socket to $0: $1",
+                            bind_addr.ToString(), ErrnoToString(err)),
+        Slice(), err);
+
+    if (s.IsNetworkError() && s.posix_code() == EADDRINUSE && bind_addr.port() != 0) {
+      TryRunLsof(bind_addr);
+    }
+    return s;
+  }
+
+  return Status::OK();
+}
+
+Status Socket::Accept(Socket *new_conn, Sockaddr *remote, int flags) {
+  TRACE_EVENT0("net", "Socket::Accept");
+  struct sockaddr_in addr;
+  socklen_t olen = sizeof(addr);
+  DCHECK_GE(fd_, 0);
+#if defined(__linux__)
+  int accept_flags = SOCK_CLOEXEC;
+  if (flags & FLAG_NONBLOCKING) {
+    accept_flags |= SOCK_NONBLOCK;
+  }
+  int fd = -1;
+  RETRY_ON_EINTR(fd, accept4(fd_, (struct sockaddr*)&addr,
+                             &olen, accept_flags));
+  if (fd < 0) {
+    int err = errno;
+    return Status::NetworkError("accept4(2) error", ErrnoToString(err), err);
+  }
+  new_conn->Reset(fd);
+
+#else
+  int fd = -1;
+  RETRY_ON_EINTR(fd, accept(fd_, (struct sockaddr*)&addr, &olen));
+  if (fd < 0) {
+    int err = errno;
+    return Status::NetworkError("accept(2) error", ErrnoToString(err), err);
+  }
+  new_conn->Reset(fd);
+  RETURN_NOT_OK(new_conn->SetNonBlocking(flags & FLAG_NONBLOCKING));
+  RETURN_NOT_OK(new_conn->SetCloseOnExec());
+#endif // defined(__linux__)
+
+  *remote = addr;
+  TRACE_EVENT_INSTANT1("net", "Accepted", TRACE_EVENT_SCOPE_THREAD,
+                       "remote", remote->ToString());
+  return Status::OK();
+}
+
+Status Socket::BindForOutgoingConnection() {
+  Sockaddr bind_host;
+  Status s = bind_host.ParseString(FLAGS_local_ip_for_outbound_sockets, 0);
+  CHECK(s.ok() && bind_host.port() == 0)
+    << "Invalid local IP set for 'local_ip_for_outbound_sockets': '"
+    << FLAGS_local_ip_for_outbound_sockets << "': " << s.ToString();
+
+  RETURN_NOT_OK(Bind(bind_host));
+  return Status::OK();
+}
+
+Status Socket::Connect(const Sockaddr &remote) {
+  TRACE_EVENT1("net", "Socket::Connect",
+               "remote", remote.ToString());
+  if (PREDICT_FALSE(!FLAGS_local_ip_for_outbound_sockets.empty())) {
+    RETURN_NOT_OK(BindForOutgoingConnection());
+  }
+
+  struct sockaddr_in addr;
+  memcpy(&addr, &remote.addr(), sizeof(sockaddr_in));
+  DCHECK_GE(fd_, 0);
+  int ret;
+  RETRY_ON_EINTR(ret, ::connect(
+      fd_, reinterpret_cast<const struct sockaddr*>(&addr), sizeof(addr)));
+  if (ret < 0) {
+    int err = errno;
+    return Status::NetworkError("connect(2) error", ErrnoToString(err), err);
+  }
+  return Status::OK();
+}
+
+Status Socket::GetSockError() const {
+  int val = 0, ret;
+  socklen_t val_len = sizeof(val);
+  DCHECK_GE(fd_, 0);
+  ret = ::getsockopt(fd_, SOL_SOCKET, SO_ERROR, &val, &val_len);
+  if (ret) {
+    int err = errno;
+    return Status::NetworkError("getsockopt(SO_ERROR) failed", ErrnoToString(err), err);
+  }
+  if (val != 0) {
+    return Status::NetworkError(ErrnoToString(val), Slice(), val);
+  }
+  return Status::OK();
+}
+
+Status Socket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
+  if (amt <= 0) {
+    return Status::NetworkError(
+              StringPrintf("invalid send of %" PRId32 " bytes",
+                           amt), Slice(), EINVAL);
+  }
+  DCHECK_GE(fd_, 0);
+  int res;
+  RETRY_ON_EINTR(res, ::send(fd_, buf, amt, MSG_NOSIGNAL));
+  if (res < 0) {
+    int err = errno;
+    return Status::NetworkError("write error", ErrnoToString(err), err);
+  }
+  *nwritten = res;
+  return Status::OK();
+}
+
+Status Socket::Writev(const struct ::iovec *iov, int iov_len,
+                      int64_t *nwritten) {
+  if (PREDICT_FALSE(iov_len <= 0)) {
+    return Status::NetworkError(
+                StringPrintf("writev: invalid io vector length of %d",
+                             iov_len),
+                Slice(), EINVAL);
+  }
+  DCHECK_GE(fd_, 0);
+
+  struct msghdr msg;
+  memset(&msg, 0, sizeof(struct msghdr));
+  msg.msg_iov = const_cast<iovec *>(iov);
+  msg.msg_iovlen = iov_len;
+  ssize_t res;
+  RETRY_ON_EINTR(res, ::sendmsg(fd_, &msg, MSG_NOSIGNAL));
+  if (PREDICT_FALSE(res < 0)) {
+    int err = errno;
+    return Status::NetworkError("sendmsg error", ErrnoToString(err), err);
+  }
+
+  *nwritten = res;
+  return Status::OK();
+}
+
+// Mostly follows writen() from Stevens (2004) or Kerrisk (2010).
+Status Socket::BlockingWrite(const uint8_t *buf, size_t buflen, size_t *nwritten,
+    const MonoTime& deadline) {
+  DCHECK_LE(buflen, std::numeric_limits<int32_t>::max()) << "Writes > INT32_MAX not supported";
+  DCHECK(nwritten);
+
+  size_t tot_written = 0;
+  while (tot_written < buflen) {
+    int32_t inc_num_written = 0;
+    int32_t num_to_write = buflen - tot_written;
+    MonoDelta timeout = deadline - MonoTime::Now();
+    if (PREDICT_FALSE(timeout.ToNanoseconds() <= 0)) {
+      return Status::TimedOut("BlockingWrite timed out");
+    }
+    RETURN_NOT_OK(SetSendTimeout(timeout));
+    Status s = Write(buf, num_to_write, &inc_num_written);
+    tot_written += inc_num_written;
+    buf += inc_num_written;
+    *nwritten = tot_written;
+
+    if (PREDICT_FALSE(!s.ok())) {
+      // Continue silently when the syscall is interrupted.
+      if (s.posix_code() == EINTR) {
+        continue;
+      }
+      if (s.posix_code() == EAGAIN) {
+        return Status::TimedOut("");
+      }
+      return s.CloneAndPrepend("BlockingWrite error");
+    }
+    if (PREDICT_FALSE(inc_num_written == 0)) {
+      // Shouldn't happen on Linux with a blocking socket. Maybe other Unices.
+      break;
+    }
+  }
+
+  if (tot_written < buflen) {
+    return Status::IOError("Wrote zero bytes on a BlockingWrite() call",
+        StringPrintf("Transferred %zu of %zu bytes", tot_written, buflen));
+  }
+  return Status::OK();
+}
+
+Status Socket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
+  if (amt <= 0) {
+    return Status::NetworkError(
+          StringPrintf("invalid recv of %d bytes", amt), Slice(), EINVAL);
+  }
+
+  // The recv() call can return fewer than the requested number of bytes.
+  // Especially when 'amt' is small, this is very unlikely to happen in
+  // the context of unit tests. So, we provide an injection hook which
+  // simulates the same behavior.
+  if (PREDICT_FALSE(FLAGS_socket_inject_short_recvs && amt > 1)) {
+    Random r(GetRandomSeed32());
+    amt = 1 + r.Uniform(amt - 1);
+  }
+
+  DCHECK_GE(fd_, 0);
+  int res;
+  RETRY_ON_EINTR(res, recv(fd_, buf, amt, 0));
+  if (res <= 0) {
+    Sockaddr remote;
+    GetPeerAddress(&remote);
+    if (res == 0) {
+      string error_message = Substitute("recv got EOF from $0", remote.ToString());
+      return Status::NetworkError(error_message, Slice(), ESHUTDOWN);
+    }
+    int err = errno;
+    string error_message = Substitute("recv error from $0", remote.ToString());
+    return Status::NetworkError(error_message, ErrnoToString(err), err);
+  }
+  *nread = res;
+  return Status::OK();
+}
+
+// Mostly follows readn() from Stevens (2004) or Kerrisk (2010).
+// One place where we deviate: we consider EOF a failure if < amt bytes are read.
+Status Socket::BlockingRecv(uint8_t *buf, size_t amt, size_t *nread, const MonoTime& deadline) {
+  DCHECK_LE(amt, std::numeric_limits<int32_t>::max()) << "Reads > INT32_MAX not supported";
+  DCHECK(nread);
+  size_t tot_read = 0;
+  while (tot_read < amt) {
+    int32_t inc_num_read = 0;
+    int32_t num_to_read = amt - tot_read;
+    MonoDelta timeout = deadline - MonoTime::Now();
+    if (PREDICT_FALSE(timeout.ToNanoseconds() <= 0)) {
+      return Status::TimedOut("");
+    }
+    RETURN_NOT_OK(SetRecvTimeout(timeout));
+    Status s = Recv(buf, num_to_read, &inc_num_read);
+    tot_read += inc_num_read;
+    buf += inc_num_read;
+    *nread = tot_read;
+
+    if (PREDICT_FALSE(!s.ok())) {
+      // Continue silently when the syscall is interrupted.
+      if (s.posix_code() == EINTR) {
+        continue;
+      }
+      if (s.posix_code() == EAGAIN) {
+        return Status::TimedOut("");
+      }
+      return s.CloneAndPrepend("BlockingRecv error");
+    }
+    if (PREDICT_FALSE(inc_num_read == 0)) {
+      // EOF.
+      break;
+    }
+  }
+
+  if (PREDICT_FALSE(tot_read < amt)) {
+    return Status::IOError("Read zero bytes on a blocking Recv() call",
+        StringPrintf("Transferred %zu of %zu bytes", tot_read, amt));
+  }
+  return Status::OK();
+}
+
+Status Socket::SetTimeout(int opt, const char* optname, const MonoDelta& timeout) {
+  if (PREDICT_FALSE(timeout.ToNanoseconds() < 0)) {
+    return Status::InvalidArgument("Timeout specified as negative to SetTimeout",
+                                   timeout.ToString());
+  }
+  struct timeval tv;
+  timeout.ToTimeVal(&tv);
+  RETURN_NOT_OK_PREPEND(SetSockOpt(SOL_SOCKET, opt, tv),
+                        Substitute("failed to set socket option $0 to $1",
+                                   optname, timeout.ToString()));
+  return Status::OK();
+}
+
+template<typename T>
+Status Socket::SetSockOpt(int level, int option, const T& value) {
+  if (::setsockopt(fd_, level, option, &value, sizeof(T)) == -1) {
+    int err = errno;
+    return Status::NetworkError(ErrnoToString(err), Slice(), err);
+  }
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/socket.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/socket.h b/be/src/kudu/util/net/socket.h
new file mode 100644
index 0000000..992c44a
--- /dev/null
+++ b/be/src/kudu/util/net/socket.h
@@ -0,0 +1,178 @@
+// 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.
+#ifndef KUDU_UTIL_NET_SOCKET_H
+#define KUDU_UTIL_NET_SOCKET_H
+
+#include <cstddef>
+#include <cstdint>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/status.h"
+
+struct iovec;
+
+namespace kudu {
+
+class MonoDelta;
+class MonoTime;
+class Sockaddr;
+
+class Socket {
+ public:
+  static const int FLAG_NONBLOCKING = 0x1;
+
+  // Create a new invalid Socket object.
+  Socket();
+
+  // Start managing a socket.
+  explicit Socket(int fd);
+
+  // Close the socket.  Errors will be ignored.
+  virtual ~Socket();
+
+  // Close the Socket, checking for errors.
+  virtual Status Close();
+
+  // call shutdown() on the socket
+  Status Shutdown(bool shut_read, bool shut_write);
+
+  // Start managing a socket.
+  void Reset(int fd);
+
+  // Stop managing the socket and return it.
+  int Release();
+
+  // Get the raw file descriptor, or -1 if there is no file descriptor being
+  // managed.
+  int GetFd() const;
+
+  // Returns true if the error is temporary and will go away if we retry on
+  // the socket.
+  static bool IsTemporarySocketError(int err);
+
+  Status Init(int flags); // See FLAG_NONBLOCKING
+
+  // Set or clear TCP_NODELAY
+  Status SetNoDelay(bool enabled);
+
+  // Set or clear TCP_CORK
+  Status SetTcpCork(bool enabled);
+
+  // Set or clear O_NONBLOCK
+  Status SetNonBlocking(bool enabled);
+  Status IsNonBlocking(bool* is_nonblock) const;
+
+  // Set SO_SENDTIMEO to the specified value. Should only be used for blocking sockets.
+  Status SetSendTimeout(const MonoDelta& timeout);
+
+  // Set SO_RCVTIMEO to the specified value. Should only be used for blocking sockets.
+  Status SetRecvTimeout(const MonoDelta& timeout);
+
+  // Sets SO_REUSEADDR to 'flag'. Should be used prior to Bind().
+  Status SetReuseAddr(bool flag);
+
+  // Sets SO_REUSEPORT to 'flag'. Should be used prior to Bind().
+  Status SetReusePort(bool flag);
+
+  // Convenience method to invoke the common sequence:
+  // 1) SetReuseAddr(true)
+  // 2) Bind()
+  // 3) Listen()
+  Status BindAndListen(const Sockaddr &sockaddr, int listen_queue_size);
+
+  // Start listening for new connections, with the given backlog size.
+  // Requires that the socket has already been bound using Bind().
+  Status Listen(int listen_queue_size);
+
+  // Call getsockname to get the address of this socket.
+  Status GetSocketAddress(Sockaddr *cur_addr) const;
+
+  // Call getpeername to get the address of the connected peer.
+  // It is virtual so that tests can override.
+  virtual Status GetPeerAddress(Sockaddr *cur_addr) const;
+
+  // Return true if this socket is determined to be a loopback connection
+  // (i.e. the local and remote peer share an IP address).
+  //
+  // If any error occurs while determining this, returns false.
+  bool IsLoopbackConnection() const;
+
+  // Call bind() to bind the socket to a given address.
+  // If bind() fails and indicates that the requested port is already in use,
+  // generates an informative log message by calling 'lsof' if available.
+  Status Bind(const Sockaddr& bind_addr);
+
+  // Call accept(2) to get a new connection.
+  Status Accept(Socket *new_conn, Sockaddr *remote, int flags);
+
+  // start connecting this socket to a remote address.
+  Status Connect(const Sockaddr &remote);
+
+  // get the error status using getsockopt(2)
+  Status GetSockError() const;
+
+  // Write up to 'amt' bytes from 'buf' to the socket. The number of bytes
+  // actually written will be stored in 'nwritten'. If an error is returned,
+  // the value of 'nwritten' is undefined.
+  virtual Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten);
+
+  // Vectorized Write.
+  // If there is an error, that error needs to be resolved before calling again.
+  // If there was no error, but not all the bytes were written, the unwritten
+  // bytes must be retried. See writev(2) for more information.
+  virtual Status Writev(const struct ::iovec *iov, int iov_len, int64_t *nwritten);
+
+  // Blocking Write call, returns IOError unless full buffer is sent.
+  // Underlying Socket expected to be in blocking mode. Fails if any Write() sends 0 bytes.
+  // Returns OK if buflen bytes were sent, otherwise IOError.
+  // Upon return, nwritten will contain the number of bytes actually written.
+  // See also writen() from Stevens (2004) or Kerrisk (2010)
+  Status BlockingWrite(const uint8_t *buf, size_t buflen, size_t *nwritten,
+      const MonoTime& deadline);
+
+  virtual Status Recv(uint8_t *buf, int32_t amt, int32_t *nread);
+
+  // Blocking Recv call, returns IOError unless requested amt bytes are read.
+  // Underlying Socket expected to be in blocking mode. Fails if any Recv() reads 0 bytes.
+  // Returns OK if amt bytes were read, otherwise IOError.
+  // Upon return, nread will contain the number of bytes actually read.
+  // See also readn() from Stevens (2004) or Kerrisk (2010)
+  Status BlockingRecv(uint8_t *buf, size_t amt, size_t *nread, const MonoTime& deadline);
+
+ private:
+  // Called internally from SetSend/RecvTimeout().
+  Status SetTimeout(int opt, const char* optname, const MonoDelta& timeout);
+
+  // Called internally during socket setup.
+  Status SetCloseOnExec();
+
+  // Bind the socket to a local address before making an outbound connection,
+  // based on the value of FLAGS_local_ip_for_outbound_sockets.
+  Status BindForOutgoingConnection();
+
+  // Set an option on the socket.
+  template<typename T>
+  Status SetSockOpt(int level, int option, const T& value);
+
+  int fd_;
+
+  DISALLOW_COPY_AND_ASSIGN(Socket);
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/nvm_cache.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/nvm_cache.cc b/be/src/kudu/util/nvm_cache.cc
new file mode 100644
index 0000000..ef52ec0
--- /dev/null
+++ b/be/src/kudu/util/nvm_cache.cc
@@ -0,0 +1,577 @@
+// This file is derived from cache.cc in the LevelDB project:
+//
+//   Some portions copyright (c) 2011 The LevelDB Authors. All rights reserved.
+//   Use of this source code is governed by a BSD-style license that can be
+//   found in the LICENSE file.
+//
+// ------------------------------------------------------------
+// This file implements a cache based on the NVML library (http://pmem.io),
+// specifically its "libvmem" component. This library makes it easy to program
+// against persistent memory hardware by exposing an API which parallels
+// malloc/free, but allocates from persistent memory instead of DRAM.
+//
+// We use this API to implement a cache which treats persistent memory or
+// non-volatile memory as if it were a larger cheaper bank of volatile memory. We
+// currently make no use of its persistence properties.
+//
+// Currently, we only store key/value in NVM. All other data structures such as the
+// ShardedLRUCache instances, hash table, etc are in DRAM. The assumption is that
+// the ratio of data stored vs overhead is quite high.
+
+#include "kudu/util/nvm_cache.h"
+
+#include <cstdint>
+#include <cstring>
+#include <iostream>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <libvmem.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/atomic_refcount.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/hash/city.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/cache_metrics.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/slice.h"
+
+DEFINE_string(nvm_cache_path, "/vmem",
+              "The path at which the NVM cache will try to allocate its memory. "
+              "This can be a tmpfs or ramfs for testing purposes.");
+TAG_FLAG(nvm_cache_path, experimental);
+
+DEFINE_int32(nvm_cache_allocation_retry_count, 10,
+             "The number of times that the NVM cache will retry attempts to allocate "
+             "memory for new entries. In between attempts, a cache entry will be "
+             "evicted.");
+TAG_FLAG(nvm_cache_allocation_retry_count, advanced);
+TAG_FLAG(nvm_cache_allocation_retry_count, experimental);
+
+DEFINE_bool(nvm_cache_simulate_allocation_failure, false,
+            "If true, the NVM cache will inject failures in calls to vmem_malloc "
+            "for testing.");
+TAG_FLAG(nvm_cache_simulate_allocation_failure, unsafe);
+
+
+namespace kudu {
+
+namespace {
+
+using std::shared_ptr;
+using std::string;
+using std::vector;
+
+typedef simple_spinlock MutexType;
+
+// LRU cache implementation
+
+// An entry is a variable length heap-allocated structure.  Entries
+// are kept in a circular doubly linked list ordered by access time.
+struct LRUHandle {
+  Cache::EvictionCallback* eviction_callback;
+  LRUHandle* next_hash;
+  LRUHandle* next;
+  LRUHandle* prev;
+  size_t charge;      // TODO(opt): Only allow uint32_t?
+  uint32_t key_length;
+  uint32_t val_length;
+  Atomic32 refs;
+  uint32_t hash;      // Hash of key(); used for fast sharding and comparisons
+  uint8_t* kv_data;
+
+  Slice key() const {
+    return Slice(kv_data, key_length);
+  }
+
+  Slice value() const {
+    return Slice(&kv_data[key_length], val_length);
+  }
+
+  uint8_t* val_ptr() {
+    return &kv_data[key_length];
+  }
+};
+
+// We provide our own simple hash table since it removes a whole bunch
+// of porting hacks and is also faster than some of the built-in hash
+// table implementations in some of the compiler/runtime combinations
+// we have tested.  E.g., readrandom speeds up by ~5% over the g++
+// 4.4.3's builtin hashtable.
+class HandleTable {
+ public:
+  HandleTable() : length_(0), elems_(0), list_(NULL) { Resize(); }
+  ~HandleTable() { delete[] list_; }
+
+  LRUHandle* Lookup(const Slice& key, uint32_t hash) {
+    return *FindPointer(key, hash);
+  }
+
+  LRUHandle* Insert(LRUHandle* h) {
+    LRUHandle** ptr = FindPointer(h->key(), h->hash);
+    LRUHandle* old = *ptr;
+    h->next_hash = (old == NULL ? NULL : old->next_hash);
+    *ptr = h;
+    if (old == NULL) {
+      ++elems_;
+      if (elems_ > length_) {
+        // Since each cache entry is fairly large, we aim for a small
+        // average linked list length (<= 1).
+        Resize();
+      }
+    }
+    return old;
+  }
+
+  LRUHandle* Remove(const Slice& key, uint32_t hash) {
+    LRUHandle** ptr = FindPointer(key, hash);
+    LRUHandle* result = *ptr;
+    if (result != NULL) {
+      *ptr = result->next_hash;
+      --elems_;
+    }
+    return result;
+  }
+
+ private:
+  // The table consists of an array of buckets where each bucket is
+  // a linked list of cache entries that hash into the bucket.
+  uint32_t length_;
+  uint32_t elems_;
+  LRUHandle** list_;
+
+  // Return a pointer to slot that points to a cache entry that
+  // matches key/hash.  If there is no such cache entry, return a
+  // pointer to the trailing slot in the corresponding linked list.
+  LRUHandle** FindPointer(const Slice& key, uint32_t hash) {
+    LRUHandle** ptr = &list_[hash & (length_ - 1)];
+    while (*ptr != NULL &&
+           ((*ptr)->hash != hash || key != (*ptr)->key())) {
+      ptr = &(*ptr)->next_hash;
+    }
+    return ptr;
+  }
+
+  void Resize() {
+    uint32_t new_length = 16;
+    while (new_length < elems_ * 1.5) {
+      new_length *= 2;
+    }
+    LRUHandle** new_list = new LRUHandle*[new_length];
+    memset(new_list, 0, sizeof(new_list[0]) * new_length);
+    uint32_t count = 0;
+    for (uint32_t i = 0; i < length_; i++) {
+      LRUHandle* h = list_[i];
+      while (h != NULL) {
+        LRUHandle* next = h->next_hash;
+        uint32_t hash = h->hash;
+        LRUHandle** ptr = &new_list[hash & (new_length - 1)];
+        h->next_hash = *ptr;
+        *ptr = h;
+        h = next;
+        count++;
+      }
+    }
+    DCHECK_EQ(elems_, count);
+    delete[] list_;
+    list_ = new_list;
+    length_ = new_length;
+  }
+};
+
+// A single shard of sharded cache.
+class NvmLRUCache {
+ public:
+  explicit NvmLRUCache(VMEM *vmp);
+  ~NvmLRUCache();
+
+  // Separate from constructor so caller can easily make an array of LRUCache
+  void SetCapacity(size_t capacity) { capacity_ = capacity; }
+
+  void SetMetrics(CacheMetrics* metrics) { metrics_ = metrics; }
+
+  Cache::Handle* Insert(LRUHandle* h, Cache::EvictionCallback* eviction_callback);
+
+  // Like Cache::Lookup, but with an extra "hash" parameter.
+  Cache::Handle* Lookup(const Slice& key, uint32_t hash, bool caching);
+  void Release(Cache::Handle* handle);
+  void Erase(const Slice& key, uint32_t hash);
+  void* AllocateAndRetry(size_t size);
+
+ private:
+  void NvmLRU_Remove(LRUHandle* e);
+  void NvmLRU_Append(LRUHandle* e);
+  // Just reduce the reference count by 1.
+  // Return true if last reference
+  bool Unref(LRUHandle* e);
+  void FreeEntry(LRUHandle* e);
+
+  // Evict the LRU item in the cache, adding it to the linked list
+  // pointed to by 'to_remove_head'.
+  void EvictOldestUnlocked(LRUHandle** to_remove_head);
+
+  // Free all of the entries in the linked list that has to_free_head
+  // as its head.
+  void FreeLRUEntries(LRUHandle* to_free_head);
+
+  // Wrapper around vmem_malloc which injects failures based on a flag.
+  void* VmemMalloc(size_t size);
+
+  // Initialized before use.
+  size_t capacity_;
+
+  // mutex_ protects the following state.
+  MutexType mutex_;
+  size_t usage_;
+
+  // Dummy head of LRU list.
+  // lru.prev is newest entry, lru.next is oldest entry.
+  LRUHandle lru_;
+
+  HandleTable table_;
+
+  VMEM* vmp_;
+
+  CacheMetrics* metrics_;
+};
+
+NvmLRUCache::NvmLRUCache(VMEM* vmp)
+  : usage_(0),
+  vmp_(vmp),
+  metrics_(NULL) {
+  // Make empty circular linked list
+  lru_.next = &lru_;
+  lru_.prev = &lru_;
+}
+
+NvmLRUCache::~NvmLRUCache() {
+  for (LRUHandle* e = lru_.next; e != &lru_; ) {
+    LRUHandle* next = e->next;
+    DCHECK_EQ(e->refs, 1);  // Error if caller has an unreleased handle
+    if (Unref(e)) {
+      FreeEntry(e);
+    }
+    e = next;
+  }
+}
+
+void* NvmLRUCache::VmemMalloc(size_t size) {
+  if (PREDICT_FALSE(FLAGS_nvm_cache_simulate_allocation_failure)) {
+    return NULL;
+  }
+  return vmem_malloc(vmp_, size);
+}
+
+bool NvmLRUCache::Unref(LRUHandle* e) {
+  DCHECK_GT(ANNOTATE_UNPROTECTED_READ(e->refs), 0);
+  return !base::RefCountDec(&e->refs);
+}
+
+void NvmLRUCache::FreeEntry(LRUHandle* e) {
+  DCHECK_EQ(ANNOTATE_UNPROTECTED_READ(e->refs), 0);
+  if (e->eviction_callback) {
+    e->eviction_callback->EvictedEntry(e->key(), e->value());
+  }
+  if (PREDICT_TRUE(metrics_)) {
+    metrics_->cache_usage->DecrementBy(e->charge);
+    metrics_->evictions->Increment();
+  }
+  vmem_free(vmp_, e);
+}
+
+// Allocate nvm memory. Try until successful or FLAGS_nvm_cache_allocation_retry_count
+// has been exceeded.
+void *NvmLRUCache::AllocateAndRetry(size_t size) {
+  void *tmp;
+  // There may be times that an allocation fails. With NVM we have
+  // a fixed size to allocate from. If we cannot allocate the size
+  // that was asked for, we will remove entries from the cache and
+  // retry up to the configured number of retries. If this fails, we
+  // return NULL, which will cause the caller to not insert anything
+  // into the cache.
+  LRUHandle *to_remove_head = NULL;
+  tmp = VmemMalloc(size);
+
+  if (tmp == NULL) {
+    std::unique_lock<MutexType> l(mutex_);
+
+    int retries_remaining = FLAGS_nvm_cache_allocation_retry_count;
+    while (tmp == NULL && retries_remaining-- > 0 && lru_.next != &lru_) {
+      EvictOldestUnlocked(&to_remove_head);
+
+      // Unlock while allocating memory.
+      l.unlock();
+      tmp = VmemMalloc(size);
+      l.lock();
+    }
+  }
+
+  // we free the entries here outside of mutex for
+  // performance reasons
+  FreeLRUEntries(to_remove_head);
+  return tmp;
+}
+
+void NvmLRUCache::NvmLRU_Remove(LRUHandle* e) {
+  e->next->prev = e->prev;
+  e->prev->next = e->next;
+  usage_ -= e->charge;
+}
+
+void NvmLRUCache::NvmLRU_Append(LRUHandle* e) {
+  // Make "e" newest entry by inserting just before lru_
+  e->next = &lru_;
+  e->prev = lru_.prev;
+  e->prev->next = e;
+  e->next->prev = e;
+  usage_ += e->charge;
+}
+
+Cache::Handle* NvmLRUCache::Lookup(const Slice& key, uint32_t hash, bool caching) {
+ LRUHandle* e;
+  {
+    std::lock_guard<MutexType> l(mutex_);
+    e = table_.Lookup(key, hash);
+    if (e != NULL) {
+      // If an entry exists, remove the old entry from the cache
+      // and re-add to the end of the linked list.
+      base::RefCountInc(&e->refs);
+      NvmLRU_Remove(e);
+      NvmLRU_Append(e);
+    }
+  }
+
+  // Do the metrics outside of the lock.
+  if (metrics_) {
+    metrics_->lookups->Increment();
+    bool was_hit = (e != NULL);
+    if (was_hit) {
+      if (caching) {
+        metrics_->cache_hits_caching->Increment();
+      } else {
+        metrics_->cache_hits->Increment();
+      }
+    } else {
+      if (caching) {
+        metrics_->cache_misses_caching->Increment();
+      } else {
+        metrics_->cache_misses->Increment();
+      }
+    }
+  }
+
+  return reinterpret_cast<Cache::Handle*>(e);
+}
+
+void NvmLRUCache::Release(Cache::Handle* handle) {
+  LRUHandle* e = reinterpret_cast<LRUHandle*>(handle);
+  bool last_reference = Unref(e);
+  if (last_reference) {
+    FreeEntry(e);
+  }
+}
+
+void NvmLRUCache::EvictOldestUnlocked(LRUHandle** to_remove_head) {
+  LRUHandle* old = lru_.next;
+  NvmLRU_Remove(old);
+  table_.Remove(old->key(), old->hash);
+  if (Unref(old)) {
+    old->next = *to_remove_head;
+    *to_remove_head = old;
+  }
+}
+
+void NvmLRUCache::FreeLRUEntries(LRUHandle* to_free_head) {
+  while (to_free_head != NULL) {
+    LRUHandle* next = to_free_head->next;
+    FreeEntry(to_free_head);
+    to_free_head = next;
+  }
+}
+
+Cache::Handle* NvmLRUCache::Insert(LRUHandle* e,
+                                   Cache::EvictionCallback* eviction_callback) {
+  DCHECK(e);
+  LRUHandle* to_remove_head = NULL;
+
+  e->refs = 2;  // One from LRUCache, one for the returned handle
+  e->eviction_callback = eviction_callback;
+  if (PREDICT_TRUE(metrics_)) {
+    metrics_->cache_usage->IncrementBy(e->charge);
+    metrics_->inserts->Increment();
+  }
+
+  {
+    std::lock_guard<MutexType> l(mutex_);
+
+    NvmLRU_Append(e);
+
+    LRUHandle* old = table_.Insert(e);
+    if (old != NULL) {
+      NvmLRU_Remove(old);
+      if (Unref(old)) {
+        old->next = to_remove_head;
+        to_remove_head = old;
+      }
+    }
+
+    while (usage_ > capacity_ && lru_.next != &lru_) {
+      EvictOldestUnlocked(&to_remove_head);
+    }
+  }
+
+  // we free the entries here outside of mutex for
+  // performance reasons
+  FreeLRUEntries(to_remove_head);
+
+  return reinterpret_cast<Cache::Handle*>(e);
+}
+
+void NvmLRUCache::Erase(const Slice& key, uint32_t hash) {
+  LRUHandle* e;
+  bool last_reference = false;
+  {
+    std::lock_guard<MutexType> l(mutex_);
+    e = table_.Remove(key, hash);
+    if (e != NULL) {
+      NvmLRU_Remove(e);
+      last_reference = Unref(e);
+    }
+  }
+  // mutex not held here
+  // last_reference will only be true if e != NULL
+  if (last_reference) {
+    FreeEntry(e);
+  }
+}
+static const int kNumShardBits = 4;
+static const int kNumShards = 1 << kNumShardBits;
+
+class ShardedLRUCache : public Cache {
+ private:
+  gscoped_ptr<CacheMetrics> metrics_;
+  vector<NvmLRUCache*> shards_;
+  VMEM* vmp_;
+
+  static inline uint32_t HashSlice(const Slice& s) {
+    return util_hash::CityHash64(
+      reinterpret_cast<const char *>(s.data()), s.size());
+  }
+
+  static uint32_t Shard(uint32_t hash) {
+    return hash >> (32 - kNumShardBits);
+  }
+
+ public:
+  explicit ShardedLRUCache(size_t capacity, const string& /*id*/, VMEM* vmp)
+        : vmp_(vmp) {
+
+    const size_t per_shard = (capacity + (kNumShards - 1)) / kNumShards;
+    for (int s = 0; s < kNumShards; s++) {
+      gscoped_ptr<NvmLRUCache> shard(new NvmLRUCache(vmp_));
+      shard->SetCapacity(per_shard);
+      shards_.push_back(shard.release());
+    }
+  }
+
+  virtual ~ShardedLRUCache() {
+    STLDeleteElements(&shards_);
+    // Per the note at the top of this file, our cache is entirely volatile.
+    // Hence, when the cache is destructed, we delete the underlying
+    // VMEM pool.
+    vmem_delete(vmp_);
+  }
+
+  virtual Handle* Insert(PendingHandle* handle,
+                         Cache::EvictionCallback* eviction_callback) OVERRIDE {
+    LRUHandle* h = reinterpret_cast<LRUHandle*>(DCHECK_NOTNULL(handle));
+    return shards_[Shard(h->hash)]->Insert(h, eviction_callback);
+  }
+  virtual Handle* Lookup(const Slice& key, CacheBehavior caching) OVERRIDE {
+    const uint32_t hash = HashSlice(key);
+    return shards_[Shard(hash)]->Lookup(key, hash, caching == EXPECT_IN_CACHE);
+  }
+  virtual void Release(Handle* handle) OVERRIDE {
+    LRUHandle* h = reinterpret_cast<LRUHandle*>(handle);
+    shards_[Shard(h->hash)]->Release(handle);
+  }
+  virtual void Erase(const Slice& key) OVERRIDE {
+    const uint32_t hash = HashSlice(key);
+    shards_[Shard(hash)]->Erase(key, hash);
+  }
+  virtual Slice Value(Handle* handle) OVERRIDE {
+    return reinterpret_cast<LRUHandle*>(handle)->value();
+  }
+  virtual uint8_t* MutableValue(PendingHandle* handle) OVERRIDE {
+    return reinterpret_cast<LRUHandle*>(handle)->val_ptr();
+  }
+
+  virtual void SetMetrics(const scoped_refptr<MetricEntity>& entity) OVERRIDE {
+    metrics_.reset(new CacheMetrics(entity));
+    for (NvmLRUCache* cache : shards_) {
+      cache->SetMetrics(metrics_.get());
+    }
+  }
+  virtual PendingHandle* Allocate(Slice key, int val_len, int charge) OVERRIDE {
+    int key_len = key.size();
+    DCHECK_GE(key_len, 0);
+    DCHECK_GE(val_len, 0);
+    LRUHandle* handle = nullptr;
+
+    // Try allocating from each of the shards -- if vmem is tight,
+    // this can cause eviction, so we might have better luck in different
+    // shards.
+    for (NvmLRUCache* cache : shards_) {
+      uint8_t* buf = static_cast<uint8_t*>(cache->AllocateAndRetry(
+          sizeof(LRUHandle) + key_len + val_len));
+      if (buf) {
+        handle = reinterpret_cast<LRUHandle*>(buf);
+        handle->kv_data = &buf[sizeof(LRUHandle)];
+        handle->val_length = val_len;
+        handle->key_length = key_len;
+        handle->charge = (charge == kAutomaticCharge) ?
+            vmem_malloc_usable_size(vmp_, buf) : charge;
+        handle->hash = HashSlice(key);
+        memcpy(handle->kv_data, key.data(), key.size());
+        return reinterpret_cast<PendingHandle*>(handle);
+      }
+    }
+    // TODO: increment a metric here on allocation failure.
+    return nullptr;
+  }
+
+  virtual void Free(PendingHandle* ph) OVERRIDE {
+    vmem_free(vmp_, ph);
+  }
+};
+
+} // end anonymous namespace
+
+Cache* NewLRUNvmCache(size_t capacity, const std::string& id) {
+  // vmem_create() will fail if the capacity is too small, but with
+  // an inscrutable error. So, we'll check ourselves.
+  CHECK_GE(capacity, VMEM_MIN_POOL)
+    << "configured capacity " << capacity << " bytes is less than "
+    << "the minimum capacity for an NVM cache: " << VMEM_MIN_POOL;
+
+  VMEM* vmp = vmem_create(FLAGS_nvm_cache_path.c_str(), capacity);
+  // If we cannot create the cache pool we should not retry.
+  PLOG_IF(FATAL, vmp == NULL) << "Could not initialize NVM cache library in path "
+                              << FLAGS_nvm_cache_path.c_str();
+
+  return new ShardedLRUCache(capacity, id, vmp);
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/nvm_cache.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/nvm_cache.h b/be/src/kudu/util/nvm_cache.h
new file mode 100644
index 0000000..9a65316
--- /dev/null
+++ b/be/src/kudu/util/nvm_cache.h
@@ -0,0 +1,31 @@
+// 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.
+#ifndef KUDU_UTIL_NVM_CACHE_H_
+#define KUDU_UTIL_NVM_CACHE_H_
+
+#include <cstddef>
+#include <string>
+
+namespace kudu {
+class Cache;
+
+// Create a cache in persistent memory with the given capacity.
+Cache* NewLRUNvmCache(size_t capacity, const std::string& id);
+
+}  // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/object_pool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/object_pool-test.cc b/be/src/kudu/util/object_pool-test.cc
new file mode 100644
index 0000000..ecfd641
--- /dev/null
+++ b/be/src/kudu/util/object_pool-test.cc
@@ -0,0 +1,86 @@
+// 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 <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/object_pool.h"
+
+namespace kudu {
+
+// Simple class which maintains a count of how many objects
+// are currently alive.
+class MyClass {
+ public:
+  MyClass() {
+    instance_count_++;
+  }
+
+  ~MyClass() {
+    instance_count_--;
+  }
+
+  static int instance_count() {
+    return instance_count_;
+  }
+
+  static void ResetCount() {
+    instance_count_ = 0;
+  }
+
+ private:
+  static int instance_count_;
+};
+int MyClass::instance_count_ = 0;
+
+TEST(TestObjectPool, TestPooling) {
+  MyClass::ResetCount();
+  {
+    ObjectPool<MyClass> pool;
+    ASSERT_EQ(0, MyClass::instance_count());
+    MyClass *a = pool.Construct();
+    ASSERT_EQ(1, MyClass::instance_count());
+    MyClass *b = pool.Construct();
+    ASSERT_EQ(2, MyClass::instance_count());
+    ASSERT_TRUE(a != b);
+    pool.Destroy(b);
+    ASSERT_EQ(1, MyClass::instance_count());
+    MyClass *c = pool.Construct();
+    ASSERT_EQ(2, MyClass::instance_count());
+    ASSERT_TRUE(c == b) << "should reuse instance";
+    pool.Destroy(c);
+
+    ASSERT_EQ(1, MyClass::instance_count());
+  }
+
+  ASSERT_EQ(0, MyClass::instance_count())
+    << "destructing pool should have cleared instances";
+}
+
+TEST(TestObjectPool, TestScopedPtr) {
+  MyClass::ResetCount();
+  ASSERT_EQ(0, MyClass::instance_count());
+  ObjectPool<MyClass> pool;
+  {
+    ObjectPool<MyClass>::scoped_ptr sptr(
+      pool.make_scoped_ptr(pool.Construct()));
+    ASSERT_EQ(1, MyClass::instance_count());
+  }
+  ASSERT_EQ(0, MyClass::instance_count());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/object_pool.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/object_pool.h b/be/src/kudu/util/object_pool.h
new file mode 100644
index 0000000..64d4b5c
--- /dev/null
+++ b/be/src/kudu/util/object_pool.h
@@ -0,0 +1,166 @@
+// 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.
+//
+// Simple pool/freelist for objects of the same type, typically used
+// in local context.
+#ifndef KUDU_UTIL_OBJECT_POOL_H
+#define KUDU_UTIL_OBJECT_POOL_H
+
+#include <glog/logging.h>
+#include <stdint.h>
+#include "kudu/gutil/manual_constructor.h"
+#include "kudu/gutil/gscoped_ptr.h"
+
+namespace kudu {
+
+template<class T>
+class ReturnToPool;
+
+// An object pool allocates and destroys a single class of objects
+// off of a free-list.
+//
+// Upon destruction of the pool, any objects allocated from this pool are
+// destroyed, regardless of whether they have been explicitly returned to the
+// pool.
+//
+// This class is similar to the boost::pool::object_pool, except that the boost
+// implementation seems to have O(n) deallocation performance and benchmarked
+// really poorly.
+//
+// This class is not thread-safe.
+template<typename T>
+class ObjectPool {
+ public:
+  typedef ReturnToPool<T> deleter_type;
+  typedef gscoped_ptr<T, deleter_type> scoped_ptr;
+
+  ObjectPool() :
+    free_list_head_(NULL),
+    alloc_list_head_(NULL),
+    deleter_(this) {
+  }
+
+  ~ObjectPool() {
+    // Delete all objects ever allocated from this pool
+    ListNode *node = alloc_list_head_;
+    while (node != NULL) {
+      ListNode *tmp = node;
+      node = node->next_on_alloc_list;
+      if (!tmp->is_on_freelist) {
+        // Have to run the actual destructor if the user forgot to free it.
+        tmp->Destroy();
+      }
+      delete tmp;
+    }
+  }
+
+  // Construct a new object instance from the pool.
+  T *Construct() {
+    base::ManualConstructor<T> *obj = GetObject();
+    obj->Init();
+    return obj->get();
+  }
+
+  template<class Arg1>
+  T *Construct(Arg1 arg1) {
+    base::ManualConstructor<T> *obj = GetObject();
+    obj->Init(arg1);
+    return obj->get();
+  }
+
+  // Destroy an object, running its destructor and returning it to the
+  // free-list.
+  void Destroy(T *t) {
+    CHECK_NOTNULL(t);
+    ListNode *node = static_cast<ListNode *>(
+      reinterpret_cast<base::ManualConstructor<T> *>(t));
+
+    node->Destroy();
+
+    DCHECK(!node->is_on_freelist);
+    node->is_on_freelist = true;
+    node->next_on_free_list = free_list_head_;
+    free_list_head_ = node;
+  }
+
+  // Create a scoped_ptr wrapper around the given pointer which came from this
+  // pool.
+  // When the scoped_ptr goes out of scope, the object will get released back
+  // to the pool.
+  scoped_ptr make_scoped_ptr(T *ptr) {
+    return scoped_ptr(ptr, deleter_);
+  }
+
+ private:
+  class ListNode : base::ManualConstructor<T> {
+    friend class ObjectPool<T>;
+
+    ListNode *next_on_free_list;
+    ListNode *next_on_alloc_list;
+
+    bool is_on_freelist;
+  };
+
+
+  base::ManualConstructor<T> *GetObject() {
+    if (free_list_head_ != NULL) {
+      ListNode *tmp = free_list_head_;
+      free_list_head_ = tmp->next_on_free_list;
+      tmp->next_on_free_list = NULL;
+      DCHECK(tmp->is_on_freelist);
+      tmp->is_on_freelist = false;
+
+      return static_cast<base::ManualConstructor<T> *>(tmp);
+    }
+    auto new_node = new ListNode();
+    new_node->next_on_free_list = NULL;
+    new_node->next_on_alloc_list = alloc_list_head_;
+    new_node->is_on_freelist = false;
+    alloc_list_head_ = new_node;
+    return new_node;
+  }
+
+  // Keeps track of free objects in this pool.
+  ListNode *free_list_head_;
+
+  // Keeps track of all objects ever allocated by this pool.
+  ListNode *alloc_list_head_;
+
+  deleter_type deleter_;
+};
+
+// Functor which returns the passed objects to a specific object pool.
+// This can be used in conjunction with scoped_ptr to automatically release
+// an object back to a pool when it goes out of scope.
+template<class T>
+class ReturnToPool {
+ public:
+  explicit ReturnToPool(ObjectPool<T> *pool) :
+    pool_(pool) {
+  }
+
+  inline void operator()(T *ptr) const {
+    pool_->Destroy(ptr);
+  }
+
+ private:
+  ObjectPool<T> *pool_;
+};
+
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/oid_generator-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/oid_generator-test.cc b/be/src/kudu/util/oid_generator-test.cc
new file mode 100644
index 0000000..be88061
--- /dev/null
+++ b/be/src/kudu/util/oid_generator-test.cc
@@ -0,0 +1,52 @@
+// 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 "kudu/util/oid_generator.h"
+
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(ObjectIdGeneratorTest, TestCanoicalizeUuid) {
+  ObjectIdGenerator gen;
+  const string kExpectedCanonicalized = "0123456789abcdef0123456789abcdef";
+  string canonicalized;
+  Status s = gen.Canonicalize("not_a_uuid", &canonicalized);
+  {
+    SCOPED_TRACE(s.ToString());
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), "invalid uuid");
+  }
+  ASSERT_OK(gen.Canonicalize(
+      "01234567-89ab-cdef-0123-456789abcdef", &canonicalized));
+  ASSERT_EQ(kExpectedCanonicalized, canonicalized);
+  ASSERT_OK(gen.Canonicalize(
+      "0123456789abcdef0123456789abcdef", &canonicalized));
+  ASSERT_EQ(kExpectedCanonicalized, canonicalized);
+  ASSERT_OK(gen.Canonicalize(
+      "0123456789AbCdEf0123456789aBcDeF", &canonicalized));
+  ASSERT_EQ(kExpectedCanonicalized, canonicalized);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/oid_generator.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/oid_generator.cc b/be/src/kudu/util/oid_generator.cc
new file mode 100644
index 0000000..eee7316
--- /dev/null
+++ b/be/src/kudu/util/oid_generator.cc
@@ -0,0 +1,65 @@
+// 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 "kudu/util/oid_generator.h"
+
+#include <cstdint>
+#include <exception>
+#include <mutex>
+#include <string>
+
+#include <boost/uuid/uuid.hpp>
+
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+namespace {
+
+string ConvertUuidToString(const boost::uuids::uuid& to_convert) {
+  const uint8_t* uuid = to_convert.data;
+  return StringPrintf("%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x",
+               uuid[0], uuid[1], uuid[2], uuid[3], uuid[4], uuid[5], uuid[6], uuid[7],
+               uuid[8], uuid[9], uuid[10], uuid[11], uuid[12], uuid[13], uuid[14], uuid[15]);
+}
+
+} // anonymous namespace
+
+string ObjectIdGenerator::Next() {
+  std::lock_guard<LockType> l(oid_lock_);
+  boost::uuids::uuid uuid = oid_generator_();
+  return ConvertUuidToString(uuid);
+}
+
+Status ObjectIdGenerator::Canonicalize(const string& input,
+                                       string* output) const {
+  try {
+    boost::uuids::uuid uuid = oid_validator_(input);
+    *output = ConvertUuidToString(uuid);
+    return Status::OK();
+  } catch (std::exception& e) {
+    return Status::InvalidArgument(Substitute("invalid uuid $0: $1",
+                                              input, e.what()));
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/oid_generator.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/oid_generator.h b/be/src/kudu/util/oid_generator.h
new file mode 100644
index 0000000..c1cc88f
--- /dev/null
+++ b/be/src/kudu/util/oid_generator.h
@@ -0,0 +1,63 @@
+// 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.
+
+#ifndef KUDU_UTIL_OID_GENERATOR_H
+#define KUDU_UTIL_OID_GENERATOR_H
+
+#include <string>
+
+#include <boost/uuid/random_generator.hpp>
+#include <boost/uuid/string_generator.hpp>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Generates a unique 32byte id, based on uuid v4.
+// This class is thread safe
+class ObjectIdGenerator {
+ public:
+  ObjectIdGenerator() {}
+  ~ObjectIdGenerator() {}
+
+  // Generates and returns a new UUID.
+  std::string Next();
+
+  // Validates an existing UUID and converts it into the format used by Kudu
+  // (that is, 16 hexadecimal bytes without any dashes).
+  Status Canonicalize(const std::string& input, std::string* output) const;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(ObjectIdGenerator);
+
+  typedef simple_spinlock LockType;
+
+  // Protects 'oid_generator_'.
+  LockType oid_lock_;
+
+  // Generates new UUIDs.
+  boost::uuids::random_generator oid_generator_;
+
+  // Validates provided UUIDs.
+  boost::uuids::string_generator oid_validator_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/once-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/once-test.cc b/be/src/kudu/util/once-test.cc
new file mode 100644
index 0000000..c0a79b1
--- /dev/null
+++ b/be/src/kudu/util/once-test.cc
@@ -0,0 +1,113 @@
+// 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 <ostream>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/once.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/thread.h"
+
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+namespace {
+
+struct Thing {
+  explicit Thing(bool should_fail)
+    : should_fail_(should_fail),
+      value_(0) {
+  }
+
+  Status Init() {
+    return once_.Init(&Thing::InitOnce, this);
+  }
+
+  Status InitOnce() {
+    if (should_fail_) {
+      return Status::IllegalState("Whoops!");
+    }
+    value_ = 1;
+    return Status::OK();
+  }
+
+  const bool should_fail_;
+  int value_;
+  KuduOnceDynamic once_;
+};
+
+} // anonymous namespace
+
+TEST(TestOnce, KuduOnceDynamicTest) {
+  {
+    Thing t(false);
+    ASSERT_EQ(0, t.value_);
+    ASSERT_FALSE(t.once_.init_succeeded());
+
+    for (int i = 0; i < 2; i++) {
+      ASSERT_OK(t.Init());
+      ASSERT_EQ(1, t.value_);
+      ASSERT_TRUE(t.once_.init_succeeded());
+    }
+  }
+
+  {
+    Thing t(true);
+    for (int i = 0; i < 2; i++) {
+      ASSERT_TRUE(t.Init().IsIllegalState());
+      ASSERT_EQ(0, t.value_);
+      ASSERT_FALSE(t.once_.init_succeeded());
+    }
+  }
+}
+
+static void InitOrGetInitted(Thing* t, int i) {
+  if (i % 2 == 0) {
+    LOG(INFO) << "Thread " << i << " initting";
+    t->Init();
+  } else {
+    LOG(INFO) << "Thread " << i << " value: " << t->once_.init_succeeded();
+  }
+}
+
+TEST(TestOnce, KuduOnceDynamicThreadSafeTest) {
+  Thing thing(false);
+
+  // The threads will read and write to thing.once_.initted. If access to
+  // it is not synchronized, TSAN will flag the access as data races.
+  vector<scoped_refptr<Thread> > threads;
+  for (int i = 0; i < 10; i++) {
+    scoped_refptr<Thread> t;
+    ASSERT_OK(Thread::Create("test", Substitute("thread $0", i),
+                             &InitOrGetInitted, &thing, i, &t));
+    threads.push_back(t);
+  }
+
+  for (const scoped_refptr<Thread>& t : threads) {
+    t->Join();
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/once.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/once.cc b/be/src/kudu/util/once.cc
new file mode 100644
index 0000000..fada777
--- /dev/null
+++ b/be/src/kudu/util/once.cc
@@ -0,0 +1,32 @@
+// 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 "kudu/util/once.h"
+
+#include "kudu/util/malloc.h"
+
+namespace kudu {
+
+size_t KuduOnceDynamic::memory_footprint_excluding_this() const {
+  return status_.memory_footprint_excluding_this();
+}
+
+size_t KuduOnceDynamic::memory_footprint_including_this() const {
+  return kudu_malloc_usable_size(this) + memory_footprint_excluding_this();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/once.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/once.h b/be/src/kudu/util/once.h
new file mode 100644
index 0000000..0f43064
--- /dev/null
+++ b/be/src/kudu/util/once.h
@@ -0,0 +1,116 @@
+// 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.
+#ifndef KUDU_UTIL_ONCE_H
+#define KUDU_UTIL_ONCE_H
+
+#include <stddef.h>
+
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class KuduOnceDynamic;
+
+namespace internal {
+
+// Cheap, single-arg "bound callback" (similar to kudu::Callback) for use
+// in KuduOnceDynamic.
+template<typename T>
+struct MemberFunc {
+  KuduOnceDynamic* once;
+  T* instance;
+  Status (T::*member_func)();
+};
+
+template<typename T>
+void InitCb(void* arg) {
+  MemberFunc<T>* mf = reinterpret_cast<MemberFunc<T>*>(arg);
+  mf->once->status_ = (mf->instance->*mf->member_func)();
+  if (PREDICT_TRUE(mf->once->status_.ok())) {
+    mf->once->set_init_succeeded();
+  }
+}
+
+} // namespace internal
+
+// More versatile version of GoogleOnceDynamic, including the following:
+// - Non-static member functions are registered and run via Init().
+// - The first time Init() is called, the registered function is run and the
+//   resulting status is stored.
+// - Regardless of whether Init() succeeded, the function will cease to run on
+//   subsequent calls to Init(), and the stored result will be returned instead.
+// - Access to initialization state is safe for concurrent use.
+class KuduOnceDynamic {
+ public:
+  KuduOnceDynamic()
+    : init_succeeded_(false) {
+  }
+
+  // If the underlying GoogleOnceDynamic has yet to be invoked, invokes the
+  // provided member function and stores its return value. Otherwise,
+  // returns the stored Status.
+  //
+  // T: the type of the member passed in.
+  template<typename T>
+  Status Init(Status (T::*member_func)(), T* instance) {
+    internal::MemberFunc<T> mf = { this, instance, member_func };
+
+    // Clang UBSAN doesn't like it when GoogleOnceDynamic handles the cast
+    // of the argument:
+    //
+    //   runtime error: call to function
+    //   kudu::cfile::BloomFileReader::InitOnceCb(kudu::cfile::BloomFileReader*)
+    //   through pointer to incorrect function type 'void (*)(void *)'
+    //
+    // So let's do the cast ourselves, to void* here and back in InitCb().
+    once_.Init(&internal::InitCb<T>, reinterpret_cast<void*>(&mf));
+    return status_;
+  }
+
+  // kMemOrderAcquire ensures that loads/stores that come after init_succeeded()
+  // aren't reordered to come before it instead. kMemOrderRelease ensures
+  // the opposite (i.e. loads/stores before set_init_succeeded() aren't reordered
+  // to come after it).
+  //
+  // Taken together, threads can safely synchronize on init_succeeded_.
+  bool init_succeeded() const { return init_succeeded_.Load(kMemOrderAcquire); }
+
+  // Returns the memory usage of this object without the object itself. Should
+  // be used when embedded inside another object.
+  size_t memory_footprint_excluding_this() const;
+
+  // Returns the memory usage of this object including the object itself.
+  // Should be used when allocated on the heap.
+  size_t memory_footprint_including_this() const;
+
+ private:
+  template<typename T>
+  friend void internal::InitCb(void* arg);
+
+  void set_init_succeeded() { init_succeeded_.Store(true, kMemOrderRelease); }
+
+  AtomicBool init_succeeded_;
+  GoogleOnceDynamic once_;
+  Status status_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/os-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/os-util-test.cc b/be/src/kudu/util/os-util-test.cc
new file mode 100644
index 0000000..a96e69d
--- /dev/null
+++ b/be/src/kudu/util/os-util-test.cc
@@ -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.
+
+#include "kudu/util/os-util.h"
+
+#include <unistd.h>
+
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/test_macros.h"
+
+using std::string;
+
+namespace kudu {
+
+void RunTest(const string& name, int user_ticks, int kernel_ticks, int io_wait) {
+  string buf = strings::Substitute(string("0 ($0) S 0 0 0 0 0 0 0") +
+                                   " 0 0 0 $1 $2 0 0 0 0 0"         +
+                                   " 0 0 0 0 0 0 0 0 0 0 "          +
+                                   " 0 0 0 0 0 0 0 0 0 0 "          +
+                                   " 0 $3 0 0 0 0 0 0 0 0 "         +
+                                   " 0 0",
+                                   name, user_ticks, kernel_ticks, io_wait);
+  ThreadStats stats;
+  string extracted_name;
+  ASSERT_OK(ParseStat(buf, &extracted_name, &stats));
+  ASSERT_EQ(name, extracted_name);
+  ASSERT_EQ(user_ticks * (1e9 / sysconf(_SC_CLK_TCK)), stats.user_ns);
+  ASSERT_EQ(kernel_ticks * (1e9 / sysconf(_SC_CLK_TCK)), stats.kernel_ns);
+  ASSERT_EQ(io_wait * (1e9 / sysconf(_SC_CLK_TCK)), stats.iowait_ns);
+}
+
+TEST(OsUtilTest, TestSelf) {
+  RunTest("test", 111, 222, 333);
+}
+
+TEST(OsUtilTest, TestSelfNameWithSpace) {
+  RunTest("a space", 111, 222, 333);
+}
+
+TEST(OsUtilTest, TestSelfNameWithParens) {
+  RunTest("a(b(c((d))e)", 111, 222, 333);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/os-util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/os-util.cc b/be/src/kudu/util/os-util.cc
new file mode 100644
index 0000000..df7761f
--- /dev/null
+++ b/be/src/kudu/util/os-util.cc
@@ -0,0 +1,185 @@
+// 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.
+//
+// Imported from Impala. Changes include:
+// - Namespace and imports.
+// - Replaced GetStrErrMsg with ErrnoToString.
+// - Replaced StringParser with strings/numbers.
+// - Fixes for cpplint.
+// - Fixed parsing when thread names have spaces.
+
+#include "kudu/util/os-util.h"
+
+#include <fcntl.h>
+#include <sys/resource.h>
+#include <unistd.h>
+
+#include <cstddef>
+#include <fstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
+
+using std::ifstream;
+using std::istreambuf_iterator;
+using std::ostringstream;
+using std::string;
+using std::vector;
+using strings::Split;
+using strings::Substitute;
+
+namespace kudu {
+
+// Ensure that Impala compiles on earlier kernels. If the target kernel does not support
+// _SC_CLK_TCK, sysconf(_SC_CLK_TCK) will return -1.
+#ifndef _SC_CLK_TCK
+#define _SC_CLK_TCK 2
+#endif
+
+static const int64_t kTicksPerSec = sysconf(_SC_CLK_TCK);
+
+// Offsets into the ../stat file array of per-thread statistics.
+//
+// They are themselves offset by two because the pid and comm fields of the
+// file are parsed separately.
+static const int64_t kUserTicks = 13 - 2;
+static const int64_t kKernelTicks = 14 - 2;
+static const int64_t kIoWait = 41 - 2;
+
+// Largest offset we are interested in, to check we get a well formed stat file.
+static const int64_t kMaxOffset = kIoWait;
+
+Status ParseStat(const std::string& buffer, std::string* name, ThreadStats* stats) {
+  DCHECK(stats != nullptr);
+
+  // The thread name should be the only field with parentheses. But the name
+  // itself may contain parentheses.
+  size_t open_paren = buffer.find('(');
+  size_t close_paren = buffer.rfind(')');
+  if (open_paren == string::npos  ||      // '(' must exist
+      close_paren == string::npos ||      // ')' must exist
+      open_paren >= close_paren   ||      // '(' must come before ')'
+      close_paren + 2 == buffer.size()) { // there must be at least two chars after ')'
+    return Status::IOError("Unrecognised /proc format");
+  }
+  string extracted_name = buffer.substr(open_paren + 1, close_paren - (open_paren + 1));
+  string rest = buffer.substr(close_paren + 2);
+  vector<string> splits = Split(rest, " ", strings::SkipEmpty());
+  if (splits.size() < kMaxOffset) {
+    return Status::IOError("Unrecognised /proc format");
+  }
+
+  int64_t tmp;
+  if (safe_strto64(splits[kUserTicks], &tmp)) {
+    stats->user_ns = tmp * (1e9 / kTicksPerSec);
+  }
+  if (safe_strto64(splits[kKernelTicks], &tmp)) {
+    stats->kernel_ns = tmp * (1e9 / kTicksPerSec);
+  }
+  if (safe_strto64(splits[kIoWait], &tmp)) {
+    stats->iowait_ns = tmp * (1e9 / kTicksPerSec);
+  }
+  if (name != nullptr) {
+    *name = extracted_name;
+  }
+  return Status::OK();
+
+}
+
+Status GetThreadStats(int64_t tid, ThreadStats* stats) {
+  DCHECK(stats != nullptr);
+  if (kTicksPerSec <= 0) {
+    return Status::NotSupported("ThreadStats not supported");
+  }
+
+  ostringstream proc_path;
+  proc_path << "/proc/self/task/" << tid << "/stat";
+  ifstream proc_file(proc_path.str().c_str());
+  if (!proc_file.is_open()) {
+    return Status::IOError("Could not open ifstream");
+  }
+
+  string buffer((istreambuf_iterator<char>(proc_file)),
+      istreambuf_iterator<char>());
+
+  return ParseStat(buffer, nullptr, stats); // don't want the name
+}
+
+void DisableCoreDumps() {
+  struct rlimit lim;
+  PCHECK(getrlimit(RLIMIT_CORE, &lim) == 0);
+  lim.rlim_cur = 0;
+  PCHECK(setrlimit(RLIMIT_CORE, &lim) == 0);
+
+  // Set coredump_filter to not dump any parts of the address space.
+  // Although the above disables core dumps to files, if core_pattern
+  // is set to a pipe rather than a file, it's not sufficient. Setting
+  // this pattern results in piping a very minimal dump into the core
+  // processor (eg abrtd), thus speeding up the crash.
+  int f;
+  RETRY_ON_EINTR(f, open("/proc/self/coredump_filter", O_WRONLY));
+  if (f >= 0) {
+    ssize_t ret;
+    RETRY_ON_EINTR(ret, write(f, "00000000", 8));
+    int close_ret;
+    RETRY_ON_EINTR(close_ret, close(f));
+  }
+}
+
+bool IsBeingDebugged() {
+#ifndef __linux__
+  return false;
+#else
+  // Look for the TracerPid line in /proc/self/status.
+  // If this is non-zero, we are being ptraced, which is indicative of gdb or strace
+  // being attached.
+  faststring buf;
+  Status s = ReadFileToString(Env::Default(), "/proc/self/status", &buf);
+  if (!s.ok()) {
+    KLOG_FIRST_N(WARNING, 1) << "could not read /proc/self/status: " << s.ToString();
+    return false;
+  }
+  StringPiece buf_sp(reinterpret_cast<const char*>(buf.data()), buf.size());
+  vector<StringPiece> lines = Split(buf_sp, "\n");
+  for (const auto& l : lines) {
+    if (!HasPrefixString(l, "TracerPid:")) continue;
+    std::pair<StringPiece, StringPiece> key_val = Split(l, "\t");
+    int64_t tracer_pid = -1;
+    if (!safe_strto64(key_val.second.data(), key_val.second.size(), &tracer_pid)) {
+      KLOG_FIRST_N(WARNING, 1) << "Invalid line in /proc/self/status: " << l;
+      return false;
+    }
+    return tracer_pid != 0;
+  }
+  KLOG_FIRST_N(WARNING, 1) << "Could not find TracerPid line in /proc/self/status";
+  return false;
+#endif // __linux__
+}
+
+} // namespace kudu


[27/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cow_object.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cow_object.h b/be/src/kudu/util/cow_object.h
new file mode 100644
index 0000000..159a8bb
--- /dev/null
+++ b/be/src/kudu/util/cow_object.h
@@ -0,0 +1,437 @@
+// 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.
+#pragma once
+
+#include <algorithm> // IWYU pragma: keep
+#include <map>
+#include <memory>
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/rwc_lock.h"
+
+namespace kudu {
+
+// An object which manages its state via copy-on-write.
+//
+// Access to this object can be done more conveniently using the
+// CowLock template class defined below.
+//
+// The 'State' template parameter must be swappable using std::swap.
+template<class State>
+class CowObject {
+ public:
+  CowObject() {}
+  ~CowObject() {}
+
+  // Lock an object for read.
+  //
+  // While locked, a mutator will be blocked when trying to commit its mutation.
+  void ReadLock() const {
+    lock_.ReadLock();
+  }
+
+  // Return whether the object is locked for read.
+  bool IsReadLocked() const {
+    return lock_.HasReaders();
+  }
+
+  // Unlock an object previously locked for read, unblocking a mutator
+  // actively trying to commit its mutation.
+  void ReadUnlock() const {
+    lock_.ReadUnlock();
+  }
+
+  // Lock the object for write (preventing concurrent mutators).
+  //
+  // We defer making a dirty copy of the state to mutable_dirty() so that the
+  // copy can be avoided if no dirty changes are actually made.
+  void StartMutation() {
+    lock_.WriteLock();
+  }
+
+  // Return whether the object is locked for read and write.
+  bool IsWriteLocked() const {
+    return lock_.HasWriteLock();
+  }
+
+  // Abort the current mutation. This drops the write lock without applying any
+  // changes made to the mutable copy.
+  void AbortMutation() {
+    DCHECK(lock_.HasWriteLock());
+    dirty_state_.reset();
+    lock_.WriteUnlock();
+  }
+
+  // Commit the current mutation. This escalates to the "Commit" lock, which
+  // blocks any concurrent readers or writers, swaps in the new version of the
+  // State, and then drops the commit lock.
+  void CommitMutation() {
+    DCHECK(lock_.HasWriteLock());
+    if (!dirty_state_) {
+      AbortMutation();
+      return;
+    }
+    lock_.UpgradeToCommitLock();
+    std::swap(state_, *dirty_state_);
+    dirty_state_.reset();
+    lock_.CommitUnlock();
+  }
+
+  // Return the current state, not reflecting any in-progress mutations.
+  State& state() {
+    DCHECK(lock_.HasReaders() || lock_.HasWriteLock());
+    return state_;
+  }
+
+  const State& state() const {
+    DCHECK(lock_.HasReaders() || lock_.HasWriteLock());
+    return state_;
+  }
+
+  // Returns the current dirty state (i.e reflecting in-progress mutations).
+  // Should only be called by a thread who previously called StartMutation().
+  State* mutable_dirty() {
+    DCHECK(lock_.HasWriteLock());
+    if (!dirty_state_) {
+      dirty_state_.reset(new State(state_));
+    }
+    return dirty_state_.get();
+  }
+
+  const State& dirty() const {
+    DCHECK(lock_.HasWriteLock());
+    if (!dirty_state_) {
+      return state_;
+    }
+    return *dirty_state_.get();
+  }
+
+ private:
+  mutable RWCLock lock_;
+
+  State state_;
+  std::unique_ptr<State> dirty_state_;
+
+  DISALLOW_COPY_AND_ASSIGN(CowObject);
+};
+
+// Lock state for the following lock-guard-like classes.
+enum class LockMode {
+  // The lock is held for reading.
+  READ,
+
+  // The lock is held for reading and writing.
+  WRITE,
+
+  // The lock is not held.
+  RELEASED
+};
+
+// Defined so LockMode is compatible with DCHECK and the like.
+std::ostream& operator<<(std::ostream& o, LockMode m);
+
+// A lock-guard-like scoped object to acquire the lock on a CowObject,
+// and obtain a pointer to the correct copy to read/write.
+//
+// Example usage:
+//
+//   CowObject<Foo> my_obj;
+//   {
+//     CowLock<Foo> l(&my_obj, LockMode::READ);
+//     l.data().get_foo();
+//     ...
+//   }
+//   {
+//     CowLock<Foo> l(&my_obj, LockMode::WRITE);
+//     l->mutable_data()->set_foo(...);
+//     ...
+//     l.Commit();
+//   }
+template<class State>
+class CowLock {
+ public:
+
+   // An unlocked CowLock. This is useful for default constructing a lock to be
+   // moved in to.
+   CowLock()
+    : cow_(nullptr),
+      mode_(LockMode::RELEASED) {
+   }
+
+  // Lock in either read or write mode.
+  CowLock(CowObject<State>* cow,
+          LockMode mode)
+    : cow_(cow),
+      mode_(mode) {
+    switch (mode) {
+      case LockMode::READ: cow_->ReadLock(); break;
+      case LockMode::WRITE: cow_->StartMutation(); break;
+      default: LOG(FATAL) << "Cannot lock in mode " << mode;
+    }
+  }
+
+  // Lock in read mode.
+  // A const object may not be locked in write mode.
+  CowLock(const CowObject<State>* info,
+          LockMode mode)
+    : cow_(const_cast<CowObject<State>*>(info)),
+      mode_(mode) {
+    switch (mode) {
+      case LockMode::READ: cow_->ReadLock(); break;
+      case LockMode::WRITE: LOG(FATAL) << "Cannot write-lock a const pointer";
+      default: LOG(FATAL) << "Cannot lock in mode " << mode;
+    }
+  }
+
+  // Disable copying.
+  CowLock(const CowLock&) = delete;
+  CowLock& operator=(const CowLock&) = delete;
+
+  // Allow moving.
+  CowLock(CowLock&& other) noexcept
+    : cow_(other.cow_),
+      mode_(other.mode_) {
+    other.cow_ = nullptr;
+    other.mode_ = LockMode::RELEASED;
+  }
+  CowLock& operator=(CowLock&& other) noexcept {
+    cow_ = other.cow_;
+    mode_ = other.mode_;
+    other.cow_ = nullptr;
+    other.mode_ = LockMode::RELEASED;
+    return *this;
+  }
+
+  // Commit the underlying object.
+  // Requires that the caller hold the lock in write mode.
+  void Commit() {
+    DCHECK_EQ(LockMode::WRITE, mode_);
+    cow_->CommitMutation();
+    mode_ = LockMode::RELEASED;
+  }
+
+  void Unlock() {
+    switch (mode_) {
+      case LockMode::READ: cow_->ReadUnlock(); break;
+      case LockMode::WRITE: cow_->AbortMutation(); break;
+      default: DCHECK_EQ(LockMode::RELEASED, mode_); break;
+    }
+    mode_ = LockMode::RELEASED;
+  }
+
+  // Obtain the underlying data. In WRITE mode, this returns the
+  // same data as mutable_data() (not the safe unchanging copy).
+  const State& data() const {
+    switch (mode_) {
+      case LockMode::READ: return cow_->state();
+      case LockMode::WRITE: return cow_->dirty();
+      default: LOG(FATAL) << "Cannot access data after committing";
+    }
+  }
+
+  // Obtain the mutable data. This may only be called in WRITE mode.
+  State* mutable_data() {
+    switch (mode_) {
+      case LockMode::READ: LOG(FATAL) << "Cannot mutate data with READ lock";
+      case LockMode::WRITE: return cow_->mutable_dirty();
+      default: LOG(FATAL) << "Cannot access data after committing";
+    }
+  }
+
+  bool is_write_locked() const {
+    return mode_ == LockMode::WRITE;
+  }
+
+  // Drop the lock. If the lock is held in WRITE mode, and the
+  // lock has not yet been released, aborts the mutation, restoring
+  // the underlying object to its original data.
+  ~CowLock() {
+    Unlock();
+  }
+
+ private:
+  CowObject<State>* cow_;
+  LockMode mode_;
+};
+
+// Scoped object that locks multiple CowObjects for reading or for writing.
+// When locked for writing and mutations are completed, can also commit those
+// mutations, which releases the lock.
+//
+// CowObjects are stored in an std::map, which provides two important properties:
+// 1. AddObject() can deduplicate CowObjects already inserted.
+// 2. When locking for writing, the deterministic iteration order provided by
+//    std::map prevents deadlocks.
+//
+// The use of std::map forces callers to provide a key for each CowObject. For
+// a key implementation to be usable, an appropriate overload of operator<
+// must be available.
+//
+// Unlike CowLock, does not mediate access to the CowObject data itself;
+// callers should access the data out of band.
+//
+// Sample usage:
+//
+//   struct Foo {
+//     string id_;
+//     string data_;
+//   };
+//
+//   vector<CowObject<Foo>> foos;
+//
+// 1. Locking a group of CowObjects for reading:
+//
+//   CowGroupLock<string, Foo> l(LockMode::RELEASED);
+//   for (const auto& f : foos) {
+//     l.AddObject(f.id_, f);
+//   }
+//   l.Lock(LockMode::READ);
+//   for (const auto& f : foos) {
+//     cout << f.state().data_ << endl;
+//   }
+//   l.Unlock();
+//
+// 2. Tracking already-write-locked CowObjects for group commit:
+//
+//   CowGroupLock<string, Foo> l(LockMode::WRITE);
+//   for (const auto& f : foos) {
+//     l.AddObject(f.id_, f);
+//     f.mutable_dirty().data_ = "modified";
+//   }
+//   l.Commit();
+//
+// 3. Aggregating unlocked CowObjects, locking them safely, and committing them together:
+//
+//   CowGroupLock<string, Foo> l(LockMode::RELEASED);
+//   for (const auto& f : foos) {
+//     l.AddObject(f.id_, f);
+//   }
+//   l.Lock(LockMode::WRITE);
+//   for (const auto& f : foos) {
+//     f.mutable_dirty().data_ = "modified";
+//   }
+//   l.Commit();
+template<class Key, class Value>
+class CowGroupLock {
+ public:
+  explicit CowGroupLock(LockMode mode)
+    : mode_(mode) {
+  }
+
+  ~CowGroupLock() {
+    Unlock();
+  }
+
+  void Unlock() {
+    switch (mode_) {
+      case LockMode::READ:
+        for (const auto& e : cows_) {
+          e.second->ReadUnlock();
+        }
+        break;
+      case LockMode::WRITE:
+        for (const auto& e : cows_) {
+          e.second->AbortMutation();
+        }
+        break;
+      default:
+        DCHECK_EQ(LockMode::RELEASED, mode_);
+        break;
+    }
+
+    cows_.clear();
+    mode_ = LockMode::RELEASED;
+  }
+
+  void Lock(LockMode new_mode) {
+    DCHECK_EQ(LockMode::RELEASED, mode_);
+
+    switch (new_mode) {
+      case LockMode::READ:
+        for (const auto& e : cows_) {
+          e.second->ReadLock();
+        }
+        break;
+      case LockMode::WRITE:
+        for (const auto& e : cows_) {
+          e.second->StartMutation();
+        }
+        break;
+      default:
+        LOG(FATAL) << "Cannot lock in mode " << new_mode;
+    }
+    mode_ = new_mode;
+  }
+
+  void Commit() {
+    DCHECK_EQ(LockMode::WRITE, mode_);
+    for (const auto& e : cows_) {
+      e.second->CommitMutation();
+    }
+    cows_.clear();
+    mode_ = LockMode::RELEASED;
+  }
+
+  // Adds a new CowObject to be tracked by the lock guard. Does nothing if a
+  // CowObject with the same key was already added.
+  //
+  // It is the responsibility of the caller to ensure:
+  // 1. That 'object' remains alive until the lock is released.
+  // 2. That if 'object' was already added, both objects point to the same
+  //    memory address.
+  // 3. That if the CowGroupLock is already locked in a particular mode,
+  //    'object' is also already locked in that mode.
+  void AddObject(Key key, const CowObject<Value>* object) {
+    AssertObjectLocked(object);
+    auto r = cows_.emplace(std::move(key), const_cast<CowObject<Value>*>(object));
+    DCHECK_EQ(r.first->second, object);
+  }
+
+  // Like the above, but for mutable objects.
+  void AddMutableObject(Key key, CowObject<Value>* object) {
+    AssertObjectLocked(object);
+    auto r = cows_.emplace(std::move(key), object);
+    DCHECK_EQ(r.first->second, object);
+  }
+
+ private:
+  void AssertObjectLocked(const CowObject<Value>* object) const {
+#ifndef NDEBUG
+    switch (mode_) {
+      case LockMode::READ:
+        DCHECK(object->IsReadLocked());
+        break;
+      case LockMode::WRITE:
+        DCHECK(object->IsWriteLocked());
+        break;
+      default:
+        DCHECK_EQ(LockMode::RELEASED, mode_);
+        break;
+    }
+#endif
+  }
+
+  std::map<Key, CowObject<Value>*> cows_;
+  LockMode mode_;
+
+  DISALLOW_COPY_AND_ASSIGN(CowGroupLock);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/crc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/crc-test.cc b/be/src/kudu/util/crc-test.cc
new file mode 100644
index 0000000..cf13268
--- /dev/null
+++ b/be/src/kudu/util/crc-test.cc
@@ -0,0 +1,112 @@
+// 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 <cstdint>
+#include <cstring>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/crc.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+namespace crc {
+
+using strings::Substitute;
+
+class CrcTest : public KuduTest {
+ protected:
+
+  // Returns pointer to data which must be deleted by caller.
+  static void GenerateBenchmarkData(const uint8_t** bufptr, size_t* buflen) {
+    const uint32_t kNumNumbers = 1000000;
+    const uint32_t kBytesPerNumber = sizeof(uint32_t);
+    const uint32_t kLength = kNumNumbers * kBytesPerNumber;
+    auto buf = new uint8_t[kLength];
+    for (uint32_t i = 0; i < kNumNumbers; i++) {
+      memcpy(buf + (i * kBytesPerNumber), &i, kBytesPerNumber);
+    }
+    *bufptr = buf;
+    *buflen = kLength;
+  }
+
+};
+
+// Basic functionality test.
+TEST_F(CrcTest, TestCRC32C) {
+  const std::string test_data("abcdefgh");
+  const uint64_t kExpectedCrc = 0xa9421b7; // Known value from crcutil usage test program.
+
+  Crc* crc32c = GetCrc32cInstance();
+  uint64_t data_crc = 0;
+  crc32c->Compute(test_data.data(), test_data.length(), &data_crc);
+  char buf[kFastToBufferSize];
+  const char* output = FastHex64ToBuffer(data_crc, buf);
+  LOG(INFO) << "CRC32C of " << test_data << " is: 0x" << output << " (full 64 bits)";
+  output = FastHex32ToBuffer(static_cast<uint32_t>(data_crc), buf);
+  LOG(INFO) << "CRC32C of " << test_data << " is: 0x" << output << " (truncated 32 bits)";
+  ASSERT_EQ(kExpectedCrc, data_crc);
+
+  // Using helper
+  uint64_t data_crc2 = Crc32c(test_data.data(), test_data.length());
+  ASSERT_EQ(kExpectedCrc, data_crc2);
+
+  // Using multiple chunks
+  size_t half_length = test_data.length() / 2;
+  uint64_t data_crc3 = Crc32c(test_data.data(), half_length);
+  data_crc3 = Crc32c(test_data.data() + half_length, half_length, data_crc3);
+  ASSERT_EQ(kExpectedCrc, data_crc3);
+}
+
+// Simple benchmark of CRC32C throughput.
+// We should expect about 8 bytes per cycle in throughput on a single core.
+TEST_F(CrcTest, BenchmarkCRC32C) {
+  gscoped_ptr<const uint8_t[]> data;
+  const uint8_t* buf;
+  size_t buflen;
+  GenerateBenchmarkData(&buf, &buflen);
+  data.reset(buf);
+  Crc* crc32c = GetCrc32cInstance();
+  int kNumRuns = 1000;
+  if (AllowSlowTests()) {
+    kNumRuns = 40000;
+  }
+  const uint64_t kNumBytes = kNumRuns * buflen;
+  Stopwatch sw;
+  sw.start();
+  for (int i = 0; i < kNumRuns; i++) {
+    uint64_t cksum;
+    crc32c->Compute(buf, buflen, &cksum);
+  }
+  sw.stop();
+  CpuTimes elapsed = sw.elapsed();
+  LOG(INFO) << Substitute("$0 runs of CRC32C on $1 bytes of data (total: $2 bytes)"
+                          " in $3 seconds; $4 bytes per millisecond, $5 bytes per nanosecond!",
+                          kNumRuns, buflen, kNumBytes, elapsed.wall_seconds(),
+                          (kNumBytes / elapsed.wall_millis()),
+                          (kNumBytes / elapsed.wall));
+}
+
+} // namespace crc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/crc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/crc.cc b/be/src/kudu/util/crc.cc
new file mode 100644
index 0000000..1534b8d
--- /dev/null
+++ b/be/src/kudu/util/crc.cc
@@ -0,0 +1,56 @@
+// 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 "kudu/util/crc.h"
+
+#include <crcutil/interface.h>
+
+#include "kudu/gutil/once.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+
+namespace kudu {
+namespace crc {
+
+using debug::ScopedLeakCheckDisabler;
+
+static GoogleOnceType crc32c_once = GOOGLE_ONCE_INIT;
+static Crc* crc32c_instance = nullptr;
+
+static void InitCrc32cInstance() {
+  ScopedLeakCheckDisabler disabler; // CRC instance is never freed.
+  // TODO: Is initial = 0 and roll window = 4 appropriate for all cases?
+  crc32c_instance = crcutil_interface::CRC::CreateCrc32c(true, 0, 4, nullptr);
+}
+
+Crc* GetCrc32cInstance() {
+  GoogleOnceInit(&crc32c_once, &InitCrc32cInstance);
+  return crc32c_instance;
+}
+
+uint32_t Crc32c(const void* data, size_t length) {
+  uint64_t crc32 = 0;
+  GetCrc32cInstance()->Compute(data, length, &crc32);
+  return static_cast<uint32_t>(crc32); // Only uses lower 32 bits.
+}
+
+uint32_t Crc32c(const void* data, size_t length, uint32_t prev_crc32) {
+  uint64_t crc_tmp = static_cast<uint64_t>(prev_crc32);
+  GetCrc32cInstance()->Compute(data, length, &crc_tmp);
+  return static_cast<uint32_t>(crc_tmp); // Only uses lower 32 bits.
+}
+
+} // namespace crc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/crc.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/crc.h b/be/src/kudu/util/crc.h
new file mode 100644
index 0000000..a5db4ea
--- /dev/null
+++ b/be/src/kudu/util/crc.h
@@ -0,0 +1,43 @@
+// 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.
+#ifndef KUDU_UTIL_CRC_H_
+#define KUDU_UTIL_CRC_H_
+
+#include <stdint.h>
+#include <stdlib.h>
+
+#include <crcutil/interface.h>
+
+namespace kudu {
+namespace crc {
+
+typedef crcutil_interface::CRC Crc;
+
+// Returns pointer to singleton instance of CRC32C implementation.
+Crc* GetCrc32cInstance();
+
+// Helper function to simply calculate a CRC32C of the given data.
+uint32_t Crc32c(const void* data, size_t length);
+
+// Given CRC value of previous chunk of data,
+// extends it to new chunk and returns the result.
+uint32_t Crc32c(const void* data, size_t length, uint32_t prev_crc32);
+
+} // namespace crc
+} // namespace kudu
+
+#endif // KUDU_UTIL_CRC_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/curl_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/curl_util.cc b/be/src/kudu/util/curl_util.cc
new file mode 100644
index 0000000..4eddb64
--- /dev/null
+++ b/be/src/kudu/util/curl_util.cc
@@ -0,0 +1,130 @@
+// 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 "kudu/util/curl_util.h"
+
+#include <cstddef>
+#include <cstdint>
+#include <ostream>
+
+#include <curl/curl.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/scoped_cleanup.h"
+
+namespace kudu {
+
+namespace {
+
+inline Status TranslateError(CURLcode code) {
+  if (code == CURLE_OK) {
+    return Status::OK();
+  }
+  return Status::NetworkError("curl error", curl_easy_strerror(code));
+}
+
+extern "C" {
+size_t WriteCallback(void* buffer, size_t size, size_t nmemb, void* user_ptr) {
+  size_t real_size = size * nmemb;
+  faststring* buf = reinterpret_cast<faststring*>(user_ptr);
+  CHECK_NOTNULL(buf)->append(reinterpret_cast<const uint8_t*>(buffer), real_size);
+  return real_size;
+}
+} // extern "C"
+
+} // anonymous namespace
+
+EasyCurl::EasyCurl() {
+  // Use our own SSL initialization, and disable curl's.
+  // Both of these calls are idempotent.
+  security::InitializeOpenSSL();
+  CHECK_EQ(0, curl_global_init(CURL_GLOBAL_DEFAULT & ~CURL_GLOBAL_SSL));
+  curl_ = curl_easy_init();
+  CHECK(curl_) << "Could not init curl";
+}
+
+EasyCurl::~EasyCurl() {
+  curl_easy_cleanup(curl_);
+}
+
+Status EasyCurl::FetchURL(const std::string& url, faststring* dst,
+                          const std::vector<std::string>& headers) {
+  return DoRequest(url, nullptr, dst, headers);
+}
+
+Status EasyCurl::PostToURL(const std::string& url,
+                           const std::string& post_data,
+                           faststring* dst) {
+  return DoRequest(url, &post_data, dst);
+}
+
+Status EasyCurl::DoRequest(const std::string& url,
+                           const std::string* post_data,
+                           faststring* dst,
+                           const std::vector<std::string>& headers) {
+  CHECK_NOTNULL(dst)->clear();
+
+  if (!verify_peer_) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(
+        curl_, CURLOPT_SSL_VERIFYHOST, 0)));
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(
+        curl_, CURLOPT_SSL_VERIFYPEER, 0)));
+  }
+
+  // Add headers if specified.
+  struct curl_slist* curl_headers = nullptr;
+  auto clean_up_curl_slist = MakeScopedCleanup([&]() {
+    curl_slist_free_all(curl_headers);
+  });
+
+  for (const auto& header : headers) {
+    curl_headers = CHECK_NOTNULL(curl_slist_append(curl_headers, header.c_str()));
+  }
+  RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_HTTPHEADER, curl_headers)));
+
+  RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_URL, url.c_str())));
+  if (return_headers_) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_HEADER, 1)));
+  }
+  RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_WRITEFUNCTION, WriteCallback)));
+  RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_WRITEDATA,
+                                                static_cast<void *>(dst))));
+  if (post_data) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_POSTFIELDS,
+                                                  post_data->c_str())));
+  }
+
+  RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_HTTPAUTH, CURLAUTH_ANY)));
+  if (timeout_.Initialized()) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_NOSIGNAL, 1)));
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(curl_, CURLOPT_TIMEOUT_MS,
+        timeout_.ToMilliseconds())));
+  }
+  RETURN_NOT_OK(TranslateError(curl_easy_perform(curl_)));
+  long rc; // NOLINT(*) curl wants a long
+  RETURN_NOT_OK(TranslateError(curl_easy_getinfo(curl_, CURLINFO_RESPONSE_CODE, &rc)));
+  if (rc != 200) {
+    return Status::RemoteError(strings::Substitute("HTTP $0", rc));
+  }
+
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/curl_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/curl_util.h b/be/src/kudu/util/curl_util.h
new file mode 100644
index 0000000..cccd2db
--- /dev/null
+++ b/be/src/kudu/util/curl_util.h
@@ -0,0 +1,92 @@
+// 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.
+#ifndef KUDU_UTIL_CURL_UTIL_H
+#define KUDU_UTIL_CURL_UTIL_H
+
+#include <string>
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+typedef void CURL;
+
+namespace kudu {
+
+class faststring;
+
+// Simple wrapper around curl's "easy" interface, allowing the user to
+// fetch web pages into memory using a blocking API.
+//
+// This is not thread-safe.
+class EasyCurl {
+ public:
+  EasyCurl();
+  ~EasyCurl();
+
+  // Fetch the given URL into the provided buffer.
+  // Any existing data in the buffer is replaced.
+  // The optional param 'headers' holds additional headers.
+  // e.g. {"Accept-Encoding: gzip"}
+  Status FetchURL(const std::string& url,
+                  faststring* dst,
+                  const std::vector<std::string>& headers = {});
+
+  // Issue an HTTP POST to the given URL with the given data.
+  // Returns results in 'dst' as above.
+  Status PostToURL(const std::string& url,
+                   const std::string& post_data,
+                   faststring* dst);
+
+  // Set whether to verify the server's SSL certificate in the case of an HTTPS
+  // connection.
+  void set_verify_peer(bool verify) {
+    verify_peer_ = verify;
+  }
+
+  void set_return_headers(bool v) {
+    return_headers_ = v;
+  }
+
+  void set_timeout(MonoDelta t) {
+    timeout_ = t;
+  }
+
+ private:
+  // Do a request. If 'post_data' is non-NULL, does a POST.
+  // Otherwise, does a GET.
+  Status DoRequest(const std::string& url,
+                   const std::string* post_data,
+                   faststring* dst,
+                   const std::vector<std::string>& headers = {});
+  CURL* curl_;
+
+  // Whether to verify the server certificate.
+  bool verify_peer_ = true;
+
+  // Whether to return the HTTP headers with the response.
+  bool return_headers_ = false;
+
+  MonoDelta timeout_;
+
+  DISALLOW_COPY_AND_ASSIGN(EasyCurl);
+};
+
+} // namespace kudu
+
+#endif /* KUDU_UTIL_CURL_UTIL_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug-util-test.cc b/be/src/kudu/util/debug-util-test.cc
new file mode 100644
index 0000000..25e4ae0
--- /dev/null
+++ b/be/src/kudu/util/debug-util-test.cc
@@ -0,0 +1,458 @@
+// 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 <dlfcn.h>
+#ifdef __linux__
+#include <link.h>
+#endif
+#include <unistd.h>
+
+#include <algorithm>
+#include <csignal>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <glog/stl_logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/array_view.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/kernel_stack_watchdog.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+using std::string;
+using std::vector;
+
+DECLARE_int32(test_timeout_after);
+DECLARE_int32(stress_cpu_threads);
+
+namespace kudu {
+
+class DebugUtilTest : public KuduTest {
+};
+
+TEST_F(DebugUtilTest, TestStackTrace) {
+  StackTrace t;
+  t.Collect(0);
+  string trace = t.Symbolize();
+  ASSERT_STR_CONTAINS(trace, "kudu::DebugUtilTest_TestStackTrace_Test::TestBody");
+}
+
+// DumpThreadStack is only supported on Linux, since the implementation relies
+// on the tgkill syscall which is not portable.
+#if defined(__linux__)
+
+namespace {
+void SleeperThread(CountDownLatch* l) {
+  // We use an infinite loop around WaitFor() instead of a normal Wait()
+  // so that this test passes in TSAN. Without this, we run into this TSAN
+  // bug which prevents the sleeping thread from handling signals:
+  // https://code.google.com/p/thread-sanitizer/issues/detail?id=91
+  while (!l->WaitFor(MonoDelta::FromMilliseconds(10))) {
+  }
+}
+
+void fake_signal_handler(int signum) {}
+
+bool IsSignalHandlerRegistered(int signum) {
+  struct sigaction cur_action;
+  CHECK_EQ(0, sigaction(signum, nullptr, &cur_action));
+  return cur_action.sa_handler != SIG_DFL;
+}
+} // anonymous namespace
+
+TEST_F(DebugUtilTest, TestStackTraceInvalidTid) {
+  string s = DumpThreadStack(1);
+  ASSERT_STR_CONTAINS(s, "unable to deliver signal");
+}
+
+TEST_F(DebugUtilTest, TestStackTraceSelf) {
+  string s = DumpThreadStack(Thread::CurrentThreadId());
+  ASSERT_STR_CONTAINS(s, "kudu::DebugUtilTest_TestStackTraceSelf_Test::TestBody()");
+}
+
+TEST_F(DebugUtilTest, TestStackTraceMainThread) {
+  string s = DumpThreadStack(getpid());
+  ASSERT_STR_CONTAINS(s, "kudu::DebugUtilTest_TestStackTraceMainThread_Test::TestBody()");
+}
+
+TEST_F(DebugUtilTest, TestSignalStackTrace) {
+  CountDownLatch l(1);
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "test thread", &SleeperThread, &l, &t));
+  auto cleanup_thr = MakeScopedCleanup([&]() {
+      // Allow the thread to finish.
+      l.CountDown();
+      t->Join();
+    });
+
+  // We have to loop a little bit because it takes a little while for the thread
+  // to start up and actually call our function.
+  ASSERT_EVENTUALLY([&]() {
+      ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), "SleeperThread");
+    });
+
+  // Test that we can change the signal and that the stack traces still work,
+  // on the new signal.
+  ASSERT_FALSE(IsSignalHandlerRegistered(SIGHUP));
+  ASSERT_OK(SetStackTraceSignal(SIGHUP));
+
+  // Should now be registered.
+  ASSERT_TRUE(IsSignalHandlerRegistered(SIGHUP));
+
+  // SIGUSR2 should be relinquished.
+  ASSERT_FALSE(IsSignalHandlerRegistered(SIGUSR2));
+
+  // Stack traces should work using the new handler.
+  ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), "SleeperThread");
+
+  // Switch back to SIGUSR2 and ensure it changes back.
+  ASSERT_OK(SetStackTraceSignal(SIGUSR2));
+  ASSERT_TRUE(IsSignalHandlerRegistered(SIGUSR2));
+  ASSERT_FALSE(IsSignalHandlerRegistered(SIGHUP));
+
+  // Stack traces should work using the new handler.
+  ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), "SleeperThread");
+
+  // Register our own signal handler on SIGHUP, and ensure that
+  // we get a bad Status if we try to use it.
+  signal(SIGHUP, &fake_signal_handler);
+  ASSERT_STR_CONTAINS(SetStackTraceSignal(SIGHUP).ToString(),
+                      "unable to install signal handler");
+  signal(SIGHUP, SIG_DFL);
+
+  // Stack traces should be disabled
+  ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), "unable to take thread stack");
+
+  // Re-enable so that other tests pass.
+  ASSERT_OK(SetStackTraceSignal(SIGUSR2));
+}
+
+// Test which dumps all known threads within this process.
+// We don't validate the results in any way -- but this verifies that we can
+// dump library threads such as the libc timer_thread and properly time out.
+TEST_F(DebugUtilTest, TestSnapshot) {
+  // HACK: prior tests in this suite start threads. Even though they Join on the
+  // threads before the test case finishes, there is actually a very short
+  // period of time after Join() returns but before the actual thread has exited
+  // and removed itself from /proc/self/task/. That means that 'ListThreads' below
+  // can sometimes show these threads from prior test cases, and then the assertions
+  // in this test case would fail.
+  //
+  // So, we have to wait here for the number of running threads to level off to the
+  // expected value.
+  // Ensure Kernel Stack Watchdog is running.
+  KernelStackWatchdog::GetInstance();
+  int initial_thread_count =
+      1 // main thread
+      + 1 // KernelStackWatchdog
+      + (FLAGS_test_timeout_after > 0 ? 1 : 0) // test timeout thread if running
+      + FLAGS_stress_cpu_threads;
+#ifdef THREAD_SANITIZER
+  initial_thread_count++; // tsan signal thread
+#endif
+  // The test and runtime environment runs various utility threads (for example,
+  // the kernel stack watchdog, the TSAN runtime thread, the test timeout thread, etc).
+  // Count them before we start any additional threads for this test.
+  ASSERT_EVENTUALLY([&]{
+      vector<pid_t> threads;
+      ASSERT_OK(ListThreads(&threads));
+      ASSERT_EQ(initial_thread_count, threads.size()) << threads;
+    });
+
+  // Start a bunch of sleeping threads.
+  const int kNumThreads = 30;
+  CountDownLatch l(1);
+  vector<scoped_refptr<Thread>> threads(kNumThreads);
+  for (int i = 0; i < kNumThreads; i++) {
+    ASSERT_OK(Thread::Create("test", "test thread", &SleeperThread, &l, &threads[i]));
+  }
+
+  SCOPED_CLEANUP({
+      // Allow the thread to finish.
+      l.CountDown();
+      for (auto& t : threads) {
+        t->Join();
+      }
+    });
+
+  StackTraceSnapshot snap;
+  ASSERT_OK(snap.SnapshotAllStacks());
+  int count = 0;
+  int groups = 0;
+  snap.VisitGroups([&](ArrayView<StackTraceSnapshot::ThreadInfo> group) {
+      groups++;
+      for (const auto& info : group) {
+        count++;
+        LOG(INFO) << info.tid << " " << info.thread_name
+                  << " (" << info.status.ToString() << ")";
+      }
+      LOG(INFO) << group[0].stack.ToHexString();
+    });
+  int tsan_threads = 0;
+#ifdef THREAD_SANITIZER
+  // TSAN starts an extra thread of its own.
+  tsan_threads++;
+#endif
+  ASSERT_EQ(kNumThreads + initial_thread_count, count);
+  // The threads might not have exactly identical stacks, but
+  // we should have far fewer groups than the total number
+  // of threads.
+  ASSERT_LE(groups, kNumThreads / 2);
+  ASSERT_EQ(tsan_threads, snap.num_failed());
+}
+
+TEST_F(DebugUtilTest, Benchmark) {
+  CountDownLatch l(1);
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "test thread", &SleeperThread, &l, &t));
+  SCOPED_CLEANUP({
+      // Allow the thread to finish.
+      l.CountDown();
+      t->Join();
+    });
+
+  for (bool symbolize : {false, true}) {
+    MonoTime end_time = MonoTime::Now() + MonoDelta::FromSeconds(1);
+    int count = 0;
+    volatile int prevent_optimize = 0;
+    while (MonoTime::Now() < end_time) {
+      StackTrace trace;
+      GetThreadStack(t->tid(), &trace);
+      if (symbolize) {
+        prevent_optimize += trace.Symbolize().size();
+      }
+      count++;
+    }
+    LOG(INFO) << "Throughput: " << count << " dumps/second (symbolize=" << symbolize << ")";
+  }
+}
+
+int TakeStackTrace(struct dl_phdr_info* /*info*/, size_t /*size*/, void* data) {
+  StackTrace* s = reinterpret_cast<StackTrace*>(data);
+  s->Collect(0);
+  return 0;
+}
+
+// Test that if we try to collect a stack trace while inside a libdl function
+// call that we properly return the bogus stack indicating the issue.
+//
+// This doesn't work in ThreadSanitizer since we don't intercept dl_iterate_phdr
+// in those builds (see note in unwind_safeness.cc).
+#ifndef THREAD_SANITIZER
+TEST_F(DebugUtilTest, TestUnwindWhileUnsafe) {
+  StackTrace s;
+  dl_iterate_phdr(&TakeStackTrace, &s);
+  ASSERT_STR_CONTAINS(s.Symbolize(), "CouldNotCollectStackTraceBecauseInsideLibDl");
+}
+#endif
+
+int DoNothingDlCallback(struct dl_phdr_info* /*info*/, size_t /*size*/, void* /*data*/) {
+  return 0;
+}
+
+// Parameterized test which performs various operations which might be dangerous to
+// collect a stack trace while the main thread tries to take stack traces.  These
+// operations are all possibly executed on normal application threads, so we need to
+// ensure that if we happen to gather the stack from a thread in the middle of the
+// function that we don't crash or deadlock.
+//
+// Example self-deadlock if we didn't have the appropriate workarounds in place:
+//  #0  __lll_lock_wait ()
+//  #1  0x00007ffff6f16e42 in __GI___pthread_mutex_lock
+//  #2  0x00007ffff6c8601f in __GI___dl_iterate_phdr
+//  #3  0x0000000000695b02 in dl_iterate_phdr
+//  #4  0x000000000056d013 in _ULx86_64_dwarf_find_proc_info
+//  #5  0x000000000056d1d5 in fetch_proc_info (c=c@ent
+//  #6  0x000000000056e2e7 in _ULx86_64_dwarf_find_save_
+//  #7  0x000000000056c1b9 in _ULx86_64_dwarf_step (c=c@
+//  #8  0x000000000056be21 in _ULx86_64_step
+//  #9  0x0000000000566b1d in google::GetStackTrace
+//  #10 0x00000000004dc4d1 in kudu::StackTrace::Collect
+//  #11 kudu::(anonymous namespace)::HandleStackTraceSignal
+//  #12 <signal handler called>
+//  #13 0x00007ffff6f16e31 in __GI___pthread_mutex_lock
+//  #14 0x00007ffff6c8601f in __GI___dl_iterate_phdr
+//  #15 0x0000000000695b02 in dl_iterate_phdr
+enum DangerousOp {
+  DLOPEN_AND_CLOSE,
+  DL_ITERATE_PHDR,
+  GET_STACK_TRACE,
+  MALLOC_AND_FREE
+};
+class RaceTest : public DebugUtilTest, public ::testing::WithParamInterface<DangerousOp> {};
+INSTANTIATE_TEST_CASE_P(DifferentRaces, RaceTest,
+                        ::testing::Values(DLOPEN_AND_CLOSE,
+                                          DL_ITERATE_PHDR,
+                                          GET_STACK_TRACE,
+                                          MALLOC_AND_FREE));
+
+void DangerousOperationThread(DangerousOp op, CountDownLatch* l) {
+  while (l->count()) {
+    switch (op) {
+      case DLOPEN_AND_CLOSE: {
+        // Check races against dlopen/dlclose.
+        void* v = dlopen("libc.so.6", RTLD_LAZY);
+        CHECK(v);
+        dlclose(v);
+        break;
+      }
+
+      case DL_ITERATE_PHDR: {
+        // Check for races against dl_iterate_phdr.
+        dl_iterate_phdr(&DoNothingDlCallback, nullptr);
+        break;
+      }
+
+      case GET_STACK_TRACE: {
+        // Check for reentrancy issues
+        GetStackTrace();
+        break;
+      }
+
+      case MALLOC_AND_FREE: {
+        // Check large allocations in tcmalloc.
+        volatile char* x = new char[1024 * 1024 * 2];
+        delete[] x;
+        break;
+      }
+      default:
+        LOG(FATAL) << "unknown op";
+    }
+  }
+}
+
+// Starts a thread performing dangerous operations and then gathers
+// its stack trace in a loop trying to trigger races.
+TEST_P(RaceTest, TestStackTraceRaces) {
+  DangerousOp op = GetParam();
+  CountDownLatch l(1);
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "test thread", &DangerousOperationThread, op, &l, &t));
+  SCOPED_CLEANUP({
+      // Allow the thread to finish.
+      l.CountDown();
+      // Crash if we can't join the thread after a reasonable amount of time.
+      // That probably indicates a deadlock.
+      CHECK_OK(ThreadJoiner(t.get()).give_up_after_ms(10000).Join());
+    });
+  MonoTime end_time = MonoTime::Now() + MonoDelta::FromSeconds(1);
+  while (MonoTime::Now() < end_time) {
+    StackTrace trace;
+    GetThreadStack(t->tid(), &trace);
+  }
+}
+
+void BlockSignalsThread() {
+  sigset_t set;
+  sigemptyset(&set);
+  sigaddset(&set, SIGUSR2);
+  for (int i = 0; i < 3; i++) {
+    CHECK_ERR(pthread_sigmask((i % 2) ? SIG_UNBLOCK : SIG_BLOCK, &set, nullptr));
+    SleepFor(MonoDelta::FromSeconds(1));
+  }
+}
+
+TEST_F(DebugUtilTest, TestThreadBlockingSignals) {
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "test thread", &BlockSignalsThread, &t));
+  SCOPED_CLEANUP({ t->Join(); });
+  string ret;
+  while (ret.find("unable to deliver signal") == string::npos) {
+    ret = DumpThreadStack(t->tid());
+    LOG(INFO) << ret;
+  }
+}
+
+// Test stack traces which time out despite the destination thread not blocking
+// signals.
+TEST_F(DebugUtilTest, TestTimeouts) {
+  const int kRunTimeSecs = AllowSlowTests() ? 5 : 1;
+
+  CountDownLatch l(1);
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "test thread", &SleeperThread, &l, &t));
+  auto cleanup_thr = MakeScopedCleanup([&]() {
+      // Allow the thread to finish.
+      l.CountDown();
+      t->Join();
+    });
+
+  // First, time a few stack traces to determine how long a non-timed-out stack
+  // trace takes.
+  vector<MicrosecondsInt64> durations;
+  for (int i = 0; i < 20; i++) {
+    StackTrace stack;
+    auto st = GetMonoTimeMicros();
+    ASSERT_OK(GetThreadStack(t->tid(), &stack));
+    auto dur = GetMonoTimeMicros() - st;
+    durations.push_back(dur);
+  }
+
+  // Compute the median to throw out outliers.
+  std::sort(durations.begin(), durations.end());
+  auto median_duration = durations[durations.size() / 2];
+  LOG(INFO) << "Median duration: " << median_duration << "us";
+
+  // Now take a bunch of stack traces with timeouts clustered around
+  // the expected time. When we time out, we adjust the timeout to be
+  // higher so the next attempt is less likely to time out. Conversely,
+  // when we succeed, we adjust the timeout to be shorter so the next
+  // attempt is more likely to time out. This has the effect of triggering
+  // all the interesting cases: (a) success, (b) timeout, (c) timeout
+  // exactly as the signal finishes.
+  int num_timeouts = 0;
+  int num_successes = 0;
+  auto end_time = MonoTime::Now() + MonoDelta::FromSeconds(kRunTimeSecs);
+  int64_t timeout_us = median_duration;
+  while (MonoTime::Now() < end_time) {
+    StackTraceCollector stc;
+    // Allocate Stack on the heap so that if we get a use-after-free it
+    // will be caught more easily by ASAN.
+    std::unique_ptr<StackTrace> stack(new StackTrace());
+    ASSERT_OK(stc.TriggerAsync(t->tid(), stack.get()));
+    Status s = stc.AwaitCollection(MonoTime::Now() + MonoDelta::FromMicroseconds(timeout_us));
+    if (s.ok()) {
+      num_successes++;
+      timeout_us--;
+    } else if (s.IsTimedOut()) {
+      num_timeouts++;
+      timeout_us++;
+    } else {
+      FAIL() << "Unexpected status: " << s.ToString();
+    }
+  }
+  LOG(INFO) << "Timed out " << num_timeouts << " times";
+  LOG(INFO) << "Succeeded " << num_successes << " times";
+}
+
+#endif
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug-util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug-util.cc b/be/src/kudu/util/debug-util.cc
new file mode 100644
index 0000000..03556d6
--- /dev/null
+++ b/be/src/kudu/util/debug-util.cc
@@ -0,0 +1,800 @@
+// 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 "kudu/util/debug-util.h"
+
+#include <dirent.h>
+#ifndef __linux__
+#include <sched.h>
+#endif
+#ifdef __linux__
+#include <syscall.h>
+#else
+#include <sys/syscall.h>
+#endif
+#include <unistd.h>
+
+#include <algorithm>
+#include <atomic>
+#include <cerrno>
+#include <climits>
+#include <csignal>
+#include <ctime>
+#include <iterator>
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+#include <glog/raw_logging.h>
+#ifdef __linux__
+#define UNW_LOCAL_ONLY
+#include <libunwind.h>
+#endif
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/hash/city.h"
+#include "kudu/gutil/linux_syscall_support.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/array_view.h"
+#include "kudu/util/debug/leak_annotations.h"
+#ifndef __linux__
+#include "kudu/util/debug/sanitizer_scopes.h"
+#endif
+#include "kudu/util/debug/unwind_safeness.h"
+#include "kudu/util/env.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/os-util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/thread.h"
+
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+#if defined(__APPLE__)
+typedef sig_t sighandler_t;
+#endif
+
+// In coverage builds, this symbol will be defined and allows us to flush coverage info
+// to disk before exiting.
+#if defined(__APPLE__)
+  // OS X does not support weak linking at compile time properly.
+  #if defined(COVERAGE_BUILD)
+extern "C" void __gcov_flush() __attribute__((weak_import));
+  #else
+extern "C" void (*__gcov_flush)() = nullptr;
+  #endif
+#else
+extern "C" {
+__attribute__((weak))
+void __gcov_flush();
+}
+#endif
+
+// Evil hack to grab a few useful functions from glog
+namespace google {
+
+extern int GetStackTrace(void** result, int max_depth, int skip_count);
+
+// Symbolizes a program counter.  On success, returns true and write the
+// symbol name to "out".  The symbol name is demangled if possible
+// (supports symbols generated by GCC 3.x or newer).  Otherwise,
+// returns false.
+bool Symbolize(void *pc, char *out, int out_size);
+
+namespace glog_internal_namespace_ {
+extern void DumpStackTraceToString(string *s);
+} // namespace glog_internal_namespace_
+} // namespace google
+
+// The %p field width for printf() functions is two characters per byte.
+// For some environments, add two extra bytes for the leading "0x".
+static const int kPrintfPointerFieldWidth = 2 + 2 * sizeof(void*);
+
+// The signal that we'll use to communicate with our other threads.
+// This can't be in used by other libraries in the process.
+static int g_stack_trace_signum = SIGUSR2;
+
+// Protects g_stack_trace_signum and the installation of the signal
+// handler.
+static base::SpinLock g_signal_handler_lock(base::LINKER_INITIALIZED);
+
+namespace kudu {
+
+bool IsCoverageBuild() {
+  return __gcov_flush != nullptr;
+}
+
+void TryFlushCoverage() {
+  static base::SpinLock lock(base::LINKER_INITIALIZED);
+
+  // Flushing coverage is not reentrant or thread-safe.
+  if (!__gcov_flush || !lock.TryLock()) {
+    return;
+  }
+
+  __gcov_flush();
+
+  lock.Unlock();
+}
+
+
+namespace stack_trace_internal {
+
+// Simple notification mechanism based on futex.
+//
+// We use this instead of a mutex and condvar because we need
+// to signal it from a signal handler, and mutexes are not async-safe.
+//
+// pthread semaphores are async-signal-safe but their timedwait function
+// only supports wall clock waiting, which is a bit dangerous since we
+// need strict timeouts here.
+class CompletionFlag {
+ public:
+
+  // Mark the flag as complete, waking all waiters.
+  void Signal() {
+    complete_ = true;
+#ifndef __APPLE__
+    sys_futex(reinterpret_cast<int32_t*>(&complete_),
+              FUTEX_WAKE | FUTEX_PRIVATE_FLAG,
+              INT_MAX, // wake all
+              0 /* ignored */);
+#endif
+  }
+
+  // Wait for the flag to be marked as complete, up until the given deadline.
+  // Returns true if the flag was marked complete before the deadline.
+  bool WaitUntil(MonoTime deadline) {
+    if (complete_) return true;
+
+    MonoTime now = MonoTime::Now();
+    while (now < deadline) {
+#ifndef __APPLE__
+      MonoDelta rem = deadline - now;
+      struct timespec ts;
+      rem.ToTimeSpec(&ts);
+      sys_futex(reinterpret_cast<int32_t*>(&complete_),
+                FUTEX_WAIT | FUTEX_PRIVATE_FLAG,
+                0, // wait if value is still 0
+                reinterpret_cast<struct kernel_timespec *>(&ts));
+#else
+      sched_yield();
+#endif
+      if (complete_) {
+        return true;
+      }
+      now = MonoTime::Now();
+    }
+    return complete_;
+  }
+
+  void Reset() {
+    complete_ = false;
+  }
+
+  bool complete() const {
+    return complete_;
+  }
+ private:
+  std::atomic<int32_t> complete_ { 0 };
+};
+
+
+// A pointer to this structure is passed as signal data to a thread when
+// a stack trace is being remotely requested.
+//
+// The state machine is as follows (each state is a tuple of 'queued_to_tid'
+// and 'result_ready' status):
+//
+//   [ kNotInUse, false ]
+//           |
+//           | (A)
+//           v                (D)
+//   [ <target tid>, false ]  --->  [ kNotInUse, false ] (leaked)
+//           |
+//           | (B)
+//           v                (E)
+//   [ kDumpStarted, false ]  --->  [ kNotInUse, false ] (tracer waits for 'result_ready')
+//           |                                 |
+//           | (C)                             | (G)
+//           v                (F)              v
+//   [ kDumpStarted, true ]   --->  [ kNotInUse, true ] (already complete)
+//
+// Transitions:
+//    (A): tracer thread sets target_tid before sending a singla
+//    (B): target thread CAS target_tid to kDumpStarted (and aborts on CAS failure)
+//    (C,G): target thread finishes collecting stacks and signals 'result_ready'
+//    (D,E,F): tracer thread exchanges 'kNotInUse' back into queued_to_tid in
+//             RevokeSigData().
+struct SignalData {
+  // The actual destination for the stack trace collected from the target thread.
+  StackTrace* stack;
+
+  static const int kNotInUse = 0;
+  static const int kDumpStarted = -1;
+  // Either one of the above constants, or if the dumper thread
+  // is waiting on a response, the tid that it is waiting on.
+  std::atomic<int64_t> queued_to_tid { kNotInUse };
+
+  // Signaled when the target thread has successfully collected its stack.
+  // The dumper thread waits for this to become true.
+  CompletionFlag result_ready;
+};
+
+} // namespace stack_trace_internal
+
+using stack_trace_internal::SignalData;
+
+namespace {
+
+// Signal handler for our stack trace signal.
+// We expect that the signal is only sent from DumpThreadStack() -- not by a user.
+void HandleStackTraceSignal(int /*signum*/, siginfo_t* info, void* /*ucontext*/) {
+  // Signal handlers may be invoked at any point, so it's important to preserve
+  // errno.
+  int save_errno = errno;
+  SCOPED_CLEANUP({
+      errno = save_errno;
+    });
+  auto* sig_data = reinterpret_cast<SignalData*>(info->si_value.sival_ptr);
+  DCHECK(sig_data);
+  if (!sig_data) {
+    // Maybe the signal was sent by a user instead of by ourself, ignore it.
+    return;
+  }
+  ANNOTATE_HAPPENS_AFTER(sig_data);
+  int64_t my_tid = Thread::CurrentThreadId();
+
+  // If we were slow to process the signal, the sender may have given up and
+  // no longer wants our stack trace. In that case, the 'sig' object will
+  // no longer contain our thread.
+  if (!sig_data->queued_to_tid.compare_exchange_strong(my_tid, SignalData::kDumpStarted)) {
+    return;
+  }
+  // Marking it as kDumpStarted ensures that the caller thread must now wait
+  // for our response, since we are writing directly into their StackTrace object.
+  sig_data->stack->Collect(/*skip_frames=*/1);
+  sig_data->result_ready.Signal();
+}
+
+bool InitSignalHandlerUnlocked(int signum) {
+  enum InitState {
+    UNINITIALIZED,
+    INIT_ERROR,
+    INITIALIZED
+  };
+  static InitState state = UNINITIALIZED;
+
+  // If we've already registered a handler, but we're being asked to
+  // change our signal, unregister the old one.
+  if (signum != g_stack_trace_signum && state == INITIALIZED) {
+    struct sigaction old_act;
+    PCHECK(sigaction(g_stack_trace_signum, nullptr, &old_act) == 0);
+    if (old_act.sa_sigaction == &HandleStackTraceSignal) {
+      signal(g_stack_trace_signum, SIG_DFL);
+    }
+  }
+
+  // If we'd previously had an error, but the signal number
+  // is changing, we should mark ourselves uninitialized.
+  if (signum != g_stack_trace_signum) {
+    g_stack_trace_signum = signum;
+    state = UNINITIALIZED;
+  }
+
+  if (state == UNINITIALIZED) {
+    struct sigaction old_act;
+    PCHECK(sigaction(g_stack_trace_signum, nullptr, &old_act) == 0);
+    if (old_act.sa_handler != SIG_DFL &&
+        old_act.sa_handler != SIG_IGN) {
+      state = INIT_ERROR;
+      LOG(WARNING) << "signal handler for stack trace signal "
+                   << g_stack_trace_signum
+                   << " is already in use: "
+                   << "Kudu will not produce thread stack traces.";
+    } else {
+      // No one appears to be using the signal. This is racy, but there is no
+      // atomic swap capability.
+      struct sigaction act;
+      memset(&act, 0, sizeof(act));
+      act.sa_sigaction = &HandleStackTraceSignal;
+      act.sa_flags = SA_SIGINFO | SA_RESTART;
+      struct sigaction old_act;
+      CHECK_ERR(sigaction(g_stack_trace_signum, &act, &old_act));
+      sighandler_t old_handler = old_act.sa_handler;
+      if (old_handler != SIG_IGN &&
+          old_handler != SIG_DFL) {
+        LOG(FATAL) << "raced against another thread installing a signal handler";
+      }
+      state = INITIALIZED;
+    }
+  }
+  return state == INITIALIZED;
+}
+
+#ifdef __linux__
+GoogleOnceType g_prime_libunwind_once;
+
+void PrimeLibunwind() {
+  // The first call into libunwind does some unsafe double-checked locking
+  // for initialization. So, we make sure that the first call is not concurrent
+  // with any other call.
+  unw_cursor_t cursor;
+  unw_context_t uc;
+  unw_getcontext(&uc);
+  RAW_CHECK(unw_init_local(&cursor, &uc) >= 0, "unw_init_local failed");
+}
+#endif
+} // anonymous namespace
+
+Status SetStackTraceSignal(int signum) {
+  base::SpinLockHolder h(&g_signal_handler_lock);
+  if (!InitSignalHandlerUnlocked(signum)) {
+    return Status::InvalidArgument("unable to install signal handler");
+  }
+  return Status::OK();
+}
+
+StackTraceCollector::StackTraceCollector(StackTraceCollector&& other) noexcept
+    : tid_(other.tid_),
+      sig_data_(other.sig_data_) {
+  other.tid_ = 0;
+  other.sig_data_ = nullptr;
+}
+
+StackTraceCollector::~StackTraceCollector() {
+  if (sig_data_) {
+    RevokeSigData();
+  }
+}
+
+#ifdef __linux__
+bool StackTraceCollector::RevokeSigData() {
+  // First, exchange the atomic variable back to 'not in use'. This ensures
+  // that, if the signalled thread hasn't started filling in the trace yet,
+  // it will see the 'kNotInUse' value and abort.
+  int64_t old_val = sig_data_->queued_to_tid.exchange(SignalData::kNotInUse);
+
+  // We now have two cases to consider.
+
+  // 1) Timed out, but signal still pending and signal handler not yet invoked.
+  //
+  //    In this case, the signal handler hasn't started collecting a stack trace, so when
+  //    we exchange 'queued_to_tid', we see that it is still "queued". In case the signal
+  //    later gets delivered, we can't free the 'sig_data_' struct itself. We intentionally
+  //    leak it. Note, however, that if the signal handler later runs, it will see that we
+  //    exchanged out its tid from 'queued_to_tid' and therefore won't attempt to write
+  //    into the 'stack' structure.
+  if (old_val == tid_) {
+    // TODO(todd) instead of leaking, we can insert these lost structs into a global
+    // free-list, and then reuse them the next time we want to send a signal. The re-use
+    // is safe since access is limited to a specific tid.
+    DLOG(WARNING) << "Leaking SignalData structure " << sig_data_ << " after lost signal "
+                  << "to thread " << tid_;
+    ANNOTATE_LEAKING_OBJECT_PTR(sig_data_);
+    sig_data_ = nullptr;
+    return false;
+  }
+
+  // 2) The signal was delivered. Either the thread is currently collecting its stack
+  //    trace (in which case we have to wait for it to finish), or it has already completed
+  //    (in which case waiting is a no-op).
+  CHECK_EQ(old_val, SignalData::kDumpStarted);
+  CHECK(sig_data_->result_ready.WaitUntil(MonoTime::Max()));
+  delete sig_data_;
+  sig_data_ = nullptr;
+  return true;
+}
+
+
+Status StackTraceCollector::TriggerAsync(int64_t tid, StackTrace* stack) {
+  CHECK(!sig_data_ && tid_ == 0) << "TriggerAsync() must not be called more than once per instance";
+
+  // Ensure that our signal handler is installed.
+  {
+    base::SpinLockHolder h(&g_signal_handler_lock);
+    if (!InitSignalHandlerUnlocked(g_stack_trace_signum)) {
+      return Status::NotSupported("unable to take thread stack: signal handler unavailable");
+    }
+  }
+  // Ensure that libunwind is primed for use before we send any signals. Otherwise
+  // we can hit a deadlock with the following stack:
+  //   GoogleOnceInit()   [waits on the 'once' to finish, but will never finish]
+  //   StackTrace::Collect()
+  //   <signal handler>
+  //   PrimeLibUnwind
+  //   GoogleOnceInit()   [not yet initted, so starts initializing]
+  //   StackTrace::Collect()
+  GoogleOnceInit(&g_prime_libunwind_once, &PrimeLibunwind);
+
+  std::unique_ptr<SignalData> data(new SignalData());
+  // Set the target TID in our communication structure, so if we end up with any
+  // delayed signal reaching some other thread, it will know to ignore it.
+  data->queued_to_tid = tid;
+  data->stack = CHECK_NOTNULL(stack);
+
+  // We use the raw syscall here instead of kill() to ensure that we don't accidentally
+  // send a signal to some other process in the case that the thread has exited and
+  // the TID been recycled.
+  siginfo_t info;
+  memset(&info, 0, sizeof(info));
+  info.si_signo = g_stack_trace_signum;
+  info.si_code = SI_QUEUE;
+  info.si_pid = getpid();
+  info.si_uid = getuid();
+  info.si_value.sival_ptr = data.get();
+  // Since we're using a signal to pass information between the two threads,
+  // we need to help TSAN out and explicitly tell it about the happens-before
+  // relationship here.
+  ANNOTATE_HAPPENS_BEFORE(data.get());
+  if (syscall(SYS_rt_tgsigqueueinfo, getpid(), tid, g_stack_trace_signum, &info) != 0) {
+    return Status::NotFound("unable to deliver signal: process may have exited");
+  }
+
+  // The signal is now pending to the target thread. We don't store it in a unique_ptr
+  // inside the class since we need to be careful to destruct it safely in case the
+  // target thread hasn't yet received the signal when this instance gets destroyed.
+  sig_data_ = data.release();
+  tid_ = tid;
+
+  return Status::OK();
+}
+
+Status StackTraceCollector::AwaitCollection(MonoTime deadline) {
+  CHECK(sig_data_) << "Must successfully call TriggerAsync() first";
+
+  // We give the thread ~1s to respond. In testing, threads typically respond within
+  // a few milliseconds, so this timeout is very conservative.
+  //
+  // The main reason that a thread would not respond is that it has blocked signals. For
+  // example, glibc's timer_thread doesn't respond to our signal, so we always time out
+  // on that one.
+  ignore_result(sig_data_->result_ready.WaitUntil(deadline));
+
+  // Whether or not we timed out above, revoke the signal data structure.
+  // It's possible that the above 'Wait' times out but it succeeds exactly
+  // after that timeout. In that case, RevokeSigData() will return true
+  // and we can return a successful result, because the destination stack trace
+  // has in fact been populated.
+  bool completed = RevokeSigData();
+  if (!completed) {
+    return Status::TimedOut("thread did not respond: maybe it is blocking signals");
+  }
+
+  return Status::OK();
+}
+
+#else  // #ifdef __linux__ ...
+Status StackTraceCollector::TriggerAsync(int64_t tid_, StackTrace* stack) {
+  return Status::NotSupported("unsupported platform");
+}
+Status StackTraceCollector::AwaitCollection(MonoTime deadline) {
+  return Status::NotSupported("unsupported platform");
+}
+bool StackTraceCollector::RevokeSigData() {
+  return false;
+}
+#endif // #ifdef __linux__ ... #else ...
+
+Status GetThreadStack(int64_t tid, StackTrace* stack) {
+  StackTraceCollector c;
+  RETURN_NOT_OK(c.TriggerAsync(tid, stack));
+  RETURN_NOT_OK(c.AwaitCollection(MonoTime::Now() + MonoDelta::FromSeconds(1)));
+  return Status::OK();
+}
+
+string DumpThreadStack(int64_t tid) {
+  StackTrace trace;
+  Status s = GetThreadStack(tid, &trace);
+  if (s.ok()) {
+    return trace.Symbolize();
+  }
+  return strings::Substitute("<$0>", s.ToString());
+}
+
+Status ListThreads(vector<pid_t> *tids) {
+#ifndef __linux__
+  return Status::NotSupported("unable to list threads on this platform");
+#else
+  DIR *dir = opendir("/proc/self/task/");
+  if (dir == NULL) {
+    return Status::IOError("failed to open task dir", ErrnoToString(errno), errno);
+  }
+  struct dirent *d;
+  while ((d = readdir(dir)) != NULL) {
+    if (d->d_name[0] != '.') {
+      uint32_t tid;
+      if (!safe_strtou32(d->d_name, &tid)) {
+        LOG(WARNING) << "bad tid found in procfs: " << d->d_name;
+        continue;
+      }
+      tids->push_back(tid);
+    }
+  }
+  closedir(dir);
+  return Status::OK();
+#endif // __linux__
+}
+
+string GetStackTrace() {
+  string s;
+  google::glog_internal_namespace_::DumpStackTraceToString(&s);
+  return s;
+}
+
+string GetStackTraceHex() {
+  char buf[1024];
+  HexStackTraceToString(buf, 1024);
+  return buf;
+}
+
+void HexStackTraceToString(char* buf, size_t size) {
+  StackTrace trace;
+  trace.Collect(1);
+  trace.StringifyToHex(buf, size);
+}
+
+string GetLogFormatStackTraceHex() {
+  StackTrace trace;
+  trace.Collect(1);
+  return trace.ToLogFormatHexString();
+}
+
+// Bogus empty function which we use below to fill in the stack trace with
+// something readable to indicate that stack trace collection was unavailable.
+void CouldNotCollectStackTraceBecauseInsideLibDl() {
+}
+
+void StackTrace::Collect(int skip_frames) {
+  if (!debug::SafeToUnwindStack()) {
+    // Build a fake stack so that the user sees an appropriate message upon symbolizing
+    // rather than seeing an empty stack.
+    uintptr_t f_ptr = reinterpret_cast<uintptr_t>(&CouldNotCollectStackTraceBecauseInsideLibDl);
+    // Increase the pointer by one byte since the return address from a function call
+    // would not be the beginning of the function itself.
+    frames_[0] = reinterpret_cast<void*>(f_ptr + 1);
+    num_frames_ = 1;
+    return;
+  }
+  const int kMaxDepth = arraysize(frames_);
+
+#ifdef __linux__
+  GoogleOnceInit(&g_prime_libunwind_once, &PrimeLibunwind);
+
+  unw_cursor_t cursor;
+  unw_context_t uc;
+  unw_getcontext(&uc);
+  RAW_CHECK(unw_init_local(&cursor, &uc) >= 0, "unw_init_local failed");
+  skip_frames++;         // Do not include the "Collect" frame
+
+  num_frames_ = 0;
+  while (num_frames_ < kMaxDepth) {
+    void *ip;
+    int ret = unw_get_reg(&cursor, UNW_REG_IP, reinterpret_cast<unw_word_t *>(&ip));
+    if (ret < 0) {
+      break;
+    }
+    if (skip_frames > 0) {
+      skip_frames--;
+    } else {
+      frames_[num_frames_++] = ip;
+    }
+    ret = unw_step(&cursor);
+    if (ret <= 0) {
+      break;
+    }
+  }
+#else
+  // On OSX, use the unwinder from glog. However, that unwinder has an issue where
+  // concurrent invocations will return no frames. See:
+  // https://github.com/google/glog/issues/298
+  // The worst result here is an empty result.
+
+  // google::GetStackTrace has a data race. This is called frequently, so better
+  // to ignore it with an annotation rather than use a suppression.
+  debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan;
+  num_frames_ = google::GetStackTrace(frames_, kMaxDepth, skip_frames + 1);
+#endif
+}
+
+void StackTrace::StringifyToHex(char* buf, size_t size, int flags) const {
+  char* dst = buf;
+
+  // Reserve kHexEntryLength for the first iteration of the loop, 1 byte for a
+  // space (which we may not need if there's just one frame), and 1 for a nul
+  // terminator.
+  char* limit = dst + size - kHexEntryLength - 2;
+  for (int i = 0; i < num_frames_ && dst < limit; i++) {
+    if (i != 0) {
+      *dst++ = ' ';
+    }
+    if (flags & HEX_0X_PREFIX) {
+      *dst++ = '0';
+      *dst++ = 'x';
+    }
+    // See note in Symbolize() below about why we subtract 1 from each address here.
+    uintptr_t addr = reinterpret_cast<uintptr_t>(frames_[i]);
+    if (addr > 0 && !(flags & NO_FIX_CALLER_ADDRESSES)) {
+      addr--;
+    }
+    FastHex64ToBuffer(addr, dst);
+    dst += kHexEntryLength;
+  }
+  *dst = '\0';
+}
+
+string StackTrace::ToHexString(int flags) const {
+  // Each frame requires kHexEntryLength, plus a space
+  // We also need one more byte at the end for '\0'
+  int len_per_frame = kHexEntryLength;
+  len_per_frame++; // For the separating space.
+  if  (flags & HEX_0X_PREFIX) {
+    len_per_frame += 2;
+  }
+  int buf_len = kMaxFrames * len_per_frame + 1;
+  char buf[buf_len];
+  StringifyToHex(buf, buf_len, flags);
+  return string(buf);
+}
+
+// Symbolization function borrowed from glog.
+string StackTrace::Symbolize() const {
+  string ret;
+  for (int i = 0; i < num_frames_; i++) {
+    void* pc = frames_[i];
+
+    char tmp[1024];
+    const char* symbol = "(unknown)";
+
+    // The return address 'pc' on the stack is the address of the instruction
+    // following the 'call' instruction. In the case of calling a function annotated
+    // 'noreturn', this address may actually be the first instruction of the next
+    // function, because the function we care about ends with the 'call'.
+    // So, we subtract 1 from 'pc' so that we're pointing at the 'call' instead
+    // of the return address.
+    //
+    // For example, compiling a C program with -O2 that simply calls 'abort()' yields
+    // the following disassembly:
+    //     Disassembly of section .text:
+    //
+    //     0000000000400440 <main>:
+    //       400440:	48 83 ec 08          	sub    $0x8,%rsp
+    //       400444:	e8 c7 ff ff ff       	callq  400410 <ab...@plt>
+    //
+    //     0000000000400449 <_start>:
+    //       400449:	31 ed                	xor    %ebp,%ebp
+    //       ...
+    //
+    // If we were to take a stack trace while inside 'abort', the return pointer
+    // on the stack would be 0x400449 (the first instruction of '_start'). By subtracting
+    // 1, we end up with 0x400448, which is still within 'main'.
+    //
+    // This also ensures that we point at the correct line number when using addr2line
+    // on logged stacks.
+    //
+    // We check that the pc is not 0 to avoid undefined behavior in the case of
+    // invalid unwinding (see KUDU-2433).
+    if (pc && google::Symbolize(
+            reinterpret_cast<char *>(pc) - 1, tmp, sizeof(tmp))) {
+      symbol = tmp;
+    }
+    StringAppendF(&ret, "    @ %*p  %s\n", kPrintfPointerFieldWidth, pc, symbol);
+  }
+  return ret;
+}
+
+string StackTrace::ToLogFormatHexString() const {
+  string ret;
+  for (int i = 0; i < num_frames_; i++) {
+    void* pc = frames_[i];
+    StringAppendF(&ret, "    @ %*p\n", kPrintfPointerFieldWidth, pc);
+  }
+  return ret;
+}
+
+uint64_t StackTrace::HashCode() const {
+  return util_hash::CityHash64(reinterpret_cast<const char*>(frames_),
+                               sizeof(frames_[0]) * num_frames_);
+}
+
+bool StackTrace::LessThan(const StackTrace& s) const {
+  return std::lexicographical_compare(frames_, &frames_[num_frames_],
+                                      s.frames_, &s.frames_[num_frames_]);
+}
+
+Status StackTraceSnapshot::SnapshotAllStacks() {
+  if (IsBeingDebugged()) {
+    return Status::Incomplete("not collecting stack trace since debugger or strace is attached");
+  }
+
+  vector<pid_t> tids;
+  RETURN_NOT_OK_PREPEND(ListThreads(&tids), "could not list threads");
+
+  collectors_.clear();
+  collectors_.resize(tids.size());
+  infos_.clear();
+  infos_.resize(tids.size());
+  for (int i = 0; i < tids.size(); i++) {
+    infos_[i].tid = tids[i];
+    infos_[i].status = collectors_[i].TriggerAsync(tids[i], &infos_[i].stack);
+  }
+
+  // Now collect the thread names while we are waiting on stack trace collection.
+  if (capture_thread_names_) {
+    for (auto& info : infos_) {
+      if (!info.status.ok()) continue;
+
+      // Get the thread's name by reading proc.
+      // TODO(todd): should we have the dumped thread fill in its own name using
+      // prctl to avoid having to open and read /proc? Or maybe we should use the
+      // Kudu ThreadMgr to get the thread names for the cases where we are using
+      // the kudu::Thread wrapper at least.
+      faststring buf;
+      Status s = ReadFileToString(Env::Default(),
+                                  strings::Substitute("/proc/self/task/$0/comm", info.tid),
+                                  &buf);
+      if (!s.ok()) {
+        info.thread_name = "<unknown name>";
+      }  else {
+        info.thread_name = buf.ToString();
+        StripTrailingNewline(&info.thread_name);
+      }
+    }
+  }
+  num_failed_ = 0;
+  MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(1);
+  for (int i = 0; i < infos_.size(); i++) {
+    infos_[i].status = infos_[i].status.AndThen([&] {
+        return collectors_[i].AwaitCollection(deadline);
+      });
+    if (!infos_[i].status.ok()) {
+      num_failed_++;
+      CHECK(!infos_[i].stack.HasCollected()) << infos_[i].status.ToString();
+    }
+  }
+  collectors_.clear();
+
+  std::sort(infos_.begin(), infos_.end(), [](const ThreadInfo& a, const ThreadInfo& b) {
+      return a.stack.LessThan(b.stack);
+    });
+  return Status::OK();
+}
+
+void StackTraceSnapshot::VisitGroups(const StackTraceSnapshot::VisitorFunc& visitor) {
+  auto group_start = infos_.begin();
+  auto group_end = group_start;
+  while (group_end != infos_.end()) {
+    do {
+      ++group_end;
+    } while (group_end != infos_.end() && group_end->stack.Equals(group_start->stack));
+    visitor(ArrayView<ThreadInfo>(&*group_start, std::distance(group_start, group_end)));
+    group_start = group_end;
+  }
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug-util.h b/be/src/kudu/util/debug-util.h
new file mode 100644
index 0000000..e8c94ea
--- /dev/null
+++ b/be/src/kudu/util/debug-util.h
@@ -0,0 +1,321 @@
+// 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.
+#ifndef KUDU_UTIL_DEBUG_UTIL_H
+#define KUDU_UTIL_DEBUG_UTIL_H
+
+#include <sys/types.h>
+
+#include <cstdint>
+#include <cstring>
+#include <functional>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/fastmem.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+template <typename T> class ArrayView;
+class MonoTime;
+class StackTrace;
+class StackTraceCollector;
+
+namespace stack_trace_internal {
+struct SignalData;
+}
+
+// Return true if coverage is enabled.
+bool IsCoverageBuild();
+
+// Try to flush coverage info. If another thread is already flushing
+// coverage, this returns without doing anything, since flushing coverage
+// is not thread-safe or re-entrant.
+void TryFlushCoverage();
+
+// Return a list of all of the thread IDs currently running in this process.
+// Not async-safe.
+Status ListThreads(std::vector<pid_t>* tids);
+
+// Set which POSIX signal number should be used internally for triggering
+// stack traces. If the specified signal handler is already in use, this
+// returns an error, and stack traces will be disabled.
+Status SetStackTraceSignal(int signum);
+
+// Return the stack trace of the given thread, stringified and symbolized.
+//
+// Note that the symbolization happens on the calling thread, not the target
+// thread, so this is relatively low-impact on the target.
+//
+// This is safe to use against the current thread, the main thread, or any other
+// thread. It requires that the target thread has not blocked POSIX signals. If
+// it has, an error message will be returned.
+//
+// This function is thread-safe.
+//
+// NOTE: if Kudu is running inside a debugger, this can be annoying to a developer since
+// it internally uses signals that will cause the debugger to stop. Consider checking
+// 'IsBeingDebugged()' from os-util.h before using this function for non-critical use
+// cases.
+std::string DumpThreadStack(int64_t tid);
+
+// Capture the thread stack of another thread
+//
+// NOTE: if Kudu is running inside a debugger, this can be annoying to a developer since
+// it internally uses signals that will cause the debugger to stop. Consider checking
+// 'IsBeingDebugged()' from os-util.h before using this function for non-critical use
+// cases.
+Status GetThreadStack(int64_t tid, StackTrace* stack);
+
+// Return the current stack trace, stringified.
+std::string GetStackTrace();
+
+// Return the current stack trace, in hex form. This is significantly
+// faster than GetStackTrace() above, so should be used in performance-critical
+// places like TRACE() calls. If you really need blazing-fast speed, though,
+// use HexStackTraceToString() into a stack-allocated buffer instead --
+// this call causes a heap allocation for the std::string.
+//
+// Note that this is much more useful in the context of a static binary,
+// since addr2line wouldn't know where shared libraries were mapped at
+// runtime.
+std::string GetStackTraceHex();
+
+// This is the same as GetStackTraceHex(), except multi-line in a format that
+// looks very similar to GetStackTrace() but without symbols. Because it's in
+// that format, the tool stacktrace_addr2line.pl in the kudu build-support
+// directory can symbolize it automatically (to the extent that addr2line(1)
+// is able to find the symbols).
+std::string GetLogFormatStackTraceHex();
+
+// Collect the current stack trace in hex form into the given buffer.
+//
+// The resulting trace just includes the hex addresses, space-separated. This is suitable
+// for later stringification by pasting into 'addr2line' for example.
+//
+// This function is async-safe.
+void HexStackTraceToString(char* buf, size_t size);
+
+// Efficient class for collecting and later stringifying a stack trace.
+//
+// Requires external synchronization.
+class StackTrace {
+ public:
+
+  // Constructs a new (uncollected) stack trace.
+  StackTrace()
+    : num_frames_(0) {
+  }
+
+  // Resets the stack trace to an uncollected state.
+  void Reset() {
+    num_frames_ = 0;
+  }
+
+  // Returns true if Collect() (but not Reset()) has been called on this stack trace.
+  bool HasCollected() const {
+    return num_frames_ > 0;
+  }
+
+  // Copies the contents of 's' into this stack trace.
+  void CopyFrom(const StackTrace& s) {
+    memcpy(this, &s, sizeof(s));
+  }
+
+  // Returns true if the stack trace 's' matches this trace.
+  bool Equals(const StackTrace& s) const {
+    return s.num_frames_ == num_frames_ &&
+      strings::memeq(frames_, s.frames_,
+                     num_frames_ * sizeof(frames_[0]));
+  }
+
+  // Comparison operator for use in sorting.
+  bool LessThan(const StackTrace& s) const;
+
+  // Collect and store the current stack trace. Skips the top 'skip_frames' frames
+  // from the stack. For example, a value of '1' will skip whichever function
+  // called the 'Collect()' function. The 'Collect' function itself is always skipped.
+  //
+  // This function is async-safe.
+  void Collect(int skip_frames = 0);
+
+  int num_frames() const {
+    return num_frames_;
+  }
+
+  void* frame(int i) const {
+    DCHECK_LE(i, num_frames_);
+    return frames_[i];
+  }
+
+  enum Flags {
+    // Do not fix up the addresses on the stack to try to point to the 'call'
+    // instructions instead of the return address. This is necessary when dumping
+    // addresses to be interpreted by 'pprof', which does this fix-up itself.
+    NO_FIX_CALLER_ADDRESSES = 1,
+
+    // Prefix each hex address with '0x'. This is required by the go version
+    // of pprof when parsing stack traces.
+    HEX_0X_PREFIX = 1 << 1,
+  };
+
+  // Stringify the trace into the given buffer.
+  // The resulting output is hex addresses suitable for passing into 'addr2line'
+  // later.
+  //
+  // Async-safe.
+  void StringifyToHex(char* buf, size_t size, int flags = 0) const;
+
+  // Same as above, but returning a std::string.
+  // This is not async-safe.
+  std::string ToHexString(int flags = 0) const;
+
+  // Return a string with a symbolized backtrace in a format suitable for
+  // printing to a log file.
+  // This is not async-safe.
+  std::string Symbolize() const;
+
+  // Return a string with a hex-only backtrace in the format typically used in
+  // log files. Similar to the format given by Symbolize(), but symbols are not
+  // resolved (only the hex addresses are given).
+  std::string ToLogFormatHexString() const;
+
+  uint64_t HashCode() const;
+
+ private:
+  enum {
+    // The maximum number of stack frames to collect.
+    kMaxFrames = 16,
+
+    // The max number of characters any frame requires in string form.
+    kHexEntryLength = 16
+  };
+
+  int num_frames_;
+  void* frames_[kMaxFrames];
+};
+
+// Utility class for gathering a process-wide snapshot of the stack traces
+// of all threads.
+class StackTraceSnapshot {
+ public:
+  // The information about each thread will be gathered in a struct.
+  struct ThreadInfo {
+    // The TID of the thread.
+    int64_t tid;
+
+    // The status of collection. If a thread exits during collection or
+    // was blocking signals, it's possible to have an error here.
+    Status status;
+
+    // The name of the thread.
+    // May be missing if 'status' is not OK or if thread name collection was
+    // disabled.
+    std::string thread_name;
+
+    // The current stack trace of the thread.
+    // Always missing if 'status' is not OK.
+    StackTrace stack;
+  };
+  using VisitorFunc = std::function<void(ArrayView<ThreadInfo> group)>;
+
+  void set_capture_thread_names(bool c) {
+    capture_thread_names_ = c;
+  }
+
+  // Snapshot the stack traces of all threads in the process. This may return a bad
+  // Status in the case that stack traces aren't supported on the platform, or if
+  // the process is running inside a debugger.
+  //
+  // NOTE: this may take some time and should not be called in a latency-sensitive
+  // context.
+  Status SnapshotAllStacks();
+
+  // After having collected stacks, visit them, grouped by shared
+  // stack trace. The visitor function will be called once per group.
+  // Each group is guaranteed to be non-empty.
+  //
+  // Any threads which failed to collect traces are returned as a single group
+  // having empty stack traces.
+  //
+  // REQUIRES: a previous successful call to SnapshotAllStacks().
+  void VisitGroups(const VisitorFunc& visitor);
+
+  // Return the number of threads which were interrogated for a stack trace.
+  //
+  // NOTE: this includes threads which failed to collect.
+  int num_threads() const { return infos_.size(); }
+
+  // Return the number of threads which failed to collect a stack trace.
+  int num_failed() const { return num_failed_; }
+
+ private:
+  std::vector<StackTraceSnapshot::ThreadInfo> infos_;
+  std::vector<StackTraceCollector> collectors_;
+  int num_failed_ = 0;
+
+  bool capture_thread_names_ = true;
+};
+
+
+// Class to collect the stack trace of another thread within this process.
+// This allows for more advanced use cases than 'DumpThreadStack(tid)' above.
+// Namely, this provides an asynchronous trigger/collect API so that many
+// stack traces can be collected from many different threads in parallel using
+// different instances of this object.
+class StackTraceCollector {
+ public:
+  StackTraceCollector() = default;
+  StackTraceCollector(StackTraceCollector&& other) noexcept;
+  ~StackTraceCollector();
+
+  // Send the asynchronous request to the the thread with TID 'tid'
+  // to collect its stack trace into '*stack'.
+  //
+  // NOTE: 'stack' must remain a valid pointer until AwaitCollection() has
+  // completed.
+  //
+  // Returns OK if the signal was sent successfully.
+  Status TriggerAsync(int64_t tid, StackTrace* stack);
+
+  // Wait for the stack trace to be collected from the target thread.
+  //
+  // REQUIRES: TriggerAsync() has returned successfully.
+  Status AwaitCollection(MonoTime deadline);
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(StackTraceCollector);
+
+  // Safely sets 'sig_data_' back to nullptr after having sent an asynchronous
+  // stack trace request. See implementation for details.
+  //
+  // Returns true if the stack trace was collected before revocation
+  // and false if it was not.
+  //
+  // POSTCONDITION: sig_data_ == nullptr
+  bool RevokeSigData();
+
+  int64_t tid_ = 0;
+  stack_trace_internal::SignalData* sig_data_ = nullptr;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/leak_annotations.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/leak_annotations.h b/be/src/kudu/util/debug/leak_annotations.h
new file mode 100644
index 0000000..2bfc3d8
--- /dev/null
+++ b/be/src/kudu/util/debug/leak_annotations.h
@@ -0,0 +1,84 @@
+// 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.
+#ifndef KUDU_UTIL_DEBUG_LEAK_ANNOTATIONS_H_
+#define KUDU_UTIL_DEBUG_LEAK_ANNOTATIONS_H_
+
+// Ignore a single leaked object, given its pointer.
+// Does nothing if LeakSanitizer is not enabled.
+#define ANNOTATE_LEAKING_OBJECT_PTR(p)
+
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+#    if defined(__linux__)
+
+#undef ANNOTATE_LEAKING_OBJECT_PTR
+#define ANNOTATE_LEAKING_OBJECT_PTR(p) __lsan_ignore_object(p);
+
+#    endif
+#  endif
+#endif
+
+// API definitions from LLVM lsan_interface.h
+
+extern "C" {
+  // Allocations made between calls to __lsan_disable() and __lsan_enable() will
+  // be treated as non-leaks. Disable/enable pairs may be nested.
+  void __lsan_disable();
+  void __lsan_enable();
+
+  // The heap object into which p points will be treated as a non-leak.
+  void __lsan_ignore_object(const void *p);
+
+  // The user may optionally provide this function to disallow leak checking
+  // for the program it is linked into (if the return value is non-zero). This
+  // function must be defined as returning a constant value; any behavior beyond
+  // that is unsupported.
+  int __lsan_is_turned_off();
+
+  // Check for leaks now. This function behaves identically to the default
+  // end-of-process leak check. In particular, it will terminate the process if
+  // leaks are found and the exitcode runtime flag is non-zero.
+  // Subsequent calls to this function will have no effect and end-of-process
+  // leak check will not run. Effectively, end-of-process leak check is moved to
+  // the time of first invocation of this function.
+  // By calling this function early during process shutdown, you can instruct
+  // LSan to ignore shutdown-only leaks which happen later on.
+  void __lsan_do_leak_check();
+
+  // Check for leaks now. Returns zero if no leaks have been found or if leak
+  // detection is disabled, non-zero otherwise.
+  // This function may be called repeatedly, e.g. to periodically check a
+  // long-running process. It prints a leak report if appropriate, but does not
+  // terminate the process. It does not affect the behavior of
+  // __lsan_do_leak_check() or the end-of-process leak check, and is not
+  // affected by them.
+  int __lsan_do_recoverable_leak_check();
+} // extern "C"
+
+namespace kudu {
+namespace debug {
+
+class ScopedLSANDisabler {
+ public:
+  ScopedLSANDisabler() { __lsan_disable(); }
+  ~ScopedLSANDisabler() { __lsan_enable(); }
+};
+
+} // namespace debug
+} // namespace kudu
+
+#endif  // KUDU_UTIL_DEBUG_LEAK_ANNOTATIONS_H_


[20/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_tags-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_tags-test.cc b/be/src/kudu/util/flag_tags-test.cc
new file mode 100644
index 0000000..4626d0d
--- /dev/null
+++ b/be/src/kudu/util/flag_tags-test.cc
@@ -0,0 +1,135 @@
+// 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 <string>
+#include <unordered_set>
+#include <vector>
+
+#include <gtest/gtest.h>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/logging_test_util.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_bool(never_fsync);
+
+DEFINE_int32(flag_with_no_tags, 0, "test flag that has no tags");
+
+DEFINE_int32(flag_with_one_tag, 0, "test flag that has 1 tag");
+TAG_FLAG(flag_with_one_tag, stable);
+
+DEFINE_int32(flag_with_two_tags, 0, "test flag that has 2 tags");
+TAG_FLAG(flag_with_two_tags, evolving);
+TAG_FLAG(flag_with_two_tags, unsafe);
+
+DEFINE_bool(test_unsafe_flag, false, "an unsafe flag");
+TAG_FLAG(test_unsafe_flag, unsafe);
+
+DEFINE_bool(test_experimental_flag, false, "an experimental flag");
+TAG_FLAG(test_experimental_flag, experimental);
+
+DEFINE_bool(test_sensitive_flag, false, "a sensitive flag");
+TAG_FLAG(test_sensitive_flag, sensitive);
+
+using std::string;
+using std::unordered_set;
+
+namespace kudu {
+
+class FlagTagsTest : public KuduTest {};
+
+TEST_F(FlagTagsTest, TestTags) {
+  unordered_set<string> tags;
+  GetFlagTags("flag_with_no_tags", &tags);
+  EXPECT_EQ(0, tags.size());
+
+  GetFlagTags("flag_with_one_tag", &tags);
+  EXPECT_EQ(1, tags.size());
+  EXPECT_TRUE(ContainsKey(tags, "stable"));
+
+  GetFlagTags("flag_with_two_tags", &tags);
+  EXPECT_EQ(2, tags.size());
+  EXPECT_TRUE(ContainsKey(tags, "evolving"));
+  EXPECT_TRUE(ContainsKey(tags, "unsafe"));
+
+  GetFlagTags("missing_flag", &tags);
+  EXPECT_EQ(0, tags.size());
+}
+
+TEST_F(FlagTagsTest, TestUnlockFlags) {
+  // Setting an unsafe flag without unlocking should crash.
+  {
+    gflags::FlagSaver s;
+    gflags::SetCommandLineOption("test_unsafe_flag", "true");
+    ASSERT_DEATH({ HandleCommonFlags(); },
+                 "Flag --test_unsafe_flag is unsafe and unsupported.*"
+                 "Use --unlock_unsafe_flags to proceed");
+  }
+
+  // Setting an unsafe flag with unlocking should proceed with a warning.
+  {
+    StringVectorSink sink;
+    ScopedRegisterSink reg(&sink);
+    gflags::FlagSaver s;
+    gflags::SetCommandLineOption("test_unsafe_flag", "true");
+    gflags::SetCommandLineOption("unlock_unsafe_flags", "true");
+    HandleCommonFlags();
+    ASSERT_EQ(1, sink.logged_msgs().size());
+    ASSERT_STR_CONTAINS(sink.logged_msgs()[0], "Enabled unsafe flag: --test_unsafe_flag");
+  }
+
+  // Setting an experimental flag without unlocking should crash.
+  {
+    gflags::FlagSaver s;
+    gflags::SetCommandLineOption("test_experimental_flag", "true");
+    ASSERT_DEATH({ HandleCommonFlags(); },
+                 "Flag --test_experimental_flag is experimental and unsupported.*"
+                 "Use --unlock_experimental_flags to proceed");
+  }
+
+  // Setting an experimental flag with unlocking should proceed with a warning.
+  {
+    StringVectorSink sink;
+    ScopedRegisterSink reg(&sink);
+    gflags::FlagSaver s;
+    gflags::SetCommandLineOption("test_experimental_flag", "true");
+    gflags::SetCommandLineOption("unlock_experimental_flags", "true");
+    HandleCommonFlags();
+    ASSERT_EQ(1, sink.logged_msgs().size());
+    ASSERT_STR_CONTAINS(sink.logged_msgs()[0],
+                        "Enabled experimental flag: --test_experimental_flag");
+  }
+}
+
+TEST_F(FlagTagsTest, TestSensitiveFlags) {
+  // Setting a sensitive flag should return a redacted value.
+  {
+    kudu::g_should_redact = kudu::RedactContext::LOG;
+    ASSERT_STR_CONTAINS(CommandlineFlagsIntoString(EscapeMode::NONE), strings::Substitute(
+                        "--test_sensitive_flag=$0", kRedactionMessage));
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_tags.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_tags.cc b/be/src/kudu/util/flag_tags.cc
new file mode 100644
index 0000000..7e11ea2
--- /dev/null
+++ b/be/src/kudu/util/flag_tags.cc
@@ -0,0 +1,91 @@
+// 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 "kudu/util/flag_tags.h"
+
+#include <map>
+#include <ostream>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/singleton.h"
+
+using std::multimap;
+using std::pair;
+using std::string;
+using std::unordered_set;
+using std::vector;
+
+namespace kudu {
+namespace flag_tags_internal {
+
+// Singleton registry storing the set of tags for each flag.
+class FlagTagRegistry {
+ public:
+  static FlagTagRegistry* GetInstance() {
+    return Singleton<FlagTagRegistry>::get();
+  }
+
+  void Tag(const string& name, const string& tag) {
+    tag_map_.insert(TagMap::value_type(name, tag));
+  }
+
+  void GetTags(const string& name, unordered_set<string>* tags) {
+    tags->clear();
+    pair<TagMap::const_iterator, TagMap::const_iterator> range =
+      tag_map_.equal_range(name);
+    for (auto it = range.first; it != range.second; ++it) {
+      if (!InsertIfNotPresent(tags, it->second)) {
+        LOG(DFATAL) << "Flag " << name << " was tagged more than once with the tag '"
+                    << it->second << "'";
+      }
+    }
+  }
+
+ private:
+  friend class Singleton<FlagTagRegistry>;
+  FlagTagRegistry() {}
+
+  typedef multimap<string, string> TagMap;
+  TagMap tag_map_;
+
+  DISALLOW_COPY_AND_ASSIGN(FlagTagRegistry);
+};
+
+
+FlagTagger::FlagTagger(const char* name, const char* tag) {
+  FlagTagRegistry::GetInstance()->Tag(name, tag);
+}
+
+FlagTagger::~FlagTagger() {
+}
+
+} // namespace flag_tags_internal
+
+using flag_tags_internal::FlagTagRegistry;
+
+void GetFlagTags(const string& flag_name,
+                 unordered_set<string>* tags) {
+  FlagTagRegistry::GetInstance()->GetTags(flag_name, tags);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_tags.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_tags.h b/be/src/kudu/util/flag_tags.h
new file mode 100644
index 0000000..bf0c400
--- /dev/null
+++ b/be/src/kudu/util/flag_tags.h
@@ -0,0 +1,169 @@
+// 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.
+//
+// Flag Tags provide a way to attach arbitrary textual tags to gflags in
+// a global registry. Kudu uses the following flag tags:
+//
+// - "stable":
+//         These flags are considered user-facing APIs. Therefore, the
+//         semantics of the flag should not be changed except between major
+//         versions. Similarly, they must not be removed except between major
+//         versions.
+//
+// - "evolving":
+//         These flags are considered user-facing APIs, but are not yet
+//         locked down. For example, they may pertain to a newly introduced
+//         feature that is still being actively developed. These may be changed
+//         between minor versions, but should be suitably release-noted.
+//
+//         This is the default assumed stability level, but can be tagged
+//         if you'd like to make it explicit.
+//
+// - "experimental":
+//         These flags are considered user-facing APIs, but are related to
+//         an experimental feature, or otherwise likely to change or be
+//         removed at any point. Users should not expect any compatibility
+//         of these flags.
+//
+//         Users must pass --unlock_experimental_flags to use any of these
+//         flags.
+//
+// - "hidden":
+//         These flags are for internal use only (e.g. testing) and should
+//         not be included in user-facing documentation.
+//
+// - "advanced":
+//         These flags are for advanced users or debugging purposes. While
+//         they aren't likely to be actively harmful (see "unsafe" below),
+//         they're also likely to be used only rarely and should be relegated
+//         to more detailed sections of documentation.
+//
+// - "unsafe":
+//         These flags are for internal use only (e.g. testing), and changing
+//         them away from the defaults may result in arbitrarily bad things
+//         happening. These flags are automatically excluded from user-facing
+//         documentation even if they are not also marked 'hidden'.
+//
+//         Users must pass --unlock_unsafe_flags to use any of these
+//         flags.
+//
+// - "runtime":
+//         These flags can be safely changed at runtime via an RPC to the
+//         server. Changing a flag at runtime that does not have this tag is allowed
+//         only if the user specifies a "force_unsafe_change" flag in the RPC.
+//
+//         NOTE: because gflags are simple global variables, it's important to
+//         think very carefully before tagging a flag with 'runtime'. In particular,
+//         if a string-type flag is marked 'runtime', you should never access it
+//         using the raw 'FLAGS_foo_bar' name. Instead, you must use the
+//         google::GetCommandLineFlagInfo(...) API to make a copy of the flag value
+//         under a lock. Otherwise, the 'std::string' instance could be mutated
+//         underneath the reader causing a crash.
+//
+//         For primitive-type flags, we assume that reading a variable is atomic.
+//         That is to say that a reader will either see the old value or the new
+//         one, but not some invalid value. However, for the runtime change to
+//         have any effect, you must be sure to use the FLAGS_foo_bar variable directly
+//         rather than initializing some instance variable during program startup.
+//
+// - "sensitive":
+//         The values of these flags are considered sensitive and will be redacted
+//         if redaction is enabled.
+//
+// A given flag may have zero or more tags associated with it. The system does
+// not make any attempt to check integrity of the tags - for example, it allows
+// you to mark a flag as both stable and unstable, even though this makes no
+// real sense. Nevertheless, you should strive to meet the following requirements:
+//
+// - A flag should have exactly no more than one of stable/evolving/experimental
+//   indicating its stability. 'evolving' is considered the default.
+// - A flag should have no more than one of advanced/hidden indicating visibility
+//   in documentation. If neither is specified, the flag will be in the main
+//   section of the documentation.
+// - It is likely that most 'experimental' flags will also be 'advanced' or 'hidden',
+//   and that 'stable' flags are not likely to be 'hidden' or 'unsafe'.
+//
+// To add a tag to a flag, use the TAG_FLAG macro. For example:
+//
+//  DEFINE_bool(sometimes_crash, false, "This flag makes Kudu crash a lot");
+//  TAG_FLAG(sometimes_crash, unsafe);
+//  TAG_FLAG(sometimes_crash, runtime);
+//
+// To fetch the list of tags associated with a flag, use 'GetFlagTags'.
+
+#ifndef KUDU_UTIL_FLAG_TAGS_H
+#define KUDU_UTIL_FLAG_TAGS_H
+
+#include "kudu/gutil/macros.h"
+
+#include <string>
+#include <unordered_set>
+
+namespace kudu {
+
+struct FlagTags {
+  enum {
+    stable,
+    evolving,
+    experimental,
+    hidden,
+    advanced,
+    unsafe,
+    runtime,
+    sensitive
+  };
+};
+
+// Tag the flag 'flag_name' with the given tag 'tag'.
+//
+// This verifies that 'flag_name' is a valid gflag, which must be defined
+// or declared above the use of the TAG_FLAG macro.
+//
+// This also validates that 'tag' is a valid flag as defined in the FlagTags
+// enum above.
+#define TAG_FLAG(flag_name, tag) \
+  COMPILE_ASSERT(sizeof(decltype(FLAGS_##flag_name)), flag_does_not_exist); \
+  COMPILE_ASSERT(sizeof(::kudu::FlagTags::tag), invalid_tag);   \
+  namespace {                                                     \
+    ::kudu::flag_tags_internal::FlagTagger t_##flag_name##_##tag( \
+        AS_STRING(flag_name), AS_STRING(tag));                    \
+  }
+
+// Fetch the list of flags associated with the given flag.
+//
+// If the flag is invalid or has no tags, sets 'tags' to be empty.
+void GetFlagTags(const std::string& flag_name,
+                 std::unordered_set<std::string>* tags);
+
+// ------------------------------------------------------------
+// Internal implementation details
+// ------------------------------------------------------------
+namespace flag_tags_internal {
+
+class FlagTagger {
+ public:
+  FlagTagger(const char* name, const char* tag);
+  ~FlagTagger();
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(FlagTagger);
+};
+
+} // namespace flag_tags_internal
+
+} // namespace kudu
+#endif /* KUDU_UTIL_FLAG_TAGS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_validators-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_validators-test.cc b/be/src/kudu/util/flag_validators-test.cc
new file mode 100644
index 0000000..77efab3
--- /dev/null
+++ b/be/src/kudu/util/flag_validators-test.cc
@@ -0,0 +1,252 @@
+// 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 <cstdlib>
+#include <functional>
+#include <map>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/flag_validators.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DEFINE_string(grouped_0, "", "First flag to set.");
+DEFINE_string(grouped_1, "", "Second flag to set.");
+DEFINE_string(grouped_2, "", "Third flag to set.");
+DEFINE_string(grouped_3, "", "Fourth flag to set.");
+
+namespace kudu {
+
+static bool CheckGroupedFlags01() {
+  const bool is_set_0 = !FLAGS_grouped_0.empty();
+  const bool is_set_1 = !FLAGS_grouped_1.empty();
+
+  if (is_set_0 != is_set_1) {
+    LOG(ERROR) << "--grouped_0 and --grouped_1 must be set as a group";
+    return false;
+  }
+
+  return true;
+}
+GROUP_FLAG_VALIDATOR(test_group_validator01, CheckGroupedFlags01)
+
+static bool CheckGroupedFlags23() {
+  const bool is_set_2 = !FLAGS_grouped_2.empty();
+  const bool is_set_3 = !FLAGS_grouped_3.empty();
+
+  if (is_set_2 != is_set_3) {
+    LOG(ERROR) << "--grouped_2 and --grouped_3 must be set as a group";
+    return false;
+  }
+
+  return true;
+}
+GROUP_FLAG_VALIDATOR(test_group_validator23, CheckGroupedFlags23)
+
+class FlagsValidatorsBasicTest : public KuduTest {
+ public:
+  void RunTest(const char** argv, int argc) {
+    char** casted_argv = const_cast<char**>(argv);
+    // ParseCommandLineFlags() calls exit(1) if it finds inconsistency in flags.
+    ASSERT_EQ(1, ParseCommandLineFlags(&argc, &casted_argv, true));
+  }
+};
+
+TEST_F(FlagsValidatorsBasicTest, Grouped) {
+  const auto& validators = GetFlagValidators();
+  ASSERT_EQ(2, validators.size());
+  const auto& it = validators.find("test_group_validator01");
+  ASSERT_NE(validators.end(), it);
+  const auto& validator = it->second;
+  EXPECT_TRUE(validator());
+  FLAGS_grouped_0 = "0";
+  EXPECT_FALSE(validator());
+  FLAGS_grouped_1 = "1";
+  EXPECT_TRUE(validator());
+  FLAGS_grouped_0 = "";
+  EXPECT_FALSE(validator());
+  FLAGS_grouped_1 = "";
+  EXPECT_TRUE(validator());
+}
+
+class FlagsValidatorsDeathTest : public KuduTest {
+ public:
+  void Run(const char** argv, int argc) {
+    debug::ScopedLeakCheckDisabler disabler;
+    char** casted_argv = const_cast<char**>(argv);
+    // ParseCommandLineFlags() calls exit(1) if one of the custom validators
+    // finds inconsistency in flags.
+    ParseCommandLineFlags(&argc, &casted_argv, true);
+    exit(0);
+  }
+
+  void RunSuccess(const char** argv, int argc) {
+    EXPECT_EXIT(Run(argv, argc), ::testing::ExitedWithCode(0), ".*");
+  }
+
+  void RunFailure(const char** argv, int argc) {
+    EXPECT_EXIT(Run(argv, argc), ::testing::ExitedWithCode(1),
+        ".* Detected inconsistency in command-line flags; exiting");
+  }
+};
+
+TEST_F(FlagsValidatorsDeathTest, GroupedSuccessNoFlags) {
+  const char* argv[] = { "argv_set_0" };
+  NO_FATALS(RunSuccess(argv, ARRAYSIZE(argv)));
+}
+
+TEST_F(FlagsValidatorsDeathTest, GroupedSuccessSimple) {
+  static const size_t kArgvSize = 1 + 2;
+  const char* argv_sets[][kArgvSize] = {
+    {
+      "argv_set_0",
+      "--grouped_0=first",
+      "--grouped_1=second",
+    },
+    {
+      "argv_set_1",
+      "--grouped_0=second",
+      "--grouped_1=first",
+    },
+    {
+      "argv_set_2",
+      "--grouped_0=",
+      "--grouped_1=",
+    },
+    {
+      "argv_set_3",
+      "--grouped_1=",
+      "--grouped_0=",
+    },
+    {
+      "argv_set_4",
+      "--grouped_2=2",
+      "--grouped_3=3",
+    },
+    {
+      "argv_set_5",
+      "--grouped_3=",
+      "--grouped_2=",
+    },
+  };
+  for (auto argv : argv_sets) {
+    RunSuccess(argv, kArgvSize);
+  }
+}
+
+TEST_F(FlagsValidatorsDeathTest, GroupedFailureSimple) {
+  static const size_t kArgvSize = 1 + 1;
+  const char* argv_sets[][kArgvSize] = {
+    {
+      "argv_set_0",
+      "--grouped_0=a",
+    },
+    {
+      "argv_set_1",
+      "--grouped_1=b",
+    },
+    {
+      "argv_set_2",
+      "--grouped_2=2",
+    },
+    {
+      "argv_set_3",
+      "--grouped_3=3",
+    },
+  };
+  for (auto argv : argv_sets) {
+    RunFailure(argv, kArgvSize);
+  }
+}
+
+// Test for correct behavior when only one of two group validators is failing.
+TEST_F(FlagsValidatorsDeathTest, GroupedFailureOneOfTwoValidators) {
+  static const size_t kArgvSize = 4 + 1;
+  const char* argv_sets[][kArgvSize] = {
+    {
+      "argv_set_0",
+      "--grouped_0=0",
+      "--grouped_1=1",
+      "--grouped_2=",
+      "--grouped_3=3",
+    },
+    {
+      "argv_set_1",
+      "--grouped_2=",
+      "--grouped_3=3",
+      "--grouped_0=0",
+      "--grouped_1=1",
+    },
+    {
+      "argv_set_2",
+      "--grouped_0=0",
+      "--grouped_1=",
+      "--grouped_2=2",
+      "--grouped_3=3",
+    },
+    {
+      "argv_set_3",
+      "--grouped_3=3",
+      "--grouped_2=2",
+      "--grouped_1=1",
+      "--grouped_0=",
+    },
+  };
+  for (auto argv : argv_sets) {
+    RunFailure(argv, kArgvSize);
+  }
+}
+
+TEST_F(FlagsValidatorsDeathTest, GroupedFailureWithEmptyValues) {
+  static const size_t kArgvSize = 1 + 2;
+  const char* argv_sets[][kArgvSize] = {
+    {
+      "argv_set_0",
+      "--grouped_0=a",
+      "--grouped_1=",
+    },
+    {
+      "argv_set_1",
+      "--grouped_1=",
+      "--grouped_0=a",
+    },
+    {
+      "argv_set_2",
+      "--grouped_0=",
+      "--grouped_1=b",
+    },
+    {
+      "argv_set_3",
+      "--grouped_1=b",
+      "--grouped_0=",
+    },
+  };
+  for (auto argv : argv_sets) {
+    RunFailure(argv, kArgvSize);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_validators.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_validators.cc b/be/src/kudu/util/flag_validators.cc
new file mode 100644
index 0000000..f90fe2e
--- /dev/null
+++ b/be/src/kudu/util/flag_validators.cc
@@ -0,0 +1,67 @@
+// 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 "kudu/util/flag_validators.h"
+
+#include <string>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/singleton.h"
+
+using std::string;
+
+namespace kudu {
+namespace flag_validation_internal {
+
+// A singleton registry for storing group flag validators.
+class FlagValidatorRegistry {
+ public:
+  static FlagValidatorRegistry* GetInstance() {
+    return Singleton<FlagValidatorRegistry>::get();
+  }
+
+  void Register(const string& name, const FlagValidator& func) {
+    InsertOrDie(&validators_, name, func);
+  }
+
+  const FlagValidatorsMap& validators() {
+    return validators_;
+  }
+
+ private:
+  friend class Singleton<FlagValidatorRegistry>;
+  FlagValidatorRegistry() {}
+
+  FlagValidatorsMap validators_;
+
+  DISALLOW_COPY_AND_ASSIGN(FlagValidatorRegistry);
+};
+
+
+Registrator::Registrator(const char* name, const FlagValidator& validator) {
+  FlagValidatorRegistry::GetInstance()->Register(name, validator);
+}
+
+} // namespace flag_validation_internal
+
+
+const FlagValidatorsMap& GetFlagValidators() {
+  using flag_validation_internal::FlagValidatorRegistry;
+  return FlagValidatorRegistry::GetInstance()->validators();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flag_validators.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flag_validators.h b/be/src/kudu/util/flag_validators.h
new file mode 100644
index 0000000..02cc2dd
--- /dev/null
+++ b/be/src/kudu/util/flag_validators.h
@@ -0,0 +1,102 @@
+// 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.
+
+#pragma once
+
+#include "kudu/gutil/macros.h"
+
+#include <functional>
+#include <map>
+#include <string>
+
+namespace kudu {
+
+// The validation function: takes no parameters and returns a boolean. A group
+// validator should return 'true' if validation was successful, or 'false'
+// otherwise.
+typedef std::function<bool(void)> FlagValidator;
+
+// The group validator registry's representation for as seen from the outside:
+// the key is the name of the group validator, the value is the validation
+// function.
+typedef std::map<std::string, FlagValidator> FlagValidatorsMap;
+
+// Register a 'group' validator for command-line flags. In contrast with the
+// standard (built-in) gflag validators registered by the DEFINE_validator()
+// macro, group validators are run at a later phase in the context of the main()
+// function. A group validator has a guarantee that all command-line flags have
+// been parsed, individually validated (via standard validators), and their
+// values are already set at the time when the validator runs.
+//
+// The first macro parameter is the name of the validator, the second parameter
+// is the validation function as is. The name must be unique across all
+// registered group validators.
+//
+// The validation function takes no parameters and returns 'true' in case of
+// successful validation, otherwise it returns 'false'. If at least one of the
+// registered group validators returns 'false', exit(1) is called.
+//
+// Usage guideline:
+//
+//   * Use the DEFINE_validator() macro if you need to validate an individual
+//     gflag's value
+//
+//   * Use the GROUP_FLAG_VALIDATOR() macro only if you need to validate a set
+//     of gflag values against one another, having the guarantee that their
+//     values are already set when the validation function runs.
+//
+// Sample usage:
+//
+//  static bool ValidateGroupedFlags() {
+//    bool has_a = !FLAGS_a.empty();
+//    bool has_b = !FLAGS_b.empty();
+//
+//    if (has_a != has_b) {
+//      LOG(ERROR) << "--a and --b must be set as a group";
+//      return false;
+//    }
+//
+//    return true;
+//  }
+//  GROUP_FLAG_VALIDATOR(grouped_flags_validator, ValidateGroupedFlags);
+//
+#define GROUP_FLAG_VALIDATOR(name, func) \
+  namespace {                                               \
+    ::kudu::flag_validation_internal::Registrator v_##name( \
+        AS_STRING(name), (func));                           \
+  }
+
+// Get all registered group flag validators.
+const FlagValidatorsMap& GetFlagValidators();
+
+namespace flag_validation_internal {
+
+// This is a utility class which registers a group validator upon instantiation.
+class Registrator {
+ public:
+  // The constructor registers a group validator with the specified name and
+  // the given validation function. The name must be unique among all group
+  // validators.
+  Registrator(const char* name, const FlagValidator& validator);
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(Registrator);
+};
+
+} // namespace flag_validation_internal
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flags-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flags-test.cc b/be/src/kudu/util/flags-test.cc
new file mode 100644
index 0000000..9ebc178
--- /dev/null
+++ b/be/src/kudu/util/flags-test.cc
@@ -0,0 +1,109 @@
+// 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 <string>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+// Test gflags
+DEFINE_string(test_nondefault_ff, "default",
+             "Check if we track non defaults from flagfile");
+DEFINE_string(test_nondefault_explicit, "default",
+             "Check if we track explicitly set non defaults");
+DEFINE_string(test_default_ff, "default",
+             "Check if we track defaults from flagfile");
+DEFINE_string(test_default_explicit, "default",
+             "Check if we track explicitly set defaults");
+DEFINE_bool(test_sensitive_flag, false, "a sensitive flag");
+TAG_FLAG(test_sensitive_flag, sensitive);
+
+DECLARE_bool(never_fsync);
+
+namespace kudu {
+
+class FlagsTest : public KuduTest {};
+
+TEST_F(FlagsTest, TestNonDefaultFlags) {
+  // Memorize the default flags
+  GFlagsMap default_flags = GetFlagsMap();
+
+  std::string flagfile_path(GetTestPath("test_nondefault_flags"));
+  std::string flagfile_contents = "--test_nondefault_ff=nondefault\n"
+                                  "--test_default_ff=default";
+
+  CHECK_OK(WriteStringToFile(Env::Default(),
+                             Slice(flagfile_contents.data(),
+                                   flagfile_contents.size()),
+                             flagfile_path));
+
+  std::string flagfile_flag = strings::Substitute("--flagfile=$0", flagfile_path);
+  int argc = 4;
+  const char* argv[4] = {
+    "some_executable_file",
+    "--test_nondefault_explicit=nondefault",
+    "--test_default_explicit=default",
+    flagfile_flag.c_str()
+  };
+
+  char** casted_argv = const_cast<char**>(argv);
+  ParseCommandLineFlags(&argc, &casted_argv, true);
+
+  std::vector<const char*> expected_flags = {
+    "--test_nondefault_explicit=nondefault",
+    "--test_nondefault_ff=nondefault",
+    flagfile_flag.c_str()
+  };
+
+  std::vector<const char*> unexpected_flags = {
+    "--test_default_explicit",
+    "--test_default_ff"
+  };
+
+  // Setting a sensitive flag with non-default value should return
+  // a redacted value.
+  FLAGS_test_sensitive_flag = true;
+  kudu::g_should_redact = kudu::RedactContext::LOG;
+  std::string result = GetNonDefaultFlags(default_flags);
+
+  for (const auto& expected : expected_flags) {
+    ASSERT_STR_CONTAINS(result, expected);
+  }
+
+  for (const auto& unexpected : unexpected_flags) {
+    ASSERT_STR_NOT_CONTAINS(result, unexpected);
+  }
+
+  ASSERT_STR_CONTAINS(result, strings::Substitute("--test_sensitive_flag=$0",
+                                                  kRedactionMessage));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flags.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flags.cc b/be/src/kudu/util/flags.cc
new file mode 100644
index 0000000..047c893
--- /dev/null
+++ b/be/src/kudu/util/flags.cc
@@ -0,0 +1,604 @@
+// 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 "kudu/util/flags.h"
+
+
+#include <cstdlib>
+#include <functional>
+#include <iostream>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <sys/stat.h>
+#include <unistd.h> // IWYU pragma: keep
+
+#include <boost/algorithm/string/predicate.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#ifdef TCMALLOC_ENABLED
+#include <gperftools/heap-profiler.h>
+#endif
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/flag_validators.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/os-util.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/string_case.h"
+#include "kudu/util/url-coding.h"
+#include "kudu/util/version_info.h"
+
+using google::CommandLineFlagInfo;
+
+using std::cout;
+using std::endl;
+using std::string;
+using std::ostringstream;
+using std::unordered_set;
+using std::vector;
+
+using strings::Substitute;
+
+// Because every binary initializes its flags here, we use it as a convenient place
+// to offer some global flags as well.
+DEFINE_bool(dump_metrics_json, false,
+            "Dump a JSON document describing all of the metrics which may be emitted "
+            "by this binary.");
+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.");
+TAG_FLAG(enable_process_lifetime_heap_profiling, stable);
+TAG_FLAG(enable_process_lifetime_heap_profiling, advanced);
+
+DEFINE_string(heap_profile_path, "", "Output path to store heap profiles. If not set " \
+    "profiles are stored in /tmp/<process-name>.<pid>.<n>.heap.");
+TAG_FLAG(heap_profile_path, stable);
+TAG_FLAG(heap_profile_path, advanced);
+
+DEFINE_int64(heap_sample_every_n_bytes, 0,
+             "Enable heap occupancy sampling. If this flag is set to some positive "
+             "value N, a memory allocation will be sampled approximately every N bytes. "
+             "Lower values of N incur larger overhead but give more accurate results. "
+             "A value such as 524288 (512KB) is a reasonable choice with relatively "
+             "low overhead.");
+TAG_FLAG(heap_sample_every_n_bytes, advanced);
+TAG_FLAG(heap_sample_every_n_bytes, experimental);
+#endif
+
+DEFINE_bool(disable_core_dumps, false, "Disable core dumps when this process crashes.");
+TAG_FLAG(disable_core_dumps, advanced);
+TAG_FLAG(disable_core_dumps, evolving);
+
+DEFINE_string(umask, "077",
+              "The umask that will be used when creating files and directories. "
+              "Permissions of top-level data directories will also be modified at "
+              "start-up to conform to the given umask. Changing this value may "
+              "enable unauthorized local users to read or modify data stored by Kudu.");
+TAG_FLAG(umask, advanced);
+
+static bool ValidateUmask(const char* /*flagname*/, const string& value) {
+  uint32_t parsed;
+  if (!safe_strtou32_base(value.c_str(), &parsed, 8)) {
+    LOG(ERROR) << "Invalid umask: must be an octal string";
+    return false;
+  }
+
+  // Verify that the umask doesn't restrict the permissions of the owner.
+  // If it did, we'd end up creating files that we can't read.
+  if ((parsed & 0700) != 0) {
+    LOG(ERROR) << "Invalid umask value: must not restrict owner permissions";
+    return false;
+  }
+  return true;
+}
+
+DEFINE_validator(umask, &ValidateUmask);
+
+DEFINE_bool(unlock_experimental_flags, false,
+            "Unlock flags marked as 'experimental'. These flags are not guaranteed to "
+            "be maintained across releases of Kudu, and may enable features or behavior "
+            "known to be unstable. Use at your own risk.");
+TAG_FLAG(unlock_experimental_flags, advanced);
+TAG_FLAG(unlock_experimental_flags, stable);
+
+DEFINE_bool(unlock_unsafe_flags, false,
+            "Unlock flags marked as 'unsafe'. These flags are not guaranteed to "
+            "be maintained across releases of Kudu, and enable features or behavior "
+            "known to be unsafe. Use at your own risk.");
+TAG_FLAG(unlock_unsafe_flags, advanced);
+TAG_FLAG(unlock_unsafe_flags, stable);
+
+DEFINE_string(redact, "all",
+              "Comma-separated list that controls redaction context. Supported options "
+              "are 'all','log', and 'none'. If 'all' is specified, sensitive data "
+              "(sensitive configuration flags and row data) will be redacted from "
+              "the web UI as well as glog and error messages. If 'log' is specified, "
+              "sensitive data will only be redacted from glog and error messages. "
+              "If 'none' is specified, no redaction will occur.");
+TAG_FLAG(redact, advanced);
+TAG_FLAG(redact, evolving);
+
+static bool ValidateRedact(const char* /*flagname*/, const string& value) {
+  kudu::g_should_redact = kudu::RedactContext::NONE;
+
+  // Flag value is case insensitive.
+  string redact_flags;
+  kudu::ToUpperCase(value, &redact_flags);
+
+  // 'all', 'none', and '' must be specified without any other option.
+  if (redact_flags == "ALL") {
+    kudu::g_should_redact = kudu::RedactContext::ALL;
+    return true;
+  }
+  if (redact_flags == "NONE" || redact_flags.empty()) {
+    return true;
+  }
+
+  for (const auto& t : strings::Split(redact_flags, ",", strings::SkipEmpty())) {
+    if (t == "LOG") {
+      kudu::g_should_redact = kudu::RedactContext::LOG;
+    } else if (t == "ALL" || t == "NONE") {
+      LOG(ERROR) << "Invalid redaction options: "
+                 << value << ", '" << t << "' must be specified by itself.";
+      return false;
+    } else {
+      LOG(ERROR) << "Invalid redaction context: " << t <<
+                    ". Available types are 'all', 'log', and 'none'.";
+      return false;
+    }
+  }
+  return true;
+}
+
+DEFINE_validator(redact, &ValidateRedact);
+// Tag a bunch of the flags that we inherit from glog/gflags.
+
+//------------------------------------------------------------
+// GLog flags
+//------------------------------------------------------------
+// Most of these are considered stable. The ones related to email are
+// marked unsafe because sending email inline from a server is a pretty
+// bad idea.
+DECLARE_string(alsologtoemail);
+TAG_FLAG(alsologtoemail, hidden);
+TAG_FLAG(alsologtoemail, unsafe);
+
+// --alsologtostderr is deprecated in favor of --stderrthreshold
+DECLARE_bool(alsologtostderr);
+TAG_FLAG(alsologtostderr, hidden);
+TAG_FLAG(alsologtostderr, runtime);
+
+DECLARE_bool(colorlogtostderr);
+TAG_FLAG(colorlogtostderr, stable);
+TAG_FLAG(colorlogtostderr, runtime);
+
+DECLARE_bool(drop_log_memory);
+TAG_FLAG(drop_log_memory, advanced);
+TAG_FLAG(drop_log_memory, runtime);
+
+DECLARE_string(log_backtrace_at);
+TAG_FLAG(log_backtrace_at, advanced);
+
+DECLARE_string(log_dir);
+TAG_FLAG(log_dir, stable);
+
+DECLARE_string(log_link);
+TAG_FLAG(log_link, stable);
+TAG_FLAG(log_link, advanced);
+
+DECLARE_bool(log_prefix);
+TAG_FLAG(log_prefix, stable);
+TAG_FLAG(log_prefix, advanced);
+TAG_FLAG(log_prefix, runtime);
+
+DECLARE_int32(logbuflevel);
+TAG_FLAG(logbuflevel, advanced);
+TAG_FLAG(logbuflevel, runtime);
+DECLARE_int32(logbufsecs);
+TAG_FLAG(logbufsecs, advanced);
+TAG_FLAG(logbufsecs, runtime);
+
+DECLARE_int32(logemaillevel);
+TAG_FLAG(logemaillevel, hidden);
+TAG_FLAG(logemaillevel, unsafe);
+
+DECLARE_string(logmailer);
+TAG_FLAG(logmailer, hidden);
+
+DECLARE_bool(logtostderr);
+TAG_FLAG(logtostderr, stable);
+TAG_FLAG(logtostderr, runtime);
+
+DECLARE_int32(max_log_size);
+TAG_FLAG(max_log_size, stable);
+TAG_FLAG(max_log_size, runtime);
+
+DECLARE_int32(minloglevel);
+TAG_FLAG(minloglevel, stable);
+TAG_FLAG(minloglevel, advanced);
+TAG_FLAG(minloglevel, runtime);
+
+DECLARE_int32(stderrthreshold);
+TAG_FLAG(stderrthreshold, stable);
+TAG_FLAG(stderrthreshold, advanced);
+TAG_FLAG(stderrthreshold, runtime);
+
+DECLARE_bool(stop_logging_if_full_disk);
+TAG_FLAG(stop_logging_if_full_disk, stable);
+TAG_FLAG(stop_logging_if_full_disk, advanced);
+TAG_FLAG(stop_logging_if_full_disk, runtime);
+
+DECLARE_int32(v);
+TAG_FLAG(v, stable);
+TAG_FLAG(v, advanced);
+TAG_FLAG(v, runtime);
+
+DECLARE_string(vmodule);
+TAG_FLAG(vmodule, stable);
+TAG_FLAG(vmodule, advanced);
+
+DECLARE_bool(symbolize_stacktrace);
+TAG_FLAG(symbolize_stacktrace, stable);
+TAG_FLAG(symbolize_stacktrace, runtime);
+TAG_FLAG(symbolize_stacktrace, advanced);
+
+//------------------------------------------------------------
+// GFlags flags
+//------------------------------------------------------------
+DECLARE_string(flagfile);
+TAG_FLAG(flagfile, stable);
+
+DECLARE_string(fromenv);
+TAG_FLAG(fromenv, stable);
+TAG_FLAG(fromenv, advanced);
+
+DECLARE_string(tryfromenv);
+TAG_FLAG(tryfromenv, stable);
+TAG_FLAG(tryfromenv, advanced);
+
+DECLARE_string(undefok);
+TAG_FLAG(undefok, stable);
+TAG_FLAG(undefok, advanced);
+
+DECLARE_int32(tab_completion_columns);
+TAG_FLAG(tab_completion_columns, stable);
+TAG_FLAG(tab_completion_columns, hidden);
+
+DECLARE_string(tab_completion_word);
+TAG_FLAG(tab_completion_word, stable);
+TAG_FLAG(tab_completion_word, hidden);
+
+DECLARE_bool(help);
+TAG_FLAG(help, stable);
+
+DECLARE_bool(helpfull);
+// We hide -helpfull because it's the same as -help for now.
+TAG_FLAG(helpfull, stable);
+TAG_FLAG(helpfull, hidden);
+
+DECLARE_string(helpmatch);
+TAG_FLAG(helpmatch, stable);
+TAG_FLAG(helpmatch, advanced);
+
+DECLARE_string(helpon);
+TAG_FLAG(helpon, stable);
+TAG_FLAG(helpon, advanced);
+
+DECLARE_bool(helppackage);
+TAG_FLAG(helppackage, stable);
+TAG_FLAG(helppackage, advanced);
+
+DECLARE_bool(helpshort);
+TAG_FLAG(helpshort, stable);
+TAG_FLAG(helpshort, advanced);
+
+DECLARE_bool(helpxml);
+TAG_FLAG(helpxml, stable);
+TAG_FLAG(helpxml, advanced);
+
+DECLARE_bool(version);
+TAG_FLAG(version, stable);
+
+//------------------------------------------------------------
+// TCMalloc flags.
+// These are tricky because tcmalloc doesn't use gflags. So we have to
+// reach into its internal namespace.
+//------------------------------------------------------------
+#define TCM_NAMESPACE FLAG__namespace_do_not_use_directly_use_DECLARE_int64_instead
+namespace TCM_NAMESPACE {
+extern int64_t FLAGS_tcmalloc_sample_parameter;
+} // namespace TCM_NAMESPACE
+
+namespace kudu {
+
+// After flags have been parsed, the umask value is filled in here.
+uint32_t g_parsed_umask = -1;
+
+namespace {
+
+void AppendXMLTag(const char* tag, const string& txt, string* r) {
+  strings::SubstituteAndAppend(r, "<$0>$1</$0>", tag, EscapeForHtmlToString(txt));
+}
+
+static string DescribeOneFlagInXML(const CommandLineFlagInfo& flag) {
+  unordered_set<string> tags;
+  GetFlagTags(flag.name, &tags);
+
+  string r("<flag>");
+  AppendXMLTag("file", flag.filename, &r);
+  AppendXMLTag("name", flag.name, &r);
+  AppendXMLTag("meaning", flag.description, &r);
+  AppendXMLTag("default", flag.default_value, &r);
+  AppendXMLTag("current", flag.current_value, &r);
+  AppendXMLTag("type", flag.type, &r);
+  AppendXMLTag("tags", JoinStrings(tags, ","), &r);
+  r += "</flag>";
+  return r;
+}
+
+void DumpFlagsXML() {
+  vector<CommandLineFlagInfo> flags;
+  GetAllFlags(&flags);
+
+  cout << "<?xml version=\"1.0\"?>" << endl;
+  cout << "<AllFlags>" << endl;
+  cout << strings::Substitute(
+      "<program>$0</program>",
+      EscapeForHtmlToString(BaseName(google::ProgramInvocationShortName()))) << endl;
+  cout << strings::Substitute(
+      "<usage>$0</usage>",
+      EscapeForHtmlToString(google::ProgramUsage())) << endl;
+
+  for (const CommandLineFlagInfo& flag : flags) {
+    cout << DescribeOneFlagInXML(flag) << endl;
+  }
+
+  cout << "</AllFlags>" << endl;
+}
+
+// Check that, if any flags tagged with 'tag' have been specified to
+// non-default values, that 'unlocked' is true. If so (i.e. if the
+// flags have been appropriately unlocked), emits a warning message
+// for each flag and returns false. Otherwise, emits an error message
+// and returns true.
+bool CheckFlagsAndWarn(const string& tag, bool unlocked) {
+  vector<CommandLineFlagInfo> flags;
+  GetAllFlags(&flags);
+
+  int use_count = 0;
+  for (const auto& f : flags) {
+    if (f.is_default) continue;
+    unordered_set<string> tags;
+    GetFlagTags(f.name, &tags);
+    if (!ContainsKey(tags, tag)) continue;
+
+    if (unlocked) {
+      LOG(WARNING) << "Enabled " << tag << " flag: --" << f.name << "=" << f.current_value;
+    } else {
+      LOG(ERROR) << "Flag --" << f.name << " is " << tag << " and unsupported.";
+      use_count++;
+    }
+  }
+
+  if (!unlocked && use_count > 0) {
+    LOG(ERROR) << use_count << " " << tag << " flag(s) in use.";
+    LOG(ERROR) << "Use --unlock_" << tag << "_flags to proceed at your own risk.";
+    return true;
+  }
+  return false;
+}
+
+// Check that any flags specified on the command line are allowed
+// to be set. This ensures that, if the user is using any unsafe
+// or experimental flags, they have explicitly unlocked them.
+void CheckFlagsAllowed() {
+  bool should_exit = false;
+  should_exit |= CheckFlagsAndWarn("unsafe", FLAGS_unlock_unsafe_flags);
+  should_exit |= CheckFlagsAndWarn("experimental", FLAGS_unlock_experimental_flags);
+  if (should_exit) {
+    exit(1);
+  }
+}
+
+// Run 'late phase' custom validators: these can be run only when all flags are
+// already parsed and individually validated.
+void RunCustomValidators() {
+  const auto& validators(GetFlagValidators());
+  bool found_inconsistency = false;
+  for (const auto& e : validators) {
+    found_inconsistency |= !e.second();
+  }
+  if (found_inconsistency) {
+    LOG(ERROR) << "Detected inconsistency in command-line flags; exiting";
+    exit(1);
+  }
+}
+
+void SetUmask() {
+  // We already validated with a nice error message using the ValidateUmask
+  // FlagValidator above.
+  CHECK(safe_strtou32_base(FLAGS_umask.c_str(), &g_parsed_umask, 8));
+  uint32_t old_mask = umask(g_parsed_umask);
+  if (old_mask != g_parsed_umask) {
+    VLOG(2) << "Changed umask from " << StringPrintf("%03o", old_mask) << " to "
+            << StringPrintf("%03o", g_parsed_umask);
+  }
+}
+
+} // anonymous namespace
+
+// If --redact indicates, redact the flag tagged as 'sensitive'.
+// Otherwise, return its value as-is. If EscapeMode is set to HTML,
+// return HTML escaped string.
+string CheckFlagAndRedact(const CommandLineFlagInfo& flag, EscapeMode mode) {
+  string ret_value;
+  unordered_set<string> tags;
+  GetFlagTags(flag.name, &tags);
+
+  if (ContainsKey(tags, "sensitive") && KUDU_SHOULD_REDACT()) {
+    ret_value = kRedactionMessage;
+  } else {
+    ret_value = flag.current_value;
+  }
+  if (mode == EscapeMode::HTML) {
+    ret_value = EscapeForHtmlToString(ret_value);
+  }
+  return ret_value;
+}
+
+int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags) {
+  // The logbufsecs default is 30 seconds which is a bit too long.
+  google::SetCommandLineOptionWithMode("logbufsecs", "5",
+                                       google::FlagSettingMode::SET_FLAGS_DEFAULT);
+
+  int ret = google::ParseCommandLineNonHelpFlags(argc, argv, remove_flags);
+  HandleCommonFlags();
+  return ret;
+}
+
+void HandleCommonFlags() {
+  if (FLAGS_helpxml) {
+    DumpFlagsXML();
+    exit(1);
+  } else if (FLAGS_dump_metrics_json) {
+    MetricPrototypeRegistry::get()->WriteAsJson();
+    exit(0);
+  } else if (FLAGS_version) {
+    cout << VersionInfo::GetAllVersionInfo() << endl;
+    exit(0);
+  }
+
+  google::HandleCommandLineHelpFlags();
+  CheckFlagsAllowed();
+  RunCustomValidators();
+
+  if (FLAGS_disable_core_dumps) {
+    DisableCoreDumps();
+  }
+
+  SetUmask();
+
+#ifdef TCMALLOC_ENABLED
+  if (FLAGS_heap_profile_path.empty()) {
+    FLAGS_heap_profile_path = strings::Substitute(
+        "/tmp/$0.$1", google::ProgramInvocationShortName(), getpid());
+  }
+
+  if (FLAGS_enable_process_lifetime_heap_profiling) {
+    HeapProfilerStart(FLAGS_heap_profile_path.c_str());
+  }
+  // Set the internal tcmalloc flag unless it was already set using the built-in
+  // environment-variable-based method. It doesn't appear that this is settable
+  // in any less hacky fashion.
+  if (!getenv("TCMALLOC_SAMPLE_PARAMETER")) {
+    TCM_NAMESPACE::FLAGS_tcmalloc_sample_parameter = FLAGS_heap_sample_every_n_bytes;
+  } else if (!google::GetCommandLineFlagInfoOrDie("heap_sample_every_n_bytes").is_default) {
+    LOG(ERROR) << "Heap sampling configured using both --heap-sample-every-n-bytes and "
+               << "TCMALLOC_SAMPLE_PARAMETER. Ignoring command line flag.";
+  }
+#endif
+}
+
+string CommandlineFlagsIntoString(EscapeMode mode) {
+  string ret_value;
+  vector<CommandLineFlagInfo> flags;
+  GetAllFlags(&flags);
+
+  for (const auto& f : flags) {
+    ret_value += "--";
+    if (mode == EscapeMode::HTML) {
+      ret_value += EscapeForHtmlToString(f.name);
+    } else if (mode == EscapeMode::NONE) {
+      ret_value += f.name;
+    }
+    ret_value += "=";
+    ret_value += CheckFlagAndRedact(f, mode);
+    ret_value += "\n";
+  }
+  return ret_value;
+}
+
+string GetNonDefaultFlags(const GFlagsMap& default_flags) {
+  ostringstream args;
+  vector<CommandLineFlagInfo> flags;
+  GetAllFlags(&flags);
+  for (const auto& flag : flags) {
+    if (!flag.is_default) {
+      // This only means that the flag has been rewritten. It doesn't
+      // mean that this has been done in the command line, or even
+      // that it's truly different from the default value.
+      // Next, we try to check both.
+      const auto& default_flag = default_flags.find(flag.name);
+      // it's very unlikely, but still possible that we don't have the flag in defaults
+      if (default_flag == default_flags.end() ||
+          flag.current_value != default_flag->second.current_value) {
+        if (!args.str().empty()) {
+          args << '\n';
+        }
+
+        // Redact the flags tagged as sensitive, if redaction is enabled.
+        string flag_value = CheckFlagAndRedact(flag, EscapeMode::NONE);
+        args << "--" << flag.name << '=' << flag_value;
+      }
+    }
+  }
+  return args.str();
+}
+
+GFlagsMap GetFlagsMap() {
+  vector<CommandLineFlagInfo> default_flags;
+  GetAllFlags(&default_flags);
+  GFlagsMap flags_by_name;
+  for (auto& flag : default_flags) {
+    flags_by_name.emplace(flag.name, std::move(flag));
+  }
+  return flags_by_name;
+}
+
+Status ParseTriState(const char* flag_name, const std::string& flag_value,
+    TriStateFlag* tri_state) {
+  if (boost::iequals(flag_value, "required")) {
+    *tri_state = TriStateFlag::REQUIRED;
+  } else if (boost::iequals(flag_value, "optional")) {
+    *tri_state = TriStateFlag::OPTIONAL;
+  } else if (boost::iequals(flag_value, "disabled")) {
+    *tri_state = TriStateFlag::DISABLED;
+  } else {
+    return Status::InvalidArgument(strings::Substitute(
+          "$0 flag must be one of 'required', 'optional', or 'disabled'",
+          flag_name));
+  }
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/flags.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flags.h b/be/src/kudu/util/flags.h
new file mode 100644
index 0000000..83cb152
--- /dev/null
+++ b/be/src/kudu/util/flags.h
@@ -0,0 +1,89 @@
+// 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.
+#ifndef KUDU_UTIL_FLAGS_H
+#define KUDU_UTIL_FLAGS_H
+
+#include <cstdint>
+#include <string>
+#include <unordered_map>
+
+#include "kudu/util/status.h"
+
+namespace google {
+  struct CommandLineFlagInfo;
+}
+
+namespace kudu {
+
+// The umask of the process, set based on the --umask flag during
+// HandleCommonFlags().
+extern uint32_t g_parsed_umask;
+
+// Looks for flags in argv and parses them.  Rearranges argv to put
+// flags first, or removes them entirely if remove_flags is true.
+// If a flag is defined more than once in the command line or flag
+// file, the last definition is used.  Returns the index (into argv)
+// of the first non-flag argument.
+//
+// This is a wrapper around google::ParseCommandLineFlags, but integrates
+// with Kudu flag tags. For example, --helpxml will include the list of
+// tags for each flag. This should be be used instead of
+// google::ParseCommandLineFlags in any user-facing binary.
+//
+// See gflags.h for more information.
+int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags);
+
+// Handle common flags such as -version, -disable_core_dumps, etc.
+// This includes the GFlags common flags such as "-help".
+//
+// Requires that flags have already been parsed using
+// google::ParseCommandLineNonHelpFlags().
+void HandleCommonFlags();
+
+enum class EscapeMode {
+  HTML,
+  NONE
+};
+
+// Stick the flags into a string. If redaction is enabled, the values of
+// flags tagged as sensitive will be redacted. Otherwise, the values
+// will be written to the string as-is. The values will be HTML escaped
+// if EscapeMode is HTML.
+std::string CommandlineFlagsIntoString(EscapeMode mode);
+
+typedef std::unordered_map<std::string, google::CommandLineFlagInfo> GFlagsMap;
+
+// Get all the flags different from their defaults. The output is a nicely
+// formatted string with --flag=value pairs per line. Redact any flags that
+// are tagged as sensitive, if redaction is enabled.
+std::string GetNonDefaultFlags(const GFlagsMap& default_flags);
+
+GFlagsMap GetFlagsMap();
+
+enum class TriStateFlag {
+  DISABLED,
+  OPTIONAL,
+  REQUIRED,
+};
+
+Status ParseTriState(const char* flag_name, const std::string& flag_value,
+    TriStateFlag* tri_state);
+
+std::string CheckFlagAndRedact(const google::CommandLineFlagInfo& flag, EscapeMode mode);
+
+} // namespace kudu
+#endif /* KUDU_UTIL_FLAGS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/group_varint-inl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/group_varint-inl.h b/be/src/kudu/util/group_varint-inl.h
new file mode 100644
index 0000000..27e289f
--- /dev/null
+++ b/be/src/kudu/util/group_varint-inl.h
@@ -0,0 +1,294 @@
+// 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.
+#ifndef KUDU_UTIL_GROUP_VARINT_INL_H
+#define KUDU_UTIL_GROUP_VARINT_INL_H
+
+#include <emmintrin.h>
+#ifdef __linux__
+#include <endian.h>
+#endif
+#include <smmintrin.h>
+#include <tmmintrin.h>
+#include <xmmintrin.h>
+
+#include <cstdint>
+#include <cstring>
+
+#include <boost/utility/binary.hpp>
+#include <boost/parameter/name.hpp>
+#include <boost/preprocessor/arithmetic/dec.hpp>
+#include <boost/preprocessor/arithmetic/inc.hpp>
+#include <boost/preprocessor/control/iif.hpp>
+#include <boost/preprocessor/control/while.hpp>
+#include <boost/preprocessor/list/fold_left.hpp>
+#include <boost/preprocessor/logical/bitand.hpp>
+#include <boost/preprocessor/logical/bool.hpp>
+#include <boost/preprocessor/logical/compl.hpp>
+#include <boost/preprocessor/seq/elem.hpp>
+#include <boost/preprocessor/seq/fold_left.hpp>
+#include <boost/preprocessor/seq/size.hpp>
+#include <boost/preprocessor/tuple/elem.hpp>
+#include <boost/preprocessor/variadic/elem.hpp>
+#include <glog/logging.h>
+
+#ifndef __linux__
+#include "kudu/gutil/port.h"
+#endif
+#include "kudu/util/faststring.h"
+
+namespace kudu {
+namespace coding {
+
+extern bool SSE_TABLE_INITTED;
+extern uint8_t SSE_TABLE[256 * 16] __attribute__((aligned(16)));
+extern uint8_t VARINT_SELECTOR_LENGTHS[256];
+
+const uint32_t MASKS[4] = { 0xff, 0xffff, 0xffffff, 0xffffffff };
+
+
+// Calculate the number of bytes to encode the given unsigned int.
+inline size_t CalcRequiredBytes32(uint32_t i) {
+  // | 1 because the result is undefined for the 0 case
+  return sizeof(uint32_t) - __builtin_clz(i|1)/8;
+}
+
+// Decode a set of 4 group-varint encoded integers from the given pointer.
+//
+// Requires that there are at up to 3 extra bytes remaining in 'src' after
+// the last integer.
+//
+// Returns a pointer following the last decoded integer.
+inline const uint8_t *DecodeGroupVarInt32(
+  const uint8_t *src,
+  uint32_t *a, uint32_t *b, uint32_t *c, uint32_t *d) {
+
+  uint8_t a_sel = (*src & BOOST_BINARY(11 00 00 00)) >> 6;
+  uint8_t b_sel = (*src & BOOST_BINARY(00 11 00 00)) >> 4;
+  uint8_t c_sel = (*src & BOOST_BINARY(00 00 11 00)) >> 2;
+  uint8_t d_sel = (*src & BOOST_BINARY(00 00 00 11 ));
+
+  src++; // skip past selector byte
+
+  *a = *reinterpret_cast<const uint32_t *>(src) & MASKS[a_sel];
+  src += a_sel + 1;
+
+  *b = *reinterpret_cast<const uint32_t *>(src) & MASKS[b_sel];
+  src += b_sel + 1;
+
+  *c = *reinterpret_cast<const uint32_t *>(src) & MASKS[c_sel];
+  src += c_sel + 1;
+
+  *d = *reinterpret_cast<const uint32_t *>(src) & MASKS[d_sel];
+  src += d_sel + 1;
+
+  return src;
+}
+
+// Decode total length of the encoded integers from the given pointer,
+// include the tag byte.
+inline size_t DecodeGroupVarInt32_GetGroupSize(const uint8_t *src) {
+  return VARINT_SELECTOR_LENGTHS[*src] + 1;
+}
+
+// Decode a set of 4 group-varint encoded integers from the given pointer.
+//
+// Returns a pointer following the last decoded integer.
+inline const uint8_t *DecodeGroupVarInt32_SlowButSafe(
+  const uint8_t *src,
+  uint32_t *a, uint32_t *b, uint32_t *c, uint32_t *d) {
+
+  // VARINT_SELECTOR_LENGTHS[] isn't initialized until SSE_TABLE_INITTED is true
+  DCHECK(SSE_TABLE_INITTED);
+
+  const size_t total_len = DecodeGroupVarInt32_GetGroupSize(src);
+
+  uint8_t safe_buf[17];
+  memcpy(safe_buf, src, total_len);
+  DecodeGroupVarInt32(safe_buf, a, b, c, d);
+  return src + total_len;
+}
+
+
+inline void DoExtractM128(__m128i results,
+                          uint32_t *a, uint32_t *b, uint32_t *c, uint32_t *d) {
+#define SSE_USE_EXTRACT_PS
+#ifdef SSE_USE_EXTRACT_PS
+  // _mm_extract_ps turns into extractps, which is slightly faster
+  // than _mm_extract_epi32 (which turns into pextrd)
+  // Apparently pextrd involves one more micro-op
+  // than extractps.
+  //
+  // A uint32 cfile macro-benchmark is about 3% faster with this code path.
+  *a = _mm_extract_ps((__v4sf)results, 0);
+  *b = _mm_extract_ps((__v4sf)results, 1);
+  *c = _mm_extract_ps((__v4sf)results, 2);
+  *d = _mm_extract_ps((__v4sf)results, 3);
+#else
+  *a = _mm_extract_epi32(results, 0);
+  *b = _mm_extract_epi32(results, 1);
+  *c = _mm_extract_epi32(results, 2);
+  *d = _mm_extract_epi32(results, 3);
+#endif
+}
+
+// Same as above, but uses SSE so may be faster.
+// TODO: remove this and just automatically pick the right implementation at runtime.
+//
+// NOTE: the src buffer must be have at least 17 bytes remaining in it, so this
+// code path is not usable at the end of a block.
+inline const uint8_t *DecodeGroupVarInt32_SSE(
+  const uint8_t *src,
+  uint32_t *a, uint32_t *b, uint32_t *c, uint32_t *d) {
+
+  DCHECK(SSE_TABLE_INITTED);
+
+  uint8_t sel_byte = *src++;
+  __m128i shuffle_mask = _mm_load_si128(
+    reinterpret_cast<__m128i *>(&SSE_TABLE[sel_byte * 16]));
+  __m128i data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
+
+  __m128i results = _mm_shuffle_epi8(data, shuffle_mask);
+
+  // It would look like the following would be most efficient,
+  // since it turns into a single movdqa instruction:
+  //   *reinterpret_cast<__m128i *>(ret) = results;
+  // (where ret is an aligned array of ints, which the user must pass)
+  // but it is actually slower than the below alternatives by a
+  // good amount -- even though these result in more instructions.
+  DoExtractM128(results, a, b, c, d);
+  src += VARINT_SELECTOR_LENGTHS[sel_byte];
+
+  return src;
+}
+
+// Optimized function which decodes a group of uint32s from 'src' into 'ret',
+// which should have enough space for 4 uint32s. During decoding, adds 'add'
+// to the vector in parallel.
+//
+// NOTE: the src buffer must be have at least 17 bytes remaining in it, so this
+// code path is not usable at the end of a block.
+inline const uint8_t *DecodeGroupVarInt32_SSE_Add(
+  const uint8_t *src,
+  uint32_t *ret,
+  __m128i add) {
+
+  DCHECK(SSE_TABLE_INITTED);
+
+  uint8_t sel_byte = *src++;
+  __m128i shuffle_mask = _mm_load_si128(
+    reinterpret_cast<__m128i *>(&SSE_TABLE[sel_byte * 16]));
+  __m128i data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
+
+  __m128i decoded_deltas = _mm_shuffle_epi8(data, shuffle_mask);
+  __m128i results = _mm_add_epi32(decoded_deltas, add);
+
+  DoExtractM128(results, &ret[0], &ret[1], &ret[2], &ret[3]);
+
+  src += VARINT_SELECTOR_LENGTHS[sel_byte];
+  return src;
+}
+
+
+// Append a set of group-varint encoded integers to the given faststring.
+inline void AppendGroupVarInt32(
+  faststring *s,
+  uint32_t a, uint32_t b, uint32_t c, uint32_t d) {
+
+  uint8_t a_tag = CalcRequiredBytes32(a) - 1;
+  uint8_t b_tag = CalcRequiredBytes32(b) - 1;
+  uint8_t c_tag = CalcRequiredBytes32(c) - 1;
+  uint8_t d_tag = CalcRequiredBytes32(d) - 1;
+
+  uint8_t prefix_byte =
+    (a_tag << 6) |
+    (b_tag << 4) |
+    (c_tag << 2) |
+    (d_tag);
+
+  uint8_t size = 1 +
+    a_tag + 1 +
+    b_tag + 1 +
+    c_tag + 1 +
+    d_tag + 1;
+
+  size_t old_size = s->size();
+
+  // Reserving 4 extra bytes means we can use simple
+  // 4-byte stores instead of variable copies here --
+  // if we hang off the end of the array into the "empty" area, it's OK.
+  // We'll chop it back off down below.
+  s->resize(old_size + size + 4);
+  uint8_t *ptr = &((*s)[old_size]);
+
+#if __BYTE_ORDER != __LITTLE_ENDIAN
+#error dont support big endian currently
+#endif
+
+  *ptr++ = prefix_byte;
+  memcpy(ptr, &a, 4);
+  ptr += a_tag + 1;
+  memcpy(ptr, &b, 4);
+  ptr += b_tag + 1;
+  memcpy(ptr, &c, 4);
+  ptr += c_tag + 1;
+  memcpy(ptr, &d, 4);
+
+  s->resize(old_size + size);
+}
+
+// Append a sequence of uint32s encoded using group-varint.
+//
+// 'frame_of_reference' is also subtracted from each integer
+// before encoding.
+//
+// If frame_of_reference is greater than any element in the array,
+// results are undefined.
+//
+// For best performance, users should already have reserved adequate
+// space in 's' (CalcRequiredBytes32 can be handy here)
+inline void AppendGroupVarInt32Sequence(faststring *s, uint32_t frame_of_reference,
+                                        uint32_t *ints, size_t size) {
+  uint32_t *p = ints;
+  while (size >= 4) {
+    AppendGroupVarInt32(s,
+                        p[0] - frame_of_reference,
+                        p[1] - frame_of_reference,
+                        p[2] - frame_of_reference,
+                        p[3] - frame_of_reference);
+    size -= 4;
+    p += 4;
+  }
+
+
+  uint32_t trailer[4] = {0, 0, 0, 0};
+  uint32_t *trailer_p = &trailer[0];
+
+  if (size > 0) {
+    while (size > 0) {
+      *trailer_p++ = *p++ - frame_of_reference;
+      size--;
+    }
+
+    AppendGroupVarInt32(s, trailer[0], trailer[1], trailer[2], trailer[3]);
+  }
+}
+
+
+} // namespace coding
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/group_varint-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/group_varint-test.cc b/be/src/kudu/util/group_varint-test.cc
new file mode 100644
index 0000000..983fb0f
--- /dev/null
+++ b/be/src/kudu/util/group_varint-test.cc
@@ -0,0 +1,144 @@
+// 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 <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <string>
+#ifdef NDEBUG
+#include <vector>
+#endif
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/faststring.h"
+#include "kudu/util/group_varint-inl.h"
+#ifdef NDEBUG
+#include "kudu/util/stopwatch.h"
+#endif
+
+namespace kudu {
+namespace coding {
+
+extern void DumpSSETable();
+
+// Encodes the given four ints as group-varint, then
+// decodes and ensures the result is the same.
+static void DoTestRoundTripGVI32(
+  uint32_t a, uint32_t b, uint32_t c, uint32_t d,
+  bool use_sse = false) {
+  faststring buf;
+  AppendGroupVarInt32(&buf, a, b, c, d);
+
+  int real_size = buf.size();
+
+  // The implementations actually read past the group varint,
+  // so append some extra padding data to ensure that it's not reading
+  // uninitialized memory. The SSE implementation uses 128-bit reads
+  // and the non-SSE one uses 32-bit reads.
+  buf.append(std::string(use_sse ? 16 : 4, 'x'));
+
+  uint32_t ret[4];
+
+  const uint8_t *end;
+
+  if (use_sse) {
+    end = DecodeGroupVarInt32_SSE(
+      buf.data(), &ret[0], &ret[1], &ret[2], &ret[3]);
+  } else {
+    end = DecodeGroupVarInt32(
+      buf.data(), &ret[0], &ret[1], &ret[2], &ret[3]);
+  }
+
+  ASSERT_EQ(a, ret[0]);
+  ASSERT_EQ(b, ret[1]);
+  ASSERT_EQ(c, ret[2]);
+  ASSERT_EQ(d, ret[3]);
+  ASSERT_EQ(end, buf.data() + real_size);
+}
+
+
+TEST(TestGroupVarInt, TestSSETable) {
+  DumpSSETable();
+  faststring buf;
+  AppendGroupVarInt32(&buf, 0, 0, 0, 0);
+  DoTestRoundTripGVI32(0, 0, 0, 0, true);
+  DoTestRoundTripGVI32(1, 2, 3, 4, true);
+  DoTestRoundTripGVI32(1, 2000, 3, 200000, true);
+}
+
+TEST(TestGroupVarInt, TestGroupVarInt) {
+  faststring buf;
+  AppendGroupVarInt32(&buf, 0, 0, 0, 0);
+  ASSERT_EQ(5UL, buf.size());
+  ASSERT_EQ(0, memcmp("\x00\x00\x00\x00\x00", buf.data(), 5));
+  buf.clear();
+
+  // All 1-byte
+  AppendGroupVarInt32(&buf, 1, 2, 3, 254);
+  ASSERT_EQ(5UL, buf.size());
+  ASSERT_EQ(0, memcmp("\x00\x01\x02\x03\xfe", buf.data(), 5));
+  buf.clear();
+
+  // Mixed 1-byte and 2-byte
+  AppendGroupVarInt32(&buf, 256, 2, 3, 65535);
+  ASSERT_EQ(7UL, buf.size());
+  ASSERT_EQ(BOOST_BINARY(01 00 00 01), buf.at(0));
+  ASSERT_EQ(256, *reinterpret_cast<const uint16_t *>(&buf[1]));
+  ASSERT_EQ(2, *reinterpret_cast<const uint8_t *>(&buf[3]));
+  ASSERT_EQ(3, *reinterpret_cast<const uint8_t *>(&buf[4]));
+  ASSERT_EQ(65535, *reinterpret_cast<const uint16_t *>(&buf[5]));
+}
+
+
+// Round-trip encode/decodes using group varint
+TEST(TestGroupVarInt, TestRoundTrip) {
+  // A few simple tests.
+  DoTestRoundTripGVI32(0, 0, 0, 0);
+  DoTestRoundTripGVI32(1, 2, 3, 4);
+  DoTestRoundTripGVI32(1, 2000, 3, 200000);
+
+  // Then a randomized test.
+  for (int i = 0; i < 10000; i++) {
+    DoTestRoundTripGVI32(random(), random(), random(), random());
+  }
+}
+
+#ifdef NDEBUG
+TEST(TestGroupVarInt, EncodingBenchmark) {
+  int n_ints = 1000000;
+
+  std::vector<uint32_t> ints;
+  ints.reserve(n_ints);
+  for (int i = 0; i < n_ints; i++) {
+    ints.push_back(i);
+  }
+
+  faststring s;
+  // conservative reservation
+  s.reserve(ints.size() * 4);
+
+  LOG_TIMING(INFO, "Benchmark") {
+    for (int i = 0; i < 100; i++) {
+      s.clear();
+      AppendGroupVarInt32Sequence(&s, 0, &ints[0], n_ints);
+    }
+  }
+}
+#endif
+} // namespace coding
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/group_varint.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/group_varint.cc b/be/src/kudu/util/group_varint.cc
new file mode 100644
index 0000000..47fbeb4
--- /dev/null
+++ b/be/src/kudu/util/group_varint.cc
@@ -0,0 +1,81 @@
+// 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 <cstdint>
+#include <cstring>
+#include <ostream>
+#include <string>
+
+#include <boost/utility/binary.hpp>
+#include <glog/logging.h>
+
+#include "kudu/util/group_varint-inl.h"
+#include "kudu/util/hexdump.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+namespace coding {
+
+bool SSE_TABLE_INITTED = false;
+uint8_t SSE_TABLE[256 * 16] __attribute__((aligned(16)));
+uint8_t VARINT_SELECTOR_LENGTHS[256];
+
+__attribute__((constructor))
+static void InitializeSSETables() {
+  memset(SSE_TABLE, 0xff, sizeof(SSE_TABLE));
+
+  for (int i = 0; i < 256; i++) {
+    uint32_t *entry = reinterpret_cast<uint32_t *>(&SSE_TABLE[i * 16]);
+
+    uint8_t selectors[] = {
+      static_cast<uint8_t>((i & BOOST_BINARY(11 00 00 00)) >> 6),
+      static_cast<uint8_t>((i & BOOST_BINARY(00 11 00 00)) >> 4),
+      static_cast<uint8_t>((i & BOOST_BINARY(00 00 11 00)) >> 2),
+      static_cast<uint8_t>((i & BOOST_BINARY(00 00 00 11))) };
+
+    // 00000000 ->
+    // 00 ff ff ff  01 ff ff ff  02 ff ff ff  03 ff ff ff
+
+    // 01000100 ->
+    // 00 01 ff ff  02 ff ff ff  03 04 ff ff  05 ff ff ff
+
+    uint8_t offset = 0;
+
+    for (int j = 0; j < 4; j++) {
+      uint8_t num_bytes = selectors[j] + 1;
+      uint8_t *entry_bytes = reinterpret_cast<uint8_t *>(&entry[j]);
+
+      for (int k = 0; k < num_bytes; k++) {
+        *entry_bytes++ = offset++;
+      }
+    }
+
+    VARINT_SELECTOR_LENGTHS[i] = offset;
+  }
+
+  SSE_TABLE_INITTED = true;
+}
+
+void DumpSSETable() {
+  LOG(INFO) << "SSE table:\n"
+            << kudu::HexDump(Slice(SSE_TABLE, sizeof(SSE_TABLE)));
+}
+
+
+
+} // namespace coding
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hash_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hash_util-test.cc b/be/src/kudu/util/hash_util-test.cc
new file mode 100644
index 0000000..4e40dd5
--- /dev/null
+++ b/be/src/kudu/util/hash_util-test.cc
@@ -0,0 +1,42 @@
+// 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 <cstdint>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/hash_util.h"
+
+namespace kudu {
+
+// Test Murmur2 Hash64 returns the expected values for inputs. These tests are
+// duplicated on the Java side to ensure that hash computations are stable
+// across both platforms.
+TEST(HashUtilTest, TestMurmur2Hash64) {
+  uint64_t hash;
+
+  hash = HashUtil::MurmurHash2_64("ab", 2, 0);
+  ASSERT_EQ(7115271465109541368, hash);
+
+  hash = HashUtil::MurmurHash2_64("abcdefg", 7, 0);
+  ASSERT_EQ(2601573339036254301, hash);
+
+  hash = HashUtil::MurmurHash2_64("quick brown fox", 15, 42);
+  ASSERT_EQ(3575930248840144026, hash);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hash_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hash_util.h b/be/src/kudu/util/hash_util.h
new file mode 100644
index 0000000..d3a513b
--- /dev/null
+++ b/be/src/kudu/util/hash_util.h
@@ -0,0 +1,71 @@
+// 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.
+
+#ifndef KUDU_UTIL_HASH_UTIL_H
+#define KUDU_UTIL_HASH_UTIL_H
+
+#include <stdint.h>
+
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+/// Utility class to compute hash values.
+class HashUtil {
+ public:
+
+  static const uint64_t MURMUR_PRIME = 0xc6a4a7935bd1e995;
+  static const int MURMUR_R = 47;
+
+  /// Murmur2 hash implementation returning 64-bit hashes.
+  ATTRIBUTE_NO_SANITIZE_INTEGER
+  static uint64_t MurmurHash2_64(const void* input, int len, uint64_t seed) {
+    uint64_t h = seed ^ (len * MURMUR_PRIME);
+
+    const uint64_t* data = reinterpret_cast<const uint64_t*>(input);
+    const uint64_t* end = data + (len / sizeof(uint64_t));
+
+    while (data != end) {
+      uint64_t k = *data++;
+      k *= MURMUR_PRIME;
+      k ^= k >> MURMUR_R;
+      k *= MURMUR_PRIME;
+      h ^= k;
+      h *= MURMUR_PRIME;
+    }
+
+    const uint8_t* data2 = reinterpret_cast<const uint8_t*>(data);
+    switch (len & 7) {
+      case 7: h ^= static_cast<uint64_t>(data2[6]) << 48;
+      case 6: h ^= static_cast<uint64_t>(data2[5]) << 40;
+      case 5: h ^= static_cast<uint64_t>(data2[4]) << 32;
+      case 4: h ^= static_cast<uint64_t>(data2[3]) << 24;
+      case 3: h ^= static_cast<uint64_t>(data2[2]) << 16;
+      case 2: h ^= static_cast<uint64_t>(data2[1]) << 8;
+      case 1: h ^= static_cast<uint64_t>(data2[0]);
+              h *= MURMUR_PRIME;
+    }
+
+    h ^= h >> MURMUR_R;
+    h *= MURMUR_PRIME;
+    h ^= h >> MURMUR_R;
+    return h;
+  }
+};
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/hdr_histogram-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/hdr_histogram-test.cc b/be/src/kudu/util/hdr_histogram-test.cc
new file mode 100644
index 0000000..5d51e98
--- /dev/null
+++ b/be/src/kudu/util/hdr_histogram-test.cc
@@ -0,0 +1,116 @@
+// 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 <cstdint>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/test_macros.h"
+
+namespace kudu {
+
+static const int kSigDigits = 2;
+
+class HdrHistogramTest : public KuduTest {
+};
+
+TEST_F(HdrHistogramTest, SimpleTest) {
+  uint64_t highest_val = 10000LU;
+
+  HdrHistogram hist(highest_val, kSigDigits);
+  ASSERT_EQ(0, hist.CountInBucketForValue(1));
+  hist.Increment(1);
+  ASSERT_EQ(1, hist.CountInBucketForValue(1));
+  hist.IncrementBy(1, 3);
+  ASSERT_EQ(4, hist.CountInBucketForValue(1));
+  hist.Increment(10);
+  ASSERT_EQ(1, hist.CountInBucketForValue(10));
+  hist.Increment(20);
+  ASSERT_EQ(1, hist.CountInBucketForValue(20));
+  ASSERT_EQ(0, hist.CountInBucketForValue(1000));
+  hist.Increment(1000);
+  hist.Increment(1001);
+  ASSERT_EQ(2, hist.CountInBucketForValue(1000));
+
+  ASSERT_EQ(1 + 1 * 3 + 10 + 20 + 1000 + 1001,
+            hist.TotalSum());
+}
+
+TEST_F(HdrHistogramTest, TestCoordinatedOmission) {
+  uint64_t interval = 1000;
+  int loop_iters = 100;
+  int64_t normal_value = 10;
+  HdrHistogram hist(1000000LU, kSigDigits);
+  for (int i = 1; i <= loop_iters; i++) {
+    // Simulate a periodic "large value" that would exhibit coordinated
+    // omission were this loop to sleep on 'interval'.
+    int64_t value = (i % normal_value == 0) ? interval * 10 : normal_value;
+
+    hist.IncrementWithExpectedInterval(value, interval);
+  }
+  ASSERT_EQ(loop_iters - (loop_iters / normal_value),
+            hist.CountInBucketForValue(normal_value));
+  for (int i = interval; i <= interval * 10; i += interval) {
+    ASSERT_EQ(loop_iters / normal_value, hist.CountInBucketForValue(i));
+  }
+}
+
+static const int kExpectedSum =
+  10 * 80 + 100 * 10 + 1000 * 5 + 10000 * 3 + 100000 * 1 + 1000000 * 1;
+static const int kExpectedMax = 1000000;
+static const int kExpectedCount = 100;
+static const int kExpectedMin = 10;
+static void load_percentiles(HdrHistogram* hist) {
+  hist->IncrementBy(10, 80);
+  hist->IncrementBy(100, 10);
+  hist->IncrementBy(1000, 5);
+  hist->IncrementBy(10000, 3);
+  hist->IncrementBy(100000, 1);
+  hist->IncrementBy(1000000, 1);
+}
+
+static void validate_percentiles(HdrHistogram* hist, uint64_t specified_max) {
+  double expected_mean =
+    static_cast<double>(kExpectedSum) / (80 + 10 + 5 + 3 + 1 + 1);
+
+  ASSERT_EQ(kExpectedMin, hist->MinValue());
+  ASSERT_EQ(kExpectedMax, hist->MaxValue());
+  ASSERT_EQ(kExpectedSum, hist->TotalSum());
+  ASSERT_NEAR(expected_mean, hist->MeanValue(), 0.001);
+  ASSERT_EQ(kExpectedCount, hist->TotalCount());
+  ASSERT_EQ(10, hist->ValueAtPercentile(80));
+  ASSERT_EQ(kExpectedCount, hist->ValueAtPercentile(90));
+  ASSERT_EQ(hist->LowestEquivalentValue(specified_max), hist->ValueAtPercentile(99));
+  ASSERT_EQ(hist->LowestEquivalentValue(specified_max), hist->ValueAtPercentile(99.99));
+  ASSERT_EQ(hist->LowestEquivalentValue(specified_max), hist->ValueAtPercentile(100));
+}
+
+TEST_F(HdrHistogramTest, PercentileAndCopyTest) {
+  uint64_t specified_max = 10000;
+  HdrHistogram hist(specified_max, kSigDigits);
+  load_percentiles(&hist);
+  NO_FATALS(validate_percentiles(&hist, specified_max));
+
+  HdrHistogram copy(hist);
+  NO_FATALS(validate_percentiles(&copy, specified_max));
+
+  ASSERT_EQ(hist.TotalSum(), copy.TotalSum());
+}
+
+} // namespace kudu


[49/51] [abbrv] impala git commit: IMPALA-4669: [KUTIL] Add kudu_util library to the build.

Posted by ta...@apache.org.
IMPALA-4669: [KUTIL] Add kudu_util library to the build.

NOTE: This commit is part of a set of changes for IMPALA-7006. It
contains pieces of a previous commit that need to be cherry picked
again after rebasing the code in be/src/kudu/{util,security,rpc}.

The original commit message is below:

A few miscellaneous changes to allow kudu_util to compile with Impala.

Add kudu_version.cc to substitute for the version.cc file that is
automatically built during the full Kudu build.

Set LZ4_DISABLE_DEPRECATE_WARNINGS to allow Kudu's compressor utility to
use deprecated names for LZ4 methods.

Add NO_NVM_SUPPORT flag to Kudu build (plan to upstream this later) to
disable building with nvm support, removing a library dependency.

Also remove imported FindOpenSSL.cmake in favour of the standard one provided
by cmake itself.

Finally, a few changes to allow compilation on RHEL5:

* Only use sched_getcpu() if supported
* Only include magic.h if available
* Workaround for kernels that don't have SOCK_NONBLOCK
* Workaround for kernels that don't have O_CLOEXEC (ignore the flag)
* Provide non-working implementation of fallocate()
* Disable inclusion of linux/fiemap.h - although this exists on RHEL5,
  it does not compile due to other #includes in env_posix.cc. We disable
  the path this is used for, since Impala does not call that code.
* Use Kudu's implementation of pipe(2), preadv(2) and pwritev(2) where
  it doesn't exist.

In most cases these changes simply force kutil to revert to a different
implementation that was already written for OSX support - this patch
generalises the logic to provide the implementation whenever the
required function doesn't exist.

This patch compiles on RHEL5.5 and 6.0, SLES11 and 12, Ubuntu 12.04 and
14.04 and Debian 7.0 and 8.0.

Change-Id: I451f02d3e4669e8a548b92fb1445cb2b322659a2
Reviewed-on: http://gerrit.cloudera.org:8080/5715
Tested-by: Impala Public Jenkins
Reviewed-by: Henry Robinson <he...@cloudera.com>
Reviewed-on: http://gerrit.cloudera.org:8080/10758
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Lars Volker <lv...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/e1d3a616
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/e1d3a616
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/e1d3a616

Branch: refs/heads/master
Commit: e1d3a616651c93a0164d3e6377f16aaae298d8cd
Parents: fcf190c
Author: Lars Volker <lv...@cloudera.com>
Authored: Tue Jul 3 16:46:23 2018 -0700
Committer: Lars Volker <lv...@cloudera.com>
Committed: Thu Jul 12 21:35:42 2018 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  |  5 +-
 LICENSE.txt                                     | 61 +++++++++++++++++++
 be/src/common/kudu_version.cc                   |  5 ++
 be/src/kudu/util/CMakeLists.txt                 | 12 ++--
 .../kudu/util/compression/compression_codec.cc  |  1 +
 be/src/kudu/util/flags.cc                       |  4 +-
 be/src/kudu/util/kudu_export.h                  | 62 ++++++++++++++++++++
 be/src/kudu/util/logging.cc                     | 11 ++--
 be/src/kudu/util/minidump.cc                    | 20 ++-----
 bin/rat_exclude_files.txt                       |  1 +
 10 files changed, 150 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 77bf210..68c7f6b 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -19,8 +19,6 @@ cmake_minimum_required(VERSION 3.2.3)
 
 # This is a Kudu-specific flag that disables Kudu targets that are test-only.
 set(NO_TESTS 1)
-# Kudu-specific, disable NVM support so that libvmem is not a dependency.
-set(NO_NVM_SUPPORT 1)
 
 # Explicitly define project() to allow modifying the compiler before the project is
 # initialized.
@@ -348,7 +346,8 @@ endif()
 if (NOT APPLE)
   find_package(LibUnwind REQUIRED)
   include_directories(SYSTEM ${LIBUNWIND_INCLUDE_DIR})
-  IMPALA_ADD_THIRDPARTY_LIB(libunwind ${LIBUNWIND_INCLUDE_DIR} ${LIBUNWIND_STATIC_LIB} "")
+  IMPALA_ADD_THIRDPARTY_LIB(libunwind ${LIBUNWIND_INCLUDE_DIR} ${LIBUNWIND_STATIC_LIB}
+    ${LIBUNWIND_SHARED_LIB})
 endif()
 
 # Required for KRPC_GENERATE, which converts protobuf to stubs.

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 789aa18..b8c2c01 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -798,6 +798,67 @@ Some portions of this module are derived from code from krb5
 
 --------------------------------------------------------------------------------
 
+be/src/kudu/util/array_view.h: 3-clause BSD license with patent grant
+
+  Copyright (c) 2015, The WebRTC project authors. All rights reserved.
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are
+  met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in
+      the documentation and/or other materials provided with the
+      distribution.
+
+    * Neither the name of Google nor the names of its contributors may
+      be used to endorse or promote products derived from this software
+      without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+  HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+  Additional IP Rights Grant (Patents)
+  ------------------------------------
+
+  "This implementation" means the copyrightable works distributed by
+  Google as part of the WebRTC code package.
+
+  Google hereby grants to you a perpetual, worldwide, non-exclusive,
+  no-charge, irrevocable (except as stated in this section) patent
+  license to make, have made, use, offer to sell, sell, import,
+  transfer, and otherwise run, modify and propagate the contents of this
+  implementation of the WebRTC code package, where such license applies
+  only to those patent claims, both currently owned by Google and
+  acquired in the future, licensable by Google that are necessarily
+  infringed by this implementation of the WebRTC code package. This
+  grant does not include claims that would be infringed only as a
+  consequence of further modification of this implementation. If you or
+  your agent or exclusive licensee institute or order or agree to the
+  institution of patent litigation against any entity (including a
+  cross-claim or counterclaim in a lawsuit) alleging that this
+  implementation of the WebRTC code package or any code incorporated
+  within this implementation of the WebRTC code package constitutes
+  direct or contributory patent infringement, or inducement of patent
+  infringement, then any patent rights granted to you under this License
+  for this implementation of the WebRTC code package shall terminate as
+  of the date such litigation is filed.
+
+--------------------------------------------------------------------------------
+
 be/src/kudu/util/x509_check_host.*: OpenSSL software license:
 
   LICENSE ISSUES

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/common/kudu_version.cc
----------------------------------------------------------------------
diff --git a/be/src/common/kudu_version.cc b/be/src/common/kudu_version.cc
index 3ffa009..3633d4f 100644
--- a/be/src/common/kudu_version.cc
+++ b/be/src/common/kudu_version.cc
@@ -27,6 +27,11 @@ void VersionInfo::GetVersionInfoPB(VersionInfoPB* pb) {
   DCHECK(false);
 }
 
+std::string VersionInfo::GetShortVersionInfo() {
+  DCHECK(false);
+  return "";
+}
+
 std::string VersionInfo::GetAllVersionInfo() {
   DCHECK(false);
   return "";

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/CMakeLists.txt b/be/src/kudu/util/CMakeLists.txt
index 94dd2fa..2ca852d 100644
--- a/be/src/kudu/util/CMakeLists.txt
+++ b/be/src/kudu/util/CMakeLists.txt
@@ -160,7 +160,7 @@ set(UTIL_SRCS
   memory/overwrite.cc
   mem_tracker.cc
   metrics.cc
-  minidump.cc
+  # minidump.cc
   monotime.cc
   mutex.cc
   net/dns_resolver.cc
@@ -198,7 +198,9 @@ 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
   website_util.cc
   zlib.cc
@@ -282,7 +284,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)
 
 if(HAVE_LIB_VMEM)
@@ -329,7 +333,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

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/compression/compression_codec.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/compression/compression_codec.cc b/be/src/kudu/util/compression/compression_codec.cc
index a2231b6..2359066 100644
--- a/be/src/kudu/util/compression/compression_codec.cc
+++ b/be/src/kudu/util/compression/compression_codec.cc
@@ -23,6 +23,7 @@
 #include <vector>
 
 #include <glog/logging.h>
+#define LZ4_DISABLE_DEPRECATE_WARNINGS
 #include <lz4.h>
 #include <snappy-sinksource.h>
 #include <snappy.h>

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/flags.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/flags.cc b/be/src/kudu/util/flags.cc
index 047c893..4b0c2d1 100644
--- a/be/src/kudu/util/flags.cc
+++ b/be/src/kudu/util/flags.cc
@@ -74,9 +74,7 @@ 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.");
+DECLARE_bool(enable_process_lifetime_heap_profiling);
 TAG_FLAG(enable_process_lifetime_heap_profiling, stable);
 TAG_FLAG(enable_process_lifetime_heap_profiling, advanced);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/kudu_export.h
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/logging.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging.cc b/be/src/kudu/util/logging.cc
index fcf035f..7d017e7 100644
--- a/be/src/kudu/util/logging.cc
+++ b/be/src/kudu/util/logging.cc
@@ -46,9 +46,7 @@
 #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]");
+DECLARE_string(log_filename);
 TAG_FLAG(log_filename, stable);
 
 DEFINE_bool(log_async, true,
@@ -61,9 +59,7 @@ 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.");
+DECLARE_int32(max_log_files);
 TAG_FLAG(max_log_files, runtime);
 TAG_FLAG(max_log_files, experimental);
 
@@ -272,7 +268,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();

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/be/src/kudu/util/minidump.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/minidump.cc b/be/src/kudu/util/minidump.cc
index d02dc5d..3ae1bf9 100644
--- a/be/src/kudu/util/minidump.cc
+++ b/be/src/kudu/util/minidump.cc
@@ -28,8 +28,8 @@
 #include <string>
 
 #if defined(__linux__)
-#include <breakpad/client/linux/handler/exception_handler.h>
-#include <breakpad/common/linux/linux_libc_support.h>
+#include <client/linux/handler/exception_handler.h>
+#include <common/linux/linux_libc_support.h>
 #endif // defined(__linux__)
 
 #include <gflags/gflags.h>
@@ -71,13 +71,7 @@ static bool ValidateMinidumpEnabled(const char* /*flagname*/, bool value) {
 }
 DEFINE_validator(enable_minidumps, &ValidateMinidumpEnabled);
 
-DEFINE_string(minidump_path, "minidumps", "Directory to write minidump files to. This "
-    "can be either an absolute path or a path relative to --log_dir. Each daemon will "
-    "create an additional sub-directory to prevent naming conflicts and to make it "
-    "easier to identify a crashing daemon. Minidump files contain crash-related "
-    "information in a compressed format. Minidumps will be written when a daemon exits "
-    "unexpectedly, for example on an unhandled exception or signal, or when a "
-    "SIGUSR1 signal is sent to the process. Cannot be set to an empty value.");
+DECLARE_string(minidump_path);
 TAG_FLAG(minidump_path, evolving);
 // The minidump path cannot be empty.
 static bool ValidateMinidumpPath(const char* /*flagname*/, const string& value) {
@@ -85,14 +79,10 @@ static bool ValidateMinidumpPath(const char* /*flagname*/, const string& value)
 }
 DEFINE_validator(minidump_path, &ValidateMinidumpPath);
 
-DEFINE_int32(max_minidumps, 9, "Maximum number of minidump files to keep per daemon. "
-    "Older files are removed first. Set to 0 to keep all minidump files.");
+DECLARE_int32(max_minidumps);
 TAG_FLAG(max_minidumps, evolving);
 
-DEFINE_int32(minidump_size_limit_hint_kb, 20480, "Size limit hint for minidump files in "
-    "KB. If a minidump exceeds this value, then breakpad will reduce the stack memory it "
-    "collects for each thread from 8KB to 2KB. However it will always include the full "
-    "stack memory for the first 20 threads, including the thread that crashed.");
+DECLARE_int32(minidump_size_limit_hint_kb);
 TAG_FLAG(minidump_size_limit_hint_kb, advanced);
 TAG_FLAG(minidump_size_limit_hint_kb, evolving);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e1d3a616/bin/rat_exclude_files.txt
----------------------------------------------------------------------
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 1c1171b..170dd24 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -40,6 +40,7 @@ www/d3.v3.min.js
 www/jquery/jquery-1.12.4.min.js
 tests/comparison/leopard/static/css/hljs.css
 tests/comparison/leopard/static/js/highlight.pack.js
+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


[36/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/serialization.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/serialization.h b/be/src/kudu/rpc/serialization.h
new file mode 100644
index 0000000..8406a1f
--- /dev/null
+++ b/be/src/kudu/rpc/serialization.h
@@ -0,0 +1,88 @@
+// 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.
+
+#ifndef KUDU_RPC_SERIALIZATION_H
+#define KUDU_RPC_SERIALIZATION_H
+
+#include <cstdint>
+#include <cstring>
+
+namespace google {
+namespace protobuf {
+class MessageLite;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Status;
+class faststring;
+class Slice;
+
+namespace rpc {
+namespace serialization {
+
+// Serialize the request param into a buffer which is allocated by this function.
+// Uses the message's cached size by calling MessageLite::GetCachedSize().
+// In : 'message' Protobuf Message to serialize
+//      'additional_size' Optional argument which increases the recorded size
+//        within param_buf. This argument is necessary if there will be
+//        additional sidecars appended onto the message (that aren't part of
+//        the protobuf itself).
+//      'use_cached_size' Additional optional argument whether to use the cached
+//        or explicit byte size by calling MessageLite::GetCachedSize() or
+//        MessageLite::ByteSize(), respectively.
+// Out: The faststring 'param_buf' to be populated with the serialized bytes.
+//        The faststring's length is only determined by the amount that
+//        needs to be serialized for the protobuf (i.e., no additional space
+//        is reserved for 'additional_size', which only affects the
+//        size indicator prefix in 'param_buf').
+void SerializeMessage(const google::protobuf::MessageLite& message,
+                      faststring* param_buf, int additional_size = 0,
+                      bool use_cached_size = false);
+
+// Serialize the request or response header into a buffer which is allocated
+// by this function.
+// Includes leading 32-bit length of the buffer.
+// In: Protobuf Header to serialize,
+//     Length of the message param following this header in the frame.
+// Out: faststring to be populated with the serialized bytes.
+void SerializeHeader(const google::protobuf::MessageLite& header,
+                     size_t param_len,
+                     faststring* header_buf);
+
+// Deserialize the request.
+// In: data buffer Slice.
+// Out: parsed_header PB initialized,
+//      parsed_main_message pointing to offset in original buffer containing
+//      the main payload.
+Status ParseMessage(const Slice& buf,
+                    google::protobuf::MessageLite* parsed_header,
+                    Slice* parsed_main_message);
+
+// Serialize the RPC connection header (magic number + flags).
+// buf must have 7 bytes available (kMagicNumberLength + kHeaderFlagsLength).
+void SerializeConnHeader(uint8_t* buf);
+
+// Validate the entire rpc header (magic number + flags).
+Status ValidateConnHeader(const Slice& slice);
+
+
+} // namespace serialization
+} // namespace rpc
+} // namespace kudu
+#endif // KUDU_RPC_SERIALIZATION_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/server_negotiation.cc b/be/src/kudu/rpc/server_negotiation.cc
new file mode 100644
index 0000000..612701f
--- /dev/null
+++ b/be/src/kudu/rpc/server_negotiation.cc
@@ -0,0 +1,989 @@
+// 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 "kudu/rpc/server_negotiation.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <set>
+#include <string>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/blocking_ops.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/init.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/fault_injection.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/trace.h"
+
+using std::set;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+// Fault injection flags.
+DEFINE_double(rpc_inject_invalid_authn_token_ratio, 0,
+              "If set higher than 0, AuthenticateByToken() randomly injects "
+              "errors replying with FATAL_INVALID_AUTHENTICATION_TOKEN code. "
+              "The flag's value corresponds to the probability of the fault "
+              "injection event. Used for only for tests.");
+TAG_FLAG(rpc_inject_invalid_authn_token_ratio, runtime);
+TAG_FLAG(rpc_inject_invalid_authn_token_ratio, unsafe);
+
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
+DEFINE_string(trusted_subnets,
+              "127.0.0.0/8,10.0.0.0/8,172.16.0.0/12,192.168.0.0/16,169.254.0.0/16",
+              "A trusted subnet whitelist. If set explicitly, all unauthenticated "
+              "or unencrypted connections are prohibited except the ones from the "
+              "specified address blocks. Otherwise, private network (127.0.0.0/8, etc.) "
+              "and local subnets of all local network interfaces will be used. Set it "
+              "to '0.0.0.0/0' to allow unauthenticated/unencrypted connections from all "
+              "remote IP addresses. However, if network access is not otherwise restricted "
+              "by a firewall, malicious users may be able to gain unauthorized access.");
+TAG_FLAG(trusted_subnets, advanced);
+TAG_FLAG(trusted_subnets, evolving);
+
+static bool ValidateTrustedSubnets(const char* /*flagname*/, const string& value) {
+  if (value.empty()) {
+    return true;
+  }
+
+  for (const auto& t : strings::Split(value, ",", strings::SkipEmpty())) {
+    kudu::Network network;
+    kudu::Status s = network.ParseCIDRString(t.ToString());
+    if (!s.ok()) {
+      LOG(ERROR) << "Invalid subnet address: " << t
+                 << ". Subnet must be specified in CIDR notation.";
+      return false;
+    }
+  }
+
+  return true;
+}
+
+DEFINE_validator(trusted_subnets, &ValidateTrustedSubnets);
+
+namespace kudu {
+namespace rpc {
+
+namespace {
+vector<Network>* g_trusted_subnets = nullptr;
+} // anonymous namespace
+
+static int ServerNegotiationGetoptCb(ServerNegotiation* server_negotiation,
+                                     const char* plugin_name,
+                                     const char* option,
+                                     const char** result,
+                                     unsigned* len) {
+  return server_negotiation->GetOptionCb(plugin_name, option, result, len);
+}
+
+static int ServerNegotiationPlainAuthCb(sasl_conn_t* conn,
+                                        ServerNegotiation* server_negotiation,
+                                        const char* user,
+                                        const char* pass,
+                                        unsigned passlen,
+                                        struct propctx* propctx) {
+  return server_negotiation->PlainAuthCb(conn, user, pass, passlen, propctx);
+}
+
+ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket,
+                                     const security::TlsContext* tls_context,
+                                     const security::TokenVerifier* token_verifier,
+                                     RpcEncryption encryption,
+                                     std::string sasl_proto_name)
+    : socket_(std::move(socket)),
+      helper_(SaslHelper::SERVER),
+      tls_context_(tls_context),
+      encryption_(encryption),
+      tls_negotiated_(false),
+      token_verifier_(token_verifier),
+      negotiated_authn_(AuthenticationType::INVALID),
+      negotiated_mech_(SaslMechanism::INVALID),
+      sasl_proto_name_(std::move(sasl_proto_name)),
+      deadline_(MonoTime::Max()) {
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
+      reinterpret_cast<int (*)()>(&ServerNegotiationGetoptCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_SERVER_USERDB_CHECKPASS,
+      reinterpret_cast<int (*)()>(&ServerNegotiationPlainAuthCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
+}
+
+Status ServerNegotiation::EnablePlain() {
+  return helper_.EnablePlain();
+}
+
+Status ServerNegotiation::EnableGSSAPI() {
+  return helper_.EnableGSSAPI();
+}
+
+SaslMechanism::Type ServerNegotiation::negotiated_mechanism() const {
+  return negotiated_mech_;
+}
+
+void ServerNegotiation::set_server_fqdn(const string& domain_name) {
+  helper_.set_server_fqdn(domain_name);
+}
+
+void ServerNegotiation::set_deadline(const MonoTime& deadline) {
+  deadline_ = deadline;
+}
+
+Status ServerNegotiation::Negotiate() {
+  TRACE("Beginning negotiation");
+
+  // Wait until starting negotiation to check that the socket, tls_context, and
+  // token_verifier are not null, since they do not need to be set for
+  // PreflightCheckGSSAPI.
+  DCHECK(socket_);
+  DCHECK(tls_context_);
+  DCHECK(token_verifier_);
+
+  // Ensure we can use blocking calls on the socket during negotiation.
+  RETURN_NOT_OK(CheckInBlockingMode(socket_.get()));
+
+  faststring recv_buf;
+
+  // Step 1: Read the connection header.
+  RETURN_NOT_OK(ValidateConnectionHeader(&recv_buf));
+
+  { // Step 2: Receive and respond to the NEGOTIATE step message.
+    NegotiatePB request;
+    RETURN_NOT_OK(RecvNegotiatePB(&request, &recv_buf));
+    RETURN_NOT_OK(HandleNegotiate(request));
+    TRACE("Negotiated authn=$0", AuthenticationTypeToString(negotiated_authn_));
+  }
+
+  // Step 3: if both ends support TLS, do a TLS handshake.
+  if (encryption_ != RpcEncryption::DISABLED &&
+      tls_context_->has_cert() &&
+      ContainsKey(client_features_, TLS)) {
+    RETURN_NOT_OK(tls_context_->InitiateHandshake(security::TlsHandshakeType::SERVER,
+                                                  &tls_handshake_));
+
+    if (negotiated_authn_ != AuthenticationType::CERTIFICATE) {
+      // The server does not need to verify the client's certificate unless it's
+      // being used for authentication.
+      tls_handshake_.set_verification_mode(security::TlsVerificationMode::VERIFY_NONE);
+    }
+
+    while (true) {
+      NegotiatePB request;
+      RETURN_NOT_OK(RecvNegotiatePB(&request, &recv_buf));
+      Status s = HandleTlsHandshake(request);
+      if (s.ok()) break;
+      if (!s.IsIncomplete()) return s;
+    }
+    tls_negotiated_ = true;
+  }
+
+  // Rejects any connection from public routable IPs if encryption
+  // is disabled. See KUDU-1875.
+  if (!tls_negotiated_) {
+    Sockaddr addr;
+    RETURN_NOT_OK(socket_->GetPeerAddress(&addr));
+
+    if (!IsTrustedConnection(addr)) {
+      // Receives client response before sending error
+      // message, even though the response is never used,
+      // to avoid risk condition that connection gets
+      // closed before client receives server's error
+      // message.
+      NegotiatePB request;
+      RETURN_NOT_OK(RecvNegotiatePB(&request, &recv_buf));
+
+      Status s = Status::NotAuthorized("unencrypted connections from publicly routable "
+                                       "IPs are prohibited. See --trusted_subnets flag "
+                                       "for more information.",
+                                       addr.ToString());
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+      return s;
+    }
+  }
+
+  // Step 4: Authentication
+  switch (negotiated_authn_) {
+    case AuthenticationType::SASL:
+      RETURN_NOT_OK(AuthenticateBySasl(&recv_buf));
+      break;
+    case AuthenticationType::TOKEN:
+      RETURN_NOT_OK(AuthenticateByToken(&recv_buf));
+      break;
+    case AuthenticationType::CERTIFICATE:
+      RETURN_NOT_OK(AuthenticateByCertificate());
+      break;
+    case AuthenticationType::INVALID: LOG(FATAL) << "unreachable";
+  }
+
+  // Step 5: Receive connection context.
+  RETURN_NOT_OK(RecvConnectionContext(&recv_buf));
+
+  TRACE("Negotiation successful");
+  return Status::OK();
+}
+
+Status ServerNegotiation::PreflightCheckGSSAPI(const std::string& sasl_proto_name) {
+  // TODO(todd): the error messages that come from this function on el6
+  // are relatively useless due to the following krb5 bug:
+  // http://krbdev.mit.edu/rt/Ticket/Display.html?id=6973
+  // This may not be useful anymore given the keytab login that happens
+  // in security/init.cc.
+
+  // Initialize a ServerNegotiation with a null socket, and enable
+  // only GSSAPI.
+  //
+  // We aren't going to actually send/receive any messages, but
+  // this makes it easier to reuse the initialization code.
+  ServerNegotiation server(
+      nullptr, nullptr, nullptr, RpcEncryption::OPTIONAL, sasl_proto_name);
+  Status s = server.EnableGSSAPI();
+  if (!s.ok()) {
+    return Status::RuntimeError(s.message());
+  }
+
+  RETURN_NOT_OK(server.InitSaslServer());
+
+  // Start the SASL server as if we were accepting a connection.
+  const char* server_out = nullptr; // ignored
+  uint32_t server_out_len = 0;
+  s = WrapSaslCall(server.sasl_conn_.get(), [&]() {
+      return sasl_server_start(
+          server.sasl_conn_.get(),
+          kSaslMechGSSAPI,
+          "", 0,  // Pass a 0-length token.
+          &server_out, &server_out_len);
+    });
+
+  // We expect 'Incomplete' status to indicate that the first step of negotiation
+  // was correct.
+  if (s.IsIncomplete()) return Status::OK();
+
+  string err_msg = s.message().ToString();
+  if (err_msg == "Permission denied") {
+    // For bad keytab permissions, we get a rather vague message. So,
+    // we make it more specific for better usability.
+    err_msg = "error accessing keytab: " + err_msg;
+  }
+  return Status::RuntimeError(err_msg);
+}
+
+Status ServerNegotiation::RecvNegotiatePB(NegotiatePB* msg, faststring* recv_buf) {
+  RequestHeader header;
+  Slice param_buf;
+  RETURN_NOT_OK(ReceiveFramedMessageBlocking(socket(), recv_buf, &header, &param_buf, deadline_));
+  Status s = helper_.CheckNegotiateCallId(header.call_id());
+  if (!s.ok()) {
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_INVALID_RPC_HEADER, s));
+    return s;
+  }
+
+  s = helper_.ParseNegotiatePB(param_buf, msg);
+  if (!s.ok()) {
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_DESERIALIZING_REQUEST, s));
+    return s;
+  }
+
+  TRACE("Received $0 NegotiatePB request", NegotiatePB::NegotiateStep_Name(msg->step()));
+  return Status::OK();
+}
+
+Status ServerNegotiation::SendNegotiatePB(const NegotiatePB& msg) {
+  ResponseHeader header;
+  header.set_call_id(kNegotiateCallId);
+
+  DCHECK(socket_);
+  DCHECK(msg.IsInitialized()) << "message must be initialized";
+  DCHECK(msg.has_step()) << "message must have a step";
+
+  TRACE("Sending $0 NegotiatePB response", NegotiatePB::NegotiateStep_Name(msg.step()));
+  return SendFramedMessageBlocking(socket(), header, msg, deadline_);
+}
+
+Status ServerNegotiation::SendError(ErrorStatusPB::RpcErrorCodePB code, const Status& err) {
+  DCHECK(!err.ok());
+
+  // Create header with negotiation-specific callId
+  ResponseHeader header;
+  header.set_call_id(kNegotiateCallId);
+  header.set_is_error(true);
+
+  // Get RPC error code from Status object
+  ErrorStatusPB msg;
+  msg.set_code(code);
+  msg.set_message(err.ToString());
+
+  TRACE("Sending RPC error: $0: $1", ErrorStatusPB::RpcErrorCodePB_Name(code), err.ToString());
+  RETURN_NOT_OK(SendFramedMessageBlocking(socket(), header, msg, deadline_));
+
+  return Status::OK();
+}
+
+Status ServerNegotiation::ValidateConnectionHeader(faststring* recv_buf) {
+  TRACE("Waiting for connection header");
+  size_t num_read;
+  const size_t conn_header_len = kMagicNumberLength + kHeaderFlagsLength;
+  recv_buf->resize(conn_header_len);
+  RETURN_NOT_OK(socket_->BlockingRecv(recv_buf->data(), conn_header_len, &num_read, deadline_));
+  DCHECK_EQ(conn_header_len, num_read);
+
+  RETURN_NOT_OK(serialization::ValidateConnHeader(*recv_buf));
+  TRACE("Connection header received");
+  return Status::OK();
+}
+
+// calls sasl_server_init() and sasl_server_new()
+Status ServerNegotiation::InitSaslServer() {
+  // TODO(unknown): Support security flags.
+  unsigned secflags = 0;
+
+  sasl_conn_t* sasl_conn = nullptr;
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(nullptr /* no conn */, [&]() {
+      return sasl_server_new(
+          // Registered name of the service using SASL. Required.
+          sasl_proto_name_.c_str(),
+          // The fully qualified domain name of this server.
+          helper_.server_fqdn(),
+          // Permits multiple user realms on server. NULL == use default.
+          nullptr,
+          // Local and remote IP address strings. We don't use any mechanisms
+          // which need these.
+          nullptr,
+          nullptr,
+          // Connection-specific callbacks.
+          &callbacks_[0],
+          // Security flags.
+          secflags,
+          &sasl_conn);
+    }), "Unable to create new SASL server");
+  sasl_conn_.reset(sasl_conn);
+  return Status::OK();
+}
+
+Status ServerNegotiation::HandleNegotiate(const NegotiatePB& request) {
+  if (request.step() != NegotiatePB::NEGOTIATE) {
+    Status s = Status::NotAuthorized("expected NEGOTIATE step",
+                                     NegotiatePB::NegotiateStep_Name(request.step()));
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+  TRACE("Received NEGOTIATE request from client");
+
+  // Fill in the set of features supported by the client.
+  for (int flag : request.supported_features()) {
+    // We only add the features that our local build knows about.
+    RpcFeatureFlag feature_flag = RpcFeatureFlag_IsValid(flag) ?
+                                  static_cast<RpcFeatureFlag>(flag) : UNKNOWN;
+    if (feature_flag != UNKNOWN) {
+      client_features_.insert(feature_flag);
+    }
+  }
+
+  if (encryption_ == RpcEncryption::REQUIRED &&
+      !ContainsKey(client_features_, RpcFeatureFlag::TLS)) {
+    Status s = Status::NotAuthorized("client does not support required TLS encryption");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  // Find the set of mutually supported authentication types.
+  set<AuthenticationType> authn_types;
+  if (request.authn_types().empty()) {
+    // If the client doesn't send any support authentication types, we assume
+    // support for SASL. This preserves backwards compatibility with clients who
+    // don't support security features.
+    authn_types.insert(AuthenticationType::SASL);
+  } else {
+    for (const auto& type : request.authn_types()) {
+      switch (type.type_case()) {
+        case AuthenticationTypePB::kSasl:
+          authn_types.insert(AuthenticationType::SASL);
+          break;
+        case AuthenticationTypePB::kToken:
+          authn_types.insert(AuthenticationType::TOKEN);
+          break;
+        case AuthenticationTypePB::kCertificate:
+          // We only provide authenticated TLS if the certificates are generated
+          // by the internal CA.
+          if (!tls_context_->is_external_cert()) {
+            authn_types.insert(AuthenticationType::CERTIFICATE);
+          }
+          break;
+        case AuthenticationTypePB::TYPE_NOT_SET: {
+          Sockaddr addr;
+          RETURN_NOT_OK(socket_->GetPeerAddress(&addr));
+          KLOG_EVERY_N_SECS(WARNING, 60)
+              << "client supports unknown authentication type, consider updating server, address: "
+              << addr.ToString();
+          break;
+        }
+      }
+    }
+
+    if (authn_types.empty()) {
+      Status s = Status::NotSupported("no mutually supported authentication types");
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+      return s;
+    }
+  }
+
+  if (encryption_ != RpcEncryption::DISABLED &&
+      ContainsKey(authn_types, AuthenticationType::CERTIFICATE) &&
+      tls_context_->has_signed_cert()) {
+    // If the client supports it and we are locally configured with TLS and have
+    // a CA-signed cert, choose cert authn.
+    // TODO(KUDU-1924): consider adding the fingerprint of the CA cert which signed
+    // the client's cert to the authentication message.
+    negotiated_authn_ = AuthenticationType::CERTIFICATE;
+  } else if (ContainsKey(authn_types, AuthenticationType::TOKEN) &&
+             token_verifier_->GetMaxKnownKeySequenceNumber() >= 0 &&
+             encryption_ != RpcEncryption::DISABLED &&
+             tls_context_->has_signed_cert()) {
+    // If the client supports it, we have a TSK to verify the client's token,
+    // and we have a signed-cert so the client can verify us, choose token authn.
+    // TODO(KUDU-1924): consider adding the TSK sequence number to the authentication
+    // message.
+    negotiated_authn_ = AuthenticationType::TOKEN;
+  } else {
+    // Otherwise we always can fallback to SASL.
+    DCHECK(ContainsKey(authn_types, AuthenticationType::SASL));
+    negotiated_authn_ = AuthenticationType::SASL;
+  }
+
+  // Fill in the NEGOTIATE step response for the client.
+  NegotiatePB response;
+  response.set_step(NegotiatePB::NEGOTIATE);
+
+  // Tell the client which features we support.
+  server_features_ = kSupportedServerRpcFeatureFlags;
+  if (tls_context_->has_cert() && encryption_ != RpcEncryption::DISABLED) {
+    server_features_.insert(TLS);
+    // If the remote peer is local, then we allow using TLS for authentication
+    // without encryption or integrity.
+    if (socket_->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections) {
+      server_features_.insert(TLS_AUTHENTICATION_ONLY);
+    }
+  }
+
+  for (RpcFeatureFlag feature : server_features_) {
+    response.add_supported_features(feature);
+  }
+
+  switch (negotiated_authn_) {
+    case AuthenticationType::CERTIFICATE:
+      response.add_authn_types()->mutable_certificate();
+      break;
+    case AuthenticationType::TOKEN:
+      response.add_authn_types()->mutable_token();
+      break;
+    case AuthenticationType::SASL: {
+      response.add_authn_types()->mutable_sasl();
+      const set<SaslMechanism::Type>& server_mechs = helper_.EnabledMechs();
+      if (PREDICT_FALSE(server_mechs.empty())) {
+        // This will happen if no mechanisms are enabled before calling Init()
+        Status s = Status::NotAuthorized("SASL server mechanism list is empty!");
+        LOG(ERROR) << s.ToString();
+        TRACE("Sending FATAL_UNAUTHORIZED response to client");
+        RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+        return s;
+      }
+
+      for (auto mechanism : server_mechs) {
+        response.add_sasl_mechanisms()->set_mechanism(SaslMechanism::name_of(mechanism));
+      }
+      break;
+    }
+    case AuthenticationType::INVALID: LOG(FATAL) << "unreachable";
+  }
+
+  return SendNegotiatePB(response);
+}
+
+Status ServerNegotiation::HandleTlsHandshake(const NegotiatePB& request) {
+  if (PREDICT_FALSE(request.step() != NegotiatePB::TLS_HANDSHAKE)) {
+    Status s =  Status::NotAuthorized("expected TLS_HANDSHAKE step",
+                                      NegotiatePB::NegotiateStep_Name(request.step()));
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  if (PREDICT_FALSE(!request.has_tls_handshake())) {
+    Status s = Status::NotAuthorized(
+        "No TLS handshake token in TLS_HANDSHAKE request from client");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  string token;
+  Status s = tls_handshake_.Continue(request.tls_handshake(), &token);
+
+  if (PREDICT_FALSE(!s.IsIncomplete() && !s.ok())) {
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  // Regardless of whether this is the final handshake roundtrip (in which case
+  // Continue would have returned OK), we still need to return a response.
+  RETURN_NOT_OK(SendTlsHandshake(std::move(token)));
+  RETURN_NOT_OK(s);
+
+  // TLS handshake is finished.
+  if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
+      ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    TRACE("Negotiated auth-only $0 with cipher $1",
+          tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+    return tls_handshake_.FinishNoWrap(*socket_);
+  }
+
+  TRACE("Negotiated $0 with cipher $1",
+        tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+  return tls_handshake_.Finish(&socket_);
+}
+
+Status ServerNegotiation::SendTlsHandshake(string tls_token) {
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::TLS_HANDSHAKE);
+  msg.mutable_tls_handshake()->swap(tls_token);
+  return SendNegotiatePB(msg);
+}
+
+Status ServerNegotiation::AuthenticateBySasl(faststring* recv_buf) {
+  RETURN_NOT_OK(InitSaslServer());
+
+  NegotiatePB request;
+  RETURN_NOT_OK(RecvNegotiatePB(&request, recv_buf));
+  Status s = HandleSaslInitiate(request);
+
+  while (s.IsIncomplete()) {
+    RETURN_NOT_OK(RecvNegotiatePB(&request, recv_buf));
+    s = HandleSaslResponse(request);
+  }
+  RETURN_NOT_OK(s);
+
+  const char* c_username = nullptr;
+  int rc = sasl_getprop(sasl_conn_.get(), SASL_USERNAME,
+                        reinterpret_cast<const void**>(&c_username));
+  // We expect that SASL_USERNAME will always get set.
+  CHECK(rc == SASL_OK && c_username != nullptr) << "No username on authenticated connection";
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    // The SASL library doesn't include the user's realm in the username if it's the
+    // same realm as the default realm of the server. So, we pass it back through the
+    // Kerberos library to add back the realm if necessary.
+    string principal = c_username;
+    RETURN_NOT_OK_PREPEND(security::CanonicalizeKrb5Principal(&principal),
+                          "could not canonicalize krb5 principal");
+
+    // Map the principal to the corresponding local username. For example, admins
+    // can set up mappings so that joe@REMOTEREALM becomes something like 'remote-joe'
+    // locally for the purposes of group mapping, ACLs, etc.
+    string local_name;
+    RETURN_NOT_OK_PREPEND(security::MapPrincipalToLocalName(principal, &local_name),
+                          strings::Substitute("could not map krb5 principal '$0' to username",
+                                              principal));
+    authenticated_user_.SetAuthenticatedByKerberos(std::move(local_name), std::move(principal));
+  } else {
+    authenticated_user_.SetUnauthenticated(c_username);
+  }
+  return Status::OK();
+}
+
+Status ServerNegotiation::AuthenticateByToken(faststring* recv_buf) {
+  // Sanity check that TLS has been negotiated. Receiving the token on an
+  // unencrypted channel is a big no-no.
+  CHECK(tls_negotiated_);
+
+  // Receive the token from the client.
+  NegotiatePB pb;
+  RETURN_NOT_OK(RecvNegotiatePB(&pb, recv_buf));
+
+  if (pb.step() != NegotiatePB::TOKEN_EXCHANGE) {
+    Status s =  Status::NotAuthorized("expected TOKEN_EXCHANGE step",
+                                      NegotiatePB::NegotiateStep_Name(pb.step()));
+  }
+  if (!pb.has_authn_token()) {
+    Status s = Status::NotAuthorized("TOKEN_EXCHANGE message must include an authentication token");
+  }
+
+  // TODO(KUDU-1924): propagate the specific token verification failure back to the client,
+  // so it knows how to intelligently retry.
+  security::TokenPB token;
+  auto verification_result = token_verifier_->VerifyTokenSignature(pb.authn_token(), &token);
+  switch (verification_result) {
+    case security::VerificationResult::VALID: break;
+
+    case security::VerificationResult::INVALID_TOKEN:
+    case security::VerificationResult::INVALID_SIGNATURE:
+    case security::VerificationResult::EXPIRED_TOKEN:
+    case security::VerificationResult::EXPIRED_SIGNING_KEY: {
+      // These errors indicate the client should get a new token and try again.
+      Status s = Status::NotAuthorized(VerificationResultToString(verification_result));
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN, s));
+      return s;
+    }
+
+    case security::VerificationResult::UNKNOWN_SIGNING_KEY: {
+      // The server doesn't recognize the signing key. This indicates that the
+      // server has not been updated with the most recent TSKs, so tell the
+      // client to try again later.
+      Status s = Status::NotAuthorized(VerificationResultToString(verification_result));
+      RETURN_NOT_OK(SendError(ErrorStatusPB::ERROR_UNAVAILABLE, s));
+      return s;
+    }
+    case security::VerificationResult::INCOMPATIBLE_FEATURE: {
+      Status s = Status::NotAuthorized(VerificationResultToString(verification_result));
+      // These error types aren't recoverable by having the client get a new token.
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+      return s;
+    }
+  }
+
+  if (!token.has_authn()) {
+    Status s = Status::NotAuthorized("non-authentication token presented for authentication");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+  if (!token.authn().has_username()) {
+    // This is a runtime error because there should be no way a client could
+    // get a signed authn token without a subject.
+    Status s = Status::RuntimeError("authentication token has no username");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN, s));
+    return s;
+  }
+
+  if (PREDICT_FALSE(FLAGS_rpc_inject_invalid_authn_token_ratio > 0)) {
+    security::VerificationResult res;
+    int sel = rand() % 4;
+    switch (sel) {
+      case 0:
+        res = security::VerificationResult::INVALID_TOKEN;
+        break;
+      case 1:
+        res = security::VerificationResult::INVALID_SIGNATURE;
+        break;
+      case 2:
+        res = security::VerificationResult::EXPIRED_TOKEN;
+        break;
+      case 3:
+        res = security::VerificationResult::EXPIRED_SIGNING_KEY;
+        break;
+    }
+    if (kudu::fault_injection::MaybeTrue(FLAGS_rpc_inject_invalid_authn_token_ratio)) {
+      Status s = Status::NotAuthorized(VerificationResultToString(res));
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN, s));
+      return s;
+    }
+  }
+
+  authenticated_user_.SetAuthenticatedByToken(token.authn().username());
+
+  // Respond with success message.
+  pb.Clear();
+  pb.set_step(NegotiatePB::TOKEN_EXCHANGE);
+  return SendNegotiatePB(pb);
+}
+
+Status ServerNegotiation::AuthenticateByCertificate() {
+  // Sanity check that TLS has been negotiated. Cert-based authentication is
+  // only possible with TLS.
+  CHECK(tls_negotiated_);
+
+  // Grab the subject from the client's cert.
+  security::Cert cert;
+  RETURN_NOT_OK(tls_handshake_.GetRemoteCert(&cert));
+
+  boost::optional<string> user_id = cert.UserId();
+  boost::optional<string> principal = cert.KuduKerberosPrincipal();
+
+  if (!user_id) {
+    Status s = Status::NotAuthorized("did not find expected X509 userId extension in cert");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN, s));
+    return s;
+  }
+
+  authenticated_user_.SetAuthenticatedByClientCert(*user_id, std::move(principal));
+
+  return Status::OK();
+}
+
+Status ServerNegotiation::HandleSaslInitiate(const NegotiatePB& request) {
+  if (PREDICT_FALSE(request.step() != NegotiatePB::SASL_INITIATE)) {
+    Status s =  Status::NotAuthorized("expected SASL_INITIATE step",
+                                      NegotiatePB::NegotiateStep_Name(request.step()));
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+  TRACE("Received SASL_INITIATE request from client");
+
+  if (request.sasl_mechanisms_size() != 1) {
+    Status s = Status::NotAuthorized(
+        "SASL_INITIATE request must include exactly one SASL mechanism, found",
+        std::to_string(request.sasl_mechanisms_size()));
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  const string& mechanism = request.sasl_mechanisms(0).mechanism();
+  TRACE("Client requested to use mechanism: $0", mechanism);
+
+  negotiated_mech_ = SaslMechanism::value_of(mechanism);
+
+  // Rejects any connection from public routable IPs if authentication mechanism
+  // is plain. See KUDU-1875.
+  if (negotiated_mech_ == SaslMechanism::PLAIN) {
+    Sockaddr addr;
+    RETURN_NOT_OK(socket_->GetPeerAddress(&addr));
+
+    if (!IsTrustedConnection(addr)) {
+      Status s = Status::NotAuthorized("unauthenticated connections from publicly "
+                                       "routable IPs are prohibited. See "
+                                       "--trusted_subnets flag for more information.",
+                                       addr.ToString());
+      RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+      return s;
+    }
+  }
+
+  // If the negotiated mechanism is GSSAPI (Kerberos), configure SASL to use
+  // integrity protection so that the channel bindings and nonce can be
+  // verified.
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    RETURN_NOT_OK(EnableProtection(sasl_conn_.get(), SaslProtection::kIntegrity));
+  }
+
+  const char* server_out = nullptr;
+  uint32_t server_out_len = 0;
+  TRACE("Calling sasl_server_start()");
+
+  Status s = WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_server_start(
+          sasl_conn_.get(),         // The SASL connection context created by init()
+          mechanism.c_str(),        // The mechanism requested by the client.
+          request.token().c_str(),  // Optional string the client gave us.
+          request.token().length(), // Client string len.
+          &server_out,              // The output of the SASL library, might not be NULL terminated
+          &server_out_len);         // Output len.
+    });
+
+  if (PREDICT_FALSE(!s.ok() && !s.IsIncomplete())) {
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  // We have a valid mechanism match
+  if (s.ok()) {
+    DCHECK(server_out_len == 0);
+    RETURN_NOT_OK(SendSaslSuccess());
+  } else { // s.IsIncomplete() (equivalent to SASL_CONTINUE)
+    RETURN_NOT_OK(SendSaslChallenge(server_out, server_out_len));
+  }
+  return s;
+}
+
+Status ServerNegotiation::HandleSaslResponse(const NegotiatePB& request) {
+  if (PREDICT_FALSE(request.step() != NegotiatePB::SASL_RESPONSE)) {
+    Status s =  Status::NotAuthorized("expected SASL_RESPONSE step",
+                                      NegotiatePB::NegotiateStep_Name(request.step()));
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+  TRACE("Received SASL_RESPONSE request from client");
+
+  if (!request.has_token()) {
+    Status s = Status::NotAuthorized("no token in SASL_RESPONSE from client");
+    RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    return s;
+  }
+
+  const char* server_out = nullptr;
+  uint32_t server_out_len = 0;
+  TRACE("Calling sasl_server_step()");
+  Status s = WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_server_step(
+          sasl_conn_.get(),         // The SASL connection context created by init()
+          request.token().c_str(),  // Optional string the client gave us
+          request.token().length(), // Client string len
+          &server_out,              // The output of the SASL library, might not be NULL terminated
+          &server_out_len);         // Output len
+    });
+
+  if (s.ok()) {
+    DCHECK(server_out_len == 0);
+    return SendSaslSuccess();
+  }
+  if (s.IsIncomplete()) {
+    return SendSaslChallenge(server_out, server_out_len);
+  }
+  RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+  return s;
+}
+
+Status ServerNegotiation::SendSaslChallenge(const char* challenge, unsigned clen) {
+  NegotiatePB response;
+  response.set_step(NegotiatePB::SASL_CHALLENGE);
+  response.mutable_token()->assign(challenge, clen);
+  RETURN_NOT_OK(SendNegotiatePB(response));
+  return Status::Incomplete("");
+}
+
+Status ServerNegotiation::SendSaslSuccess() {
+  NegotiatePB response;
+  response.set_step(NegotiatePB::SASL_SUCCESS);
+
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    // Send a nonce to the client.
+    nonce_ = string();
+    RETURN_NOT_OK(security::GenerateNonce(nonce_.get_ptr()));
+    response.set_nonce(*nonce_);
+
+    if (tls_negotiated_) {
+      // Send the channel bindings to the client.
+      security::Cert cert;
+      RETURN_NOT_OK(tls_handshake_.GetLocalCert(&cert));
+
+      string plaintext_channel_bindings;
+      RETURN_NOT_OK(cert.GetServerEndPointChannelBindings(&plaintext_channel_bindings));
+
+      Slice ciphertext;
+      RETURN_NOT_OK(SaslEncode(sasl_conn_.get(),
+                               plaintext_channel_bindings,
+                               &ciphertext));
+      *response.mutable_channel_bindings() = ciphertext.ToString();
+    }
+  }
+
+  RETURN_NOT_OK(SendNegotiatePB(response));
+  return Status::OK();
+}
+
+Status ServerNegotiation::RecvConnectionContext(faststring* recv_buf) {
+  TRACE("Waiting for connection context");
+  RequestHeader header;
+  Slice param_buf;
+  RETURN_NOT_OK(ReceiveFramedMessageBlocking(socket(), recv_buf, &header, &param_buf, deadline_));
+  DCHECK(header.IsInitialized());
+
+  if (header.call_id() != kConnectionContextCallId) {
+    return Status::NotAuthorized("expected ConnectionContext callid, received",
+                                 std::to_string(header.call_id()));
+  }
+
+  ConnectionContextPB conn_context;
+  if (!conn_context.ParseFromArray(param_buf.data(), param_buf.size())) {
+    return Status::NotAuthorized("invalid ConnectionContextPB message, missing fields",
+                                 conn_context.InitializationErrorString());
+  }
+
+  if (nonce_) {
+    Status s;
+    // Validate that the client returned the correct SASL protected nonce.
+    if (!conn_context.has_encoded_nonce()) {
+      return Status::NotAuthorized("ConnectionContextPB wrapped nonce missing");
+    }
+
+    Slice decoded_nonce;
+    s = SaslDecode(sasl_conn_.get(), conn_context.encoded_nonce(), &decoded_nonce);
+    if (!s.ok()) {
+      return Status::NotAuthorized("failed to decode nonce", s.message());
+    }
+
+    if (*nonce_ != decoded_nonce) {
+      Sockaddr addr;
+      RETURN_NOT_OK(socket_->GetPeerAddress(&addr));
+      LOG(WARNING) << "Received an invalid connection nonce from client "
+                   << addr.ToString()
+                   << ", this could indicate a replay attack";
+      return Status::NotAuthorized("nonce mismatch");
+    }
+  }
+
+  return Status::OK();
+}
+
+int ServerNegotiation::GetOptionCb(const char* plugin_name,
+                                   const char* option,
+                                   const char** result,
+                                   unsigned* len) {
+  return helper_.GetOptionCb(plugin_name, option, result, len);
+}
+
+int ServerNegotiation::PlainAuthCb(sasl_conn_t* /*conn*/,
+                                   const char* user,
+                                   const char* /*pass*/,
+                                   unsigned /*passlen*/,
+                                   struct propctx*  /*propctx*/) {
+  TRACE("Received PLAIN auth, user=$0", user);
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "Password authentication callback called while PLAIN auth disabled";
+    return SASL_BADPARAM;
+  }
+  // We always allow PLAIN authentication to succeed.
+  return SASL_OK;
+}
+
+bool ServerNegotiation::IsTrustedConnection(const Sockaddr& addr) {
+  static std::once_flag once;
+  std::call_once(once, [] {
+    g_trusted_subnets = new vector<Network>();
+    CHECK_OK(Network::ParseCIDRStrings(FLAGS_trusted_subnets, g_trusted_subnets));
+
+    // If --trusted_subnets is not set explicitly, local subnets of all local network
+    // interfaces as well as the default private subnets will be used.
+    if (google::GetCommandLineFlagInfoOrDie("trusted_subnets").is_default) {
+      std::vector<Network> local_networks;
+      WARN_NOT_OK(GetLocalNetworks(&local_networks),
+                  "Unable to get local networks.");
+
+      g_trusted_subnets->insert(g_trusted_subnets->end(),
+                                local_networks.begin(),
+                                local_networks.end());
+    }
+  });
+
+  return std::any_of(g_trusted_subnets->begin(), g_trusted_subnets->end(),
+                     [&](const Network& t) { return t.WithinNetwork(addr); });
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/server_negotiation.h b/be/src/kudu/rpc/server_negotiation.h
new file mode 100644
index 0000000..2582af1
--- /dev/null
+++ b/be/src/kudu/rpc/server_negotiation.h
@@ -0,0 +1,259 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <set>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/sasl_helper.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Sockaddr;
+class faststring;
+
+namespace security {
+class TlsContext;
+class TokenVerifier;
+}
+
+namespace rpc {
+
+// Class for doing KRPC negotiation with a remote client over a bidirectional socket.
+// Operations on this class are NOT thread-safe.
+class ServerNegotiation {
+ public:
+  // Creates a new server negotiation instance, taking ownership of the
+  // provided socket. After completing the negotiation process by setting the
+  // desired options and calling Negotiate((), the socket can be retrieved with
+  // release_socket().
+  //
+  // The provided TlsContext must outlive this negotiation instance.
+  ServerNegotiation(std::unique_ptr<Socket> socket,
+                    const security::TlsContext* tls_context,
+                    const security::TokenVerifier* token_verifier,
+                    RpcEncryption encryption,
+                    std::string sasl_proto_name);
+
+  // Enable PLAIN authentication.
+  // Despite PLAIN authentication taking a username and password, we disregard
+  // the password and use this as a "unauthenticated" mode.
+  // Must be called before Negotiate().
+  Status EnablePlain();
+
+  // Enable GSSAPI (Kerberos) authentication.
+  // Must be called before Negotiate().
+  Status EnableGSSAPI();
+
+  // Returns mechanism negotiated by this connection.
+  // Must be called after Negotiate().
+  SaslMechanism::Type negotiated_mechanism() const;
+
+  // Returns the negotiated authentication type for the connection.
+  // Must be called after Negotiate().
+  AuthenticationType negotiated_authn() const {
+    DCHECK_NE(negotiated_authn_, AuthenticationType::INVALID);
+    return negotiated_authn_;
+  }
+
+  // Returns true if TLS was negotiated.
+  // Must be called after Negotiate().
+  bool tls_negotiated() const {
+    return tls_negotiated_;
+  }
+
+  // Returns the set of RPC system features supported by the remote client.
+  // Must be called after Negotiate().
+  std::set<RpcFeatureFlag> client_features() const {
+    return client_features_;
+  }
+
+  // Returns the set of RPC system features supported by the remote client.
+  // Must be called after Negotiate().
+  // Subsequent calls to this method or client_features() will return an empty set.
+  std::set<RpcFeatureFlag> take_client_features() {
+    return std::move(client_features_);
+  }
+
+  // Name of the user that was authenticated.
+  // Must be called after a successful Negotiate().
+  //
+  // Subsequent calls will return bogus data.
+  RemoteUser take_authenticated_user() {
+    return std::move(authenticated_user_);
+  }
+
+  // Specify the fully-qualified domain name of the remote server.
+  // Must be called before Negotiate(). Required for some mechanisms.
+  void set_server_fqdn(const std::string& domain_name);
+
+  // Set deadline for connection negotiation.
+  void set_deadline(const MonoTime& deadline);
+
+  Socket* socket() const { return socket_.get(); }
+
+  // Returns the socket owned by this server negotiation. The caller will own
+  // the socket after this call, and the negotiation instance should no longer
+  // be used. Must be called after Negotiate().
+  std::unique_ptr<Socket> release_socket() { return std::move(socket_); }
+
+  // Negotiate with the remote client. Should only be called once per
+  // ServerNegotiation and socket instance, after all options have been set.
+  //
+  // Returns OK on success, otherwise may return NotAuthorized, NotSupported, or
+  // another non-OK status.
+  Status Negotiate() WARN_UNUSED_RESULT;
+
+  // SASL callback for plugin options, supported mechanisms, etc.
+  // Returns SASL_FAIL if the option is not handled, which does not fail the handshake.
+  int GetOptionCb(const char* plugin_name, const char* option,
+                  const char** result, unsigned* len);
+
+  // SASL callback for PLAIN authentication via SASL_CB_SERVER_USERDB_CHECKPASS.
+  int PlainAuthCb(sasl_conn_t* conn, const char* user, const char* pass,
+                  unsigned passlen, struct propctx* propctx);
+
+  // Perform a "pre-flight check" that everything required to act as a Kerberos
+  // server is properly set up.
+  static Status PreflightCheckGSSAPI(const std::string& sasl_proto_name) WARN_UNUSED_RESULT;
+
+ private:
+
+  // Parse a negotiate request from the client, deserializing it into 'msg'.
+  // If the request is malformed, sends an error message to the client.
+  Status RecvNegotiatePB(NegotiatePB* msg, faststring* recv_buf) WARN_UNUSED_RESULT;
+
+  // Encode and send the specified negotiate response message to the server.
+  Status SendNegotiatePB(const NegotiatePB& msg) WARN_UNUSED_RESULT;
+
+  // Encode and send the specified RPC error message to the client.
+  // Calls Status.ToString() for the embedded error message.
+  Status SendError(ErrorStatusPB::RpcErrorCodePB code, const Status& err) WARN_UNUSED_RESULT;
+
+  // Parse and validate connection header.
+  Status ValidateConnectionHeader(faststring* recv_buf) WARN_UNUSED_RESULT;
+
+  // Initialize the SASL server negotiation instance.
+  Status InitSaslServer() WARN_UNUSED_RESULT;
+
+  // Handle case when client sends NEGOTIATE request. Builds the set of
+  // client-supported RPC features, determines a mutually supported
+  // authentication type to use for the connection, and sends a NEGOTIATE
+  // response.
+  Status HandleNegotiate(const NegotiatePB& request) WARN_UNUSED_RESULT;
+
+  // Handle a TLS_HANDSHAKE request message from the server.
+  Status HandleTlsHandshake(const NegotiatePB& request) WARN_UNUSED_RESULT;
+
+  // Send a TLS_HANDSHAKE response message to the server with the provided token.
+  Status SendTlsHandshake(std::string tls_token) WARN_UNUSED_RESULT;
+
+  // Authenticate the client using SASL. Populates the 'authenticated_user_'
+  // field with the SASL principal.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateBySasl(faststring* recv_buf) WARN_UNUSED_RESULT;
+
+  // Authenticate the client using a token. Populates the
+  // 'authenticated_user_' field with the token's principal.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateByToken(faststring* recv_buf) WARN_UNUSED_RESULT;
+
+  // Authenticate the client using the client's TLS certificate. Populates the
+  // 'authenticated_user_' field with the certificate's subject.
+  Status AuthenticateByCertificate() WARN_UNUSED_RESULT;
+
+  // Handle case when client sends SASL_INITIATE request.
+  // Returns Status::OK if the SASL negotiation is complete, or
+  // Status::Incomplete if a SASL_RESPONSE step is expected.
+  Status HandleSaslInitiate(const NegotiatePB& request) WARN_UNUSED_RESULT;
+
+  // Handle case when client sends SASL_RESPONSE request.
+  Status HandleSaslResponse(const NegotiatePB& request) WARN_UNUSED_RESULT;
+
+  // Send a SASL_CHALLENGE response to the client with a challenge token.
+  Status SendSaslChallenge(const char* challenge, unsigned clen) WARN_UNUSED_RESULT;
+
+  // Send a SASL_SUCCESS response to the client.
+  Status SendSaslSuccess() WARN_UNUSED_RESULT;
+
+  // Receive and validate the ConnectionContextPB.
+  Status RecvConnectionContext(faststring* recv_buf) WARN_UNUSED_RESULT;
+
+  // Returns true if connection is from trusted subnets or local networks.
+  bool IsTrustedConnection(const Sockaddr& addr);
+
+  // The socket to the remote client.
+  std::unique_ptr<Socket> socket_;
+
+  // SASL state.
+  std::vector<sasl_callback_t> callbacks_;
+  std::unique_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
+  SaslHelper helper_;
+  boost::optional<std::string> nonce_;
+
+  // TLS state.
+  const security::TlsContext* tls_context_;
+  security::TlsHandshake tls_handshake_;
+  const RpcEncryption encryption_;
+  bool tls_negotiated_;
+
+  // TSK state.
+  const security::TokenVerifier* token_verifier_;
+
+  // The set of features supported by the client and server. Filled in during negotiation.
+  std::set<RpcFeatureFlag> client_features_;
+  std::set<RpcFeatureFlag> server_features_;
+
+  // The successfully-authenticated user, if applicable. Filled in during
+  // negotiation.
+  RemoteUser authenticated_user_;
+
+  // The authentication type. Filled in during negotiation.
+  AuthenticationType negotiated_authn_;
+
+  // The SASL mechanism. Filled in during negotiation if the negotiated
+  // authentication type is SASL.
+  SaslMechanism::Type negotiated_mech_;
+
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
+  // Negotiation timeout deadline.
+  MonoTime deadline_;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_if.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_if.cc b/be/src/kudu/rpc/service_if.cc
new file mode 100644
index 0000000..008c478
--- /dev/null
+++ b/be/src/kudu/rpc/service_if.cc
@@ -0,0 +1,160 @@
+// 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 "kudu/rpc/service_if.h"
+
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/result_tracker.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+// TODO remove this once we have fully cluster-tested this.
+// Despite being on by default, this is left in in case we discover
+// any issues in 0.10.0, we'll have an easy workaround to disable the feature.
+DEFINE_bool(enable_exactly_once, true, "Whether to enable exactly once semantics.");
+TAG_FLAG(enable_exactly_once, hidden);
+
+using google::protobuf::Message;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+ServiceIf::~ServiceIf() {
+}
+
+void ServiceIf::Shutdown() {
+}
+
+bool ServiceIf::SupportsFeature(uint32_t feature) const {
+  return false;
+}
+
+bool ServiceIf::ParseParam(InboundCall *call, google::protobuf::Message *message) {
+  Slice param(call->serialized_request());
+  if (PREDICT_FALSE(!message->ParseFromArray(param.data(), param.size()))) {
+    string err = Substitute("invalid parameter for call $0: missing fields: $1",
+                            call->remote_method().ToString(),
+                            message->InitializationErrorString().c_str());
+    LOG(WARNING) << err;
+    call->RespondFailure(ErrorStatusPB::ERROR_INVALID_REQUEST,
+                         Status::InvalidArgument(err));
+    return false;
+  }
+  return true;
+}
+
+void ServiceIf::RespondBadMethod(InboundCall *call) {
+  Sockaddr local_addr, remote_addr;
+
+  CHECK_OK(call->connection()->socket()->GetSocketAddress(&local_addr));
+  CHECK_OK(call->connection()->socket()->GetPeerAddress(&remote_addr));
+  string err = Substitute("Call on service $0 received at $1 from $2 with an "
+                          "invalid method name: $3",
+                          call->remote_method().service_name(),
+                          local_addr.ToString(),
+                          remote_addr.ToString(),
+                          call->remote_method().method_name());
+  LOG(WARNING) << err;
+  call->RespondFailure(ErrorStatusPB::ERROR_NO_SUCH_METHOD,
+                       Status::InvalidArgument(err));
+}
+
+GeneratedServiceIf::~GeneratedServiceIf() {
+}
+
+
+void GeneratedServiceIf::Handle(InboundCall *call) {
+  const RpcMethodInfo* method_info = call->method_info();
+  if (!method_info) {
+    RespondBadMethod(call);
+    return;
+  }
+  unique_ptr<Message> req(method_info->req_prototype->New());
+  if (PREDICT_FALSE(!ParseParam(call, req.get()))) {
+    return;
+  }
+  Message* resp = method_info->resp_prototype->New();
+
+  bool track_result = call->header().has_request_id()
+                      && method_info->track_result
+                      && FLAGS_enable_exactly_once;
+  RpcContext* ctx = new RpcContext(call,
+                                   req.release(),
+                                   resp,
+                                   track_result ? result_tracker_ : nullptr);
+  if (!method_info->authz_method(ctx->request_pb(), resp, ctx)) {
+    // The authz_method itself should have responded to the RPC.
+    return;
+  }
+
+  if (track_result) {
+    RequestIdPB request_id(call->header().request_id());
+    ResultTracker::RpcState state = ctx->result_tracker()->TrackRpc(
+        call->header().request_id(),
+        resp,
+        ctx);
+    switch (state) {
+      case ResultTracker::NEW:
+        // Fall out of the 'if' statement to the normal path.
+        break;
+      case ResultTracker::COMPLETED:
+      case ResultTracker::IN_PROGRESS:
+      case ResultTracker::STALE:
+        // ResultTracker has already responded to the RPC and deleted
+        // 'ctx'.
+        return;
+      default:
+        LOG(FATAL) << "Unknown state: " << state;
+    }
+  }
+  method_info->func(ctx->request_pb(), resp, ctx);
+}
+
+
+RpcMethodInfo* GeneratedServiceIf::LookupMethod(const RemoteMethod& method) {
+  DCHECK_EQ(method.service_name(), service_name());
+  const auto& it = methods_by_name_.find(method.method_name());
+  if (PREDICT_FALSE(it == methods_by_name_.end())) {
+    return nullptr;
+  }
+  return it->second.get();
+}
+
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_if.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_if.h b/be/src/kudu/rpc/service_if.h
new file mode 100644
index 0000000..9156b4a
--- /dev/null
+++ b/be/src/kudu/rpc/service_if.h
@@ -0,0 +1,134 @@
+// 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.
+#ifndef KUDU_RPC_SERVICE_IF_H
+#define KUDU_RPC_SERVICE_IF_H
+
+#include <cstdint>
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include <google/protobuf/message.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/metrics.h"
+
+namespace kudu {
+namespace rpc {
+
+class InboundCall;
+class RemoteMethod;
+class ResultTracker;
+class RpcContext;
+
+// Generated services define an instance of this class for each
+// method that they implement. The generic server code implemented
+// by GeneratedServiceIf look up the RpcMethodInfo in order to handle
+// each RPC.
+struct RpcMethodInfo : public RefCountedThreadSafe<RpcMethodInfo> {
+  // Prototype protobufs for requests and responses.
+  // These are empty protobufs which are cloned in order to provide an
+  // instance for each request.
+  std::unique_ptr<google::protobuf::Message> req_prototype;
+  std::unique_ptr<google::protobuf::Message> resp_prototype;
+
+  scoped_refptr<Histogram> handler_latency_histogram;
+
+  // Whether we should track this method's result, using ResultTracker.
+  bool track_result;
+
+  // The authorization function for this RPC. If this function
+  // returns false, the RPC has already been handled (i.e. rejected)
+  // by the authorization function.
+  std::function<bool(const google::protobuf::Message* req,
+                     google::protobuf::Message* resp,
+                     RpcContext* ctx)> authz_method;
+
+  // The actual function to be called.
+  std::function<void(const google::protobuf::Message* req,
+                     google::protobuf::Message* resp,
+                     RpcContext* ctx)> func;
+};
+
+// Handles incoming messages that initiate an RPC.
+class ServiceIf {
+ public:
+  virtual ~ServiceIf();
+  virtual void Handle(InboundCall* incoming) = 0;
+  virtual void Shutdown();
+  virtual std::string service_name() const = 0;
+
+  // The service should return true if it supports the provided application
+  // specific feature flag.
+  virtual bool SupportsFeature(uint32_t feature) const;
+
+  // Look up the method being requested by the remote call.
+  //
+  // If this returns nullptr, then certain functionality like
+  // metrics collection will not be performed for this call.
+  virtual RpcMethodInfo* LookupMethod(const RemoteMethod& method) {
+    return nullptr;
+  }
+
+  // Default authorization method, which just allows all RPCs.
+  //
+  // See docs/design-docs/rpc.md for details on how to add custom
+  // authorization checks to a service.
+  bool AuthorizeAllowAll(const google::protobuf::Message* /*req*/,
+                         google::protobuf::Message* /*resp*/,
+                         RpcContext* /*ctx*/) {
+    return true;
+  }
+
+ protected:
+  bool ParseParam(InboundCall* call, google::protobuf::Message* message);
+  void RespondBadMethod(InboundCall* call);
+};
+
+
+// Base class for code-generated service classes.
+class GeneratedServiceIf : public ServiceIf {
+ public:
+  virtual ~GeneratedServiceIf();
+
+  // Looks up the appropriate method in 'methods_by_name_' and executes
+  // it on the current thread.
+  //
+  // If no such method is found, responds with an error.
+  void Handle(InboundCall* incoming) override;
+
+  RpcMethodInfo* LookupMethod(const RemoteMethod& method) override;
+
+  // Returns the mapping from method names to method infos.
+  typedef std::unordered_map<std::string, scoped_refptr<RpcMethodInfo>> MethodInfoMap;
+  const MethodInfoMap& methods_by_name() const { return methods_by_name_; }
+
+ protected:
+  // For each method, stores the relevant information about how to handle the
+  // call. Methods are inserted by the constructor of the generated subclass.
+  // After construction, this map is accessed by multiple threads and therefore
+  // must not be modified.
+  MethodInfoMap methods_by_name_;
+
+  // The result tracker for this service's methods.
+  scoped_refptr<ResultTracker> result_tracker_;
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_pool.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_pool.cc b/be/src/kudu/rpc/service_pool.cc
new file mode 100644
index 0000000..62d46d6
--- /dev/null
+++ b/be/src/kudu/rpc/service_pool.cc
@@ -0,0 +1,234 @@
+// 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 "kudu/rpc/service_pool.h"
+
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/service_queue.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace.h"
+
+using std::shared_ptr;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+METRIC_DEFINE_histogram(server, rpc_incoming_queue_time,
+                        "RPC Queue Time",
+                        kudu::MetricUnit::kMicroseconds,
+                        "Number of microseconds incoming RPC requests spend in the worker queue",
+                        60000000LU, 3);
+
+METRIC_DEFINE_counter(server, rpcs_timed_out_in_queue,
+                      "RPC Queue Timeouts",
+                      kudu::MetricUnit::kRequests,
+                      "Number of RPCs whose timeout elapsed while waiting "
+                      "in the service queue, and thus were not processed.");
+
+METRIC_DEFINE_counter(server, rpcs_queue_overflow,
+                      "RPC Queue Overflows",
+                      kudu::MetricUnit::kRequests,
+                      "Number of RPCs dropped because the service queue "
+                      "was full.");
+
+namespace kudu {
+namespace rpc {
+
+ServicePool::ServicePool(gscoped_ptr<ServiceIf> service,
+                         const scoped_refptr<MetricEntity>& entity,
+                         size_t service_queue_length)
+  : service_(std::move(service)),
+    service_queue_(service_queue_length),
+    incoming_queue_time_(METRIC_rpc_incoming_queue_time.Instantiate(entity)),
+    rpcs_timed_out_in_queue_(METRIC_rpcs_timed_out_in_queue.Instantiate(entity)),
+    rpcs_queue_overflow_(METRIC_rpcs_queue_overflow.Instantiate(entity)),
+    closing_(false) {
+}
+
+ServicePool::~ServicePool() {
+  Shutdown();
+}
+
+Status ServicePool::Init(int num_threads) {
+  for (int i = 0; i < num_threads; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("service pool", "rpc worker",
+        &ServicePool::RunThread, this, &new_thread));
+    threads_.push_back(new_thread);
+  }
+  return Status::OK();
+}
+
+void ServicePool::Shutdown() {
+  service_queue_.Shutdown();
+
+  MutexLock lock(shutdown_lock_);
+  if (closing_) return;
+  closing_ = true;
+  // TODO: Use a proper thread pool implementation.
+  for (scoped_refptr<kudu::Thread>& thread : threads_) {
+    CHECK_OK(ThreadJoiner(thread.get()).Join());
+  }
+
+  // Now we must drain the service queue.
+  Status status = Status::ServiceUnavailable("Service is shutting down");
+  std::unique_ptr<InboundCall> incoming;
+  while (service_queue_.BlockingGet(&incoming)) {
+    incoming.release()->RespondFailure(ErrorStatusPB::FATAL_SERVER_SHUTTING_DOWN, status);
+  }
+
+  service_->Shutdown();
+}
+
+void ServicePool::RejectTooBusy(InboundCall* c) {
+  string err_msg =
+      Substitute("$0 request on $1 from $2 dropped due to backpressure. "
+                 "The service queue is full; it has $3 items.",
+                 c->remote_method().method_name(),
+                 service_->service_name(),
+                 c->remote_address().ToString(),
+                 service_queue_.max_size());
+  rpcs_queue_overflow_->Increment();
+  KLOG_EVERY_N_SECS(WARNING, 1) << err_msg;
+  c->RespondFailure(ErrorStatusPB::ERROR_SERVER_TOO_BUSY,
+                    Status::ServiceUnavailable(err_msg));
+  DLOG(INFO) << err_msg << " Contents of service queue:\n"
+             << service_queue_.ToString();
+
+  if (too_busy_hook_) {
+    too_busy_hook_();
+  }
+}
+
+RpcMethodInfo* ServicePool::LookupMethod(const RemoteMethod& method) {
+  return service_->LookupMethod(method);
+}
+
+Status ServicePool::QueueInboundCall(gscoped_ptr<InboundCall> call) {
+  InboundCall* c = call.release();
+
+  vector<uint32_t> unsupported_features;
+  for (uint32_t feature : c->GetRequiredFeatures()) {
+    if (!service_->SupportsFeature(feature)) {
+      unsupported_features.push_back(feature);
+    }
+  }
+
+  if (!unsupported_features.empty()) {
+    c->RespondUnsupportedFeature(unsupported_features);
+    return Status::NotSupported("call requires unsupported application feature flags",
+                                JoinMapped(unsupported_features,
+                                           [] (uint32_t flag) { return std::to_string(flag); },
+                                           ", "));
+  }
+
+  TRACE_TO(c->trace(), "Inserting onto call queue");
+
+  // Queue message on service queue
+  boost::optional<InboundCall*> evicted;
+  auto queue_status = service_queue_.Put(c, &evicted);
+  if (queue_status == QUEUE_FULL) {
+    RejectTooBusy(c);
+    return Status::OK();
+  }
+
+  if (PREDICT_FALSE(evicted != boost::none)) {
+    RejectTooBusy(*evicted);
+  }
+
+  if (PREDICT_TRUE(queue_status == QUEUE_SUCCESS)) {
+    // NB: do not do anything with 'c' after it is successfully queued --
+    // a service thread may have already dequeued it, processed it, and
+    // responded by this point, in which case the pointer would be invalid.
+    return Status::OK();
+  }
+
+  Status status = Status::OK();
+  if (queue_status == QUEUE_SHUTDOWN) {
+    status = Status::ServiceUnavailable("Service is shutting down");
+    c->RespondFailure(ErrorStatusPB::FATAL_SERVER_SHUTTING_DOWN, status);
+  } else {
+    status = Status::RuntimeError(Substitute("Unknown error from BlockingQueue: $0", queue_status));
+    c->RespondFailure(ErrorStatusPB::FATAL_UNKNOWN, status);
+  }
+  return status;
+}
+
+void ServicePool::RunThread() {
+  while (true) {
+    std::unique_ptr<InboundCall> incoming;
+    if (!service_queue_.BlockingGet(&incoming)) {
+      VLOG(1) << "ServicePool: messenger shutting down.";
+      return;
+    }
+
+    incoming->RecordHandlingStarted(incoming_queue_time_.get());
+    ADOPT_TRACE(incoming->trace());
+
+    if (PREDICT_FALSE(incoming->ClientTimedOut())) {
+      TRACE_TO(incoming->trace(), "Skipping call since client already timed out");
+      rpcs_timed_out_in_queue_->Increment();
+
+      // Respond as a failure, even though the client will probably ignore
+      // the response anyway.
+      incoming->RespondFailure(
+        ErrorStatusPB::ERROR_SERVER_TOO_BUSY,
+        Status::TimedOut("Call waited in the queue past client deadline"));
+
+      // Must release since RespondFailure above ends up taking ownership
+      // of the object.
+      ignore_result(incoming.release());
+      continue;
+    }
+
+    TRACE_TO(incoming->trace(), "Handling call");
+
+    // Release the InboundCall pointer -- when the call is responded to,
+    // it will get deleted at that point.
+    service_->Handle(incoming.release());
+  }
+}
+
+const string ServicePool::service_name() const {
+  return service_->service_name();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_pool.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_pool.h b/be/src/kudu/rpc/service_pool.h
new file mode 100644
index 0000000..2bc8873
--- /dev/null
+++ b/be/src/kudu/rpc/service_pool.h
@@ -0,0 +1,117 @@
+// 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.
+
+#ifndef KUDU_SERVICE_POOL_H
+#define KUDU_SERVICE_POOL_H
+
+#include <cstddef>
+#include <functional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/rpc_service.h"
+#include "kudu/rpc/service_queue.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Counter;
+class Histogram;
+class MetricEntity;
+class Thread;
+
+namespace rpc {
+
+class InboundCall;
+class RemoteMethod;
+class ServiceIf;
+
+struct RpcMethodInfo;
+
+// A pool of threads that handle new incoming RPC calls.
+// Also includes a queue that calls get pushed onto for handling by the pool.
+class ServicePool : public RpcService {
+ public:
+  ServicePool(gscoped_ptr<ServiceIf> service,
+              const scoped_refptr<MetricEntity>& metric_entity,
+              size_t service_queue_length);
+  virtual ~ServicePool();
+
+  // Set a hook function to be called when any RPC gets rejected because
+  // the service queue is full.
+  //
+  // NOTE: This hook runs on a reactor thread so must execute quickly.
+  // Additionally, if a service queue is overflowing, the server is likely
+  // under a lot of load, so hooks should be careful to throttle their own
+  // execution.
+  void set_too_busy_hook(std::function<void(void)> hook) {
+    too_busy_hook_ = std::move(hook);
+  }
+
+  // Start up the thread pool.
+  virtual Status Init(int num_threads);
+
+  // Shut down the queue and the thread pool.
+  virtual void Shutdown();
+
+  RpcMethodInfo* LookupMethod(const RemoteMethod& method) override;
+
+  virtual Status QueueInboundCall(gscoped_ptr<InboundCall> call) OVERRIDE;
+
+  const Counter* RpcsTimedOutInQueueMetricForTests() const {
+    return rpcs_timed_out_in_queue_.get();
+  }
+
+  const Histogram* IncomingQueueTimeMetricForTests() const {
+    return incoming_queue_time_.get();
+  }
+
+  const Counter* RpcsQueueOverflowMetric() const {
+    return rpcs_queue_overflow_.get();
+  }
+
+  const std::string service_name() const;
+
+ private:
+  void RunThread();
+  void RejectTooBusy(InboundCall* c);
+
+  gscoped_ptr<ServiceIf> service_;
+  std::vector<scoped_refptr<kudu::Thread> > threads_;
+  LifoServiceQueue service_queue_;
+  scoped_refptr<Histogram> incoming_queue_time_;
+  scoped_refptr<Counter> rpcs_timed_out_in_queue_;
+  scoped_refptr<Counter> rpcs_queue_overflow_;
+
+  mutable Mutex shutdown_lock_;
+  bool closing_;
+
+  std::function<void(void)> too_busy_hook_;
+
+  DISALLOW_COPY_AND_ASSIGN(ServicePool);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_queue-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_queue-test.cc b/be/src/kudu/rpc/service_queue-test.cc
new file mode 100644
index 0000000..f1450fd
--- /dev/null
+++ b/be/src/kudu/rpc/service_queue-test.cc
@@ -0,0 +1,151 @@
+// 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 <atomic>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/port.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/service_queue.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_util.h"
+
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+DEFINE_int32(num_producers, 4,
+             "Number of producer threads");
+
+DEFINE_int32(num_consumers, 20,
+             "Number of consumer threads");
+
+DEFINE_int32(max_queue_size, 50,
+             "Max queue length");
+
+namespace kudu {
+namespace rpc {
+
+static std::atomic<uint32_t> inprogress;
+
+static std::atomic<uint32_t> total;
+
+template <typename Queue>
+void ProducerThread(Queue* queue) {
+  int max_inprogress = FLAGS_max_queue_size - FLAGS_num_producers;
+  while (true) {
+    while (inprogress > max_inprogress) {
+      base::subtle::PauseCPU();
+    }
+    inprogress++;
+    InboundCall* call = new InboundCall(nullptr);
+    boost::optional<InboundCall*> evicted;
+    auto status = queue->Put(call, &evicted);
+    if (status == QUEUE_FULL) {
+      LOG(INFO) << "queue full: producer exiting";
+      delete call;
+      break;
+    }
+
+    if (PREDICT_FALSE(evicted != boost::none)) {
+      LOG(INFO) << "call evicted: producer exiting";
+      delete evicted.get();
+      break;
+    }
+
+    if (PREDICT_TRUE(status == QUEUE_SHUTDOWN)) {
+      delete call;
+      break;
+    }
+  }
+}
+
+template <typename Queue>
+void ConsumerThread(Queue* queue) {
+  unique_ptr<InboundCall> call;
+  while (queue->BlockingGet(&call)) {
+    inprogress--;
+    total++;
+    call.reset();
+  }
+}
+
+TEST(TestServiceQueue, LifoServiceQueuePerf) {
+  LifoServiceQueue queue(FLAGS_max_queue_size);
+  vector<std::thread> producers;
+  vector<std::thread> consumers;
+
+  for (int i = 0; i < FLAGS_num_producers; i++) {
+    producers.emplace_back(&ProducerThread<LifoServiceQueue>, &queue);
+  }
+
+  for (int i = 0; i < FLAGS_num_consumers; i++) {
+    consumers.emplace_back(&ConsumerThread<LifoServiceQueue>, &queue);
+  }
+
+  int seconds = AllowSlowTests() ? 10 : 1;
+  uint64_t total_sample = 0;
+  uint64_t total_queue_len = 0;
+  uint64_t total_idle_workers = 0;
+  Stopwatch sw(Stopwatch::ALL_THREADS);
+  sw.start();
+  int32_t before = total;
+
+  for (int i = 0; i < seconds * 50; i++) {
+    SleepFor(MonoDelta::FromMilliseconds(20));
+    total_sample++;
+    total_queue_len += queue.estimated_queue_length();
+    total_idle_workers += queue.estimated_idle_worker_count();
+  }
+
+  sw.stop();
+  int32_t delta = total - before;
+
+  queue.Shutdown();
+  for (int i = 0; i < FLAGS_num_producers; i++) {
+    producers[i].join();
+  }
+  for (int i = 0; i < FLAGS_num_consumers; i++) {
+    consumers[i].join();
+  }
+
+  float reqs_per_second = static_cast<float>(delta / sw.elapsed().wall_seconds());
+  float user_cpu_micros_per_req = static_cast<float>(sw.elapsed().user / 1000.0 / delta);
+  float sys_cpu_micros_per_req = static_cast<float>(sw.elapsed().system / 1000.0 / delta);
+
+  LOG(INFO) << "Reqs/sec:         " << (int32_t)reqs_per_second;
+  LOG(INFO) << "User CPU per req: " << user_cpu_micros_per_req << "us";
+  LOG(INFO) << "Sys CPU per req:  " << sys_cpu_micros_per_req << "us";
+  LOG(INFO) << "Avg rpc queue length: " << total_queue_len / static_cast<double>(total_sample);
+  LOG(INFO) << "Avg idle workers:     " << total_idle_workers / static_cast<double>(total_sample);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_queue.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_queue.cc b/be/src/kudu/rpc/service_queue.cc
new file mode 100644
index 0000000..29c0516
--- /dev/null
+++ b/be/src/kudu/rpc/service_queue.cc
@@ -0,0 +1,145 @@
+// 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 "kudu/rpc/service_queue.h"
+
+#include <mutex>
+#include <ostream>
+
+#include <boost/optional/optional.hpp>
+
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+namespace rpc {
+
+__thread LifoServiceQueue::ConsumerState* LifoServiceQueue::tl_consumer_ = nullptr;
+
+LifoServiceQueue::LifoServiceQueue(int max_size)
+   : shutdown_(false),
+     max_queue_size_(max_size) {
+  CHECK_GT(max_queue_size_, 0);
+}
+
+LifoServiceQueue::~LifoServiceQueue() {
+  DCHECK(queue_.empty())
+      << "ServiceQueue holds bare pointers at destruction time";
+}
+
+bool LifoServiceQueue::BlockingGet(std::unique_ptr<InboundCall>* out) {
+  auto consumer = tl_consumer_;
+  if (PREDICT_FALSE(!consumer)) {
+    consumer = tl_consumer_ = new ConsumerState(this);
+    std::lock_guard<simple_spinlock> l(lock_);
+    consumers_.emplace_back(consumer);
+  }
+
+  while (true) {
+    {
+      std::lock_guard<simple_spinlock> l(lock_);
+      if (!queue_.empty()) {
+        auto it = queue_.begin();
+        out->reset(*it);
+        queue_.erase(it);
+        return true;
+      }
+      if (PREDICT_FALSE(shutdown_)) {
+        return false;
+      }
+      consumer->DCheckBoundInstance(this);
+      waiting_consumers_.push_back(consumer);
+    }
+    InboundCall* call = consumer->Wait();
+    if (call != nullptr) {
+      out->reset(call);
+      return true;
+    }
+    // if call == nullptr, this means we are shutting down the queue.
+    // Loop back around and re-check 'shutdown_'.
+  }
+}
+
+QueueStatus LifoServiceQueue::Put(InboundCall* call,
+                                  boost::optional<InboundCall*>* evicted) {
+  std::unique_lock<simple_spinlock> l(lock_);
+  if (PREDICT_FALSE(shutdown_)) {
+    return QUEUE_SHUTDOWN;
+  }
+
+  DCHECK(!(waiting_consumers_.size() > 0 && queue_.size() > 0));
+
+  // fast path
+  if (queue_.empty() && waiting_consumers_.size() > 0) {
+    auto consumer = waiting_consumers_[waiting_consumers_.size() - 1];
+    waiting_consumers_.pop_back();
+    // Notify condition var(and wake up consumer thread) takes time,
+    // so put it out of spinlock scope.
+    l.unlock();
+    consumer->Post(call);
+    return QUEUE_SUCCESS;
+  }
+
+  if (PREDICT_FALSE(queue_.size() >= max_queue_size_)) {
+    // eviction
+    DCHECK_EQ(queue_.size(), max_queue_size_);
+    auto it = queue_.end();
+    --it;
+    if (DeadlineLess(*it, call)) {
+      return QUEUE_FULL;
+    }
+
+    *evicted = *it;
+    queue_.erase(it);
+  }
+
+  queue_.insert(call);
+  return QUEUE_SUCCESS;
+}
+
+void LifoServiceQueue::Shutdown() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  shutdown_ = true;
+
+  // Post a nullptr to wake up any consumers which are waiting.
+  for (auto* cs : waiting_consumers_) {
+    cs->Post(nullptr);
+  }
+  waiting_consumers_.clear();
+}
+
+bool LifoServiceQueue::empty() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return queue_.empty();
+}
+
+int LifoServiceQueue::max_size() const {
+  return max_queue_size_;
+}
+
+std::string LifoServiceQueue::ToString() const {
+  std::string ret;
+
+  std::lock_guard<simple_spinlock> l(lock_);
+  for (const auto* t : queue_) {
+    ret.append(t->ToString());
+    ret.append("\n");
+  }
+  return ret;
+}
+
+} // namespace rpc
+} // namespace kudu


[33/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/security-test-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security-test-util.h b/be/src/kudu/security/security-test-util.h
new file mode 100644
index 0000000..5862bec
--- /dev/null
+++ b/be/src/kudu/security/security-test-util.h
@@ -0,0 +1,56 @@
+// 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.
+
+#pragma once
+
+#include <ostream>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+class Cert;
+class PrivateKey;
+class TlsContext;
+
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert);
+
+// Describes the options for configuring a TlsContext.
+enum class PkiConfig {
+  // The TLS context has no TLS cert and no trusted certs.
+  NONE,
+  // The TLS context has a self-signed TLS cert and no trusted certs.
+  SELF_SIGNED,
+  // The TLS context has no TLS cert and a trusted cert.
+  TRUSTED,
+  // The TLS context has a signed TLS cert and trusts the corresponding signing cert.
+  SIGNED,
+  // The TLS context has a externally signed TLS cert and trusts the corresponding signing cert.
+  EXTERNALLY_SIGNED,
+};
+
+// PkiConfig pretty-printer.
+std::ostream& operator<<(std::ostream& o, PkiConfig c);
+
+Status ConfigureTlsContext(PkiConfig config,
+                           const Cert& ca_cert,
+                           const PrivateKey& ca_key,
+                           TlsContext* tls_context);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/security_flags.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security_flags.cc b/be/src/kudu/security/security_flags.cc
new file mode 100644
index 0000000..acdd662
--- /dev/null
+++ b/be/src/kudu/security/security_flags.cc
@@ -0,0 +1,42 @@
+// 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 "kudu/security/security_flags.h"
+
+namespace kudu {
+namespace security {
+
+// This is the "modern compatibility" cipher list of the Mozilla Security
+// Server Side TLS recommendations, accessed Feb. 2017, with the addition of
+// the non ECDH/DH AES cipher suites from the "intermediate compatibility"
+// list. These additional ciphers maintain compatibility with RHEL 6.5 and
+// below. The DH AES ciphers are not included since we are not configured to
+// use DH key agreement.
+const char* const SecurityDefaults::SecurityDefaults::kDefaultTlsCiphers =
+                                   "ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:"
+                                   "ECDHE-ECDSA-CHACHA20-POLY1305:ECDHE-RSA-CHACHA20-POLY1305:"
+                                   "ECDHE-ECDSA-AES128-GCM-SHA256:ECDHE-RSA-AES128-GCM-SHA256:"
+                                   "ECDHE-ECDSA-AES256-SHA384:ECDHE-RSA-AES256-SHA384:"
+                                   "ECDHE-ECDSA-AES128-SHA256:ECDHE-RSA-AES128-SHA256:"
+                                   "AES256-GCM-SHA384:AES128-GCM-SHA256:"
+                                   "AES256-SHA256:AES128-SHA256:"
+                                   "AES256-SHA:AES128-SHA";
+
+const char* const SecurityDefaults::SecurityDefaults::kDefaultTlsMinVersion = "TLSv1";
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/security_flags.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security_flags.h b/be/src/kudu/security/security_flags.h
new file mode 100644
index 0000000..e64536d
--- /dev/null
+++ b/be/src/kudu/security/security_flags.h
@@ -0,0 +1,36 @@
+// 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.
+#pragma once
+
+#include "kudu/util/flags.h"
+
+namespace kudu {
+namespace security {
+
+// Authentication configuration for RPC connections.
+typedef TriStateFlag RpcAuthentication;
+
+// Encryption configuration for RPC connections.
+typedef TriStateFlag RpcEncryption;
+
+struct SecurityDefaults {
+  static const char* const kDefaultTlsCiphers;
+  static const char* const kDefaultTlsMinVersion;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/simple_acl.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/simple_acl.cc b/be/src/kudu/security/simple_acl.cc
new file mode 100644
index 0000000..09cc6ab
--- /dev/null
+++ b/be/src/kudu/security/simple_acl.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/security/simple_acl.h"
+
+#include <cctype>
+#include <cstring>
+#include <utility>
+#include <vector>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/status.h"
+
+using std::set;
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+SimpleAcl::SimpleAcl() {
+}
+
+SimpleAcl::~SimpleAcl() {
+}
+
+Status SimpleAcl::ParseFlag(const string& flag) {
+  vector<StringPiece> fields = strings::Split(flag, ",", strings::SkipWhitespace());
+  set<string> users;
+  for (const auto& field : fields) {
+    if (field.empty()) continue;
+    // if any field is a wildcard, no need to include the rest.
+    if (flag == "*") {
+      Reset({"*"});
+      return Status::OK();
+    }
+
+
+    // Leave open the use of various special characters at the start of each
+    // username. We reserve some special characters that might be useful in
+    // ACLs:
+    // '!': might be interpreted as "not"
+    // '@': often used to read data from a file
+    // '#': comments
+    // '$': maybe variable expansion?
+    // '%': used by sudoers for groups
+    // '*': only allowed for special wildcard ACL above
+    // '-', '+', '=': useful for allow/deny style ACLs
+    // <quote characters>: in case we want to add quoted strings
+    // whitespace: down right confusing
+    static const char* kReservedStartingCharacters = "!@#$%*-=+'\"";
+    if (strchr(kReservedStartingCharacters, field[0]) ||
+        isspace(field[0])) {
+      return Status::NotSupported("invalid username", field.ToString());
+    }
+
+    users.insert(field.ToString());
+  }
+
+  Reset(std::move(users));
+  return Status::OK();
+}
+
+void SimpleAcl::Reset(set<string> users) {
+  users_ = std::move(users);
+}
+
+bool SimpleAcl::UserAllowed(const string& username) {
+  return ContainsKey(users_, "*") || ContainsKey(users_, username);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/simple_acl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/simple_acl.h b/be/src/kudu/security/simple_acl.h
new file mode 100644
index 0000000..e27a8ca
--- /dev/null
+++ b/be/src/kudu/security/simple_acl.h
@@ -0,0 +1,58 @@
+// 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.
+#pragma once
+
+#include <set>
+#include <string>
+
+namespace kudu {
+class Status;
+
+namespace security {
+
+// Represent a very simple access control list which contains a set of users.
+//
+// This is basically just a wrapper around a set<string> with a bit of parsing logic and
+// support for the '*' wildcard.
+class SimpleAcl {
+ public:
+  SimpleAcl();
+  ~SimpleAcl();
+
+  // Parse a flag value, which should be of the form 'user1,user2,user3' to indicate a
+  // list of users, or '*' to indicate a wildcard. This syntax may be expanded later to
+  // include groups, "allow/deny" style access, etc.
+  //
+  // Thread-unsafe: must be called before the ACL may be consulted, and may not be
+  // called a second time concurrent with reads of the ACL.
+  Status ParseFlag(const std::string& flag);
+
+  // Return true if the given user is allowed by the ACL.
+  //
+  // Thread-safe after initialization.
+  bool UserAllowed(const std::string& username);
+
+  // Reset the ACL to the specific set of usernames.
+  void Reset(std::set<std::string> users);
+
+ private:
+  // The set of users, or a set with the single value '*' for the wildcard.
+  std::set<std::string> users_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/mini_kdc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/mini_kdc-test.cc b/be/src/kudu/security/test/mini_kdc-test.cc
new file mode 100644
index 0000000..e0ba455
--- /dev/null
+++ b/be/src/kudu/security/test/mini_kdc-test.cc
@@ -0,0 +1,144 @@
+// 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 <string>
+
+#include <boost/optional/optional.hpp>
+#include <gtest/gtest.h>
+
+#include "kudu/security/init.h"
+#include "kudu/security/test/mini_kdc.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+
+namespace kudu {
+
+class MiniKdcTest : public KuduTest {};
+
+TEST_F(MiniKdcTest, TestBasicOperation) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+  ASSERT_OK(kdc.Start());
+  ASSERT_GT(kdc.port(), 0);
+  ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+  ASSERT_OK(kdc.Kinit("alice"));
+
+  ASSERT_OK(kdc.Stop());
+  ASSERT_OK(kdc.Start());
+
+  // Check that alice is kinit'd.
+  string klist;
+  ASSERT_OK(kdc.Klist(&klist));
+  ASSERT_STR_CONTAINS(klist, "alice@KRBTEST.COM");
+
+  ASSERT_OK(kdc.CreateUserPrincipal("bob"));
+  ASSERT_OK(kdc.Kinit("bob"));
+
+  // Check that bob has replaced alice as the kinit'd principal.
+  ASSERT_OK(kdc.Klist(&klist));
+  ASSERT_STR_NOT_CONTAINS(klist, "alice@KRBTEST.COM");
+  ASSERT_STR_CONTAINS(klist, "bob@KRBTEST.COM");
+  ASSERT_STR_CONTAINS(klist, "krbtgt/KRBTEST.COM@KRBTEST.COM");
+
+  // Drop 'bob' credentials. We'll get a RuntimeError because klist
+  // exits with a non-zero exit code if there are no cached credentials.
+  ASSERT_OK(kdc.Kdestroy());
+  ASSERT_TRUE(kdc.Klist(&klist).IsRuntimeError());
+
+  // Test keytab creation.
+  const string kSPN = "kudu/foo.example.com";
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab(kSPN, &kt_path));
+  SCOPED_TRACE(kt_path);
+  ASSERT_OK(kdc.KlistKeytab(kt_path, &klist));
+  ASSERT_STR_CONTAINS(klist, "kudu/foo.example.com@KRBTEST.COM");
+
+  // Test programmatic keytab login.
+  kdc.SetKrb5Environment();
+  ASSERT_OK(security::InitKerberosForServer(kSPN, kt_path));
+  ASSERT_EQ("kudu/foo.example.com@KRBTEST.COM", *security::GetLoggedInPrincipalFromKeytab());
+
+  // Test principal canonicalization.
+  string princ = "foo";
+  ASSERT_OK(security::CanonicalizeKrb5Principal(&princ));
+  ASSERT_EQ("foo@KRBTEST.COM", princ);
+
+  // Test auth-to-local mapping for a user from the local realm as well as a remote realm.
+  {
+    string local_user;
+    ASSERT_OK(security::MapPrincipalToLocalName("foo@KRBTEST.COM", &local_user));
+    ASSERT_EQ("foo", local_user);
+
+    ASSERT_OK(security::MapPrincipalToLocalName("foo/host@KRBTEST.COM", &local_user));
+    ASSERT_EQ("foo", local_user);
+
+    // The Heimdal implementation in macOS does not correctly implement auth to
+    // local mapping (see init.cc).
+#ifndef __APPLE__
+    ASSERT_OK(security::MapPrincipalToLocalName("foo@OTHERREALM.COM", &local_user));
+    ASSERT_EQ("other-foo", local_user);
+#endif
+  }
+}
+
+// Regression test to ensure that dropping a stopped MiniKdc doesn't panic.
+TEST_F(MiniKdcTest, TestStopDrop) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+}
+
+TEST_F(MiniKdcTest, TestOperationsWhenKdcNotRunning) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+  ASSERT_OK(kdc.Start());
+  ASSERT_OK(kdc.Stop());
+
+  // MiniKdc::CreateUserPrincipal() works directly with the local files,
+  // so it should work fine even if KDC is shut down.
+  ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+
+  {
+    // Without running KDC it should not be possible to obtain and cache an
+    // initial ticket-granting ticket for principal.
+    const Status s = kdc.Kinit("alice");
+    ASSERT_TRUE(s.IsRuntimeError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "process exited with non-zero status");
+  }
+  {
+    // Without running KDC klist should fail.
+    string klist;
+    const Status s = kdc.Klist(&klist);
+    ASSERT_TRUE(s.IsRuntimeError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "process exited with non-zero status");
+  }
+
+  ASSERT_OK(kdc.Start());
+
+  // Once KDC has started, 'kinit' and 'klist' should work with no issues.
+  ASSERT_OK(kdc.Kinit("alice"));
+  {
+    // Check that alice is kinit'd.
+    string klist;
+    ASSERT_OK(kdc.Klist(&klist));
+    ASSERT_STR_CONTAINS(klist, "alice@KRBTEST.COM");
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/mini_kdc.cc b/be/src/kudu/security/test/mini_kdc.cc
new file mode 100644
index 0000000..904695a
--- /dev/null
+++ b/be/src/kudu/security/test/mini_kdc.cc
@@ -0,0 +1,315 @@
+// 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 "kudu/security/test/mini_kdc.h"
+
+#include <csignal>
+#include <cstdlib>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_util.h"
+
+using std::map;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+string MiniKdcOptions::ToString() const {
+  return strings::Substitute("{ realm: $0, data_root: $1, port: $2, "
+      "ticket_lifetime: $3, renew_lifetime: $4 }",
+      realm, data_root, port, ticket_lifetime, renew_lifetime);
+}
+
+MiniKdc::MiniKdc()
+    : MiniKdc(MiniKdcOptions()) {
+}
+
+MiniKdc::MiniKdc(MiniKdcOptions options)
+    : options_(std::move(options)) {
+  if (options_.realm.empty()) {
+    options_.realm = "KRBTEST.COM";
+  }
+  if (options_.data_root.empty()) {
+    options_.data_root = JoinPathSegments(GetTestDataDirectory(), "krb5kdc");
+  }
+  if (options_.ticket_lifetime.empty()) {
+    options_.ticket_lifetime = "24h";
+  }
+  if (options_.renew_lifetime.empty()) {
+    options_.renew_lifetime = "7d";
+  }
+}
+
+MiniKdc::~MiniKdc() {
+  if (kdc_process_) {
+    WARN_NOT_OK(Stop(), "Unable to stop MiniKdc");
+  }
+}
+
+map<string, string> MiniKdc::GetEnvVars() const {
+  return {
+    {"KRB5_CONFIG", JoinPathSegments(options_.data_root, "krb5.conf")},
+    {"KRB5_KDC_PROFILE", JoinPathSegments(options_.data_root, "kdc.conf")},
+    {"KRB5CCNAME", JoinPathSegments(options_.data_root, "krb5cc")},
+    // Enable the workaround for MIT krb5 1.10 bugs from krb5_realm_override.cc.
+    {"KUDU_ENABLE_KRB5_REALM_FIX", "yes"}
+  };
+}
+
+vector<string> MiniKdc::MakeArgv(const vector<string>& in_argv) {
+  vector<string> real_argv = { "env" };
+  for (const auto& p : GetEnvVars()) {
+    real_argv.push_back(Substitute("$0=$1", p.first, p.second));
+  }
+  for (const string& a : in_argv) {
+    real_argv.push_back(a);
+  }
+  return real_argv;
+}
+
+namespace {
+// Attempts to find the path to the specified Kerberos binary, storing it in 'path'.
+Status GetBinaryPath(const string& binary, string* path) {
+  static const vector<string> kCommonLocations = {
+    "/usr/local/opt/krb5/sbin", // Homebrew
+    "/usr/local/opt/krb5/bin", // Homebrew
+    "/opt/local/sbin", // Macports
+    "/opt/local/bin", // Macports
+    "/usr/lib/mit/sbin", // SLES
+    "/usr/sbin", // Linux
+  };
+  return FindExecutable(binary, kCommonLocations, path);
+}
+} // namespace
+
+Status MiniKdc::Start() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "starting KDC");
+  CHECK(!kdc_process_);
+  VLOG(1) << "Starting Kerberos KDC: " << options_.ToString();
+
+  if (!Env::Default()->FileExists(options_.data_root)) {
+    VLOG(1) << "Creating KDC database and configuration files";
+    RETURN_NOT_OK(Env::Default()->CreateDir(options_.data_root));
+
+    RETURN_NOT_OK(CreateKdcConf());
+    RETURN_NOT_OK(CreateKrb5Conf());
+
+    // Create the KDC database using the kdb5_util tool.
+    string kdb5_util_bin;
+    RETURN_NOT_OK(GetBinaryPath("kdb5_util", &kdb5_util_bin));
+
+    RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+        kdb5_util_bin, "create",
+        "-s", // Stash the master password.
+        "-P", "masterpw", // Set a password.
+        "-W", // Use weak entropy (since we don't need real security).
+    })));
+  }
+
+  // Start the Kerberos KDC.
+  string krb5kdc_bin;
+  RETURN_NOT_OK(GetBinaryPath("krb5kdc", &krb5kdc_bin));
+
+  kdc_process_.reset(new Subprocess(
+      MakeArgv({
+      krb5kdc_bin,
+      "-n", // Do not daemonize.
+  })));
+
+  RETURN_NOT_OK(kdc_process_->Start());
+
+  const bool need_config_update = (options_.port == 0);
+  // Wait for KDC to start listening on its ports and commencing operation.
+  RETURN_NOT_OK(WaitForUdpBind(kdc_process_->pid(), &options_.port, MonoDelta::FromSeconds(1)));
+
+  if (need_config_update) {
+    // If we asked for an ephemeral port, grab the actual ports and
+    // rewrite the configuration so that clients can connect.
+    RETURN_NOT_OK(CreateKrb5Conf());
+    RETURN_NOT_OK(CreateKdcConf());
+  }
+
+  return Status::OK();
+}
+
+Status MiniKdc::Stop() {
+  if (!kdc_process_) {
+    return Status::OK();
+  }
+  VLOG(1) << "Stopping KDC";
+  unique_ptr<Subprocess> proc(kdc_process_.release());
+  RETURN_NOT_OK(proc->Kill(SIGKILL));
+  RETURN_NOT_OK(proc->Wait());
+
+  return Status::OK();
+}
+
+// Creates a kdc.conf file according to the provided options.
+Status MiniKdc::CreateKdcConf() const {
+  static const string kFileTemplate = R"(
+[kdcdefaults]
+kdc_ports = $2
+kdc_tcp_ports = ""
+
+[realms]
+$1 = {
+        acl_file = $0/kadm5.acl
+        admin_keytab = $0/kadm5.keytab
+        database_name = $0/principal
+        key_stash_file = $0/.k5.$1
+        max_renewable_life = 7d 0h 0m 0s
+}
+  )";
+  string file_contents = strings::Substitute(kFileTemplate, options_.data_root,
+                                             options_.realm, options_.port);
+  return WriteStringToFile(Env::Default(), file_contents,
+                           JoinPathSegments(options_.data_root, "kdc.conf"));
+}
+
+// Creates a krb5.conf file according to the provided options.
+Status MiniKdc::CreateKrb5Conf() const {
+  static const string kFileTemplate = R"(
+[logging]
+    kdc = FILE:/dev/stderr
+
+[libdefaults]
+    default_realm = $1
+    dns_lookup_kdc = false
+    dns_lookup_realm = false
+    forwardable = true
+    renew_lifetime = $2
+    ticket_lifetime = $3
+
+    # Disable aes256 since Java does not support it without JCE. Java is only
+    # one of several minicluster consumers, but disabling aes256 doesn't
+    # appreciably hurt Kudu code coverage, so we disable it universally.
+    #
+    # For more details, see:
+    # https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/jgss-features.html
+    default_tkt_enctypes = aes128-cts des3-cbc-sha1
+    default_tgs_enctypes = aes128-cts des3-cbc-sha1
+    permitted_enctypes = aes128-cts des3-cbc-sha1
+
+    # In miniclusters, we start daemons on local loopback IPs that
+    # have no reverse DNS entries. So, disable reverse DNS.
+    rdns = false
+
+    # The server side will start its GSSAPI server using the local FQDN.
+    # However, in tests, we connect to it via a non-matching loopback IP.
+    # This enables us to connect despite that mismatch.
+    ignore_acceptor_hostname = true
+
+[realms]
+    $1 = {
+        kdc = 127.0.0.1:$0
+        # This super-arcane syntax can be found documented in various Hadoop
+        # vendors' security guides and very briefly in the MIT krb5 docs.
+        # Basically, this one says to map anyone coming in as foo@OTHERREALM.COM
+        # and map them to a local user 'other-foo'
+        auth_to_local = RULE:[1:other-$$1@$$0](.*@OTHERREALM.COM$$)s/@.*//
+    }
+  )";
+  string file_contents = strings::Substitute(kFileTemplate, options_.port, options_.realm,
+                                             options_.renew_lifetime, options_.ticket_lifetime);
+  return WriteStringToFile(Env::Default(), file_contents,
+                           JoinPathSegments(options_.data_root, "krb5.conf"));
+}
+
+Status MiniKdc::CreateUserPrincipal(const string& username) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating user principal $0", username));
+  string kadmin;
+  RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("add_principal -pw $0 $0", username)})));
+  return Status::OK();
+}
+
+Status MiniKdc::CreateServiceKeytab(const string& spn,
+                                    string* path) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating service keytab for $0", spn));
+  string kt_path = spn;
+  StripString(&kt_path, "/", '_');
+  kt_path = JoinPathSegments(options_.data_root, kt_path) + ".keytab";
+
+  string kadmin;
+  RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("add_principal -randkey $0", spn)})));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("ktadd -k $0 $1", kt_path, spn)})));
+  *path = kt_path;
+  return Status::OK();
+}
+
+Status MiniKdc::Kinit(const string& username) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("kinit for $0", username));
+  string kinit;
+  RETURN_NOT_OK(GetBinaryPath("kinit", &kinit));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ kinit, username }), username));
+  return Status::OK();
+}
+
+Status MiniKdc::Kdestroy() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "kdestroy");
+  string kdestroy;
+  RETURN_NOT_OK(GetBinaryPath("kdestroy", &kdestroy));
+  return Subprocess::Call(MakeArgv({ kdestroy, "-A" }));
+}
+
+Status MiniKdc::Klist(string* output) {
+  string klist;
+  RETURN_NOT_OK(GetBinaryPath("klist", &klist));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ klist, "-A" }), "", output));
+  return Status::OK();
+}
+
+Status MiniKdc::KlistKeytab(const string& keytab_path, string* output) {
+  string klist;
+  RETURN_NOT_OK(GetBinaryPath("klist", &klist));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ klist, "-k", keytab_path }), "", output));
+  return Status::OK();
+}
+
+Status MiniKdc::SetKrb5Environment() const {
+  if (!kdc_process_) {
+    return Status::IllegalState("KDC not started");
+  }
+  for (const auto& p : GetEnvVars()) {
+    CHECK_ERR(setenv(p.first.c_str(), p.second.c_str(), 1 /*overwrite*/));
+  }
+
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/mini_kdc.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/mini_kdc.h b/be/src/kudu/security/test/mini_kdc.h
new file mode 100644
index 0000000..e282cc4
--- /dev/null
+++ b/be/src/kudu/security/test/mini_kdc.h
@@ -0,0 +1,134 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Subprocess;
+
+struct MiniKdcOptions {
+
+  // Kerberos Realm.
+  //
+  // Default: "KRBTEST.COM".
+  std::string realm;
+
+  // Directory in which to store data.
+  //
+  // Default: "", which auto-generates a unique path for this KDC.
+  // The default may only be used from a gtest unit test.
+  std::string data_root;
+
+  // KDC port.
+  //
+  // Default: 0 (ephemeral port).
+  uint16_t port = 0;
+
+  // The default lifetime for initial ticket requests.
+  std::string ticket_lifetime;
+
+  // The default renewable lifetime for initial ticket requests.
+  std::string renew_lifetime;
+
+  // Returns a string representation of the options suitable for debug printing.
+  std::string ToString() const;
+};
+
+class MiniKdc {
+ public:
+  // Creates a new MiniKdc with the default options.
+  MiniKdc();
+
+  // Creates a new MiniKdc with the provided options.
+  explicit MiniKdc(MiniKdcOptions options);
+
+  ~MiniKdc();
+
+  // Starts the mini Kerberos KDC.
+  Status Start() WARN_UNUSED_RESULT;
+
+  // Stops the mini Kerberos KDC.
+  Status Stop() WARN_UNUSED_RESULT;
+
+  uint16_t port() const {
+    CHECK(kdc_process_) << "must start first";
+    return options_.port;
+  }
+
+  // Creates a new user with the given username.
+  // The password is the same as the username.
+  Status CreateUserPrincipal(const std::string& username) WARN_UNUSED_RESULT;
+
+  // Creates a new service principal and associated keytab, returning its
+  // path in 'path'. 'spn' is the desired service principal name
+  // (e.g. "kudu/foo.example.com"). If the principal already exists, its key
+  // will be reset and a new keytab will be generated.
+  Status CreateServiceKeytab(const std::string& spn, std::string* path);
+
+  // Kinit a user to the mini KDC.
+  Status Kinit(const std::string& username) WARN_UNUSED_RESULT;
+
+  // Destroy any credentials in the current ticket cache.
+  // Equivalent to 'kdestroy -A'.
+  Status Kdestroy() WARN_UNUSED_RESULT;
+
+  // Call the 'klist' utility.  This is useful for logging the local ticket
+  // cache state.
+  Status Klist(std::string* output) WARN_UNUSED_RESULT;
+
+  // Call the 'klist' utility to list the contents of a specific keytab.
+  Status KlistKeytab(const std::string& keytab_path,
+                     std::string* output) WARN_UNUSED_RESULT;
+
+  // Sets the environment variables used by the krb5 library
+  // in the current process. This points the SASL library at the
+  // configuration associated with this KDC.
+  Status SetKrb5Environment() const;
+
+  // Returns a map of the Kerberos environment variables which configure
+  // a process to use this KDC.
+  std::map<std::string, std::string> GetEnvVars() const;
+
+ private:
+
+  // Prepends required Kerberos environment variables to the process arguments.
+  std::vector<std::string> MakeArgv(const std::vector<std::string>& in_argv);
+
+  // Creates a kdc.conf in the data root.
+  Status CreateKrb5Conf() const WARN_UNUSED_RESULT;
+
+  // Creates a krb5.conf in the data root.
+  Status CreateKdcConf() const WARN_UNUSED_RESULT;
+
+  std::unique_ptr<Subprocess> kdc_process_;
+  MiniKdcOptions options_;
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/test/test_certs.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/test_certs.cc b/be/src/kudu/security/test/test_certs.cc
new file mode 100644
index 0000000..88cf5cf
--- /dev/null
+++ b/be/src/kudu/security/test/test_certs.cc
@@ -0,0 +1,969 @@
+// 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 "kudu/security/test/test_certs.h"
+
+#include <string>
+
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+namespace security {
+
+//
+// The easiest way to create RSA private key and CA self-signed certificate pair
+// is using the couple of commands below:
+//
+//   openssl genrsa -out ca.pkey.pem 2048
+//   openssl req -new -x509 -batch -days 3650 -key ca.pkey.pem -out ca.cert.pem
+//
+// NOTE:
+//   The latter command uses configuration properties from default configuration
+//   file of the OpenSSL library.  Also, it runs in batch mode due to the
+//   '-batch' flag. To specify custom certificate subject properties, omit
+//   the '-batch' flag and run the command in interactive mode. If more
+//   customization is needed, see the other methods below.
+//
+////////////////////////////////////////////////////////////////////////////
+//
+// The other way to create RSA private key and CA self-signed certificate pair
+// is using OpenSSL's CA.sh script in $OPENSSL_SRC_ROOT/apps:
+//
+//   cp $OPENSSL_SRC_ROOT/CA.sh .
+//   chmod +x CA.sh
+//   ./CA.sh -newca
+//
+// Find the newly generated files at the following locations:
+//   * demoCA/cacert.pem:         self-signed CA certificate
+//   * demoCA/private/cakey.pem:  encrypted CA private key
+//
+// To decrypt the generated private key, run the following command and provide
+// the pass phrase (assuming that was an RSA key):
+//
+//   openssl rsa -in ./demoCA/private/cakey.pem
+//
+////////////////////////////////////////////////////////////////////////////
+//
+// Besides, the following sequence of commands can used to create
+// a private key and CA certficate with custom properties.
+//
+//  * Create a separate directory, e.g.:
+//
+//      mkdir /tmp/cert && cd /tmp/cert
+//
+//  * Create custom my.cnf configuration file for the OpenSSL library, copying
+//    the default one and modifying the result, if necessary.
+//
+//      cp $OPENSSL_CFG_ROOT/etc/openssl.cnf my.cnf
+//      vim my.cnf
+//
+//  * Create the CA directory structure which matches the directory structure
+//    of the 'default_ca' section from the configuration file, e.g.:
+//
+//      mkdir -p demoCA/certs demoCA/crl demoCA/newcerts demoCA/private
+//      touch demoCA/index.txt
+//
+//  * Create private key and certificate signing request (CSR):
+//
+//      openssl req -new -keyout ca.pkey.pem -out ca.req.pem \
+//        -subj "/C=US/ST=CA/O=MyCompany/CN=MyName/emailAddress=my@email.com" \
+//        -passin pass:mega_pass -passout pass:mega_pass -batch
+//
+//  * Create a self-signed certificate using the newly generated CSR as input:
+//
+//      openssl ca -config my.cnf -create_serial -days 3650 \
+//        -keyfile ca.pkey.pem -selfsign -extensions v3_ca \
+//        -outdir ./ -out ca.cert.pem -passin pass:mega_pass -batch \
+//        -infiles ca.req.pem
+//
+// The encryped private key is in ca.pkey.pem, the certificate is in
+// ca.cert.pem.  To decrypt the generated private key, execute the following
+// (assuming that was an RSA key):
+//
+//   openssl rsa -passin pass:mega_pass -in ./ca.pkey.pem
+//
+const char kCaCert[] = R"***(
+-----BEGIN CERTIFICATE-----
+MIIDizCCAnOgAwIBAgIJAIsQXjBhvdPoMA0GCSqGSIb3DQEBCwUAMFwxCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTESMBAGA1UECgwJTXlDb21wYW55MQ8wDQYDVQQD
+DAZNeU5hbWUxGzAZBgkqhkiG9w0BCQEWDG15QGVtYWlsLmNvbTAeFw0xNjEwMjUw
+NjAxNThaFw0yNjEwMjMwNjAxNThaMFwxCzAJBgNVBAYTAlVTMQswCQYDVQQIDAJD
+QTESMBAGA1UECgwJTXlDb21wYW55MQ8wDQYDVQQDDAZNeU5hbWUxGzAZBgkqhkiG
+9w0BCQEWDG15QGVtYWlsLmNvbTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
+ggEBAKexXVOe0SfNexxl1nqMBRy8MCYWTl1kbRt5VQ698aXYcPNBC7gnEBW+8Yaa
+2f3Hl1Ye51zUGnOl4FU6HFDiIq59/lKCNG2X3amlYjzkImXn4M56r+5rEWs+HoHW
+kuqmMaxnrJatM86Of0K3j5QrOUft/qT5R6vSPnFH/pz+6ccBkAGV0UFVdshYSGkx
+KziVTdJ2Ri8oZgyeuReGxLkXOqKHzcOUFinvQ8fe8yaQr1kRAaPRo1eFqORXAMAU
+4KyvfiVjZMEGj0p47IekJHVPVVMopEmMMjhzRfbrxrKrMcIG6e4acF1KAd4wGI9A
+pCR3e1vcfbghDO7GhTMswLCnMYUCAwEAAaNQME4wHQYDVR0OBBYEFDc1+ybIwvG2
+IvEuAusZ9GGMlga/MB8GA1UdIwQYMBaAFDc1+ybIwvG2IvEuAusZ9GGMlga/MAwG
+A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAJT9fL/vtayfAHpdzFvdWBe+
+R6y5HsVQQTBNF9x1eM6M0vGAlsXGgip3+RH7OMwurxNetL2mc03+PECas5LxB5Pr
+u1+kwtmv5YyfQzou0VwztjcbK2OEpWKj16XX6NO403iKoRF4fLn0DjZQcB0oXw4s
+vBxhNfz+SAsjsAMNgLHHXonJfg7wcdmNSp2N3TslGL/DH0bXMhsKx2CuMA3rd9WZ
+mJjItRIk8qNjazlmG0KYxQclP3lGagIMHxU6tY+iBXs1JR1/AUnPl/GaPeayCJSR
+3PB7R+MMrI0hfWFWkBt0D+UAKVa9to/N06wp4JqxEgOooU08PguXLIVDlW0xBcw=
+-----END CERTIFICATE-----
+)***";
+
+
+// See the comment for kCaCert_
+const char kCaPrivateKey[] = R"***(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEogIBAAKCAQEAp7FdU57RJ817HGXWeowFHLwwJhZOXWRtG3lVDr3xpdhw80EL
+uCcQFb7xhprZ/ceXVh7nXNQac6XgVTocUOIirn3+UoI0bZfdqaViPOQiZefgznqv
+7msRaz4egdaS6qYxrGeslq0zzo5/QrePlCs5R+3+pPlHq9I+cUf+nP7pxwGQAZXR
+QVV2yFhIaTErOJVN0nZGLyhmDJ65F4bEuRc6oofNw5QWKe9Dx97zJpCvWREBo9Gj
+V4Wo5FcAwBTgrK9+JWNkwQaPSnjsh6QkdU9VUyikSYwyOHNF9uvGsqsxwgbp7hpw
+XUoB3jAYj0CkJHd7W9x9uCEM7saFMyzAsKcxhQIDAQABAoIBABuZQ0TZ5I5qcRKR
+aCUvGkBKcJo0HZ2dQ5+77lXIyRaEcsJ2OBmMxEbv8Aw5PBtaV/vihi1u8xOJf0xH
+jhV5wj95mPu3Vi2bSu36vBpNaaPf783Lv1y73lgKFzdDO1bHF3HKdksuIlKifStb
+zpOSMZE3CCvaowMSTRiTwsHP6mXIBdQ/TwAZHqGVTWDVGxc8JvoJ/3GjSgUIPKzy
+I2aS/5DQ+zmLktuP61GFMJg9tCSrwZPDi/XAatpoAOC9eA7AqF/l1TiaXsQN95mr
+mz2DkCoWRzAuDbya2Sh6nTJvpOMPAeXJ/MMZh9TWswJc4OAO2kZZsFfd0H6M1TKy
+1eAYKVkCgYEA1JhkKQ2h4cVzqQ9A5+4C0q5+j/RFDUOVnNlIjQiM73RchNu713mK
+zzhsom9S/6ZU8OH3TxzD54i2hHtX+QIJqVG0412QgAqAqnAKXGGkkAXiXGfGZhEW
+UB3OuTMbhfVqrkpj0wAPiEJAAuek7zES2B+gURUC24aAfOWU8xMkSjMCgYEAye4U
+e0NQ4HhhWRgWbgFYeAzsC/ezvlx30JjXiLPCNXGoLLJUCMjqWCPGYUvDonIJbxbj
++MYFkvYSDFGwTobKsB7FyT8DxPNus40zOh47y8QUK7jTL4nAmnBa3W9Oj00ceKpo
+wKe/adc2xPrS7mnVpz3ZkJ4I9z/MbEinyV5UTWcCgYAy8gXmlJ67dM6/r6kVK0M/
+65Lmulml0RFUUfmB2o+zfkYBjIqaG0U5XUMjNdxE6T4nr27NZY5IuMlMPCabxHI+
+Qhc/+Rb8qAenUEwbUUbXQKG7FR9FLEkVj98PIIEy+9nBxI/ha31NYNroF0y+CRuD
+8ShA5fEWXEgEJhwol+i1YwKBgEnGeiUuyvW4BZkPe+JlC3WRAwy8SydZkUzdCqIf
+Su1LwS3TWXB8N2JMb8ZMcAWBtICp1FCnyJGQ5bcqgUevZ45BL/H+29mxNtjS1cx+
+D0q7MMNom3/azEugkRImAIXKnoRXfj4lC4IX5yLAoSAJ+s1Hg52an5v16zIEuYiQ
+tiwxAoGAOP8/yjMzit1hzk27k9IfQSLD+1SqKCsRdGbAIhFRFlz4RUQOly1dEX8M
+qVmStlQ7N5gQWJSyDTe6rTe8pG9r030kNDJ+etr2KWpATGNaVWSmLWSYBXrPtejK
+gmbcYCewtt7dFP9tvx6k7aUQ6CKzg0GxaIHQecNzjxYrw8sb4Js=
+-----END RSA PRIVATE KEY-----
+)***";
+
+// Corresponding public key for the kCaPrivateKey
+const char kCaPublicKey[] = R"***(
+-----BEGIN PUBLIC KEY-----
+MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAp7FdU57RJ817HGXWeowF
+HLwwJhZOXWRtG3lVDr3xpdhw80ELuCcQFb7xhprZ/ceXVh7nXNQac6XgVTocUOIi
+rn3+UoI0bZfdqaViPOQiZefgznqv7msRaz4egdaS6qYxrGeslq0zzo5/QrePlCs5
+R+3+pPlHq9I+cUf+nP7pxwGQAZXRQVV2yFhIaTErOJVN0nZGLyhmDJ65F4bEuRc6
+oofNw5QWKe9Dx97zJpCvWREBo9GjV4Wo5FcAwBTgrK9+JWNkwQaPSnjsh6QkdU9V
+UyikSYwyOHNF9uvGsqsxwgbp7hpwXUoB3jAYj0CkJHd7W9x9uCEM7saFMyzAsKcx
+hQIDAQAB
+-----END PUBLIC KEY-----
+)***";
+
+// See the comment for kCaCert_
+// (but use '-1' as number of days for the certificate expiration).
+const char kCaExpiredCert[] = R"***(
+-----BEGIN CERTIFICATE-----
+MIIDjTCCAnWgAwIBAgIJALNJes+nGWH9MA0GCSqGSIb3DQEBCwUAMF0xCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTETMBEGA1UECgwKRXhwQ29tcGFueTEQMA4GA1UE
+AwwHRXhwTmFtZTEaMBgGCSqGSIb3DQEJARYLZXhwQGV4cC5jb20wHhcNMTYxMDI1
+MTkzOTM4WhcNMTYxMDI0MTkzOTM4WjBdMQswCQYDVQQGEwJVUzELMAkGA1UECAwC
+Q0ExEzARBgNVBAoMCkV4cENvbXBhbnkxEDAOBgNVBAMMB0V4cE5hbWUxGjAYBgkq
+hkiG9w0BCQEWC2V4cEBleHAuY29tMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
+CgKCAQEAzqPj5nRm57mr9YtZDvHREuVFHTwPcKzDeff9fnrKKwOJPSF0Bou/BjS1
+S7yQYAtmT/EMi7qxEWjgrR1qW+muR8QN+zAwNdkdLrFK3SJigQ4a/OeSH86aHXUD
+ekV8mgBgzP90osbHf7AiqrGzkYWq+ApTO/IgnXgaWbbdt5znGTW5lKQ4O2CYhpcM
+MC1sBBjW7Qqx+Gi8iXub0zlJ2mVI8o+zb9qvSDb8fa0JYxasRDn/nB0wKZC3f/Gf
+Rs+lJZUTEy5+eMhVdj1RjVBE+mgW7L27On24ViPU7B3DjM0SYnD6ZOUWMH0mtwO8
+W3OoK8MJhPvFP7Lr5QfSjiBH+ryLOwIDAQABo1AwTjAdBgNVHQ4EFgQUsp8OZLl1
+2Z/2aXBQRH0Z+nWxqXcwHwYDVR0jBBgwFoAUsp8OZLl12Z/2aXBQRH0Z+nWxqXcw
+DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEArWvFi13iqmvnY0xkgt3R
+acurTvWcQzcUOgVPF8u1atj9d+0zrMk7Don1KQp6uYLdeNLL8NbL4oLxtidW/Yap
+ZEbHVDQTeZAsT7Hr+0uD3vMUndsjG7C85tOhZMiGukFPhuaHE5KmQEy6nUCaJiAv
+opZlNj1mEOGyshSXHsBATl9o33WLTLfPqrO3/12jExApHiADcON4RsPUV6M6k5A2
+/KghYEPYAuFfXTsqj+W7HRL1UuiHJxW96ySQqYzQ86aRN2ZZlTdbDnIU5Jrb6YJB
+hUALcxIUhtodui61zsJFIkVauxTxk7jNCwRvj4I1dSSFWA63t9eh7sKilLRCayNl
+yQ==
+-----END CERTIFICATE-----
+)***";
+
+// See the comment for kCaExpiredCert_
+const char kCaExpiredPrivateKey[] = R"***(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpAIBAAKCAQEAzqPj5nRm57mr9YtZDvHREuVFHTwPcKzDeff9fnrKKwOJPSF0
+Bou/BjS1S7yQYAtmT/EMi7qxEWjgrR1qW+muR8QN+zAwNdkdLrFK3SJigQ4a/OeS
+H86aHXUDekV8mgBgzP90osbHf7AiqrGzkYWq+ApTO/IgnXgaWbbdt5znGTW5lKQ4
+O2CYhpcMMC1sBBjW7Qqx+Gi8iXub0zlJ2mVI8o+zb9qvSDb8fa0JYxasRDn/nB0w
+KZC3f/GfRs+lJZUTEy5+eMhVdj1RjVBE+mgW7L27On24ViPU7B3DjM0SYnD6ZOUW
+MH0mtwO8W3OoK8MJhPvFP7Lr5QfSjiBH+ryLOwIDAQABAoIBABszgcWNXxpz24oI
+HOIVvPLi0VVG2bV4WIcOuQTUPxaocYFljPNro+q6N39PxCWQephdX8xo9/QVvTWs
+oJqWyUVTLo/5SO9dtDS4S+WOKC9a3vyZsyeSt8DW7W1EBmHzWMrDeeQPjKVnVzjn
+CX9HfDkIiupiNh7kd3uF0evgsJ8lsZ65HtBq9MWu+mIR1H0EpRLxywdoRJLJ+JdW
+g1fLFRuhnWo0GcEyBK45kLCoVJsRbCkFGf6uPDOOC0g5mIyxGclWeF6ps1OFnFyu
+FWsYeMLSt5tYZfB0/QR46X9HQOhfLunjA04VBkScSRjlohGO4d20ZW7HlPY20CbR
+1PHhEvkCgYEA98FYoovNezx8OgkcAtNOOTK7GpUaUfh3Xl5yPGgCqxoG8G+BTmKF
+MGlIf6URKQA0BUtNdjIvfIcaIctj56qFwjHL6CbzR5MkXUZLlyl0XzYFXm/lavr4
+Z5DHWdFo+GyFaiXIiVof93jAnOFgjSxdhHaEhQqj7pmaBoHVZqtwHFcCgYEA1YRH
+xTzcWErp06KJTt+/P4YtWRh9GDBhhlO3oaGOANkEab8cGjRO9LJP24wyo7exXqGb
+UjtEifEHtzhj6a/UwSAMsFcNhlQRvy525HD1gJmQ2m4wZ3GxztK4IZ4rVDjsB5/D
+SMMBsDfs1r1iRwdSMHAOhrVH2l/DMFQLnx1x+b0CgYEAlQm6SA3RjlDUahUQxKJY
+bBAYfeUz8BuHsz0dezkWYddGVVy+bGjXtkefVSn3KLL2mDi0YGXQKxkanzm636G0
+1R0fjIfh0Syys2mWD1jgqGXW1Ph7Cd/vjl2Jjn5qpwahOzl/aSDOGhCJzdXGPyZx
+Gz4wedfsxZuhDEkOFrUKvAECgYEAxHYYy8V6Qct8Z30wtmBuSvcdFtPPlsg9lCnH
+13MdhG4q/1oXc40Z8VF45VyU48uL6rTsg7eBEyOyo8XBOS7Opnzk8ATJrwX/5lfM
+kdnWK2QhwrqM00HsB5AgWN5+o9pUY5d/Sp4UGZ77z4MmwJBd8a/Jze1Tlf1zTi6n
+GtsvGkkCgYAfILUAPf+ujgB9zdsJa+4l9XCEq0j39/Usfj0VrInNAk7RN8W0qNw7
+ZLs3Qt2fgPO0CeMeVUVKcvdjlXq3EbrWKrsJLxy3Gb8ruBjIlJqncJn6mKslXS+l
+H/sbP2R+P6RvQceLEEtk6ZZLiuScVmLtVOpUoUZb3Rx6a7GKbec7oQ==
+-----END RSA PRIVATE KEY-----
+)***";
+
+// Corresponding public part of the kCaExpiredPrivateKey
+const char kCaExpiredPublicKey[] = R"***(
+-----BEGIN PUBLIC KEY-----
+MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAzqPj5nRm57mr9YtZDvHR
+EuVFHTwPcKzDeff9fnrKKwOJPSF0Bou/BjS1S7yQYAtmT/EMi7qxEWjgrR1qW+mu
+R8QN+zAwNdkdLrFK3SJigQ4a/OeSH86aHXUDekV8mgBgzP90osbHf7AiqrGzkYWq
++ApTO/IgnXgaWbbdt5znGTW5lKQ4O2CYhpcMMC1sBBjW7Qqx+Gi8iXub0zlJ2mVI
+8o+zb9qvSDb8fa0JYxasRDn/nB0wKZC3f/GfRs+lJZUTEy5+eMhVdj1RjVBE+mgW
+7L27On24ViPU7B3DjM0SYnD6ZOUWMH0mtwO8W3OoK8MJhPvFP7Lr5QfSjiBH+ryL
+OwIDAQAB
+-----END PUBLIC KEY-----
+)***";
+
+const char kCertDnsHostnamesInSan[] = R"***(
+-----BEGIN CERTIFICATE-----
+MIIEPzCCAyegAwIBAgIJAJoczuNKGspGMA0GCSqGSIb3DQEBCwUAMFwxCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTESMBAGA1UECgwJTXlDb21wYW55MQ8wDQYDVQQD
+DAZNeU5hbWUxGzAZBgkqhkiG9w0BCQEWDG15QGVtYWlsLmNvbTAeFw0xNzA0Mjgx
+OTUwNTVaFw0yNzA0MjYxOTUwNTVaMAAwXDANBgkqhkiG9w0BAQEFAANLADBIAkEA
+rpJhLdS/Euf2cu0hPXkvkocLO0XbNtFwXNjkOOjuJZd65FHqLb6TmmxxDpL7fB94
+Mq1fD20fqdAgSVzljOyvuwIDAQABo4ICJjCCAiIwDgYDVR0PAQH/BAQDAgWgMCAG
+A1UdJQEB/wQWMBQGCCsGAQUFBwMBBggrBgEFBQcDAjAMBgNVHRMBAf8EAjAAMIIB
+3gYDVR0RBIIB1TCCAdGCDG1lZ2EuZ2lnYS5pb4ILZm9vLmJhci5jb22CggGydG9v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vLmxvb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29v
+b29vb29vb29vb29vb29vb29vb29vb29vb29vb29vb29vbmcuaG9zdG5hbWUuaW8w
+DQYJKoZIhvcNAQELBQADggEBAIKVVABj3nTyqDEnDKDfvS6QNEQSa1mw7WdFTpzH
+6cbMfiixVyyLqfAV4NZ+PnIa4mpsWP5LrsrWFVK/HtiTX7Y8oW0qdA04WtYd9VUT
+BgWKHyLygbA+PSZ6GdXFjZd8wDthK0qlT2MfZJUwD36eYnBxuonU8a4lmxaUG2zC
+L8FplhNJUEt6XfJ0zZGx1VHe12LLjgMz3ShDAmD9DlHHFjJ1aQ/17OGmmjWmbWnm
+an4ys5seqeHuK2WzP3NAx7LOwe/R1kHpEAX/Al6xyLIY3h7BBzurpgfrO6hTTECF
+561gUMp+cAvogw074thF5j4b+uEK5Bl8nzN2h8BwwxwGzUo=
+-----END CERTIFICATE-----
+)***";
+
+//
+// The reference signatures were obtained by using the following sequence:
+//  0. The reference private key was saved into /tmp/ca.pkey.pem file.
+//  1. Put the input data into /tmp/in.txt file.
+//  2. To sign the input data, run
+//    openssl dgst -sign /tmp/ca.pkey.pem -sha512 -out /tmp/out /tmp/in.txt
+//  3. To capture the signature in text format, run
+//    base64 -b 60 /tmp/out
+//
+const char kDataTiny[] = "Tiny";
+const char kSignatureTinySHA512[] =
+    "omtvSpfj9tKo0RdI4zJwasWSQnXl++aKVjhH19ABJCd0haKT8RXNuhnxcbZU"
+    "Y1ILE5F9YjVj+tN/7ah5WQZR5qlJ6GMFfCFBhOzvi/vf5PSbUrFfwFvFD6sq"
+    "Bu0PWdwKM3t8/YFE2HcZWSzGCcasKlG/aw2eQCN3Kdv8QVMlC28CFA/EqQBt"
+    "8Sfye1DLba33SzDpJqR2DduTFrEW2UffumpYIbkEcMwUSBFzfdp5hgWPowFb"
+    "LrnKvyWKpEPMFGQmf5siyXSkbBIfL774tynhWN/lAUWykwXSUfGgi2G0NQvj"
+    "xmuHhbxWpbW/31uMGssw92OfVQ/+aQ4pNmY9GbibcA==";
+
+const char kDataShort[] = "ShortRefInputData";
+const char kSignatureShortSHA512[] =
+    "BHaDipr8ibn40BMD6+DlatKsjbmsGZsJIDlheppBjqv66eBDLKOVjpmpMLl9"
+    "9lXCGUlVS+cNcVP4RPDzXNoXkpzUOJD3UQSnxCAm6tV1eGjD3SHi3fk6PCNc"
+    "MhM/+09fA0WHdIdZm93cpHt6c9MFzB/dUjHJByhQ7Csmz2zdITyMIl3/D+bi"
+    "ocW0aIibk0wNGn/FmXfgFDP+3pBS2bpS0AdFnckX8AqXHFMJnvqKYODpYCW8"
+    "NWFSD1TgZOumu/gzxm+HySPezQ2j9tdR6nb9swfShvN+o0oBVGq5vgtgZMTM"
+    "7Ws+BrasLfvQFkvtGMWB9VeH/rDlGOym8RwUrCIJJQ==";
+
+const char kDataLong[] =
+R"***(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.
+)***";
+const char kSignatureLongSHA512[] =
+    "kc62qPHApVFbueR1xSCQJR5NomqDRzVA+4Xi9egVyfkKgpVhDAYGxbMl8OTY/YCb"
+    "eQuwY+B7RGxF9sj3gvsq/dvrbIjLT3QDhs0bv+lXTtBQ5r9zrals3de0tEFrPoLr"
+    "CkKPhVZaG+zwmUVltfsdlsqvepy6rNW7BocehvgpPTbzxgsZg4nUANsjSy8HBoDb"
+    "xWyfbkMgBY4aWIH1g+wksq1DHzdTNdZCYstupRwVw/ESC+zrFQiZPFeRE/wCSeG/"
+    "bd0L8TcotQHJchZ8THW0rEbuCg79I7Crd1KQYljBpOOhMYZEDEdM9L19JlaMlw+Z"
+    "leyLfL8Bw3wCg9cMfNmQfQ==";
+
+Status CreateTestSSLCertWithPlainKey(const string& dir,
+                                     string* cert_file,
+                                     string* key_file) {
+  const char* kCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIEejCCA2KgAwIBAgIJAKMdvDR5PL82MA0GCSqGSIb3DQEBBQUAMIGEMQswCQYD
+VQQGEwJVUzETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UEBxMNU2FuIEZyYW5j
+aXNjbzERMA8GA1UEChMIQ2xvdWRlcmExEjAQBgNVBAMTCWxvY2FsaG9zdDEhMB8G
+CSqGSIb3DQEJARYSaGVucnlAY2xvdWRlcmEuY29tMB4XDTEzMDkyMjAwMjUxOFoX
+DTQxMDIwNzAwMjUxOFowgYQxCzAJBgNVBAYTAlVTMRMwEQYDVQQIEwpDYWxpZm9y
+bmlhMRYwFAYDVQQHEw1TYW4gRnJhbmNpc2NvMREwDwYDVQQKEwhDbG91ZGVyYTES
+MBAGA1UEAxMJbG9jYWxob3N0MSEwHwYJKoZIhvcNAQkBFhJoZW5yeUBjbG91ZGVy
+YS5jb20wggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQCoUj3pMQ2ELkrz
+zq+koixljVFBAEEqwUWSjA+GEKwfFb/UPRjeO/wrKndp2r83jc6KRt66rvAIl8cr
+b54yTOsJ/ZcARrjTwG3IG8Tely/54ZQyH0ImdJyEbCSoI04zX3ovjlppz3g5xanj
+WmpAh6pzPgBOTfisCLMPD70xQ8F//QWZdNatoly54STkTWoJv/Oll/UpXcBY8JOR
++ytX82eGgG4F8YoQqmbjrrN5JAmqLRiBAkr3WUena6ekqJBalJRzex/Wh8a9XEV7
+9HFVVngBhezsOJgf81hzBzzhULKfxuXl8uaUj3Z9cZg39CDsyz+ULYbsPm8VoMUI
+VCf7MUVTAgMBAAGjgewwgekwHQYDVR0OBBYEFK94kea7jIKQawAIb+0DqsA1rf6n
+MIG5BgNVHSMEgbEwga6AFK94kea7jIKQawAIb+0DqsA1rf6noYGKpIGHMIGEMQsw
+CQYDVQQGEwJVUzETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UEBxMNU2FuIEZy
+YW5jaXNjbzERMA8GA1UEChMIQ2xvdWRlcmExEjAQBgNVBAMTCWxvY2FsaG9zdDEh
+MB8GCSqGSIb3DQEJARYSaGVucnlAY2xvdWRlcmEuY29tggkAox28NHk8vzYwDAYD
+VR0TBAUwAwEB/zANBgkqhkiG9w0BAQUFAAOCAQEAEtkPPncCnN2IFVJvz04K+VsX
+b6w3qwPynQKc67+++JkNb3TYKrh/0UVM1NrEOu3TGplqOrKgAlITuaWNqNOSBu1R
+WJtrz85YkonED5awjjuALVEY82+c7pOXkuv5G5421RINfRn2hNzgw8VFb5CEvxHH
+jER80Vx6UGKr/S649qTQ8AzVzTwWS86VsGI2azAD7D67G/IDGf+0P7FsXonKY+vl
+vKzkfaO1+qEOLtDHV9mwlsxl3Re/MNym4ExWHi9txynCNiRZHqWoZUS+KyYqIR2q
+seCrQwgi1Fer9Ekd5XNjWjigC3VC3SjMqWaxeKbZ2/AuABJMz5xSiRkgwphXEQ==
+-----END CERTIFICATE-----
+  )";
+  const char* kKey = R"(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpAIBAAKCAQEAqFI96TENhC5K886vpKIsZY1RQQBBKsFFkowPhhCsHxW/1D0Y
+3jv8Kyp3adq/N43Oikbeuq7wCJfHK2+eMkzrCf2XAEa408BtyBvE3pcv+eGUMh9C
+JnSchGwkqCNOM196L45aac94OcWp41pqQIeqcz4ATk34rAizDw+9MUPBf/0FmXTW
+raJcueEk5E1qCb/zpZf1KV3AWPCTkfsrV/NnhoBuBfGKEKpm466zeSQJqi0YgQJK
+91lHp2unpKiQWpSUc3sf1ofGvVxFe/RxVVZ4AYXs7DiYH/NYcwc84VCyn8bl5fLm
+lI92fXGYN/Qg7Ms/lC2G7D5vFaDFCFQn+zFFUwIDAQABAoIBABNTpiIxbLDhs998
+uvQ3XsumR08kXVcwa/GgvWOSZIEJOUaAYWubDaBTNvTjlhMl6DI+YvKihZMnAkp9
+fXefF1nFUWJJvI0ryi8w6RD54RtbCG4c4raRqysVU7wumZsSenAdc0o09UQE6zXc
+uth/+1VSKCzVjRkLwquXDg0rD3vHfQHWQvQlzwUh3OACA3LfLezVqzrEB02YVRxm
+xwg5veeMg6Aod8vsvsajry9eE0hKeFGonRANerL9lwZxzD2ZjU2fSEJYY3xxKVgi
+o+QVTKaAt9pivOs10YVZDcIDH0xmDpxAkaLb5ZAbnjwhf7WGYgEm8VruAHkJxyXX
+yPf3rpkCgYEA0dp/Xv5KBIkD6JJao8hnhtP5x9U7o/pTzRxaO3WUflvTI6DtC0nk
+cTOwFVs4WljT76T937q2x4stuX0woHzvIaZ6wwZ2vv81ikDY6pE8eLWsH/xFAmkx
+HBfkSijFgJV6EpTqUnFD7QKU89tzWrh/kxaMO1WgFaBhxPPs3K1LDTUCgYEAzVW5
+3yjfVHNgjWTeAbnbToGvUihOidvIvS5cVo5q0Dhfabz0tiXFxAoQUGErUGPC8Nu2
+n/HxTI3b0PbCCwjBsDNqX2kzxTSe5aTGIrBUWbped1bxme8jggXuWYbg8vvLpsYf
+wAJPxnGIxW/v/aftHUhbTIuVfZX2+UnilrwiwWcCgYEAg8paz4eXaH277KVtMwq6
+qZwac/tgNz0Qv/GcYVcYaLq2QNvhwoMnakhxvxfIrkS25PuTTJxwCaVIlAMhNMkB
+TPrGghBfJtgUAb1z/Ow1NAG0FWpS1I7HfsMqZcBxOK2nOmA3QItNg11pujQJn+Ha
+jL9OVj0SCkLs48nk6ToTtjkCgYEAh8YCtNwq6IWuN3CWGCAUMpIwIqxCWof48Zch
+OZ7MZEiSVrG6QmMxpRJefTfzUyHUOj2eQZ7SxqMa0c8IuhEdOeyVjudaczD7TLAq
+z68252oDovfbo8Tr/sL7OzmjryfuHqXtQqKEq5xRKvR8hYavlGhO7otx2uv5thcz
+/CYE+UsCgYAsgewfzbcBqJwsAUs98DK99EU8VqKuaYlU5wNvAVb27O6sVeysSokM
+G1TGIXJPphA3dSAe4Pf/j4ff/eGaS20FAFhs4BPpw0fAeCHpmD0BjIba0lxBS/gY
+dc+JVPKL8Fe4a8fmsI6ndcZQ9qpOdZM5WOD0ldKRc+SsrYKkTmOOJQ==
+-----END RSA PRIVATE KEY-----
+  )";
+
+  *cert_file = JoinPathSegments(dir, "test.cert");
+  *key_file = JoinPathSegments(dir, "test.key");
+
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCert, *cert_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kKey, *key_file));
+  return Status::OK();
+}
+
+Status CreateTestSSLCertWithEncryptedKey(const string& dir,
+                                         string* cert_file,
+                                         string* key_file,
+                                         string* key_password) {
+  const char* kCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIFuTCCA6GgAwIBAgIJAMboiIQH/LDlMA0GCSqGSIb3DQEBCwUAMHMxCzAJBgNV
+BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX
+aWRnaXRzIFB0eSBMdGQxDzANBgNVBAMMBk15TmFtZTEbMBkGCSqGSIb3DQEJARYM
+bXlAZW1haWwuY29tMB4XDTE3MDQxNDE3MzEzOVoXDTI3MDQxMjE3MzEzOVowczEL
+MAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVy
+bmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGTXlOYW1lMRswGQYJKoZIhvcN
+AQkBFgxteUBlbWFpbC5jb20wggIiMA0GCSqGSIb3DQEBAQUAA4ICDwAwggIKAoIC
+AQC+Av808cmgkgtLpH+ERAaJLgpSQ+l2UfUHTB4XeFXcWcxlsRyXqqNTh5NwkjMI
+c6Ei8p12PBol9I2j//l9sCmLXWDXq2EFFkZ+tcszPjQiTmqzPeruAnamYzuQFioZ
+mnNbPypD9qdk/IWY4XWMWOL/qIhnkNQvswSCqu7JA37xaiOqdLBYt/nSN9h5cOwi
+iHQODY15OmrgAB4JO9brHdBp/fzoN3DkHpQ0V5rlEZ+5Ud9qDs3UEQMgo+ZV8wYL
+KVb9/sUyWu+i1NJIAIhNt5oC8AXJJt+C5Bqme3+7mkWnnBo9DwsvnqDOjOY6AvpO
+NHDeRgEBBelj8rGOGQAFgfTlv+w3kDas25oxmoeVXSPF94eU75bu/EE6GGNpe1EB
+ZtfGUSfRLZwBMAeTZ7f1b9xgNygNpBGmwt9bg+qOZ6PYWkGIrP5+Umhjyss26j5r
+PzJSummB93+69QoESLnF68WcFrR7fxN+oVra63kic/wvC3RH+P3lAIaYw9dKGtec
+D3/F2xBp9+Q3nMJQ5MGDdv4wbWQ9lA63uwcWSGIP3R3KKrs4ULtvHIVQz3tgKbwu
+Lw5LM7x3KnV1iMwfJC09I+lv8MxJBS7cxGU7UEyIasIirsZblPTBshjoKPax2RWR
+I/VI9HwdA4cCk+zbvklK3hHgemCagVLIGwT5+tU7tY4UOQIDAQABo1AwTjAdBgNV
+HQ4EFgQU+4nVu5ZsCzs3wxbyS8LNmkL849AwHwYDVR0jBBgwFoAU+4nVu5ZsCzs3
+wxbyS8LNmkL849AwDAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAgEAWHLA
+OCjmeqf6AWpmoOOx3fUaGzaR0SG1Cn85nDMypdTrZtNUn9mkDDDXH40xP31vUyjx
+i2Z68dvHsJCdqYL1KAwvvWIEt8xzIWI9ALCANl7JD1L224lUdXI/SgOVaba5bxf4
+nVhqeGH5G7WB6g4ZoOkiICluCnjc5FWaZlENJ4oQLLS9mQE7sREouCy+lDnlUW0x
+Mf1A5hiERTmvuy26b/lkjXfdGw0z/lNXNAS2k59cKHZ11FqzSLwzK2betmpJYzcq
+N4kPMbfUDrN5x/pjAl//GATQgCiXCUjwGvKnhhXedLjnLUC7bxrAoDwKj986iKnO
+v9wzukBC6t/Ao1COodDISzTLORTMIWLOjyg9bPVKSjdFxmKhpCUQQ3Jt6k9JOZtR
+hvKVmDZBCB10eCJALsHlDWAy0DgjRrD1dnnXrOIUgq6ZLqtzAKGkQF5Y5sYEXyTm
+fCFgiXHtU2haGzp5x+i/vz/E6bBsxJhUVzaWlP149WhQs4RO1YL3Iqsdcy0AcMA9
+FUNW6C+37fVk6w1OJGcI4uTfgMpSJL7iTCSzuspR4lEPUHLIvB3kZKyjr7/eAiMg
+NU9t8oyYtGfrXWXHEZ+d8vK7KCnvMZ2ezNtMC88tC8NtnJ8yBPxBLS7k1f/IrYVL
+OHUKIgiZvAfTg3GSj/iiNespDd665okkzRb0QNQ=
+-----END CERTIFICATE-----
+)";
+  const char* kKey = R"(
+-----BEGIN ENCRYPTED PRIVATE KEY-----
+MIIJljBABgkqhkiG9w0BBQ0wMzAbBgkqhkiG9w0BBQwwDgQIQfgyMe//utcCAggA
+MBQGCCqGSIb3DQMHBAiO+0dos6mhJQSCCVDmzHR2xvhXpuw/CxE8Qs49VndyqC3p
+U5jIukFD79/Cyt6I7uH0TOqj2e+0ucVbYuMrx29xaD6WOef+SXV+Q5WMDdQ/5rYW
+lY5Mgl8QxRYnIppWLk1Hn48289wzqkhhBSspRjWfQfDlSP8c1+FPzNJ8l0bk2N8l
+erfkScbEAtI54e31nBjJb3Z7YEWbttPLD4FraX2bteA2F5Vgn5m5LoEmhmS8KO55
+HIMO6HQVLpwpEX0k5oxi96+Sqh26ZkO72qBDhQ6hvhHSRfxXl73+oURyFcDZeRfF
+CNAZHVBPSN7I76R+vK6HvkIS9nLIm3YQ5DGck9XzjdIaY49srBLT3ZTY4ObPtuay
+MKtnfrfLSGD9VBuGcpJx0e5WqJgb1eVKcfRNRC+mp6vyJMqTVmoC+cjRIDg7As8D
++Mze0V3jYQfZtv6Cl6TZvCWmhtiVqdVnvz37NpExFe5PCqij+tDvAoUHLPltzkgw
++ivXbb08G+VjYVwGTWjvbMhnJ19LvOiKnfrE4MLmjiDvJToBD4uBN1NLjB2wE2Nb
+J33m8OTTJyGPSbXBsb3L53nDhs1rB8y3YXIFU5e1W0IbnX3jU1Z9Bb+/YniGBcyQ
+H1MX4Bz7mZBsU7wMT7z5g5qQfMxG2saRqcNseKOqVA2e9T9hUGmaac0//JHvrWTO
+fJyiAL4HOa24uP6cmht03Ui+tBiEu6hAOlLrDsaXcdEZ4zv7pf784+XotY+J4S9E
+emJLQnhxBFqwIy+6DSvKBhnVohCz5VCC/9ssRmpAhiCObFbUU8Np5nINOp9uzr66
+n2QVEH6eTGccXDmx1K5z/+HlImVcKcpBJvYKdTpW0VxlDxjOp6DwxCiY1uvWytX4
+GQ6qxtnCsA6K+j7UcgAHHOwN/ltkVUMOlFkwebu/AT337jR0tGXOxQLU3GT/nNLq
+i+2L7I8yUpxVLxYshDhBFd4gKiaU5Uy9ADBbv8qAVOhmrCYfCqbdO4GGLQTlVhwA
+0LAqsCWo6aZPPYeoJRCy4SGIW428+qOVx2AkopT6SlCi9mypuvar07JoN0aNBj0C
++9n0/WBQ5BmY1+vpFYyos7JOcopGg1ownF1nZ0IccZhyBgTk0S7E+rYh2ovzzy7K
+1PRh/zo+bWKJmBKhClAgr+104AM0oVCfUdG8G+JY2ckoInA8ticv8a4JMYHnxCRD
+ZJ/5TpBw4YLimgBRLj9iDOf5V9HeIb7weUp+q8NZ2BEjG9ODX4/kcVVxWQPJS8Ig
+u0dBl+T61nq7Tg45PhE0nzyEoGGPL2xC5QayF5/eAhFtGUXpPVAE52AuCLrT5KKB
+ksfiIbqq6gKrK0clNZFgn7TyadGZL6NKdlb3Gk0ZY+F7/E23ayjEJ28GgHo+yLXz
+p8oGMn1F2IuzQH+H1vNhb0iFDDcE6lalq6TOhtGE/sxkll4JXTosAJIJ2gimpNOJ
+18qHpB8cbl/X2haKbURLTKppLqxSJYAwhttJ37oeq5t88u1W481bHZKlOD7OZ1l5
+r7BWFUY7nFBYVmNixWeda/EBuFQth+XZ1hfKO5M2F6F2jcLbkElbST30Fj1aU/pA
+Yq0PBW01sq1EdlxRszjCEtYjFePmXvDpOKW9mqvAKHYNY/U2vUS3go5BYwXOpK3Z
+ijWBuwJLisbFSzxC7eCWu7S8y4W96lENOz+Xf4aD0n3rjYeCVmj4VXsF1Tcl9H2k
+S4p9SP4OC/IMK8TmFWTy3Ilp3la3IJnFAqDezaJeLA32w756QbQ6ziJKgGUZYWhg
+RYM/8ha06LxOLhRA4qvSCEs6NOii3stKB4pQTBcTqYp5jZFnoR9YhpV++rKwyQC7
+8vQO+MZnJ3mBuwoXA01VBI9spmvTC8w7S6z1nIr7mjuItluYZHZOXpwaiL0kB+0O
+Ttv8l/uRlT85Q0vwpqk9tY8uNKdqOidrLHuv2ICHUeTU7AylDzZrWvK0/NNd+Sfx
+2/Mqu6jbuEUJwsFGHB3zPJoT8v9+jHiTi0upv/OEqu/LXfw7/Arhbup4ujyymrpu
+aD9i/5vu042XXzM1IcF4FrXITb36vRvBOfFIgsdIXstvXXxyLYZ8orO1kreVQB1J
+TRbWDOk+/IDQMnYef/p84y+DKHiCVUYIbYQe0x7synYrtlzSjf6SOrMKiFlZWZhl
+Du0YAMRnxHp6CiFVc+Zpt8hFC2GEgeffL+qUgfJyGH1R0hknSWyBN2b/84kcs9wJ
+YIqAzzPz1HSE80MVCbv5lJt1I0PLNam5AwEeTqcntZuu/4LWlbYSFZ/R4kF/qaAb
+EzE6PQeLBNIzNEemQWD7b2XvludB8cAiMa/HNvpjOaWqZZLdDR8VLYb2WzPjWuO/
+gFc9z95G0AZM8aUadt3w/bgoc4QgZX3sZmBiWBtKI6XNnhVD54w04gAirrXYjxF0
+pb79pb2w0IVCAdgm2Ybd8QB83EU2bwAXL4e9infojHpRZMk/j7TKHnaJOpr0xTsQ
+CY+D6+3EkbM+jl3OdtaMORY2fFik2Ujf6DctQG5IR8LB0z7xp2HbedGNI5x2eY5i
+13veBE+U/Z9d6uzs9Vgxk1maKLQXu7+h8IN+jMjVJsZ/WkeERcbbt8nihBS+66er
+JFjFGRejvWEuBdhKl8rSWf8OxNoajZyWcmUgxNvg/6o3xGYdj6Wu8VT/t2juoC4F
+fY2yvlIq7+Zx4q5IuSW+Qm5wROtHvKLvBwDSzAoJ4ai/Yyse3USEOZCv7rS+59Si
+p3/rlnipLAs29R8SYAt8Q2ntSmZOkfWaz+/IV1KeEoH5qUmI/rKDueNjPnKNEmLX
+9Q3oDEvNmsYuAaCN2wvFDDpwkmhjepxUR5FewlCwbHVNfF0KVdlX9XmvVIQXfbF6
+uRlmBalqiXW1xyd9agQnV3+t8Mvuddn+KKEX9nqZ4teloVByfyNE9gAatBUcT33V
+0xGW1CIRzkYykT3HSbf/irfzXHU090M//P8bX6z430Jrg3DxiVUQZiAo2NoVetFH
+x/h68BZ6/j8O0F36V2W+0sE+qN8Wrh1lDxBykzexJAWS063xe5oVbINbZplsO5tK
+ghdr7RWGVApmwaXBhKIxtapSQUMLFhBKEGLL4iDw+jwPAlcBQl3lw+xEOu/No+aE
+XPwFoojhSS9XuE3M8Sc9gKvLcZbdKAegMnYz+py5OwdJ8RiaoaetCTJist08FUg2
+TOQYXv+dMtOkYg==
+-----END ENCRYPTED PRIVATE KEY-----
+)";
+  const char* kKeyPassword = "test";
+
+  *cert_file = JoinPathSegments(dir, "test.cert");
+  *key_file = JoinPathSegments(dir, "test.key");
+  *key_password = kKeyPassword;
+
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCert, *cert_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kKey, *key_file));
+  return Status::OK();
+}
+
+//
+// These certificates were generated by following the steps outlined in this tutorial
+// for creating the Root CA, Intermediate CA and end-user cert:
+// https://raymii.org/s/tutorials/ \
+// OpenSSL_command_line_Root_and_Intermediate_CA_including_OCSP_CRL%20and_revocation.html
+//
+// The parts relating to the OSCP and CRL were omitted.
+//
+// | serverCert TRUSTS intermediateCA TRUSTS rootCA |
+//
+// The 'cert_file' here contains the serverCert and intermediateCA.
+// The 'ca_cert_file' contains the rootCA and the same intermediateCA.
+// This was added to test KUDU-2091 and KUDU-2220.
+Status CreateTestSSLCertSignedByChain(const string& dir,
+                                      string* cert_file,
+                                      string* key_file,
+                                      string* ca_cert_file) {
+  const char* kCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIFizCCA3OgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwUTEXMBUGA1UEAwwOSW50
+ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNBMQswCQYDVQQGEwJVUzENMAsGA1UECgwE
+QWNtZTENMAsGA1UECwwES3VkdTAeFw0xNzA4MTEyMTM4MDZaFw00NDEyMjYyMTM4
+MDZaMEwxEjAQBgNVBAMMCWxvY2FsaG9zdDELMAkGA1UECAwCQ0ExCzAJBgNVBAYT
+AlVTMQ0wCwYDVQQKDARBY21lMQ0wCwYDVQQLDARLdWR1MIICIjANBgkqhkiG9w0B
+AQEFAAOCAg8AMIICCgKCAgEAqevNYH73n4kARZtMsHRucdKmqVd/xxztMlK5VOor
+ERUBhKVVOw3kpDrN9z80ldIkpOrtrfE7Ame/nA9v4k6P3minPEm1qCA/kvaAodtT
+4HjAkrPc+fto6VO6+aUV6l+ckAV/79lOuc7AutNlvvPtBQQcgOKvlNUSRKwM7ndy
+dO4ZAa+uP9Wtsd0gl8b5F3P8vwevD3a0+iDvwSd3pi2s/BeVgRwvOxJzud8ipZ/A
+ZmZN8Df9nHw5lsqLdNnqHXjTVCNXLnYXQC4gKU56fzyZL595liuefyQxiGY+dCCn
+CpqlSsHboJVC/F3OaQi3xVRTB5l2Nwb149EIadwCF0OulZCuYljJ5y9H2bECXEjP
+e5aOdz9d8W3/T7p9vBKWctToeCpqKXUd+8RPudh0D0sUHuwQ4u4S1K6X+eK+gGhT
+HOnPwt+P8ytG0M463z5Gh9feW9ZDIYoiFckheFBAHxsgDWhjYpFmYireLLXMbyaM
+s5v/AxPNRAsx3vAAd0M0vGOpdgEJ9V1MsKmxkPO/tDC3zmnv6uJhtJfrOAKxwiGC
+fDe4IoSC6H5fTxeAgw6BG5onS1UPLADL8NA/M1y8qiSCZS/5S0cHoJp5AxDfZSSR
+O49ispjqcONRwckcRJ5Pbl0IA+wGyg2DuI9LaqS5kKWp5AE8VCLPz7yepDkRnnjO
+3m8CAwEAAaNyMHAwDAYDVR0TAQH/BAIwADAdBgNVHQ4EFgQUZBZLZZaUfyIK/8B7
+GIIWDqeEvDgwHwYDVR0jBBgwFoAU8KctfaqAq0887CHqDsIC0Rkg7oQwCwYDVR0P
+BAQDAgWgMBMGA1UdJQQMMAoGCCsGAQUFBwMBMA0GCSqGSIb3DQEBBQUAA4ICAQA3
+XJXk9CbzdZUQugPI43LY88g+WjbTJfc/KtPSkHN3GjBBh8C0He7A2tp6Xj/LELmx
+crq62FzcFBnq8/iSdFITaYWRo0V/mXlpv2cpPebtwqbARCXUHGvF4/dGk/kw7uK/
+ohZJbeNySuQmQ5SQyfTdVA30Z0OSZ4jp24jC8uME7L8XOcFDgCRw01QNOISpi/5J
+BqeuFihmu/odYMHiEJdCXqe+4qIFfTh0mbgQ57l/geZm0K8uCEiOdTzSMoO8YdO2
+tm6EGNnc4yrVywjIHIvSy6YtNzd4ZM1a1CkEfPvGwe/wI1DI/zl3aJ721kcMPken
+rgEA4xXTPh6gZNMELIGZfu/mOTCFObe8rrh4QSaW4L+xa/VrLEnQRxuXAYGnmDWF
+e79aA+uXdS4+3OysNgEf4qDBt/ZquS/31DBdfJ59VfXWxp2yxMcGhcfiOdnx2Jy5
+KO8wdpXJA/7uwTJzsjLrIgfZnserOiBwE4luaHhDmKDGNVQvhkMq5tdtMdzuwn3/
+n6P1UwbFPiRGIzEAo0SSC1PRT8phv+5y0B1+gcj/peFymZVE+gRcrv9irVQqUpAY
+Lo9xrClAJ2xx4Ouz1GprKPoHdVyqtgcLXN4Oyi8Tehu96Zf6GytSEfTXsbQp+GgR
+TGRhKnDySjPhLp/uObfVwioyuAyA5mVCwjsZ/cvUUA==
+-----END CERTIFICATE-----
+-----BEGIN CERTIFICATE-----
+MIIHmDCCA4CgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwVjELMAkGA1UEBhMCVVMx
+CzAJBgNVBAgMAkNBMQswCQYDVQQHDAJTRjENMAsGA1UECgwEQWNtZTENMAsGA1UE
+CwwES3VkdTEPMA0GA1UEAwwGUk9PVENBMB4XDTE3MDgxMTIxMzUzNVoXDTQ0MTIy
+NzIxMzUzNVowUTEXMBUGA1UEAwwOSW50ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNB
+MQswCQYDVQQGEwJVUzENMAsGA1UECgwEQWNtZTENMAsGA1UECwwES3VkdTCCAiIw
+DQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM1X35LT/eBWBt0Uqqh3DSUyY3K8
+HLIlX3ZXg2Nx6y8yqhw5UGVFZl0uYBDo2DSlTl4sey+AxLIbpQI9ArRA+xqmFynV
+jheB9otudnA8hVwi/e9o+m+VSjG+HPRjSS5hwdPgpJG8DCPSmGyUUFtf3v0NxkUq
+Is+fB5qhQ36aQkI+MwQsSlHR+YrrKKVnE3f911wr9OScQP5KHjrZLQex8OmpWD9G
+v4P9jfVSUwmNEXXjmXDhNG/1R4ofX6HogZR6lBmRNGbcjjWRZQmPrOe9YcdkMLD0
+CdaUyKikqqW6Ilxs7scfuCGqwBWqh66tY18MBMHnt0bL26atTPduKYqulJ1pijio
+DUrzqtAzm7PirqPZ4aOJ9PNjdQs9zH3Zad3pcjfjpdKj4a/asX0st631J5jE6MLB
+LcbAerb/Csr/+tD0TOxwWlA+p/6wPb8ECflQLkvDDEY5BrRGdqYDpEOdm1F9DWQh
+y0RB8rWJMkxC/tTqYHfeaphzCxndLRsZQKVcPiqWCT7b431umIjPaDhsykNlcU3N
+f0V7V/fLY6wwuACngS0BLQuMrXy5FyhmWnUBeWwHfAeTxCkHlF+cVT6wHmeOuGbC
+c1piq7O7puKdC3UjO7Nn+WoOb2B6Qm/dajHpj5myxYJa5tGQGeUnWPwjjMQR557k
+HzugGAzkuG1ASQrhAgMBAAGjdTBzMA8GA1UdEwEB/wQFMAMBAf8wHQYDVR0OBBYE
+FPCnLX2qgKtPPOwh6g7CAtEZIO6EMB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeU
+HW46abboMAsGA1UdDwQEAwIBpjATBgNVHSUEDDAKBggrBgEFBQcDATANBgkqhkiG
+9w0BAQUFAAOCBAEAIaD2yzjTFdn61A4Qi+ek3fBJaDNQZytd0rHb49v3T+mdj/MI
+yShI1qezDFkg2FP1LfNgjuQl/T+g0BloXatAhZ/dj20Y8oN6bmilV+r2YLJbvbTn
+3hI+MxNf3Ue3FmIrwKK3QdkWcDBURpyYaDO71oxPl9QNfdhWCGHB/oWKU2y4Qt/O
+aPy+CmBAZEclX+hsdUBDJG5vuujpv4myCFwpLgFKNQX3XqCPLc4SRjfyla2YmeZv
+j7KKYh8XOWBbBF0BnWD94WzUDIBmFlUfS32aJTvd7tVaWXwH8rGwDfLN8i05UD9G
+zc3uuFH+UdzWVymk/4svKIPlB2nw9vPV8hvRRah0yFN3EQqAF0vQtwVJF/VwtZdg
+ahH0DykYTf7cKtFXE40xB7YgwDLXd3UiXfo3USW28uKqsrO52xYuUTBn+xkilds1
+tNKwtpXFWP2PUk92ficxoqi1cJnHxIIt5HKskFPgfIpzkpR8IM/vsom1a5fn4TT1
+aJbO5FsZTXQMxFLYWiSOMhTZMp3iNduxMYPosngjjKPEIkTQHKkedpF+CAGIMOKE
+BVa0vHyF34laKMMDT8d9yxwBJLqjlBohNsLLZa/Y90ThaMw+QYn/GZATB+7ng+ip
+VdGAQrghsGSxP+47HZ6WgBrlRdUWN1d1tlN2NBMHLucpbra5THGzl5MlaSVBYZb6
+yXI+2lwcTnnEkKv2zoA4ZHWdtLn/b1y4NKNg205TA+sOZcl6B1BgMe/rFuXdZe9Q
+/b6Tjz65qL4y1ByBVBJNhQQairw6cypHzwzC3w6ub1ZXtFqnTlU8fFcHGeOyydYS
+NfoepF0w2v0ounqD+6rN1CH/ERVb4FCEN19HQ3z+rAj19z2h6m/l5QEKI7bz8ghD
+8yxyqJz+L9XpfOo1yZfHQJckilY6BBIGWyeetJBmvkwv2WPt+3pX1u7h5LkvNRj2
+3fItf486zqtzUi+i/E//rS4gD/rRr4a85U8GSfp3LSAbtmfC0LNYUYA9Dcc0LSpl
+9alNuEpBHSHXlCVh4bcOb0L9n5XNdMcUYBo14hQdP0K1G7TounuAXFKYIQeyNyoi
+OAZ+eb7Y2xNnkY/ps/kyhsZgOJyiDZhdcruK3FIUGYlg5aVjQTB8H0c3/5SZnSky
+6779yMKztFXj9ctYU0YyJXWdF0xP/vi1gjQx/hJnDfXFfIOmeJdQSC08BGyK/PeC
+8zAS380bgzOza/eBL6IK0RqytbWgdoLrUQQfa1+f7AQxDDdoOkUenM0HSWjKfCuG
+m1/N7KUDHtnjVIHWqRefTPg1/tQjVY8/zgxN8MyAy+D95y4rawjsJf1dL6c0+zGv
+Wd40Cr+wAdHKN6t/oransoxu0EZ3HcSOI1umFg==
+-----END CERTIFICATE-----
+)";
+  const char* kKey = R"(
+-----BEGIN RSA PRIVATE KEY-----
+MIIJKAIBAAKCAgEAqevNYH73n4kARZtMsHRucdKmqVd/xxztMlK5VOorERUBhKVV
+Ow3kpDrN9z80ldIkpOrtrfE7Ame/nA9v4k6P3minPEm1qCA/kvaAodtT4HjAkrPc
++fto6VO6+aUV6l+ckAV/79lOuc7AutNlvvPtBQQcgOKvlNUSRKwM7ndydO4ZAa+u
+P9Wtsd0gl8b5F3P8vwevD3a0+iDvwSd3pi2s/BeVgRwvOxJzud8ipZ/AZmZN8Df9
+nHw5lsqLdNnqHXjTVCNXLnYXQC4gKU56fzyZL595liuefyQxiGY+dCCnCpqlSsHb
+oJVC/F3OaQi3xVRTB5l2Nwb149EIadwCF0OulZCuYljJ5y9H2bECXEjPe5aOdz9d
+8W3/T7p9vBKWctToeCpqKXUd+8RPudh0D0sUHuwQ4u4S1K6X+eK+gGhTHOnPwt+P
+8ytG0M463z5Gh9feW9ZDIYoiFckheFBAHxsgDWhjYpFmYireLLXMbyaMs5v/AxPN
+RAsx3vAAd0M0vGOpdgEJ9V1MsKmxkPO/tDC3zmnv6uJhtJfrOAKxwiGCfDe4IoSC
+6H5fTxeAgw6BG5onS1UPLADL8NA/M1y8qiSCZS/5S0cHoJp5AxDfZSSRO49ispjq
+cONRwckcRJ5Pbl0IA+wGyg2DuI9LaqS5kKWp5AE8VCLPz7yepDkRnnjO3m8CAwEA
+AQKCAgAE3oL2Hu1Nnwlo9ThPXibEEDtCYwWAWS3a4U/6RPOS+70dZfd5R76jjiPU
+z/TbzjfKmgjRkTYVrY9qE28rVwD8aJdSPPJ9rN7lgTbSbIyMxCkQiyLr7u5ksUeM
+W9Sy8KZ14hJ2dw2weWJAeEpUHH1QRXvjnZtWcnyhhySfuMCI5UHGMJiXr7HYhPOo
+JcWBjItTlg7ILKim+kakjFL7aheo6awZFQutb6vtSZ2ejWNgC9Jz7cbQsyabUZaJ
+dK0mxw2XPaQD6tJjvm6hgGQ2PTBOkw1S5lEWZ50bwYJMpZrjzOarq751bZGL1cxS
+ajOJ7g6rCxS+Iu7s5lKNZgaRUBkymATYccoigfZDR//fHKAmdgjgPstqy1NJL+uX
+bIuNE0bR+mBM2JQzNjPIcE67PG+0aQdO4dj0TnTzkTP1JSsa6Tz4ckOUgt7IBK8j
+ilCQpHgOB900hXC6xVRnAaU/uuSYEtfi2eFBKHT02OqH51yNZ2jsgOQJCvxNrrdI
+OmA0AaZ2tVXTTXe6qebaNjjp2cziiO5ma+5mdI1vmLQAA9v0micO+eKp/pi5e0r6
+j+9uyR2Oo4qnHg04TIfDyglW3uKz1eI0RPfBN/dx3WLpydxKeywXPH6EzfBFk8pz
+ST2sy+1ZN4vn0bDSTjSLsLBW+xBKYINqKkBD2Kg6B7aeNINjEQKCAQEA4BQByyZV
+9va91T+rQiNPifuI4PKgmLTl0wxM1Jg/H4YCyLtuEkmuvwfeAjaUaUuk2MDs3xfQ
+4pu8vtAnDapq5vJ/lMg0m3+NIwoks+xee9F//M4du9j67USvX5Qw/6Cnx1zAvrWR
+TyZncKUocQowcXM9VU0xcv4qVCMaH5F5Q/1VvG7uAtGCnB8TOHzV7GUaj9xxSqDc
+f3+p9BzsecpPZsdpVi01dQbCKi9neZwFv1Vr3MvieNDOGqSGn8X5EjSHY5PzCaXL
+S+/HoFWOSzWcuNdzKJRjVkC8U+eHoEabaRnD47dfJntN3qOQ6Mwf6P2jMN9GqlQu
+DQlvpMxBwQT1/QKCAQEAwiC4hr6EZKaLmeZBLggsS+oifHReXQy6tf2n7kCAwAgL
++65K8UW+YUtQyHL6UFfD27vvW5yp6LZxMRUD3ghguHPMQcejgoQTfGmth1bCb9tx
+zqfxuWUoNauqZiP4/kgxoh815Kt+kC8BRCXNIWjH38zxY+ncv0b4oKP7lYna/7Xk
+URLmMFr92QVAydRxY9kQTHQTCd+ZQrFT97xEoosgzkKhmynSfWNx7ymYmCrHzscA
+TutpD26u4CA4Wh4ZdVPEF10lGR531SAFEqXCkaUvIfwPphPmOtum2LZdEYho9C76
+71kLzzoJOJUNo2L9ORd5I4tOrMT0tmN+MpS1cPXb2wKCAQAu3aBeZ9807vhXQKDG
+DXKWTmibe8OBDNzAnmL3V/xj0HiGmUT1SDnnNHMHjXjO6QZKW1dvdaC3tJDua8Sv
+RARl1zQ93v25xBy1xmpUw0wjo3acXlOztTcOJv5zBCCXZneQ5+JcQMdqgYLC+ZgS
+xGnLYKnkTGfaQDSEMm9FSPzO7o5fAeh/6Gfj1VAE0X9AmQjMK/P6Atj7Ra07JE2F
+T3355h0u6/exST+U6SNAORSupuQPYwkz8aAZzG1nv1VPrHLgrdH4I5f4gucCrsI7
+ErR7qHwqcZaxNIrvFY61Q+8/NSdWWkTpXIK13Qny1raZ2WqnTxuNhlu3WFDka+AY
+ybvVAoIBACOaxL1R7A5ZzXjolkPPE/DIfJK+9R+z2frPDyHPis2trCT5Dp254PUP
+Tz20eNyLfEys53WyAifAbnpGFHOArdymwGvAJekmODy1VTJhY0AIy5LPkrIiL4HI
+fnRFXMGmlBPcDZJnMctYE69gD4N1KFOPzyY4Gliqt6ce7GG86wHDZqDICpgL2EsZ
+f4yE/lcF1Mtw7pz8+asVwwTI7v2w7s9lwSYoQYbl2lu3EVm3XvY54YCYBKjj8AcD
+YdKFer3eIzT1zHwS7n+UY9nLtSfpV/+vr18Sf0OETdGpgOBaWIWQqE2F03iqeE58
+aAfze+YgvAMc5c0iQo/BJ8A3LiANt8kCggEBAK8cFEBm+Z1g5s1IaWxqTIylR5XF
+5RFty9HyUXtkAd2d1qVzBaBG3SXg4cRsW0EhcUV2XP3iTFIPXEEABRRu5U6DEal6
+wQclrhfP4hiRQHp2Ny6jDj70NCSeUmyEu2lmwEJJYsDSOCnVmtt+qlgmk4yI1Dua
+nXhLcPLqopuhEZs2V/l2Q6E5i4vrs71Y7of+vsAvvt42Vx5wsGdPQihc5E1MI7GB
+hxmQys1MwG3Jyd7Zk88MVNveASeEIc7UAmr/TGL+RIv4bxNi/1HrgekBf1jnFUU3
+4fsdqKy0W+rOgOlGN8TX7NYCz3B41UEiyf+/gZ/TcLKAyGnoAO727Ngayqo=
+-----END RSA PRIVATE KEY-----
+)";
+  const char* kCaChainCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIJfzCCBWegAwIBAgIJAOquFl/JjmRLMA0GCSqGSIb3DQEBCwUAMFYxCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTELMAkGA1UEBwwCU0YxDTALBgNVBAoMBEFjbWUx
+DTALBgNVBAsMBEt1ZHUxDzANBgNVBAMMBlJPT1RDQTAeFw0xNzA4MTEyMTMyMTla
+Fw00NDEyMjcyMTMyMTlaMFYxCzAJBgNVBAYTAlVTMQswCQYDVQQIDAJDQTELMAkG
+A1UEBwwCU0YxDTALBgNVBAoMBEFjbWUxDTALBgNVBAsMBEt1ZHUxDzANBgNVBAMM
+BlJPT1RDQTCCBCIwDQYJKoZIhvcNAQEBBQADggQPADCCBAoCggQBAOHxuOsGDBJt
+quCb+lrJ3uXvvBv6f1w1aP+WqDEqveQlOi68sV/DVUR3s+en/MHA5jYAVo2D+eR7
+v/zjrAzCeqCpipbDcxA2e00+kggGHc1BoLtXXTPPCcTQt/0jjX26GXlaJRrY5MAy
+ZJ35vkJ5wCTw7DttfyRzR/RplI6DfO3t2kkSFpSsjGFJZQRZn/L2OM8Ii/tEhede
+UP/Rv8KIKM2+P9RS0VIutiI+/mOpH0QZHHEgnHy7x/CcNCd+wDG516YoJXp9c/1n
+aRLK+jA0bNCf0ZktMpuifoFzpNJ3pvDkjgTLhbiMkS8VKc66Z/Mv0EVOrdiMla/X
+0OSWqEZctxIcVIGDbMqngy62dghMBmxpVkfNmu6RqyS3HmPFrhRXJIIogdBo8mdJ
+xFCCvOgA6suaZnQtQC0mlRi5XGnTocpeHYUZ1c1hO2ZdVrFTh3atJsD80kVYxYuK
+YMq3QaK2zZUK6TUIFue1UqLf2dpIFzskLY6bEVob7Rdl8AHdFBJ8cGOyYKpG+rwO
+n3XQudt8YwDUCvw+8MGRXQiwUnzT/3gSuLNjlQOdcqN78wT5mdp6QZwareptyRwT
+yk/wWnfZlcFO33aPnUhvzzI5TzTB6EqG+3oNYkuXXy/glvOFluyQcPfsYXVOnXOj
+xF0hjKcpx10KQSvXjT9SRYr8NcOC7Yjy3f+WF+nwV+EzevqC2iTr1u8ymqUvpgFJ
+snvO8G/tycfxrwjI/4IghBgwqhcqD4wp/NleXy3A7GE3kFusL10i1bjwxBlz5qER
+uKaxU164HXPl4gv+Qt3eGqJE8KHDwTp8x+619S0+Gd8fY6Yj6/v9WyDef0SKGscm
+t3iqYNA39yNHAj++cjcCrJwBfINVvnTsVFKsCwUpjVuNOGRfZv0uHLAv6LaePQk5
+FKHwlLlPRx7ZcwHpkzTvp/ixYPb/cNJOw8fVW5CoWXYEzDUJY0oU8BWlQNHQ/e4q
+V7Yxa/vourUUvOACDzyQ6hCO95dQdDMCDQqC2VVL45+TUJ3eU1gDHge4T2js/qL8
+iJ+auZapiZjUQzLFse4XkgDrkMrD4fkOQOw4x9AhJF/SrnI8UPNjNOmAyOlqGTdd
+IyLesKXgnOGASSmc7JRk+YBpu9PQXIgHHQZIao1zPGP5k/ylp8XPYitC9MKzRRam
+67aJmutJxEtw7VJhoyz5m5LhLysPLY+R01+QqZK9/7qwWaX6PvMmm42zq9YKncOM
+cC/4eAPnwbj6yhpFoaUD5qzloci3+tvYgb+wr3f1F9SPrpe5xJz3NTXdQj8FsGjl
+ShL+bybUQ7wzZQObUhWtXSayagQg1MAxUCn7Aglmo/A/1+teacyuCEIbrhmODYM3
+Okji9gmGv+cCAwEAAaNQME4wHQYDVR0OBBYEFE/9XKaDey5kC8f3bCeUHW46abbo
+MB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeUHW46abboMAwGA1UdEwQFMAMBAf8w
+DQYJKoZIhvcNAQELBQADggQBAMXuMpJzYV3QMH780os0dZyJ+zi4FETVjAWFAPME
+qzN27W0L9+HZcGpz5i5FLdmc0F3u1cyCrJ4nCCWMrEIVmrLInFRaH1u9HUO78jPw
+Uw/MZLF+sf7uE8IAdVzLQG0A3QjAVoOX3IEOxEaQYYlVQHonyq2pBt8mkGqiPI3R
+E9cTnP/R1Ncd1wZkFL/n5qSNGTr/eg1O/jFB5xH96xm18Z7HgJDku2JCKQK6kqTM
+B7LjAFwWzTg8cnewVFRzIvJe83w9lHs1SW3R9fz7fIEBbZQ3z+n1cSj5jDjaT1+U
+YWTj+gAklZT4M/vImXF0XqbZytUOqe16HfBInc0G/kakUIcs6le2hmfhccJdG25I
+e5TH6ZdMumt7//hVPBPN5fhYKc2uHpzbtmxUjuKG8Na8/w+y2O+sW5CfpNtrYLyB
+WATHGtBB3ckBAICLlhoQiY/ku9r6BfczI86MbSy5vG5CD2sYGhVEl3PQXAnvBKax
+svZS3z9f16SZm61FWwz+r0XCe7LBiKe9YpODyE8lFDymZyW0vKYzDLzCy/mXhU/j
+locrf5r4YK0cOxNQC/jK7MLDFxPQbYg2SuAPW4DF2QzgKn2RuatdOB12S05afawj
+bhrbikIfEtD3erUMMJwaV9dxhHL835rfexxbRzBCdbjWg7Qiw4r8+PJB/mSSvsVO
+RAI02p8IqW5O+iXkU4V2Mapzdpo6b8O6TplHRXbRxWuen87g87KHhgZaC6TmWgvT
+It3ARZx3tkBoJwf41ELmWcakqiT9aQslc5weafw3SZp6+w0QU0qqFwCFLJWHETa5
+/PVHDEkBoXDMnqMlu7E9PUks4Op9T2f7bNy94GZXRbSp2VKjV68sds739DhVIZ+M
+MIaEutz3UndEuGGlcVuqXlda+H5xp57RnMZSKbT240kGdci51WahhfkX7dLY6c/b
+jaNWyGSfM+wFlky97t7ANbPP85SDgrrSyb6rTIt1zU2c5+vvjNVvDhlS6n7ls/Pi
+lMWWs5Ka66E8oZFwYygfIiEv6FcNWrSZ/vCMuS02WJovsZd4YrYtNbpkx6shaA5t
+BOIpuelPbQNPlOaJ+YnRuuppomPnXx5X3RlHld7xsExHDNsi57H0PBDq/W5O1S4t
+rHm3SItJQ4ndFHBGjZ7kSOyHtCLWZ8cB75sPduVC2PnRL/kt3tmfFFVsUurLGz4n
+wgCg1OuflNcc9wIF8lZMjm0TZkQMGYBIfBA7x8/Vs2XSFuaT9vbWoC07LXftl13g
+HsMg1UUSqnMBUQStG42lbVFF1yIfPZThEPxD2RJTCw8FTLBmNrJyBsZ0BGagwe1C
+KH5H1VGmllMdZDHOamHHKA8mEDI4eAKY3HoOS4rfioT8Tks=
+-----END CERTIFICATE-----
+-----BEGIN CERTIFICATE-----
+MIIHmDCCA4CgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwVjELMAkGA1UEBhMCVVMx
+CzAJBgNVBAgMAkNBMQswCQYDVQQHDAJTRjENMAsGA1UECgwEQWNtZTENMAsGA1UE
+CwwES3VkdTEPMA0GA1UEAwwGUk9PVENBMB4XDTE3MDgxMTIxMzUzNVoXDTQ0MTIy
+NzIxMzUzNVowUTEXMBUGA1UEAwwOSW50ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNB
+MQswCQYDVQQGEwJVUzENMAsGA1UECgwEQWNtZTENMAsGA1UECwwES3VkdTCCAiIw
+DQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM1X35LT/eBWBt0Uqqh3DSUyY3K8
+HLIlX3ZXg2Nx6y8yqhw5UGVFZl0uYBDo2DSlTl4sey+AxLIbpQI9ArRA+xqmFynV
+jheB9otudnA8hVwi/e9o+m+VSjG+HPRjSS5hwdPgpJG8DCPSmGyUUFtf3v0NxkUq
+Is+fB5qhQ36aQkI+MwQsSlHR+YrrKKVnE3f911wr9OScQP5KHjrZLQex8OmpWD9G
+v4P9jfVSUwmNEXXjmXDhNG/1R4ofX6HogZR6lBmRNGbcjjWRZQmPrOe9YcdkMLD0
+CdaUyKikqqW6Ilxs7scfuCGqwBWqh66tY18MBMHnt0bL26atTPduKYqulJ1pijio
+DUrzqtAzm7PirqPZ4aOJ9PNjdQs9zH3Zad3pcjfjpdKj4a/asX0st631J5jE6MLB
+LcbAerb/Csr/+tD0TOxwWlA+p/6wPb8ECflQLkvDDEY5BrRGdqYDpEOdm1F9DWQh
+y0RB8rWJMkxC/tTqYHfeaphzCxndLRsZQKVcPiqWCT7b431umIjPaDhsykNlcU3N
+f0V7V/fLY6wwuACngS0BLQuMrXy5FyhmWnUBeWwHfAeTxCkHlF+cVT6wHmeOuGbC
+c1piq7O7puKdC3UjO7Nn+WoOb2B6Qm/dajHpj5myxYJa5tGQGeUnWPwjjMQR557k
+HzugGAzkuG1ASQrhAgMBAAGjdTBzMA8GA1UdEwEB/wQFMAMBAf8wHQYDVR0OBBYE
+FPCnLX2qgKtPPOwh6g7CAtEZIO6EMB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeU
+HW46abboMAsGA1UdDwQEAwIBpjATBgNVHSUEDDAKBggrBgEFBQcDATANBgkqhkiG
+9w0BAQUFAAOCBAEAIaD2yzjTFdn61A4Qi+ek3fBJaDNQZytd0rHb49v3T+mdj/MI
+yShI1qezDFkg2FP1LfNgjuQl/T+g0BloXatAhZ/dj20Y8oN6bmilV+r2YLJbvbTn
+3hI+MxNf3Ue3FmIrwKK3QdkWcDBURpyYaDO71oxPl9QNfdhWCGHB/oWKU2y4Qt/O
+aPy+CmBAZEclX+hsdUBDJG5vuujpv4myCFwpLgFKNQX3XqCPLc4SRjfyla2YmeZv
+j7KKYh8XOWBbBF0BnWD94WzUDIBmFlUfS32aJTvd7tVaWXwH8rGwDfLN8i05UD9G
+zc3uuFH+UdzWVymk/4svKIPlB2nw9vPV8hvRRah0yFN3EQqAF0vQtwVJF/VwtZdg
+ahH0DykYTf7cKtFXE40xB7YgwDLXd3UiXfo3USW28uKqsrO52xYuUTBn+xkilds1
+tNKwtpXFWP2PUk92ficxoqi1cJnHxIIt5HKskFPgfIpzkpR8IM/vsom1a5fn4TT1
+aJbO5FsZTXQMxFLYWiSOMhTZMp3iNduxMYPosngjjKPEIkTQHKkedpF+CAGIMOKE
+BVa0vHyF34laKMMDT8d9yxwBJLqjlBohNsLLZa/Y90ThaMw+QYn/GZATB+7ng+ip
+VdGAQrghsGSxP+47HZ6WgBrlRdUWN1d1tlN2NBMHLucpbra5THGzl5MlaSVBYZb6
+yXI+2lwcTnnEkKv2zoA4ZHWdtLn/b1y4NKNg205TA+sOZcl6B1BgMe/rFuXdZe9Q
+/b6Tjz65qL4y1ByBVBJNhQQairw6cypHzwzC3w6ub1ZXtFqnTlU8fFcHGeOyydYS
+NfoepF0w2v0ounqD+6rN1CH/ERVb4FCEN19HQ3z+rAj19z2h6m/l5QEKI7bz8ghD
+8yxyqJz+L9XpfOo1yZfHQJckilY6BBIGWyeetJBmvkwv2WPt+3pX1u7h5LkvNRj2
+3fItf486zqtzUi+i/E//rS4gD/rRr4a85U8GSfp3LSAbtmfC0LNYUYA9Dcc0LSpl
+9alNuEpBHSHXlCVh4bcOb0L9n5XNdMcUYBo14hQdP0K1G7TounuAXFKYIQeyNyoi
+OAZ+eb7Y2xNnkY/ps/kyhsZgOJyiDZhdcruK3FIUGYlg5aVjQTB8H0c3/5SZnSky
+6779yMKztFXj9ctYU0YyJXWdF0xP/vi1gjQx/hJnDfXFfIOmeJdQSC08BGyK/PeC
+8zAS380bgzOza/eBL6IK0RqytbWgdoLrUQQfa1+f7AQxDDdoOkUenM0HSWjKfCuG
+m1/N7KUDHtnjVIHWqRefTPg1/tQjVY8/zgxN8MyAy+D95y4rawjsJf1dL6c0+zGv
+Wd40Cr+wAdHKN6t/oransoxu0EZ3HcSOI1umFg==
+-----END CERTIFICATE-----
+)";
+
+  *cert_file = JoinPathSegments(dir, "test.cert");
+  *key_file = JoinPathSegments(dir, "test.key");
+  *ca_cert_file = JoinPathSegments(dir, "testchainca.cert");
+
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCert, *cert_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kKey, *key_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCaChainCert, *ca_cert_file));
+  return Status::OK();
+}
+
+//
+// These certificates were generated by following the steps outlined in this tutorial
+// for creating the Root CA, Intermediate CA and end-user cert:
+// https://raymii.org/s/tutorials/ \
+// OpenSSL_command_line_Root_and_Intermediate_CA_including_OCSP_CRL%20and_revocation.html
+//
+// The parts relating to the OSCP and CRL were omitted.
+//
+// | serverCert TRUSTS intermediateCA TRUSTS rootCA |
+//
+// The 'cert_file' here contains the serverCert and intermediateCA.
+// The 'ca_cert_file' contains only the rootCA.
+// This was added to test KUDU-2041.
+Status CreateTestSSLCertWithChainSignedByRoot(const string& dir,
+                                              string* cert_file,
+                                              string* key_file,
+                                              string* ca_cert_file) {
+  const char* kCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIFizCCA3OgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwUTEXMBUGA1UEAwwOSW50
+ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNBMQswCQYDVQQGEwJVUzENMAsGA1UECgwE
+QWNtZTENMAsGA1UECwwES3VkdTAeFw0xNzA4MTEyMTM4MDZaFw00NDEyMjYyMTM4
+MDZaMEwxEjAQBgNVBAMMCWxvY2FsaG9zdDELMAkGA1UECAwCQ0ExCzAJBgNVBAYT
+AlVTMQ0wCwYDVQQKDARBY21lMQ0wCwYDVQQLDARLdWR1MIICIjANBgkqhkiG9w0B
+AQEFAAOCAg8AMIICCgKCAgEAqevNYH73n4kARZtMsHRucdKmqVd/xxztMlK5VOor
+ERUBhKVVOw3kpDrN9z80ldIkpOrtrfE7Ame/nA9v4k6P3minPEm1qCA/kvaAodtT
+4HjAkrPc+fto6VO6+aUV6l+ckAV/79lOuc7AutNlvvPtBQQcgOKvlNUSRKwM7ndy
+dO4ZAa+uP9Wtsd0gl8b5F3P8vwevD3a0+iDvwSd3pi2s/BeVgRwvOxJzud8ipZ/A
+ZmZN8Df9nHw5lsqLdNnqHXjTVCNXLnYXQC4gKU56fzyZL595liuefyQxiGY+dCCn
+CpqlSsHboJVC/F3OaQi3xVRTB5l2Nwb149EIadwCF0OulZCuYljJ5y9H2bECXEjP
+e5aOdz9d8W3/T7p9vBKWctToeCpqKXUd+8RPudh0D0sUHuwQ4u4S1K6X+eK+gGhT
+HOnPwt+P8ytG0M463z5Gh9feW9ZDIYoiFckheFBAHxsgDWhjYpFmYireLLXMbyaM
+s5v/AxPNRAsx3vAAd0M0vGOpdgEJ9V1MsKmxkPO/tDC3zmnv6uJhtJfrOAKxwiGC
+fDe4IoSC6H5fTxeAgw6BG5onS1UPLADL8NA/M1y8qiSCZS/5S0cHoJp5AxDfZSSR
+O49ispjqcONRwckcRJ5Pbl0IA+wGyg2DuI9LaqS5kKWp5AE8VCLPz7yepDkRnnjO
+3m8CAwEAAaNyMHAwDAYDVR0TAQH/BAIwADAdBgNVHQ4EFgQUZBZLZZaUfyIK/8B7
+GIIWDqeEvDgwHwYDVR0jBBgwFoAU8KctfaqAq0887CHqDsIC0Rkg7oQwCwYDVR0P
+BAQDAgWgMBMGA1UdJQQMMAoGCCsGAQUFBwMBMA0GCSqGSIb3DQEBBQUAA4ICAQA3
+XJXk9CbzdZUQugPI43LY88g+WjbTJfc/KtPSkHN3GjBBh8C0He7A2tp6Xj/LELmx
+crq62FzcFBnq8/iSdFITaYWRo0V/mXlpv2cpPebtwqbARCXUHGvF4/dGk/kw7uK/
+ohZJbeNySuQmQ5SQyfTdVA30Z0OSZ4jp24jC8uME7L8XOcFDgCRw01QNOISpi/5J
+BqeuFihmu/odYMHiEJdCXqe+4qIFfTh0mbgQ57l/geZm0K8uCEiOdTzSMoO8YdO2
+tm6EGNnc4yrVywjIHIvSy6YtNzd4ZM1a1CkEfPvGwe/wI1DI/zl3aJ721kcMPken
+rgEA4xXTPh6gZNMELIGZfu/mOTCFObe8rrh4QSaW4L+xa/VrLEnQRxuXAYGnmDWF
+e79aA+uXdS4+3OysNgEf4qDBt/ZquS/31DBdfJ59VfXWxp2yxMcGhcfiOdnx2Jy5
+KO8wdpXJA/7uwTJzsjLrIgfZnserOiBwE4luaHhDmKDGNVQvhkMq5tdtMdzuwn3/
+n6P1UwbFPiRGIzEAo0SSC1PRT8phv+5y0B1+gcj/peFymZVE+gRcrv9irVQqUpAY
+Lo9xrClAJ2xx4Ouz1GprKPoHdVyqtgcLXN4Oyi8Tehu96Zf6GytSEfTXsbQp+GgR
+TGRhKnDySjPhLp/uObfVwioyuAyA5mVCwjsZ/cvUUA==
+-----END CERTIFICATE-----
+-----BEGIN CERTIFICATE-----
+MIIHmDCCA4CgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwVjELMAkGA1UEBhMCVVMx
+CzAJBgNVBAgMAkNBMQswCQYDVQQHDAJTRjENMAsGA1UECgwEQWNtZTENMAsGA1UE
+CwwES3VkdTEPMA0GA1UEAwwGUk9PVENBMB4XDTE3MDgxMTIxMzUzNVoXDTQ0MTIy
+NzIxMzUzNVowUTEXMBUGA1UEAwwOSW50ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNB
+MQswCQYDVQQGEwJVUzENMAsGA1UECgwEQWNtZTENMAsGA1UECwwES3VkdTCCAiIw
+DQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM1X35LT/eBWBt0Uqqh3DSUyY3K8
+HLIlX3ZXg2Nx6y8yqhw5UGVFZl0uYBDo2DSlTl4sey+AxLIbpQI9ArRA+xqmFynV
+jheB9otudnA8hVwi/e9o+m+VSjG+HPRjSS5hwdPgpJG8DCPSmGyUUFtf3v0NxkUq
+Is+fB5qhQ36aQkI+MwQsSlHR+YrrKKVnE3f911wr9OScQP5KHjrZLQex8OmpWD9G
+v4P9jfVSUwmNEXXjmXDhNG/1R4ofX6HogZR6lBmRNGbcjjWRZQmPrOe9YcdkMLD0
+CdaUyKikqqW6Ilxs7scfuCGqwBWqh66tY18MBMHnt0bL26atTPduKYqulJ1pijio
+DUrzqtAzm7PirqPZ4aOJ9PNjdQs9zH3Zad3pcjfjpdKj4a/asX0st631J5jE6MLB
+LcbAerb/Csr/+tD0TOxwWlA+p/6wPb8ECflQLkvDDEY5BrRGdqYDpEOdm1F9DWQh
+y0RB8rWJMkxC/tTqYHfeaphzCxndLRsZQKVcPiqWCT7b431umIjPaDhsykNlcU3N
+f0V7V/fLY6wwuACngS0BLQuMrXy5FyhmWnUBeWwHfAeTxCkHlF+cVT6wHmeOuGbC
+c1piq7O7puKdC3UjO7Nn+WoOb2B6Qm/dajHpj5myxYJa5tGQGeUnWPwjjMQR557k
+HzugGAzkuG1ASQrhAgMBAAGjdTBzMA8GA1UdEwEB/wQFMAMBAf8wHQYDVR0OBBYE
+FPCnLX2qgKtPPOwh6g7CAtEZIO6EMB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeU
+HW46abboMAsGA1UdDwQEAwIBpjATBgNVHSUEDDAKBggrBgEFBQcDATANBgkqhkiG
+9w0BAQUFAAOCBAEAIaD2yzjTFdn61A4Qi+ek3fBJaDNQZytd0rHb49v3T+mdj/MI
+yShI1qezDFkg2FP1LfNgjuQl/T+g0BloXatAhZ/dj20Y8oN6bmilV+r2YLJbvbTn
+3hI+MxNf3Ue3FmIrwKK3QdkWcDBURpyYaDO71oxPl9QNfdhWCGHB/oWKU2y4Qt/O
+aPy+CmBAZEclX+hsdUBDJG5vuujpv4myCFwpLgFKNQX3XqCPLc4SRjfyla2YmeZv
+j7KKYh8XOWBbBF0BnWD94WzUDIBmFlUfS32aJTvd7tVaWXwH8rGwDfLN8i05UD9G
+zc3uuFH+UdzWVymk/4svKIPlB2nw9vPV8hvRRah0yFN3EQqAF0vQtwVJF/VwtZdg
+ahH0DykYTf7cKtFXE40xB7YgwDLXd3UiXfo3USW28uKqsrO52xYuUTBn+xkilds1
+tNKwtpXFWP2PUk92ficxoqi1cJnHxIIt5HKskFPgfIpzkpR8IM/vsom1a5fn4TT1
+aJbO5FsZTXQMxFLYWiSOMhTZMp3iNduxMYPosngjjKPEIkTQHKkedpF+CAGIMOKE
+BVa0vHyF34laKMMDT8d9yxwBJLqjlBohNsLLZa/Y90ThaMw+QYn/GZATB+7ng+ip
+VdGAQrghsGSxP+47HZ6WgBrlRdUWN1d1tlN2NBMHLucpbra5THGzl5MlaSVBYZb6
+yXI+2lwcTnnEkKv2zoA4ZHWdtLn/b1y4NKNg205TA+sOZcl6B1BgMe/rFuXdZe9Q
+/b6Tjz65qL4y1ByBVBJNhQQairw6cypHzwzC3w6ub1ZXtFqnTlU8fFcHGeOyydYS
+NfoepF0w2v0ounqD+6rN1CH/ERVb4FCEN19HQ3z+rAj19z2h6m/l5QEKI7bz8ghD
+8yxyqJz+L9XpfOo1yZfHQJckilY6BBIGWyeetJBmvkwv2WPt+3pX1u7h5LkvNRj2
+3fItf486zqtzUi+i/E//rS4gD/rRr4a85U8GSfp3LSAbtmfC0LNYUYA9Dcc0LSpl
+9alNuEpBHSHXlCVh4bcOb0L9n5XNdMcUYBo14hQdP0K1G7TounuAXFKYIQeyNyoi
+OAZ+eb7Y2xNnkY/ps/kyhsZgOJyiDZhdcruK3FIUGYlg5aVjQTB8H0c3/5SZnSky
+6779yMKztFXj9ctYU0YyJXWdF0xP/vi1gjQx/hJnDfXFfIOmeJdQSC08BGyK/PeC
+8zAS380bgzOza/eBL6IK0RqytbWgdoLrUQQfa1+f7AQxDDdoOkUenM0HSWjKfCuG
+m1/N7KUDHtnjVIHWqRefTPg1/tQjVY8/zgxN8MyAy+D95y4rawjsJf1dL6c0+zGv
+Wd40Cr+wAdHKN6t/oransoxu0EZ3HcSOI1umFg==
+-----END CERTIFICATE-----
+)";
+  const char* kKey = R"(
+-----BEGIN RSA PRIVATE KEY-----
+MIIJKAIBAAKCAgEAqevNYH73n4kARZtMsHRucdKmqVd/xxztMlK5VOorERUBhKVV
+Ow3kpDrN9z80ldIkpOrtrfE7Ame/nA9v4k6P3minPEm1qCA/kvaAodtT4HjAkrPc
++fto6VO6+aUV6l+ckAV/79lOuc7AutNlvvPtBQQcgOKvlNUSRKwM7ndydO4ZAa+u
+P9Wtsd0gl8b5F3P8vwevD3a0+iDvwSd3pi2s/BeVgRwvOxJzud8ipZ/AZmZN8Df9
+nHw5lsqLdNnqHXjTVCNXLnYXQC4gKU56fzyZL595liuefyQxiGY+dCCnCpqlSsHb
+oJVC/F3OaQi3xVRTB5l2Nwb149EIadwCF0OulZCuYljJ5y9H2bECXEjPe5aOdz9d
+8W3/T7p9vBKWctToeCpqKXUd+8RPudh0D0sUHuwQ4u4S1K6X+eK+gGhTHOnPwt+P
+8ytG0M463z5Gh9feW9ZDIYoiFckheFBAHxsgDWhjYpFmYireLLXMbyaMs5v/AxPN
+RAsx3vAAd0M0vGOpdgEJ9V1MsKmxkPO/tDC3zmnv6uJhtJfrOAKxwiGCfDe4IoSC
+6H5fTxeAgw6BG5onS1UPLADL8NA/M1y8qiSCZS/5S0cHoJp5AxDfZSSRO49ispjq
+cONRwckcRJ5Pbl0IA+wGyg2DuI9LaqS5kKWp5AE8VCLPz7yepDkRnnjO3m8CAwEA
+AQKCAgAE3oL2Hu1Nnwlo9ThPXibEEDtCYwWAWS3a4U/6RPOS+70dZfd5R76jjiPU
+z/TbzjfKmgjRkTYVrY9qE28rVwD8aJdSPPJ9rN7lgTbSbIyMxCkQiyLr7u5ksUeM
+W9Sy8KZ14hJ2dw2weWJAeEpUHH1QRXvjnZtWcnyhhySfuMCI5UHGMJiXr7HYhPOo
+JcWBjItTlg7ILKim+kakjFL7aheo6awZFQutb6vtSZ2ejWNgC9Jz7cbQsyabUZaJ
+dK0mxw2XPaQD6tJjvm6hgGQ2PTBOkw1S5lEWZ50bwYJMpZrjzOarq751bZGL1cxS
+ajOJ7g6rCxS+Iu7s5lKNZgaRUBkymATYccoigfZDR//fHKAmdgjgPstqy1NJL+uX
+bIuNE0bR+mBM2JQzNjPIcE67PG+0aQdO4dj0TnTzkTP1JSsa6Tz4ckOUgt7IBK8j
+ilCQpHgOB900hXC6xVRnAaU/uuSYEtfi2eFBKHT02OqH51yNZ2jsgOQJCvxNrrdI
+OmA0AaZ2tVXTTXe6qebaNjjp2cziiO5ma+5mdI1vmLQAA9v0micO+eKp/pi5e0r6
+j+9uyR2Oo4qnHg04TIfDyglW3uKz1eI0RPfBN/dx3WLpydxKeywXPH6EzfBFk8pz
+ST2sy+1ZN4vn0bDSTjSLsLBW+xBKYINqKkBD2Kg6B7aeNINjEQKCAQEA4BQByyZV
+9va91T+rQiNPifuI4PKgmLTl0wxM1Jg/H4YCyLtuEkmuvwfeAjaUaUuk2MDs3xfQ
+4pu8vtAnDapq5vJ/lMg0m3+NIwoks+xee9F//M4du9j67USvX5Qw/6Cnx1zAvrWR
+TyZncKUocQowcXM9VU0xcv4qVCMaH5F5Q/1VvG7uAtGCnB8TOHzV7GUaj9xxSqDc
+f3+p9BzsecpPZsdpVi01dQbCKi9neZwFv1Vr3MvieNDOGqSGn8X5EjSHY5PzCaXL
+S+/HoFWOSzWcuNdzKJRjVkC8U+eHoEabaRnD47dfJntN3qOQ6Mwf6P2jMN9GqlQu
+DQlvpMxBwQT1/QKCAQEAwiC4hr6EZKaLmeZBLggsS+oifHReXQy6tf2n7kCAwAgL
++65K8UW+YUtQyHL6UFfD27vvW5yp6LZxMRUD3ghguHPMQcejgoQTfGmth1bCb9tx
+zqfxuWUoNauqZiP4/kgxoh815Kt+kC8BRCXNIWjH38zxY+ncv0b4oKP7lYna/7Xk
+URLmMFr92QVAydRxY9kQTHQTCd+ZQrFT97xEoosgzkKhmynSfWNx7ymYmCrHzscA
+TutpD26u4CA4Wh4ZdVPEF10lGR531SAFEqXCkaUvIfwPphPmOtum2LZdEYho9C76
+71kLzzoJOJUNo2L9ORd5I4tOrMT0tmN+MpS1cPXb2wKCAQAu3aBeZ9807vhXQKDG
+DXKWTmibe8OBDNzAnmL3V/xj0HiGmUT1SDnnNHMHjXjO6QZKW1dvdaC3tJDua8Sv
+RARl1zQ93v25xBy1xmpUw0wjo3acXlOztTcOJv5zBCCXZneQ5+JcQMdqgYLC+ZgS
+xGnLYKnkTGfaQDSEMm9FSPzO7o5fAeh/6Gfj1VAE0X9AmQjMK/P6Atj7Ra07JE2F
+T3355h0u6/exST+U6SNAORSupuQPYwkz8aAZzG1nv1VPrHLgrdH4I5f4gucCrsI7
+ErR7qHwqcZaxNIrvFY61Q+8/NSdWWkTpXIK13Qny1raZ2WqnTxuNhlu3WFDka+AY
+ybvVAoIBACOaxL1R7A5ZzXjolkPPE/DIfJK+9R+z2frPDyHPis2trCT5Dp254PUP
+Tz20eNyLfEys53WyAifAbnpGFHOArdymwGvAJekmODy1VTJhY0AIy5LPkrIiL4HI
+fnRFXMGmlBPcDZJnMctYE69gD4N1KFOPzyY4Gliqt6ce7GG86wHDZqDICpgL2EsZ
+f4yE/lcF1Mtw7pz8+asVwwTI7v2w7s9lwSYoQYbl2lu3EVm3XvY54YCYBKjj8AcD
+YdKFer3eIzT1zHwS7n+UY9nLtSfpV/+vr18Sf0OETdGpgOBaWIWQqE2F03iqeE58
+aAfze+YgvAMc5c0iQo/BJ8A3LiANt8kCggEBAK8cFEBm+Z1g5s1IaWxqTIylR5XF
+5RFty9HyUXtkAd2d1qVzBaBG3SXg4cRsW0EhcUV2XP3iTFIPXEEABRRu5U6DEal6
+wQclrhfP4hiRQHp2Ny6jDj70NCSeUmyEu2lmwEJJYsDSOCnVmtt+qlgmk4yI1Dua
+nXhLcPLqopuhEZs2V/l2Q6E5i4vrs71Y7of+vsAvvt42Vx5wsGdPQihc5E1MI7GB
+hxmQys1MwG3Jyd7Zk88MVNveASeEIc7UAmr/TGL+RIv4bxNi/1HrgekBf1jnFUU3
+4fsdqKy0W+rOgOlGN8TX7NYCz3B41UEiyf+/gZ/TcLKAyGnoAO727Ngayqo=
+-----END RSA PRIVATE KEY-----
+)";
+  const char* kRootCaCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIJfzCCBWegAwIBAgIJAOquFl/JjmRLMA0GCSqGSIb3DQEBCwUAMFYxCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTELMAkGA1UEBwwCU0YxDTALBgNVBAoMBEFjbWUx
+DTALBgNVBAsMBEt1ZHUxDzANBgNVBAMMBlJPT1RDQTAeFw0xNzA4MTEyMTMyMTla
+Fw00NDEyMjcyMTMyMTlaMFYxCzAJBgNVBAYTAlVTMQswCQYDVQQIDAJDQTELMAkG
+A1UEBwwCU0YxDTALBgNVBAoMBEFjbWUxDTALBgNVBAsMBEt1ZHUxDzANBgNVBAMM
+BlJPT1RDQTCCBCIwDQYJKoZIhvcNAQEBBQADggQPADCCBAoCggQBAOHxuOsGDBJt
+quCb+lrJ3uXvvBv6f1w1aP+WqDEqveQlOi68sV/DVUR3s+en/MHA5jYAVo2D+eR7
+v/zjrAzCeqCpipbDcxA2e00+kggGHc1BoLtXXTPPCcTQt/0jjX26GXlaJRrY5MAy
+ZJ35vkJ5wCTw7DttfyRzR/RplI6DfO3t2kkSFpSsjGFJZQRZn/L2OM8Ii/tEhede
+UP/Rv8KIKM2+P9RS0VIutiI+/mOpH0QZHHEgnHy7x/CcNCd+wDG516YoJXp9c/1n
+aRLK+jA0bNCf0ZktMpuifoFzpNJ3pvDkjgTLhbiMkS8VKc66Z/Mv0EVOrdiMla/X
+0OSWqEZctxIcVIGDbMqngy62dghMBmxpVkfNmu6RqyS3HmPFrhRXJIIogdBo8mdJ
+xFCCvOgA6suaZnQtQC0mlRi5XGnTocpeHYUZ1c1hO2ZdVrFTh3atJsD80kVYxYuK
+YMq3QaK2zZUK6TUIFue1UqLf2dpIFzskLY6bEVob7Rdl8AHdFBJ8cGOyYKpG+rwO
+n3XQudt8YwDUCvw+8MGRXQiwUnzT/3gSuLNjlQOdcqN78wT5mdp6QZwareptyRwT
+yk/wWnfZlcFO33aPnUhvzzI5TzTB6EqG+3oNYkuXXy/glvOFluyQcPfsYXVOnXOj
+xF0hjKcpx10KQSvXjT9SRYr8NcOC7Yjy3f+WF+nwV+EzevqC2iTr1u8ymqUvpgFJ
+snvO8G/tycfxrwjI/4IghBgwqhcqD4wp/NleXy3A7GE3kFusL10i1bjwxBlz5qER
+uKaxU164HXPl4gv+Qt3eGqJE8KHDwTp8x+619S0+Gd8fY6Yj6/v9WyDef0SKGscm
+t3iqYNA39yNHAj++cjcCrJwBfINVvnTsVFKsCwUpjVuNOGRfZv0uHLAv6LaePQk5
+FKHwlLlPRx7ZcwHpkzTvp/ixYPb/cNJOw8fVW5CoWXYEzDUJY0oU8BWlQNHQ/e4q
+V7Yxa/vourUUvOACDzyQ6hCO95dQdDMCDQqC2VVL45+TUJ3eU1gDHge4T2js/qL8
+iJ+auZapiZjUQzLFse4XkgDrkMrD4fkOQOw4x9AhJF/SrnI8UPNjNOmAyOlqGTdd
+IyLesKXgnOGASSmc7JRk+YBpu9PQXIgHHQZIao1zPGP5k/ylp8XPYitC9MKzRRam
+67aJmutJxEtw7VJhoyz5m5LhLysPLY+R01+QqZK9/7qwWaX6PvMmm42zq9YKncOM
+cC/4eAPnwbj6yhpFoaUD5qzloci3+tvYgb+wr3f1F9SPrpe5xJz3NTXdQj8FsGjl
+ShL+bybUQ7wzZQObUhWtXSayagQg1MAxUCn7Aglmo/A/1+teacyuCEIbrhmODYM3
+Okji9gmGv+cCAwEAAaNQME4wHQYDVR0OBBYEFE/9XKaDey5kC8f3bCeUHW46abbo
+MB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeUHW46abboMAwGA1UdEwQFMAMBAf8w
+DQYJKoZIhvcNAQELBQADggQBAMXuMpJzYV3QMH780os0dZyJ+zi4FETVjAWFAPME
+qzN27W0L9+HZcGpz5i5FLdmc0F3u1cyCrJ4nCCWMrEIVmrLInFRaH1u9HUO78jPw
+Uw/MZLF+sf7uE8IAdVzLQG0A3QjAVoOX3IEOxEaQYYlVQHonyq2pBt8mkGqiPI3R
+E9cTnP/R1Ncd1wZkFL/n5qSNGTr/eg1O/jFB5xH96xm18Z7HgJDku2JCKQK6kqTM
+B7LjAFwWzTg8cnewVFRzIvJe83w9lHs1SW3R9fz7fIEBbZQ3z+n1cSj5jDjaT1+U
+YWTj+gAklZT4M/vImXF0XqbZytUOqe16HfBInc0G/kakUIcs6le2hmfhccJdG25I
+e5TH6ZdMumt7//hVPBPN5fhYKc2uHpzbtmxUjuKG8Na8/w+y2O+sW5CfpNtrYLyB
+WATHGtBB3ckBAICLlhoQiY/ku9r6BfczI86MbSy5vG5CD2sYGhVEl3PQXAnvBKax
+svZS3z9f16SZm61FWwz+r0XCe7LBiKe9YpODyE8lFDymZyW0vKYzDLzCy/mXhU/j
+locrf5r4YK0cOxNQC/jK7MLDFxPQbYg2SuAPW4DF2QzgKn2RuatdOB12S05afawj
+bhrbikIfEtD3erUMMJwaV9dxhHL835rfexxbRzBCdbjWg7Qiw4r8+PJB/mSSvsVO
+RAI02p8IqW5O+iXkU4V2Mapzdpo6b8O6TplHRXbRxWuen87g87KHhgZaC6TmWgvT
+It3ARZx3tkBoJwf41ELmWcakqiT9aQslc5weafw3SZp6+w0QU0qqFwCFLJWHETa5
+/PVHDEkBoXDMnqMlu7E9PUks4Op9T2f7bNy94GZXRbSp2VKjV68sds739DhVIZ+M
+MIaEutz3UndEuGGlcVuqXlda+H5xp57RnMZSKbT240kGdci51WahhfkX7dLY6c/b
+jaNWyGSfM+wFlky97t7ANbPP85SDgrrSyb6rTIt1zU2c5+vvjNVvDhlS6n7ls/Pi
+lMWWs5Ka66E8oZFwYygfIiEv6FcNWrSZ/vCMuS02WJovsZd4YrYtNbpkx6shaA5t
+BOIpuelPbQNPlOaJ+YnRuuppomPnXx5X3RlHld7xsExHDNsi57H0PBDq/W5O1S4t
+rHm3SItJQ4ndFHBGjZ7kSOyHtCLWZ8cB75sPduVC2PnRL/kt3tmfFFVsUurLGz4n
+wgCg1OuflNcc9wIF8lZMjm0TZkQMGYBIfBA7x8/Vs2XSFuaT9vbWoC07LXftl13g
+HsMg1UUSqnMBUQStG42lbVFF1yIfPZThEPxD2RJTCw8FTLBmNrJyBsZ0BGagwe1C
+KH5H1VGmllMdZDHOamHHKA8mEDI4eAKY3HoOS4rfioT8Tks=
+-----END CERTIFICATE-----
+)";
+
+  *cert_file = JoinPathSegments(dir, "test.cert");
+  *key_file = JoinPathSegments(dir, "test.key");
+  *ca_cert_file = JoinPathSegments(dir, "testchainca.cert");
+
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCert, *cert_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kKey, *key_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kRootCaCert, *ca_cert_file));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu


[44/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/inbound_call.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/inbound_call.cc b/be/src/kudu/rpc/inbound_call.cc
new file mode 100644
index 0000000..6920071
--- /dev/null
+++ b/be/src/kudu/rpc/inbound_call.cc
@@ -0,0 +1,345 @@
+// 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 "kudu/rpc/inbound_call.h"
+
+#include <cstdint>
+#include <memory>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <google/protobuf/message.h>
+#include <google/protobuf/message_lite.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/rpcz_store.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/trace.h"
+
+namespace google {
+namespace protobuf {
+class FieldDescriptor;
+}
+}
+
+using google::protobuf::FieldDescriptor;
+using google::protobuf::Message;
+using google::protobuf::MessageLite;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+InboundCall::InboundCall(Connection* conn)
+  : conn_(conn),
+    trace_(new Trace),
+    method_info_(nullptr),
+    deadline_(MonoTime::Max()) {
+  RecordCallReceived();
+}
+
+InboundCall::~InboundCall() {}
+
+Status InboundCall::ParseFrom(gscoped_ptr<InboundTransfer> transfer) {
+  TRACE_EVENT_FLOW_BEGIN0("rpc", "InboundCall", this);
+  TRACE_EVENT0("rpc", "InboundCall::ParseFrom");
+  RETURN_NOT_OK(serialization::ParseMessage(transfer->data(), &header_, &serialized_request_));
+
+  // Adopt the service/method info from the header as soon as it's available.
+  if (PREDICT_FALSE(!header_.has_remote_method())) {
+    return Status::Corruption("Non-connection context request header must specify remote_method");
+  }
+  if (PREDICT_FALSE(!header_.remote_method().IsInitialized())) {
+    return Status::Corruption("remote_method in request header is not initialized",
+                              header_.remote_method().InitializationErrorString());
+  }
+  remote_method_.FromPB(header_.remote_method());
+
+  // Compute and cache the call deadline.
+  if (header_.has_timeout_millis() && header_.timeout_millis() != 0) {
+    deadline_ = timing_.time_received + MonoDelta::FromMilliseconds(header_.timeout_millis());
+  }
+
+  if (header_.sidecar_offsets_size() > TransferLimits::kMaxSidecars) {
+    return Status::Corruption(strings::Substitute(
+            "Received $0 additional payload slices, expected at most %d",
+            header_.sidecar_offsets_size(), TransferLimits::kMaxSidecars));
+  }
+
+  RETURN_NOT_OK(RpcSidecar::ParseSidecars(
+          header_.sidecar_offsets(), serialized_request_, inbound_sidecar_slices_));
+  if (header_.sidecar_offsets_size() > 0) {
+    // Trim the request to just the message
+    serialized_request_ = Slice(serialized_request_.data(), header_.sidecar_offsets(0));
+  }
+
+  // Retain the buffer that we have a view into.
+  transfer_.swap(transfer);
+  return Status::OK();
+}
+
+void InboundCall::RespondSuccess(const MessageLite& response) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondSuccess");
+  Respond(response, true);
+}
+
+void InboundCall::RespondUnsupportedFeature(const vector<uint32_t>& unsupported_features) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondUnsupportedFeature");
+  ErrorStatusPB err;
+  err.set_message("unsupported feature flags");
+  err.set_code(ErrorStatusPB::ERROR_INVALID_REQUEST);
+  for (uint32_t feature : unsupported_features) {
+    err.add_unsupported_feature_flags(feature);
+  }
+
+  Respond(err, false);
+}
+
+void InboundCall::RespondFailure(ErrorStatusPB::RpcErrorCodePB error_code,
+                                 const Status& status) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondFailure");
+  ErrorStatusPB err;
+  err.set_message(status.ToString());
+  err.set_code(error_code);
+
+  Respond(err, false);
+}
+
+void InboundCall::RespondApplicationError(int error_ext_id, const std::string& message,
+                                          const MessageLite& app_error_pb) {
+  ErrorStatusPB err;
+  ApplicationErrorToPB(error_ext_id, message, app_error_pb, &err);
+  Respond(err, false);
+}
+
+void InboundCall::ApplicationErrorToPB(int error_ext_id, const std::string& message,
+                                       const google::protobuf::MessageLite& app_error_pb,
+                                       ErrorStatusPB* err) {
+  err->set_message(message);
+  const FieldDescriptor* app_error_field =
+    err->GetReflection()->FindKnownExtensionByNumber(error_ext_id);
+  if (app_error_field != nullptr) {
+    err->GetReflection()->MutableMessage(err, app_error_field)->CheckTypeAndMergeFrom(app_error_pb);
+  } else {
+    LOG(DFATAL) << "Unable to find application error extension ID " << error_ext_id
+                << " (message=" << message << ")";
+  }
+}
+
+void InboundCall::Respond(const MessageLite& response,
+                          bool is_success) {
+  TRACE_EVENT_FLOW_END0("rpc", "InboundCall", this);
+  SerializeResponseBuffer(response, is_success);
+
+  TRACE_EVENT_ASYNC_END1("rpc", "InboundCall", this,
+                         "method", remote_method_.method_name());
+  TRACE_TO(trace_, "Queueing $0 response", is_success ? "success" : "failure");
+  RecordHandlingCompleted();
+  conn_->rpcz_store()->AddCall(this);
+  conn_->QueueResponseForCall(gscoped_ptr<InboundCall>(this));
+}
+
+void InboundCall::SerializeResponseBuffer(const MessageLite& response,
+                                          bool is_success) {
+  if (PREDICT_FALSE(!response.IsInitialized())) {
+    LOG(ERROR) << "Invalid RPC response for " << ToString()
+               << ": protobuf missing required fields: "
+               << response.InitializationErrorString();
+    // Send it along anyway -- the client will also notice the missing fields
+    // and produce an error on the other side, but this will at least
+    // make it clear on both sides of the RPC connection what kind of error
+    // happened.
+  }
+
+  uint32_t protobuf_msg_size = response.ByteSize();
+
+  ResponseHeader resp_hdr;
+  resp_hdr.set_call_id(header_.call_id());
+  resp_hdr.set_is_error(!is_success);
+  int32_t sidecar_byte_size = 0;
+  for (const unique_ptr<RpcSidecar>& car : outbound_sidecars_) {
+    resp_hdr.add_sidecar_offsets(sidecar_byte_size + protobuf_msg_size);
+    int32_t sidecar_bytes = car->AsSlice().size();
+    DCHECK_LE(sidecar_byte_size, TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes);
+    sidecar_byte_size += sidecar_bytes;
+  }
+
+  serialization::SerializeMessage(response, &response_msg_buf_,
+                                  sidecar_byte_size, true);
+  int64_t main_msg_size = sidecar_byte_size + response_msg_buf_.size();
+  serialization::SerializeHeader(resp_hdr, main_msg_size,
+                                 &response_hdr_buf_);
+}
+
+size_t InboundCall::SerializeResponseTo(TransferPayload* slices) const {
+  TRACE_EVENT0("rpc", "InboundCall::SerializeResponseTo");
+  DCHECK_GT(response_hdr_buf_.size(), 0);
+  DCHECK_GT(response_msg_buf_.size(), 0);
+  size_t n_slices = 2 + outbound_sidecars_.size();
+  DCHECK_LE(n_slices, slices->size());
+  auto slice_iter = slices->begin();
+  *slice_iter++ = Slice(response_hdr_buf_);
+  *slice_iter++ = Slice(response_msg_buf_);
+  for (auto& sidecar : outbound_sidecars_) {
+    *slice_iter++ = sidecar->AsSlice();
+  }
+  DCHECK_EQ(slice_iter - slices->begin(), n_slices);
+  return n_slices;
+}
+
+Status InboundCall::AddOutboundSidecar(unique_ptr<RpcSidecar> car, int* idx) {
+  // Check that the number of sidecars does not exceed the number of payload
+  // slices that are free (two are used up by the header and main message
+  // protobufs).
+  if (outbound_sidecars_.size() > TransferLimits::kMaxSidecars) {
+    return Status::ServiceUnavailable("All available sidecars already used");
+  }
+  int64_t sidecar_bytes = car->AsSlice().size();
+  if (outbound_sidecars_total_bytes_ >
+      TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes) {
+    return Status::RuntimeError(Substitute("Total size of sidecars $0 would exceed limit $1",
+        static_cast<int64_t>(outbound_sidecars_total_bytes_) + sidecar_bytes,
+        TransferLimits::kMaxTotalSidecarBytes));
+  }
+
+  outbound_sidecars_.emplace_back(std::move(car));
+  outbound_sidecars_total_bytes_ += sidecar_bytes;
+  DCHECK_GE(outbound_sidecars_total_bytes_, 0);
+  *idx = outbound_sidecars_.size() - 1;
+  return Status::OK();
+}
+
+string InboundCall::ToString() const {
+  if (header_.has_request_id()) {
+    return Substitute("Call $0 from $1 (ReqId={client: $2, seq_no=$3, attempt_no=$4})",
+                      remote_method_.ToString(),
+                      conn_->remote().ToString(),
+                      header_.request_id().client_id(),
+                      header_.request_id().seq_no(),
+                      header_.request_id().attempt_no());
+  }
+  return Substitute("Call $0 from $1 (request call id $2)",
+                      remote_method_.ToString(),
+                      conn_->remote().ToString(),
+                      header_.call_id());
+}
+
+void InboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
+                         RpcCallInProgressPB* resp) {
+  resp->mutable_header()->CopyFrom(header_);
+  if (req.include_traces() && trace_) {
+    resp->set_trace_buffer(trace_->DumpToString());
+  }
+  resp->set_micros_elapsed((MonoTime::Now() - timing_.time_received)
+                           .ToMicroseconds());
+}
+
+const RemoteUser& InboundCall::remote_user() const {
+  return conn_->remote_user();
+}
+
+const Sockaddr& InboundCall::remote_address() const {
+  return conn_->remote();
+}
+
+const scoped_refptr<Connection>& InboundCall::connection() const {
+  return conn_;
+}
+
+Trace* InboundCall::trace() {
+  return trace_.get();
+}
+
+void InboundCall::RecordCallReceived() {
+  TRACE_EVENT_ASYNC_BEGIN0("rpc", "InboundCall", this);
+  DCHECK(!timing_.time_received.Initialized());  // Protect against multiple calls.
+  timing_.time_received = MonoTime::Now();
+}
+
+void InboundCall::RecordHandlingStarted(Histogram* incoming_queue_time) {
+  DCHECK(incoming_queue_time != nullptr);
+  DCHECK(!timing_.time_handled.Initialized());  // Protect against multiple calls.
+  timing_.time_handled = MonoTime::Now();
+  incoming_queue_time->Increment(
+      (timing_.time_handled - timing_.time_received).ToMicroseconds());
+}
+
+void InboundCall::RecordHandlingCompleted() {
+  DCHECK(!timing_.time_completed.Initialized());  // Protect against multiple calls.
+  timing_.time_completed = MonoTime::Now();
+
+  if (!timing_.time_handled.Initialized()) {
+    // Sometimes we respond to a call before we begin handling it (e.g. due to queue
+    // overflow, etc). These cases should not be counted against the histogram.
+    return;
+  }
+
+  if (method_info_) {
+    method_info_->handler_latency_histogram->Increment(
+        (timing_.time_completed - timing_.time_handled).ToMicroseconds());
+  }
+}
+
+bool InboundCall::ClientTimedOut() const {
+  return MonoTime::Now() >= deadline_;
+}
+
+MonoTime InboundCall::GetTimeReceived() const {
+  return timing_.time_received;
+}
+
+vector<uint32_t> InboundCall::GetRequiredFeatures() const {
+  vector<uint32_t> features;
+  for (uint32_t feature : header_.required_feature_flags()) {
+    features.push_back(feature);
+  }
+  return features;
+}
+
+Status InboundCall::GetInboundSidecar(int idx, Slice* sidecar) const {
+  DCHECK(transfer_) << "Sidecars have been discarded";
+  if (idx < 0 || idx >= header_.sidecar_offsets_size()) {
+    return Status::InvalidArgument(strings::Substitute(
+            "Index $0 does not reference a valid sidecar", idx));
+  }
+  *sidecar = inbound_sidecar_slices_[idx];
+  return Status::OK();
+}
+
+void InboundCall::DiscardTransfer() {
+  transfer_.reset();
+}
+
+size_t InboundCall::GetTransferSize() {
+  if (!transfer_) return 0;
+  return transfer_->data().size();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/inbound_call.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/inbound_call.h b/be/src/kudu/rpc/inbound_call.h
new file mode 100644
index 0000000..0db4c37
--- /dev/null
+++ b/be/src/kudu/rpc/inbound_call.h
@@ -0,0 +1,286 @@
+// 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.
+#ifndef KUDU_RPC_INBOUND_CALL_H
+#define KUDU_RPC_INBOUND_CALL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace google {
+namespace protobuf {
+class MessageLite;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Histogram;
+class Sockaddr;
+class Trace;
+
+namespace rpc {
+
+class Connection;
+class DumpRunningRpcsRequestPB;
+class RemoteUser;
+class RpcCallInProgressPB;
+class RpcSidecar;
+
+struct InboundCallTiming {
+  MonoTime time_received;   // Time the call was first accepted.
+  MonoTime time_handled;    // Time the call handler was kicked off.
+  MonoTime time_completed;  // Time the call handler completed.
+
+  MonoDelta TotalDuration() const {
+    return time_completed - time_received;
+  }
+};
+
+// Inbound call on server
+class InboundCall {
+ public:
+  explicit InboundCall(Connection* conn);
+  ~InboundCall();
+
+  // Parse an inbound call message.
+  //
+  // This only deserializes the call header, populating the 'header_' and
+  // 'serialized_request_' member variables. The actual call parameter is
+  // not deserialized, as this may be CPU-expensive, and this is called
+  // from the reactor thread.
+  Status ParseFrom(gscoped_ptr<InboundTransfer> transfer);
+
+  // Return the serialized request parameter protobuf.
+  const Slice& serialized_request() const {
+    DCHECK(transfer_) << "Transfer discarded before parameter parsing";
+    return serialized_request_;
+  }
+
+  const RemoteMethod& remote_method() const {
+    return remote_method_;
+  }
+
+  const int32_t call_id() const {
+    return header_.call_id();
+  }
+
+  // Serializes 'response' into the InboundCall's internal buffer, and marks
+  // the call as a success. Enqueues the response back to the connection
+  // that made the call.
+  //
+  // This method deletes the InboundCall object, so no further calls may be
+  // made after this one.
+  void RespondSuccess(const google::protobuf::MessageLite& response);
+
+  // Serializes a failure response into the internal buffer, marking the
+  // call as a failure. Enqueues the response back to the connection that
+  // made the call.
+  //
+  // This method deletes the InboundCall object, so no further calls may be
+  // made after this one.
+  void RespondFailure(ErrorStatusPB::RpcErrorCodePB error_code,
+                      const Status &status);
+
+  void RespondUnsupportedFeature(const std::vector<uint32_t>& unsupported_features);
+
+  void RespondApplicationError(int error_ext_id, const std::string& message,
+                               const google::protobuf::MessageLite& app_error_pb);
+
+  // Convert an application error extension to an ErrorStatusPB.
+  // These ErrorStatusPB objects are what are returned in application error responses.
+  static void ApplicationErrorToPB(int error_ext_id, const std::string& message,
+                                   const google::protobuf::MessageLite& app_error_pb,
+                                   ErrorStatusPB* err);
+
+  // Serialize the response packet for the finished call into 'slices'.
+  // The resulting slices refer to memory in this object.
+  // Returns the number of slices in the serialized response.
+  size_t SerializeResponseTo(TransferPayload* slices) const;
+
+  // See RpcContext::AddRpcSidecar()
+  Status AddOutboundSidecar(std::unique_ptr<RpcSidecar> car, int* idx);
+
+  std::string ToString() const;
+
+  void DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp);
+
+  const RemoteUser& remote_user() const;
+
+  const Sockaddr& remote_address() const;
+
+  const scoped_refptr<Connection>& connection() const;
+
+  Trace* trace();
+
+  const InboundCallTiming& timing() const {
+    return timing_;
+  }
+
+  const RequestHeader& header() const {
+    return header_;
+  }
+
+  // Associate this call with a particular method that will be invoked
+  // by the service.
+  void set_method_info(scoped_refptr<RpcMethodInfo> info) {
+    method_info_ = std::move(info);
+  }
+
+  // Return the method associated with this call. This is set just before
+  // the call is enqueued onto the service queue, and therefore may be
+  // 'nullptr' for much of the lifecycle of a call.
+  RpcMethodInfo* method_info() {
+    return method_info_.get();
+  }
+
+  // When this InboundCall was received (instantiated).
+  // Should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordCallReceived();
+
+  // When RPC call Handle() was called on the server side.
+  // Updates the Histogram with time elapsed since the call was received,
+  // and should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordHandlingStarted(Histogram* incoming_queue_time);
+
+  // Return true if the deadline set by the client has already elapsed.
+  // In this case, the server may stop processing the call, since the
+  // call response will be ignored anyway.
+  bool ClientTimedOut() const;
+
+  // Return an upper bound on the client timeout deadline. This does not
+  // account for transmission delays between the client and the server.
+  // If the client did not specify a deadline, returns MonoTime::Max().
+  MonoTime GetClientDeadline() const {
+    return deadline_;
+  }
+
+  // Return the time when this call was received.
+  MonoTime GetTimeReceived() const;
+
+  // Returns the set of application-specific feature flags required to service
+  // the RPC.
+  std::vector<uint32_t> GetRequiredFeatures() const;
+
+  // Get a sidecar sent as part of the request. If idx < 0 || idx > num sidecars - 1,
+  // returns an error.
+  Status GetInboundSidecar(int idx, Slice* sidecar) const;
+
+  // Releases the buffer that contains the request + sidecar data. It is an error to
+  // access sidecars or serialized_request() after this method is called.
+  void DiscardTransfer();
+
+  // Returns the size of the transfer buffer that backs this call. If the transfer does
+  // not exist (e.g. GetTransferSize() is called after DiscardTransfer()), returns 0.
+  size_t GetTransferSize();
+
+ private:
+  friend class RpczStore;
+
+  // Serialize and queue the response.
+  void Respond(const google::protobuf::MessageLite& response,
+               bool is_success);
+
+  // Serialize a response message for either success or failure. If it is a success,
+  // 'response' should be the user-defined response type for the call. If it is a
+  // failure, 'response' should be an ErrorStatusPB instance.
+  void SerializeResponseBuffer(const google::protobuf::MessageLite& response,
+                               bool is_success);
+
+  // When RPC call Handle() completed execution on the server side.
+  // Updates the Histogram with time elapsed since the call was started,
+  // and should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordHandlingCompleted();
+
+  // The connection on which this inbound call arrived.
+  scoped_refptr<Connection> conn_;
+
+  // The header of the incoming call. Set by ParseFrom()
+  RequestHeader header_;
+
+  // The serialized bytes of the request param protobuf. Set by ParseFrom().
+  // This references memory held by 'transfer_'.
+  Slice serialized_request_;
+
+  // The transfer that produced the call.
+  // This is kept around because it retains the memory referred to
+  // by 'serialized_request_' above.
+  gscoped_ptr<InboundTransfer> transfer_;
+
+  // The buffers for serialized response. Set by SerializeResponseBuffer().
+  faststring response_hdr_buf_;
+  faststring response_msg_buf_;
+
+  // Vector of additional sidecars that are tacked on to the call's response
+  // after serialization of the protobuf. See rpc/rpc_sidecar.h for more info.
+  std::vector<std::unique_ptr<RpcSidecar>> outbound_sidecars_;
+
+  // Total size of sidecars in outbound_sidecars_. This is limited to a maximum
+  // of TransferLimits::kMaxTotalSidecarBytes.
+  int32_t outbound_sidecars_total_bytes_ = 0;
+
+  // Inbound sidecars from the request. The slices are views onto transfer_. There are as
+  // many slices as header_.sidecar_offsets_size().
+  Slice inbound_sidecar_slices_[TransferLimits::kMaxSidecars];
+
+  // The trace buffer.
+  scoped_refptr<Trace> trace_;
+
+  // Timing information related to this RPC call.
+  InboundCallTiming timing_;
+
+  // Proto service this calls belongs to. Used for routing.
+  // This field is filled in when the inbound request header is parsed.
+  RemoteMethod remote_method_;
+
+  // After the method has been looked up within the service, this is filled in
+  // to point to the information about this method. Acts as a pointer back to
+  // per-method info such as tracing.
+  scoped_refptr<RpcMethodInfo> method_info_;
+
+  // A time at which the client will time out, or MonoTime::Max if the
+  // client did not pass a timeout.
+  MonoTime deadline_;
+
+  DISALLOW_COPY_AND_ASSIGN(InboundCall);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/messenger.cc b/be/src/kudu/rpc/messenger.cc
new file mode 100644
index 0000000..17ac0c5
--- /dev/null
+++ b/be/src/kudu/rpc/messenger.cc
@@ -0,0 +1,502 @@
+// 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 "kudu/rpc/messenger.h"
+
+#include <cstdlib>
+#include <functional>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_service.h"
+#include "kudu/rpc/rpcz_store.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread_restrictions.h"
+#include "kudu/util/threadpool.h"
+
+using std::string;
+using std::shared_ptr;
+using std::make_shared;
+using strings::Substitute;
+
+namespace boost {
+template <typename Signature> class function;
+}
+
+namespace kudu {
+namespace rpc {
+
+MessengerBuilder::MessengerBuilder(std::string name)
+    : name_(std::move(name)),
+      connection_keepalive_time_(MonoDelta::FromMilliseconds(65000)),
+      num_reactors_(4),
+      min_negotiation_threads_(0),
+      max_negotiation_threads_(4),
+      coarse_timer_granularity_(MonoDelta::FromMilliseconds(100)),
+      rpc_negotiation_timeout_ms_(3000),
+      sasl_proto_name_("kudu"),
+      rpc_authentication_("optional"),
+      rpc_encryption_("optional"),
+      rpc_tls_ciphers_(kudu::security::SecurityDefaults::kDefaultTlsCiphers),
+      rpc_tls_min_protocol_(kudu::security::SecurityDefaults::kDefaultTlsMinVersion),
+      enable_inbound_tls_(false),
+      reuseport_(false) {
+}
+
+MessengerBuilder& MessengerBuilder::set_connection_keepalive_time(const MonoDelta &keepalive) {
+  connection_keepalive_time_ = keepalive;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_num_reactors(int num_reactors) {
+  num_reactors_ = num_reactors;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_min_negotiation_threads(int min_negotiation_threads) {
+  min_negotiation_threads_ = min_negotiation_threads;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_max_negotiation_threads(int max_negotiation_threads) {
+  max_negotiation_threads_ = max_negotiation_threads;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_coarse_timer_granularity(const MonoDelta &granularity) {
+  coarse_timer_granularity_ = granularity;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_metric_entity(
+    const scoped_refptr<MetricEntity>& metric_entity) {
+  metric_entity_ = metric_entity;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_connection_keep_alive_time(int32_t time_in_ms) {
+  connection_keepalive_time_ = MonoDelta::FromMilliseconds(time_in_ms);
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_negotiation_timeout_ms(int64_t time_in_ms) {
+  rpc_negotiation_timeout_ms_ = time_in_ms;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_sasl_proto_name(const std::string& sasl_proto_name) {
+  sasl_proto_name_ = sasl_proto_name;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_authentication(const std::string& rpc_authentication) {
+  rpc_authentication_ = rpc_authentication;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_encryption(const std::string& rpc_encryption) {
+  rpc_encryption_ = rpc_encryption;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_tls_ciphers(const std::string& rpc_tls_ciphers) {
+  rpc_tls_ciphers_ = rpc_tls_ciphers;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_tls_min_protocol(
+    const std::string& rpc_tls_min_protocol) {
+  rpc_tls_min_protocol_ = rpc_tls_min_protocol;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_cert_key_files(
+    const std::string& cert, const std::string& private_key) {
+  rpc_certificate_file_ = cert;
+  rpc_private_key_file_ = private_key;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_certificate_authority_file(const std::string& ca) {
+  rpc_ca_certificate_file_ = ca;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_private_password_key_cmd(const std::string& cmd) {
+  rpc_private_key_password_cmd_ = cmd;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_keytab_file(const std::string& keytab_file) {
+  keytab_file_ = keytab_file;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::enable_inbound_tls() {
+  enable_inbound_tls_ = true;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_reuseport() {
+  reuseport_ = true;
+  return *this;
+}
+
+Status MessengerBuilder::Build(shared_ptr<Messenger> *msgr) {
+  // Initialize SASL library before we start making requests
+  RETURN_NOT_OK(SaslInit(!keytab_file_.empty()));
+
+  Messenger* new_msgr(new Messenger(*this));
+
+  auto cleanup = MakeScopedCleanup([&] () {
+      new_msgr->AllExternalReferencesDropped();
+  });
+
+  RETURN_NOT_OK(ParseTriState("--rpc_authentication",
+                              rpc_authentication_,
+                              &new_msgr->authentication_));
+
+  RETURN_NOT_OK(ParseTriState("--rpc_encryption",
+                              rpc_encryption_,
+                              &new_msgr->encryption_));
+
+  RETURN_NOT_OK(new_msgr->Init());
+  if (new_msgr->encryption_ != RpcEncryption::DISABLED && enable_inbound_tls_) {
+    auto* tls_context = new_msgr->mutable_tls_context();
+
+    if (!rpc_certificate_file_.empty()) {
+      CHECK(!rpc_private_key_file_.empty());
+      CHECK(!rpc_ca_certificate_file_.empty());
+
+      // TODO(KUDU-1920): should we try and enforce that the server
+      // is in the subject or alt names of the cert?
+      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(rpc_ca_certificate_file_));
+      if (rpc_private_key_password_cmd_.empty()) {
+        RETURN_NOT_OK(tls_context->LoadCertificateAndKey(rpc_certificate_file_,
+                                                         rpc_private_key_file_));
+      } else {
+        RETURN_NOT_OK(tls_context->LoadCertificateAndPasswordProtectedKey(
+            rpc_certificate_file_, rpc_private_key_file_,
+            [&](){
+              string ret;
+              WARN_NOT_OK(security::GetPasswordFromShellCommand(
+                  rpc_private_key_password_cmd_, &ret),
+                  "could not get RPC password from configured command");
+              return ret;
+            }
+        ));
+      }
+    } else {
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+    }
+  }
+
+  // See docs on Messenger::retain_self_ for info about this odd hack.
+  cleanup.cancel();
+  *msgr = shared_ptr<Messenger>(new_msgr, std::mem_fun(&Messenger::AllExternalReferencesDropped));
+  return Status::OK();
+}
+
+// See comment on Messenger::retain_self_ member.
+void Messenger::AllExternalReferencesDropped() {
+  // The last external ref may have been dropped in the context of a task
+  // running on a reactor thread. If that's the case, a SYNC shutdown here
+  // would deadlock.
+  //
+  // If a SYNC shutdown is desired, Shutdown() should be called explicitly.
+  ShutdownInternal(ShutdownMode::ASYNC);
+
+  CHECK(retain_self_.get());
+  // If we have no more external references, then we no longer
+  // need to retain ourself. We'll destruct as soon as all our
+  // internal-facing references are dropped (ie those from reactor
+  // threads).
+  retain_self_.reset();
+}
+
+void Messenger::Shutdown() {
+  ShutdownInternal(ShutdownMode::SYNC);
+}
+
+void Messenger::ShutdownInternal(ShutdownMode mode) {
+  if (mode == ShutdownMode::SYNC) {
+    ThreadRestrictions::AssertWaitAllowed();
+  }
+
+  // Since we're shutting down, it's OK to block.
+  //
+  // TODO(adar): this ought to be removed (i.e. if ASYNC, waiting should be
+  // forbidden, and if SYNC, we already asserted above), but that's not
+  // possible while shutting down thread and acceptor pools still involves
+  // joining threads.
+  ThreadRestrictions::ScopedAllowWait allow_wait;
+
+  acceptor_vec_t pools_to_shutdown;
+  RpcServicesMap services_to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    if (closing_) {
+      return;
+    }
+    VLOG(1) << "shutting down messenger " << name_;
+    closing_ = true;
+
+    services_to_release = std::move(rpc_services_);
+    pools_to_shutdown = std::move(acceptor_pools_);
+  }
+
+  // Destroy state outside of the lock.
+  services_to_release.clear();
+  for (const auto& p : pools_to_shutdown) {
+    p->Shutdown();
+  }
+
+  // Need to shut down negotiation pool before the reactors, since the
+  // reactors close the Connection sockets, and may race against the negotiation
+  // threads' blocking reads & writes.
+  client_negotiation_pool_->Shutdown();
+  server_negotiation_pool_->Shutdown();
+
+  for (Reactor* reactor : reactors_) {
+    reactor->Shutdown(mode);
+  }
+}
+
+Status Messenger::AddAcceptorPool(const Sockaddr &accept_addr,
+                                  shared_ptr<AcceptorPool>* pool) {
+  // Before listening, if we expect to require Kerberos, we want to verify
+  // that everything is set up correctly. This way we'll generate errors on
+  // startup rather than later on when we first receive a client connection.
+  if (!keytab_file_.empty()) {
+    RETURN_NOT_OK_PREPEND(ServerNegotiation::PreflightCheckGSSAPI(sasl_proto_name()),
+                          "GSSAPI/Kerberos not properly configured");
+  }
+
+  Socket sock;
+  RETURN_NOT_OK(sock.Init(0));
+  RETURN_NOT_OK(sock.SetReuseAddr(true));
+  if (reuseport_) {
+    RETURN_NOT_OK(sock.SetReusePort(true));
+  }
+  RETURN_NOT_OK(sock.Bind(accept_addr));
+  Sockaddr remote;
+  RETURN_NOT_OK(sock.GetSocketAddress(&remote));
+  auto acceptor_pool(make_shared<AcceptorPool>(this, &sock, remote));
+
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  acceptor_pools_.push_back(acceptor_pool);
+  pool->swap(acceptor_pool);
+  return Status::OK();
+}
+
+// Register a new RpcService to handle inbound requests.
+Status Messenger::RegisterService(const string& service_name,
+                                  const scoped_refptr<RpcService>& service) {
+  DCHECK(service);
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  if (InsertIfNotPresent(&rpc_services_, service_name, service)) {
+    return Status::OK();
+  } else {
+    return Status::AlreadyPresent("This service is already present");
+  }
+}
+
+void Messenger::UnregisterAllServices() {
+  RpcServicesMap to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    to_release = std::move(rpc_services_);
+  }
+  // Release the map outside of the lock.
+}
+
+Status Messenger::UnregisterService(const string& service_name) {
+  scoped_refptr<RpcService> to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    to_release = EraseKeyReturnValuePtr(&rpc_services_, service_name);
+    if (!to_release) {
+      return Status::ServiceUnavailable(Substitute(
+          "service $0 not registered on $1", service_name, name_));
+    }
+  }
+  // Release the service outside of the lock.
+  return Status::OK();
+}
+
+void Messenger::QueueOutboundCall(const shared_ptr<OutboundCall> &call) {
+  Reactor *reactor = RemoteToReactor(call->conn_id().remote());
+  reactor->QueueOutboundCall(call);
+}
+
+void Messenger::QueueInboundCall(gscoped_ptr<InboundCall> call) {
+  shared_lock<rw_spinlock> guard(lock_.get_lock());
+  scoped_refptr<RpcService>* service = FindOrNull(rpc_services_,
+                                                  call->remote_method().service_name());
+  if (PREDICT_FALSE(!service)) {
+    Status s =  Status::ServiceUnavailable(Substitute("service $0 not registered on $1",
+                                                      call->remote_method().service_name(), name_));
+    LOG(INFO) << s.ToString();
+    call.release()->RespondFailure(ErrorStatusPB::ERROR_NO_SUCH_SERVICE, s);
+    return;
+  }
+
+  call->set_method_info((*service)->LookupMethod(call->remote_method()));
+
+  // The RpcService will respond to the client on success or failure.
+  WARN_NOT_OK((*service)->QueueInboundCall(std::move(call)), "Unable to handle RPC call");
+}
+
+void Messenger::QueueCancellation(const shared_ptr<OutboundCall> &call) {
+  Reactor *reactor = RemoteToReactor(call->conn_id().remote());
+  reactor->QueueCancellation(call);
+}
+
+void Messenger::RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote) {
+  Reactor *reactor = RemoteToReactor(remote);
+  reactor->RegisterInboundSocket(new_socket, remote);
+}
+
+Messenger::Messenger(const MessengerBuilder &bld)
+  : name_(bld.name_),
+    closing_(false),
+    authentication_(RpcAuthentication::REQUIRED),
+    encryption_(RpcEncryption::REQUIRED),
+    tls_context_(new security::TlsContext(bld.rpc_tls_ciphers_, bld.rpc_tls_min_protocol_)),
+    token_verifier_(new security::TokenVerifier()),
+    rpcz_store_(new RpczStore()),
+    metric_entity_(bld.metric_entity_),
+    rpc_negotiation_timeout_ms_(bld.rpc_negotiation_timeout_ms_),
+    sasl_proto_name_(bld.sasl_proto_name_),
+    keytab_file_(bld.keytab_file_),
+    reuseport_(bld.reuseport_),
+    retain_self_(this) {
+  for (int i = 0; i < bld.num_reactors_; i++) {
+    reactors_.push_back(new Reactor(retain_self_, i, bld));
+  }
+  CHECK_OK(ThreadPoolBuilder("client-negotiator")
+      .set_min_threads(bld.min_negotiation_threads_)
+      .set_max_threads(bld.max_negotiation_threads_)
+      .Build(&client_negotiation_pool_));
+  CHECK_OK(ThreadPoolBuilder("server-negotiator")
+      .set_min_threads(bld.min_negotiation_threads_)
+      .set_max_threads(bld.max_negotiation_threads_)
+      .Build(&server_negotiation_pool_));
+}
+
+Messenger::~Messenger() {
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  CHECK(closing_) << "Should have already shut down";
+  STLDeleteElements(&reactors_);
+}
+
+Reactor* Messenger::RemoteToReactor(const Sockaddr &remote) {
+  uint32_t hashCode = remote.HashCode();
+  int reactor_idx = hashCode % reactors_.size();
+  // This is just a static partitioning; we could get a lot
+  // fancier with assigning Sockaddrs to Reactors.
+  return reactors_[reactor_idx];
+}
+
+Status Messenger::Init() {
+  RETURN_NOT_OK(tls_context_->Init());
+  for (Reactor* r : reactors_) {
+    RETURN_NOT_OK(r->Init());
+  }
+
+  return Status::OK();
+}
+
+Status Messenger::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                                  DumpRunningRpcsResponsePB* resp) {
+  shared_lock<rw_spinlock> guard(lock_.get_lock());
+  for (Reactor* reactor : reactors_) {
+    RETURN_NOT_OK(reactor->DumpRunningRpcs(req, resp));
+  }
+  return Status::OK();
+}
+
+void Messenger::ScheduleOnReactor(const boost::function<void(const Status&)>& func,
+                                  MonoDelta when) {
+  DCHECK(!reactors_.empty());
+
+  // If we're already running on a reactor thread, reuse it.
+  Reactor* chosen = nullptr;
+  for (Reactor* r : reactors_) {
+    if (r->IsCurrentThread()) {
+      chosen = r;
+    }
+  }
+  if (chosen == nullptr) {
+    // Not running on a reactor thread, pick one at random.
+    chosen = reactors_[rand() % reactors_.size()];
+  }
+
+  DelayedTask* task = new DelayedTask(func, when);
+  chosen->ScheduleReactorTask(task);
+}
+
+const scoped_refptr<RpcService> Messenger::rpc_service(const string& service_name) const {
+  scoped_refptr<RpcService> service;
+  {
+    shared_lock<rw_spinlock> guard(lock_.get_lock());
+    if (!FindCopy(rpc_services_, service_name, &service)) {
+      return scoped_refptr<RpcService>(nullptr);
+    }
+  }
+  return service;
+}
+
+ThreadPool* Messenger::negotiation_pool(Connection::Direction dir) {
+  switch (dir) {
+    case Connection::CLIENT: return client_negotiation_pool_.get();
+    case Connection::SERVER: return server_negotiation_pool_.get();
+  }
+  DCHECK(false) << "Unknown Connection::Direction value: " << dir;
+  return nullptr;
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/messenger.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/messenger.h b/be/src/kudu/rpc/messenger.h
new file mode 100644
index 0000000..64a804b
--- /dev/null
+++ b/be/src/kudu/rpc/messenger.h
@@ -0,0 +1,460 @@
+// 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.
+#ifndef KUDU_RPC_MESSENGER_H
+#define KUDU_RPC_MESSENGER_H
+
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/status.h"
+
+namespace boost {
+template <typename Signature>
+class function;
+}
+
+namespace kudu {
+
+class Socket;
+class ThreadPool;
+
+namespace security {
+class TlsContext;
+class TokenVerifier;
+}
+
+namespace rpc {
+
+using security::RpcAuthentication;
+using security::RpcEncryption;
+
+class AcceptorPool;
+class DumpRunningRpcsRequestPB;
+class DumpRunningRpcsResponsePB;
+class InboundCall;
+class Messenger;
+class OutboundCall;
+class Reactor;
+class RpcService;
+class RpczStore;
+
+struct AcceptorPoolInfo {
+ public:
+  explicit AcceptorPoolInfo(Sockaddr bind_address)
+      : bind_address_(bind_address) {}
+
+  Sockaddr bind_address() const {
+    return bind_address_;
+  }
+
+ private:
+  Sockaddr bind_address_;
+};
+
+// Used to construct a Messenger.
+class MessengerBuilder {
+ public:
+  friend class Messenger;
+  friend class ReactorThread;
+
+  explicit MessengerBuilder(std::string name);
+
+  // Set the length of time we will keep a TCP connection will alive with no traffic.
+  MessengerBuilder &set_connection_keepalive_time(const MonoDelta &keepalive);
+
+  // Set the number of reactor threads that will be used for sending and
+  // receiving.
+  MessengerBuilder &set_num_reactors(int num_reactors);
+
+  // Set the minimum number of connection-negotiation threads that will be used
+  // to handle the blocking connection-negotiation step.
+  MessengerBuilder &set_min_negotiation_threads(int min_negotiation_threads);
+
+  // Set the maximum number of connection-negotiation threads that will be used
+  // to handle the blocking connection-negotiation step.
+  MessengerBuilder &set_max_negotiation_threads(int max_negotiation_threads);
+
+  // Set the granularity with which connections are checked for keepalive.
+  MessengerBuilder &set_coarse_timer_granularity(const MonoDelta &granularity);
+
+  // Set metric entity for use by RPC systems.
+  MessengerBuilder &set_metric_entity(const scoped_refptr<MetricEntity>& metric_entity);
+
+  // Set the time in milliseconds after which an idle connection from a client will be
+  // disconnected by the server.
+  MessengerBuilder &set_connection_keep_alive_time(int32_t time_in_ms);
+
+  // Set the timeout for negotiating an RPC connection.
+  MessengerBuilder &set_rpc_negotiation_timeout_ms(int64_t time_in_ms);
+
+  // Set the SASL protocol name that is used for the SASL negotiation.
+  MessengerBuilder &set_sasl_proto_name(const std::string& sasl_proto_name);
+
+  // Set the state of authentication required. If 'optional', authentication will be used when
+  // the remote end supports it. If 'required', connections which are not able to authenticate
+  // (because the remote end lacks support) are rejected.
+  MessengerBuilder &set_rpc_authentication(const std::string& rpc_authentication);
+
+  // Set the state of encryption required. If 'optional', encryption will be used when the
+  // remote end supports it. If 'required', connections which are not able to use encryption
+  // (because the remote end lacks support) are rejected. If 'disabled', encryption will not
+  // be used, and RPC authentication (--rpc_authentication) must also be disabled as well.
+  MessengerBuilder &set_rpc_encryption(const std::string& rpc_encryption);
+
+  // Set the cipher suite preferences to use for TLS-secured RPC connections. Uses the OpenSSL
+  // cipher preference list format. See man (1) ciphers for more information.
+  MessengerBuilder &set_rpc_tls_ciphers(const std::string& rpc_tls_ciphers);
+
+  // Set the minimum protocol version to allow when for securing RPC connections with TLS. May be
+  // one of 'TLSv1', 'TLSv1.1', or 'TLSv1.2'.
+  MessengerBuilder &set_rpc_tls_min_protocol(const std::string& rpc_tls_min_protocol);
+
+  // Set the TLS server certificate and private key files paths. If this is set in conjunction
+  // with enable_inbound_tls(), internal PKI will not be used for encrypted communication and
+  // external PKI will be used instead.
+  MessengerBuilder &set_epki_cert_key_files(
+      const std::string& cert, const std::string& private_key);
+
+  // Set the TLS Certificate Authority file path. Must always be set with set_epki_cert_key_files().
+  // If this is set in conjunction with enable_inbound_tls(), internal PKI will not be used for
+  // encrypted communication and external PKI will be used instead.
+  MessengerBuilder &set_epki_certificate_authority_file(const std::string& ca);
+
+  // Set a Unix command whose output returns the password used to decrypt the RPC server's private
+  // key file specified via set_epki_cert_key_files(). If the .PEM key file is not
+  // password-protected, this flag does not need to be set. Trailing whitespace will be trimmed
+  // before it is used to decrypt the private key.
+  MessengerBuilder &set_epki_private_password_key_cmd(const std::string& cmd);
+
+  // Set the path to the Kerberos Keytab file for this server.
+  MessengerBuilder &set_keytab_file(const std::string& keytab_file);
+
+  // Configure the messenger to enable TLS encryption on inbound connections.
+  MessengerBuilder& enable_inbound_tls();
+
+  // Configure the messenger to set the SO_REUSEPORT socket option.
+  MessengerBuilder& set_reuseport();
+
+  Status Build(std::shared_ptr<Messenger> *msgr);
+
+ private:
+  const std::string name_;
+  MonoDelta connection_keepalive_time_;
+  int num_reactors_;
+  int min_negotiation_threads_;
+  int max_negotiation_threads_;
+  MonoDelta coarse_timer_granularity_;
+  scoped_refptr<MetricEntity> metric_entity_;
+  int64_t rpc_negotiation_timeout_ms_;
+  std::string sasl_proto_name_;
+  std::string rpc_authentication_;
+  std::string rpc_encryption_;
+  std::string rpc_tls_ciphers_;
+  std::string rpc_tls_min_protocol_;
+  std::string rpc_certificate_file_;
+  std::string rpc_private_key_file_;
+  std::string rpc_ca_certificate_file_;
+  std::string rpc_private_key_password_cmd_;
+  std::string keytab_file_;
+  bool enable_inbound_tls_;
+  bool reuseport_;
+};
+
+// A Messenger is a container for the reactor threads which run event loops
+// for the RPC services. If the process is a server, a Messenger can also have
+// one or more attached AcceptorPools which accept RPC connections. In this case,
+// calls received over the connection are enqueued into the messenger's service_queue
+// for processing by a ServicePool.
+//
+// Users do not typically interact with the Messenger directly except to create
+// one as a singleton, and then make calls using Proxy objects.
+//
+// See rpc-test.cc and rpc-bench.cc for example usages.
+class Messenger {
+ public:
+  friend class MessengerBuilder;
+  friend class Proxy;
+  friend class Reactor;
+  friend class ReactorThread;
+  typedef std::vector<std::shared_ptr<AcceptorPool> > acceptor_vec_t;
+  typedef std::unordered_map<std::string, scoped_refptr<RpcService> > RpcServicesMap;
+
+  static const uint64_t UNKNOWN_CALL_ID = 0;
+
+  ~Messenger();
+
+  // Stops all communication and prevents further use. If called explicitly,
+  // also waits for outstanding tasks running on reactor threads to finish,
+  // which means it may  not be called from a reactor task.
+  //
+  // It's not required to call this -- dropping the shared_ptr provided
+  // from MessengerBuilder::Build will automatically call this method.
+  void Shutdown();
+
+  // Add a new acceptor pool listening to the given accept address.
+  // You can create any number of acceptor pools you want, including none.
+  //
+  // The created pool is returned in *pool. The Messenger also retains
+  // a reference to the pool, so the caller may safely drop this reference
+  // and the pool will remain live.
+  //
+  // NOTE: the returned pool is not initially started. You must call
+  // pool->Start(...) to begin accepting connections.
+  //
+  // If Kerberos is enabled, this also runs a pre-flight check that makes
+  // sure the environment is appropriately configured to authenticate
+  // clients via Kerberos. If not, this returns a RuntimeError.
+  Status AddAcceptorPool(const Sockaddr &accept_addr,
+                         std::shared_ptr<AcceptorPool>* pool);
+
+  // Register a new RpcService to handle inbound requests.
+  //
+  // Returns an error if a service with the same name is already registered.
+  Status RegisterService(const std::string& service_name,
+                         const scoped_refptr<RpcService>& service);
+
+  // Unregister an RpcService by name.
+  //
+  // Returns an error if no service with this name can be found.
+  Status UnregisterService(const std::string& service_name);
+
+  // Unregisters all RPC services.
+  void UnregisterAllServices();
+
+  // Queue a call for transmission. This will pick the appropriate reactor,
+  // and enqueue a task on that reactor to assign and send the call.
+  void QueueOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // Enqueue a call for processing on the server.
+  void QueueInboundCall(gscoped_ptr<InboundCall> call);
+
+  // Queue a cancellation for the given outbound call.
+  void QueueCancellation(const std::shared_ptr<OutboundCall> &call);
+
+  // Take ownership of the socket via Socket::Release
+  void RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote);
+
+  // Dump the current RPCs into the given protobuf.
+  Status DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                         DumpRunningRpcsResponsePB* resp);
+
+  // Run 'func' on a reactor thread after 'when' time elapses.
+  //
+  // The status argument conveys whether 'func' was run correctly (i.e.
+  // after the elapsed time) or not.
+  void ScheduleOnReactor(const boost::function<void(const Status&)>& func,
+                         MonoDelta when);
+
+  const security::TlsContext& tls_context() const { return *tls_context_; }
+  security::TlsContext* mutable_tls_context() { return tls_context_.get(); }
+
+  const security::TokenVerifier& token_verifier() const { return *token_verifier_; }
+  security::TokenVerifier* mutable_token_verifier() { return token_verifier_.get(); }
+  std::shared_ptr<security::TokenVerifier> shared_token_verifier() const {
+    return token_verifier_;
+  }
+
+  boost::optional<security::SignedTokenPB> authn_token() const {
+    std::lock_guard<simple_spinlock> l(authn_token_lock_);
+    return authn_token_;
+  }
+  void set_authn_token(const security::SignedTokenPB& token) {
+    std::lock_guard<simple_spinlock> l(authn_token_lock_);
+    authn_token_ = token;
+  }
+
+  RpcAuthentication authentication() const { return authentication_; }
+  RpcEncryption encryption() const { return encryption_; }
+
+  ThreadPool* negotiation_pool(Connection::Direction dir);
+
+  RpczStore* rpcz_store() { return rpcz_store_.get(); }
+
+  int num_reactors() const { return reactors_.size(); }
+
+  const std::string& name() const {
+    return name_;
+  }
+
+  bool closing() const {
+    shared_lock<rw_spinlock> l(lock_.get_lock());
+    return closing_;
+  }
+
+  scoped_refptr<MetricEntity> metric_entity() const { return metric_entity_; }
+
+  const int64_t rpc_negotiation_timeout_ms() const { return rpc_negotiation_timeout_ms_; }
+
+  const std::string& sasl_proto_name() const {
+    return sasl_proto_name_;
+  }
+
+  const std::string& keytab_file() const { return keytab_file_; }
+
+  const scoped_refptr<RpcService> rpc_service(const std::string& service_name) const;
+
+ private:
+  FRIEND_TEST(TestRpc, TestConnectionKeepalive);
+  FRIEND_TEST(TestRpc, TestConnectionAlwaysKeepalive);
+  FRIEND_TEST(TestRpc, TestClientConnectionsMetrics);
+  FRIEND_TEST(TestRpc, TestCredentialsPolicy);
+  FRIEND_TEST(TestRpc, TestReopenOutboundConnections);
+
+  explicit Messenger(const MessengerBuilder &bld);
+
+  Reactor* RemoteToReactor(const Sockaddr &remote);
+  Status Init();
+  void RunTimeoutThread();
+  void UpdateCurTime();
+
+  // Shuts down the messenger.
+  //
+  // Depending on 'mode', may or may not wait on any outstanding reactor tasks.
+  enum class ShutdownMode {
+    SYNC,
+    ASYNC,
+  };
+  void ShutdownInternal(ShutdownMode mode);
+
+  // Called by external-facing shared_ptr when the user no longer holds
+  // any references. See 'retain_self_' for more info.
+  void AllExternalReferencesDropped();
+
+  const std::string name_;
+
+  // Protects closing_, acceptor_pools_, rpc_services_.
+  mutable percpu_rwlock lock_;
+
+  bool closing_;
+
+  // Whether to require authentication and encryption on the connections managed
+  // by this messenger.
+  // TODO(KUDU-1928): scope these to individual proxies, so that messengers can be
+  // reused by different clients.
+  RpcAuthentication authentication_;
+  RpcEncryption encryption_;
+
+  // Pools which are listening on behalf of this messenger.
+  // Note that the user may have called Shutdown() on one of these
+  // pools, so even though we retain the reference, it may no longer
+  // be listening.
+  acceptor_vec_t acceptor_pools_;
+
+  // RPC services that handle inbound requests.
+  RpcServicesMap rpc_services_;
+
+  std::vector<Reactor*> reactors_;
+
+  // Separate client and server negotiation pools to avoid possibility of distributed
+  // deadlock. See KUDU-2041.
+  gscoped_ptr<ThreadPool> client_negotiation_pool_;
+  gscoped_ptr<ThreadPool> server_negotiation_pool_;
+
+  std::unique_ptr<security::TlsContext> tls_context_;
+
+  // A TokenVerifier, which can verify client provided authentication tokens.
+  std::shared_ptr<security::TokenVerifier> token_verifier_;
+
+  // An optional token, which can be used to authenticate to a server.
+  mutable simple_spinlock authn_token_lock_;
+  boost::optional<security::SignedTokenPB> authn_token_;
+
+  std::unique_ptr<RpczStore> rpcz_store_;
+
+  scoped_refptr<MetricEntity> metric_entity_;
+
+  // Timeout in milliseconds after which an incomplete connection negotiation will timeout.
+  const int64_t rpc_negotiation_timeout_ms_;
+
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
+  // Path to the Kerberos Keytab file for this server.
+  const std::string keytab_file_;
+
+  // Whether to set SO_REUSEPORT on the listening sockets.
+  bool reuseport_;
+
+  // The ownership of the Messenger object is somewhat subtle. The pointer graph
+  // looks like this:
+  //
+  //    [User Code ]             |      [ Internal code ]
+  //                             |
+  //     shared_ptr[1]           |
+  //         |                   |
+  //         v
+  //      Messenger    <------------ shared_ptr[2] --- Reactor
+  //       ^    |      ------------- bare pointer  --> Reactor
+  //        \__/
+  //     shared_ptr[2]
+  //     (retain_self_)
+  //
+  // shared_ptr[1] instances use Messenger::AllExternalReferencesDropped()
+  //   as a deleter.
+  // shared_ptr[2] are "traditional" shared_ptrs which call 'delete' on the
+  //   object.
+  //
+  // The teardown sequence is as follows:
+  // Option 1): User calls "Shutdown()" explicitly:
+  //  - Messenger::Shutdown tells Reactors to shut down.
+  //  - When each reactor thread finishes, it drops its shared_ptr[2].
+  //  - the Messenger::retain_self instance remains, keeping the Messenger
+  //    alive.
+  //  - Before returning, Messenger::Shutdown waits for Reactors to shut down.
+  //  - The user eventually drops its shared_ptr[1], which calls
+  //    Messenger::AllExternalReferencesDropped. This drops retain_self_
+  //    and results in object destruction.
+  // Option 2): User drops all of its shared_ptr[1] references
+  //  - Though the Reactors still reference the Messenger, AllExternalReferencesDropped
+  //    will get called, which triggers Messenger::Shutdown.
+  //  - AllExternalReferencesDropped drops retain_self_, so the only remaining
+  //    references are from Reactor threads. But the reactor threads are shutting down.
+  //  - When the last Reactor thread dies, there will be no more shared_ptr[1] references
+  //    and the Messenger will be destroyed.
+  //
+  // The main goal of all of this confusion is that when using option 2, the
+  // reactor threads need to be able to shut down asynchronously, and we need
+  // to keep the Messenger alive until they do so. If normal shared_ptrs were
+  // handed out to users, the Messenger destructor may be forced to Join() the
+  // reactor threads, which deadlocks if the user destructs the Messenger from
+  // within a Reactor thread itself.
+  std::shared_ptr<Messenger> retain_self_;
+
+  DISALLOW_COPY_AND_ASSIGN(Messenger);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/mt-rpc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/mt-rpc-test.cc b/be/src/kudu/rpc/mt-rpc-test.cc
new file mode 100644
index 0000000..7427850
--- /dev/null
+++ b/be/src/kudu/rpc/mt-rpc-test.cc
@@ -0,0 +1,318 @@
+// 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 "kudu/rpc/rpc-test-base.h"
+
+#include <cstddef>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/proxy.h"
+#include "kudu/rpc/rpc_service.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/service_pool.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/thread.h"
+
+
+METRIC_DECLARE_counter(rpc_connections_accepted);
+METRIC_DECLARE_counter(rpcs_queue_overflow);
+
+using std::string;
+using std::shared_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+class MultiThreadedRpcTest : public RpcTestBase {
+ public:
+  // Make a single RPC call.
+  void SingleCall(Sockaddr server_addr, const char* method_name,
+                  Status* result, CountDownLatch* latch) {
+    LOG(INFO) << "Connecting to " << server_addr.ToString();
+    shared_ptr<Messenger> client_messenger;
+    CHECK_OK(CreateMessenger("ClientSC", &client_messenger));
+    Proxy p(client_messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
+    *result = DoTestSyncCall(p, method_name);
+    latch->CountDown();
+  }
+
+  // Make RPC calls until we see a failure.
+  void HammerServer(Sockaddr server_addr, const char* method_name,
+                    Status* last_result) {
+    shared_ptr<Messenger> client_messenger;
+    CHECK_OK(CreateMessenger("ClientHS", &client_messenger));
+    HammerServerWithMessenger(server_addr, method_name, last_result, client_messenger);
+  }
+
+  void HammerServerWithMessenger(
+      Sockaddr server_addr, const char* method_name, Status* last_result,
+      const shared_ptr<Messenger>& messenger) {
+    LOG(INFO) << "Connecting to " << server_addr.ToString();
+    Proxy p(messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
+
+    int i = 0;
+    while (true) {
+      i++;
+      Status s = DoTestSyncCall(p, method_name);
+      if (!s.ok()) {
+        // Return on first failure.
+        LOG(INFO) << "Call failed. Shutting down client thread. Ran " << i << " calls: "
+            << s.ToString();
+        *last_result = s;
+        return;
+      }
+    }
+  }
+};
+
+static void AssertShutdown(kudu::Thread* thread, const Status* status) {
+  ASSERT_OK(ThreadJoiner(thread).warn_every_ms(500).Join());
+  string msg = status->ToString();
+  ASSERT_TRUE(msg.find("Service unavailable") != string::npos ||
+              msg.find("Network error") != string::npos)
+              << "Status is actually: " << msg;
+}
+
+// Test making several concurrent RPC calls while shutting down.
+// Simply verify that we don't hit any CHECK errors.
+TEST_F(MultiThreadedRpcTest, TestShutdownDuringService) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  const int kNumThreads = 4;
+  scoped_refptr<kudu::Thread> threads[kNumThreads];
+  Status statuses[kNumThreads];
+  for (int i = 0; i < kNumThreads; i++) {
+    ASSERT_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+      &MultiThreadedRpcTest::HammerServer, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &statuses[i], &threads[i]));
+  }
+
+  SleepFor(MonoDelta::FromMilliseconds(50));
+
+  // Shut down server.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (int i = 0; i < kNumThreads; i++) {
+    AssertShutdown(threads[i].get(), &statuses[i]);
+  }
+}
+
+// Test shutting down the client messenger exactly as a thread is about to start
+// a new connection. This is a regression test for KUDU-104.
+TEST_F(MultiThreadedRpcTest, TestShutdownClientWhileCallsPending) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger));
+
+  scoped_refptr<kudu::Thread> thread;
+  Status status;
+  ASSERT_OK(kudu::Thread::Create("test", "test",
+      &MultiThreadedRpcTest::HammerServerWithMessenger, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &status, client_messenger, &thread));
+
+  // Shut down the messenger after a very brief sleep. This often will race so that the
+  // call gets submitted to the messenger before shutdown, but the negotiation won't have
+  // started yet. In a debug build this fails about half the time without the bug fix.
+  // See KUDU-104.
+  SleepFor(MonoDelta::FromMicroseconds(10));
+  client_messenger->Shutdown();
+  client_messenger.reset();
+
+  ASSERT_OK(ThreadJoiner(thread.get()).warn_every_ms(500).Join());
+  ASSERT_TRUE(status.IsAborted() ||
+              status.IsServiceUnavailable());
+  string msg = status.ToString();
+  SCOPED_TRACE(msg);
+  ASSERT_TRUE(msg.find("Client RPC Messenger shutting down") != string::npos ||
+              msg.find("reactor is shutting down") != string::npos ||
+              msg.find("Unable to start connection negotiation thread") != string::npos)
+              << "Status is actually: " << msg;
+}
+
+// This bogus service pool leaves the service queue full.
+class BogusServicePool : public ServicePool {
+ public:
+  BogusServicePool(gscoped_ptr<ServiceIf> service,
+                   const scoped_refptr<MetricEntity>& metric_entity,
+                   size_t service_queue_length)
+    : ServicePool(std::move(service), metric_entity, service_queue_length) {
+  }
+  virtual Status Init(int num_threads) OVERRIDE {
+    // Do nothing
+    return Status::OK();
+  }
+};
+
+void IncrementBackpressureOrShutdown(const Status* status, int* backpressure, int* shutdown) {
+  string msg = status->ToString();
+  if (msg.find("service queue is full") != string::npos) {
+    ++(*backpressure);
+  } else if (msg.find("shutting down") != string::npos) {
+    ++(*shutdown);
+  } else if (msg.find("got EOF from remote") != string::npos) {
+    ++(*shutdown);
+  } else {
+    FAIL() << "Unexpected status message: " << msg;
+  }
+}
+
+// Test that we get a Service Unavailable error when we max out the incoming RPC service queue.
+TEST_F(MultiThreadedRpcTest, TestBlowOutServiceQueue) {
+  const size_t kMaxConcurrency = 2;
+
+  MessengerBuilder bld("messenger1");
+  bld.set_num_reactors(kMaxConcurrency);
+  bld.set_metric_entity(metric_entity_);
+  CHECK_OK(bld.Build(&server_messenger_));
+
+  shared_ptr<AcceptorPool> pool;
+  ASSERT_OK(server_messenger_->AddAcceptorPool(Sockaddr(), &pool));
+  ASSERT_OK(pool->Start(kMaxConcurrency));
+  Sockaddr server_addr = pool->bind_address();
+
+  gscoped_ptr<ServiceIf> service(new GenericCalculatorService());
+  service_name_ = service->service_name();
+  service_pool_ = new BogusServicePool(std::move(service),
+                                      server_messenger_->metric_entity(),
+                                      kMaxConcurrency);
+  ASSERT_OK(service_pool_->Init(n_worker_threads_));
+  server_messenger_->RegisterService(service_name_, service_pool_);
+
+  scoped_refptr<kudu::Thread> threads[3];
+  Status status[3];
+  CountDownLatch latch(1);
+  for (int i = 0; i < 3; i++) {
+    ASSERT_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+      &MultiThreadedRpcTest::SingleCall, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &status[i], &latch, &threads[i]));
+  }
+
+  // One should immediately fail due to backpressure. The latch is only initialized
+  // to wait for the first of three threads to finish.
+  latch.Wait();
+
+  // The rest would time out after 10 sec, but we help them along.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (const auto& thread : threads) {
+    ASSERT_OK(ThreadJoiner(thread.get()).warn_every_ms(500).Join());
+  }
+
+  // Verify that one error was due to backpressure.
+  int errors_backpressure = 0;
+  int errors_shutdown = 0;
+
+  for (const auto& s : status) {
+    IncrementBackpressureOrShutdown(&s, &errors_backpressure, &errors_shutdown);
+  }
+
+  ASSERT_EQ(1, errors_backpressure);
+  ASSERT_EQ(2, errors_shutdown);
+
+  // Check that RPC queue overflow metric is 1
+  Counter *rpcs_queue_overflow =
+    METRIC_rpcs_queue_overflow.Instantiate(server_messenger_->metric_entity()).get();
+  ASSERT_EQ(1, rpcs_queue_overflow->value());
+}
+
+static void HammerServerWithTCPConns(const Sockaddr& addr) {
+  while (true) {
+    Socket socket;
+    CHECK_OK(socket.Init(0));
+    Status s;
+    LOG_SLOW_EXECUTION(INFO, 100, "Connect took long") {
+      s = socket.Connect(addr);
+    }
+    if (!s.ok()) {
+      CHECK(s.IsNetworkError()) << "Unexpected error: " << s.ToString();
+      return;
+    }
+    CHECK_OK(socket.Close());
+  }
+}
+
+// Regression test for KUDU-128.
+// Test that shuts down the server while new TCP connections are incoming.
+TEST_F(MultiThreadedRpcTest, TestShutdownWithIncomingConnections) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  // Start a number of threads which just hammer the server with TCP connections.
+  vector<scoped_refptr<kudu::Thread> > threads;
+  for (int i = 0; i < 8; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+        &HammerServerWithTCPConns, server_addr, &new_thread));
+    threads.push_back(new_thread);
+  }
+
+  // Sleep until the server has started to actually accept some connections from the
+  // test threads.
+  scoped_refptr<Counter> conns_accepted =
+    METRIC_rpc_connections_accepted.Instantiate(server_messenger_->metric_entity());
+  while (conns_accepted->value() == 0) {
+    SleepFor(MonoDelta::FromMicroseconds(100));
+  }
+
+  // Shutdown while there are still new connections appearing.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (scoped_refptr<kudu::Thread>& t : threads) {
+    ASSERT_OK(ThreadJoiner(t.get()).warn_every_ms(500).Join());
+  }
+}
+
+} // namespace rpc
+} // namespace kudu
+


[46/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
IMPALA-7006: Add KRPC folders from kudu@334ecafd

cp -a ~/checkout/kudu/src/kudu/{rpc,util,security} be/src/kudu/

Change-Id: I232db2b4ccf5df9aca87b21dea31bfb2735d1ab7
Reviewed-on: http://gerrit.cloudera.org:8080/10757
Reviewed-by: Lars Volker <lv...@cloudera.com>
Tested-by: Lars Volker <lv...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/fcf190c4
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/fcf190c4
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/fcf190c4

Branch: refs/heads/master
Commit: fcf190c4de1fcc291a5356634fd7cd12efa64852
Parents: 39870d4
Author: Lars Volker <lv...@cloudera.com>
Authored: Tue Jul 3 15:10:52 2018 -0700
Committer: Lars Volker <lv...@cloudera.com>
Committed: Thu Jul 12 21:35:42 2018 +0000

----------------------------------------------------------------------
 be/src/kudu/rpc/CMakeLists.txt                  |  138 +
 be/src/kudu/rpc/acceptor_pool.cc                |  175 ++
 be/src/kudu/rpc/acceptor_pool.h                 |   84 +
 be/src/kudu/rpc/blocking_ops.cc                 |  126 +
 be/src/kudu/rpc/blocking_ops.h                  |   58 +
 be/src/kudu/rpc/client_negotiation.cc           |  853 ++++++
 be/src/kudu/rpc/client_negotiation.h            |  263 ++
 be/src/kudu/rpc/connection.cc                   |  767 ++++++
 be/src/kudu/rpc/connection.h                    |  391 +++
 be/src/kudu/rpc/connection_id.cc                |   85 +
 be/src/kudu/rpc/connection_id.h                 |   84 +
 be/src/kudu/rpc/constants.cc                    |   37 +
 be/src/kudu/rpc/constants.h                     |   60 +
 be/src/kudu/rpc/exactly_once_rpc-test.cc        |  629 +++++
 be/src/kudu/rpc/inbound_call.cc                 |  345 +++
 be/src/kudu/rpc/inbound_call.h                  |  286 ++
 be/src/kudu/rpc/messenger.cc                    |  502 ++++
 be/src/kudu/rpc/messenger.h                     |  460 ++++
 be/src/kudu/rpc/mt-rpc-test.cc                  |  318 +++
 be/src/kudu/rpc/negotiation-test.cc             | 1346 ++++++++++
 be/src/kudu/rpc/negotiation.cc                  |  327 +++
 be/src/kudu/rpc/negotiation.h                   |   58 +
 be/src/kudu/rpc/outbound_call.cc                |  531 ++++
 be/src/kudu/rpc/outbound_call.h                 |  348 +++
 be/src/kudu/rpc/periodic-test.cc                |  295 +++
 be/src/kudu/rpc/periodic.cc                     |  219 ++
 be/src/kudu/rpc/periodic.h                      |  215 ++
 be/src/kudu/rpc/protoc-gen-krpc.cc              |  691 +++++
 be/src/kudu/rpc/proxy.cc                        |  116 +
 be/src/kudu/rpc/proxy.h                         |  126 +
 be/src/kudu/rpc/reactor-test.cc                 |  112 +
 be/src/kudu/rpc/reactor.cc                      |  918 +++++++
 be/src/kudu/rpc/reactor.h                       |  427 +++
 be/src/kudu/rpc/remote_method.cc                |   53 +
 be/src/kudu/rpc/remote_method.h                 |   51 +
 be/src/kudu/rpc/remote_user.cc                  |   40 +
 be/src/kudu/rpc/remote_user.h                   |   99 +
 be/src/kudu/rpc/request_tracker-test.cc         |   86 +
 be/src/kudu/rpc/request_tracker.cc              |   55 +
 be/src/kudu/rpc/request_tracker.h               |   87 +
 be/src/kudu/rpc/response_callback.h             |   31 +
 be/src/kudu/rpc/result_tracker.cc               |  595 +++++
 be/src/kudu/rpc/result_tracker.h                |  401 +++
 be/src/kudu/rpc/retriable_rpc.h                 |  296 +++
 be/src/kudu/rpc/rpc-bench.cc                    |  298 +++
 be/src/kudu/rpc/rpc-test-base.h                 |  661 +++++
 be/src/kudu/rpc/rpc-test.cc                     | 1364 ++++++++++
 be/src/kudu/rpc/rpc.cc                          |  101 +
 be/src/kudu/rpc/rpc.h                           |  221 ++
 be/src/kudu/rpc/rpc_context.cc                  |  217 ++
 be/src/kudu/rpc/rpc_context.h                   |  245 ++
 be/src/kudu/rpc/rpc_controller.cc               |  177 ++
 be/src/kudu/rpc/rpc_controller.h                |  282 ++
 be/src/kudu/rpc/rpc_header.proto                |  365 +++
 be/src/kudu/rpc/rpc_introspection.proto         |  110 +
 be/src/kudu/rpc/rpc_service.h                   |   47 +
 be/src/kudu/rpc/rpc_sidecar.cc                  |  115 +
 be/src/kudu/rpc/rpc_sidecar.h                   |   73 +
 be/src/kudu/rpc/rpc_stub-test.cc                |  726 ++++++
 be/src/kudu/rpc/rpcz_store.cc                   |  272 ++
 be/src/kudu/rpc/rpcz_store.h                    |   74 +
 be/src/kudu/rpc/rtest.proto                     |  160 ++
 be/src/kudu/rpc/rtest_diff_package.proto        |   26 +
 be/src/kudu/rpc/sasl_common.cc                  |  470 ++++
 be/src/kudu/rpc/sasl_common.h                   |  158 ++
 be/src/kudu/rpc/sasl_helper.cc                  |  134 +
 be/src/kudu/rpc/sasl_helper.h                   |  109 +
 be/src/kudu/rpc/serialization.cc                |  223 ++
 be/src/kudu/rpc/serialization.h                 |   88 +
 be/src/kudu/rpc/server_negotiation.cc           |  989 +++++++
 be/src/kudu/rpc/server_negotiation.h            |  259 ++
 be/src/kudu/rpc/service_if.cc                   |  160 ++
 be/src/kudu/rpc/service_if.h                    |  134 +
 be/src/kudu/rpc/service_pool.cc                 |  234 ++
 be/src/kudu/rpc/service_pool.h                  |  117 +
 be/src/kudu/rpc/service_queue-test.cc           |  151 ++
 be/src/kudu/rpc/service_queue.cc                |  145 ++
 be/src/kudu/rpc/service_queue.h                 |  225 ++
 be/src/kudu/rpc/transfer.cc                     |  283 ++
 be/src/kudu/rpc/transfer.h                      |  212 ++
 be/src/kudu/rpc/user_credentials.cc             |   64 +
 be/src/kudu/rpc/user_credentials.h              |   53 +
 be/src/kudu/security/CMakeLists.txt             |  141 +
 be/src/kudu/security/ca/cert_management-test.cc |  294 +++
 be/src/kudu/security/ca/cert_management.cc      |  423 +++
 be/src/kudu/security/ca/cert_management.h       |  226 ++
 be/src/kudu/security/cert-test.cc               |  165 ++
 be/src/kudu/security/cert.cc                    |  301 +++
 be/src/kudu/security/cert.h                     |  119 +
 be/src/kudu/security/crypto-test.cc             |  257 ++
 be/src/kudu/security/crypto.cc                  |  276 ++
 be/src/kudu/security/crypto.h                   |  103 +
 be/src/kudu/security/init.cc                    |  465 ++++
 be/src/kudu/security/init.h                     |   84 +
 be/src/kudu/security/kerberos_util.cc           |   37 +
 be/src/kudu/security/kerberos_util.h            |   29 +
 be/src/kudu/security/krb5_realm_override.cc     |  105 +
 be/src/kudu/security/openssl_util.cc            |  322 +++
 be/src/kudu/security/openssl_util.h             |  217 ++
 be/src/kudu/security/openssl_util_bio.h         |  129 +
 be/src/kudu/security/security-test-util.cc      |  103 +
 be/src/kudu/security/security-test-util.h       |   56 +
 be/src/kudu/security/security_flags.cc          |   42 +
 be/src/kudu/security/security_flags.h           |   36 +
 be/src/kudu/security/simple_acl.cc              |   89 +
 be/src/kudu/security/simple_acl.h               |   58 +
 be/src/kudu/security/test/mini_kdc-test.cc      |  144 ++
 be/src/kudu/security/test/mini_kdc.cc           |  315 +++
 be/src/kudu/security/test/mini_kdc.h            |  134 +
 be/src/kudu/security/test/test_certs.cc         |  969 +++++++
 be/src/kudu/security/test/test_certs.h          |   86 +
 be/src/kudu/security/test/test_pass.cc          |   40 +
 be/src/kudu/security/test/test_pass.h           |   37 +
 be/src/kudu/security/tls_context.cc             |  520 ++++
 be/src/kudu/security/tls_context.h              |  202 ++
 be/src/kudu/security/tls_handshake-test.cc      |  390 +++
 be/src/kudu/security/tls_handshake.cc           |  274 ++
 be/src/kudu/security/tls_handshake.h            |  171 ++
 be/src/kudu/security/tls_socket-test.cc         |  366 +++
 be/src/kudu/security/tls_socket.cc              |  185 ++
 be/src/kudu/security/tls_socket.h               |   60 +
 be/src/kudu/security/token-test.cc              |  677 +++++
 be/src/kudu/security/token.proto                |   97 +
 be/src/kudu/security/token_signer.cc            |  299 +++
 be/src/kudu/security/token_signer.h             |  316 +++
 be/src/kudu/security/token_signing_key.cc       |  110 +
 be/src/kudu/security/token_signing_key.h        |  103 +
 be/src/kudu/security/token_verifier.cc          |  173 ++
 be/src/kudu/security/token_verifier.h           |  126 +
 be/src/kudu/security/x509_check_host.cc         |  439 ++++
 be/src/kudu/security/x509_check_host.h          |   50 +
 be/src/kudu/util/CMakeLists.txt                 |  482 ++++
 be/src/kudu/util/alignment.h                    |   28 +
 be/src/kudu/util/array_view.h                   |  133 +
 be/src/kudu/util/async_logger.cc                |  151 ++
 be/src/kudu/util/async_logger.h                 |  206 ++
 be/src/kudu/util/async_util-test.cc             |  129 +
 be/src/kudu/util/async_util.h                   |   99 +
 be/src/kudu/util/atomic-test.cc                 |  135 +
 be/src/kudu/util/atomic.cc                      |   56 +
 be/src/kudu/util/atomic.h                       |  322 +++
 be/src/kudu/util/auto_release_pool.h            |   99 +
 be/src/kudu/util/barrier.h                      |   68 +
 be/src/kudu/util/bit-stream-utils.h             |  150 ++
 be/src/kudu/util/bit-stream-utils.inline.h      |  211 ++
 be/src/kudu/util/bit-util-test.cc               |   45 +
 be/src/kudu/util/bit-util.h                     |   57 +
 be/src/kudu/util/bitmap-test.cc                 |  230 ++
 be/src/kudu/util/bitmap.cc                      |  136 +
 be/src/kudu/util/bitmap.h                       |  219 ++
 be/src/kudu/util/blocking_queue-test.cc         |  249 ++
 be/src/kudu/util/blocking_queue.h               |  256 ++
 be/src/kudu/util/bloom_filter-test.cc           |   92 +
 be/src/kudu/util/bloom_filter.cc                |   89 +
 be/src/kudu/util/bloom_filter.h                 |  254 ++
 be/src/kudu/util/boost_mutex_utils.h            |   45 +
 be/src/kudu/util/cache-bench.cc                 |  191 ++
 be/src/kudu/util/cache-test.cc                  |  246 ++
 be/src/kudu/util/cache.cc                       |  572 ++++
 be/src/kudu/util/cache.h                        |  216 ++
 be/src/kudu/util/cache_metrics.cc               |   69 +
 be/src/kudu/util/cache_metrics.h                |   42 +
 be/src/kudu/util/callback_bind-test.cc          |  119 +
 be/src/kudu/util/coding-inl.h                   |  120 +
 be/src/kudu/util/coding.cc                      |  142 +
 be/src/kudu/util/coding.h                       |  113 +
 .../kudu/util/compression/compression-test.cc   |   90 +
 be/src/kudu/util/compression/compression.proto  |   29 +
 .../kudu/util/compression/compression_codec.cc  |  286 ++
 .../kudu/util/compression/compression_codec.h   |   78 +
 be/src/kudu/util/condition_variable.cc          |  142 +
 be/src/kudu/util/condition_variable.h           |  118 +
 be/src/kudu/util/countdown_latch-test.cc        |   74 +
 be/src/kudu/util/countdown_latch.h              |  137 +
 be/src/kudu/util/cow_object.cc                  |   34 +
 be/src/kudu/util/cow_object.h                   |  437 ++++
 be/src/kudu/util/crc-test.cc                    |  112 +
 be/src/kudu/util/crc.cc                         |   56 +
 be/src/kudu/util/crc.h                          |   43 +
 be/src/kudu/util/curl_util.cc                   |  130 +
 be/src/kudu/util/curl_util.h                    |   92 +
 be/src/kudu/util/debug-util-test.cc             |  458 ++++
 be/src/kudu/util/debug-util.cc                  |  800 ++++++
 be/src/kudu/util/debug-util.h                   |  321 +++
 be/src/kudu/util/debug/leak_annotations.h       |   84 +
 be/src/kudu/util/debug/leakcheck_disabler.h     |   48 +
 be/src/kudu/util/debug/sanitizer_scopes.h       |   47 +
 be/src/kudu/util/debug/trace_event.h            | 1501 +++++++++++
 be/src/kudu/util/debug/trace_event_impl.cc      | 2436 ++++++++++++++++++
 be/src/kudu/util/debug/trace_event_impl.h       |  726 ++++++
 .../util/debug/trace_event_impl_constants.cc    |   14 +
 be/src/kudu/util/debug/trace_event_memory.h     |   28 +
 .../util/debug/trace_event_synthetic_delay.cc   |  238 ++
 .../util/debug/trace_event_synthetic_delay.h    |  166 ++
 be/src/kudu/util/debug/trace_logging.h          |  132 +
 be/src/kudu/util/debug/unwind_safeness.cc       |  164 ++
 be/src/kudu/util/debug/unwind_safeness.h        |   29 +
 be/src/kudu/util/debug_ref_counted.h            |   56 +
 be/src/kudu/util/decimal_util-test.cc           |   81 +
 be/src/kudu/util/decimal_util.cc                |   89 +
 be/src/kudu/util/decimal_util.h                 |   69 +
 be/src/kudu/util/easy_json-test.cc              |  106 +
 be/src/kudu/util/easy_json.cc                   |  212 ++
 be/src/kudu/util/easy_json.h                    |  190 ++
 be/src/kudu/util/env-test.cc                    | 1173 +++++++++
 be/src/kudu/util/env.cc                         |   93 +
 be/src/kudu/util/env.h                          |  681 +++++
 be/src/kudu/util/env_posix.cc                   | 1852 +++++++++++++
 be/src/kudu/util/env_util-test.cc               |  192 ++
 be/src/kudu/util/env_util.cc                    |  320 +++
 be/src/kudu/util/env_util.h                     |  112 +
 be/src/kudu/util/errno-test.cc                  |   50 +
 be/src/kudu/util/errno.cc                       |   52 +
 be/src/kudu/util/errno.h                        |   36 +
 be/src/kudu/util/faststring-test.cc             |   65 +
 be/src/kudu/util/faststring.cc                  |   72 +
 be/src/kudu/util/faststring.h                   |  259 ++
 be/src/kudu/util/fault_injection.cc             |   78 +
 be/src/kudu/util/fault_injection.h              |   98 +
 be/src/kudu/util/file_cache-stress-test.cc      |  402 +++
 be/src/kudu/util/file_cache-test-util.h         |   92 +
 be/src/kudu/util/file_cache-test.cc             |  361 +++
 be/src/kudu/util/file_cache.cc                  |  654 +++++
 be/src/kudu/util/file_cache.h                   |  209 ++
 be/src/kudu/util/flag_tags-test.cc              |  135 +
 be/src/kudu/util/flag_tags.cc                   |   91 +
 be/src/kudu/util/flag_tags.h                    |  169 ++
 be/src/kudu/util/flag_validators-test.cc        |  252 ++
 be/src/kudu/util/flag_validators.cc             |   67 +
 be/src/kudu/util/flag_validators.h              |  102 +
 be/src/kudu/util/flags-test.cc                  |  109 +
 be/src/kudu/util/flags.cc                       |  604 +++++
 be/src/kudu/util/flags.h                        |   89 +
 be/src/kudu/util/group_varint-inl.h             |  294 +++
 be/src/kudu/util/group_varint-test.cc           |  144 ++
 be/src/kudu/util/group_varint.cc                |   81 +
 be/src/kudu/util/hash_util-test.cc              |   42 +
 be/src/kudu/util/hash_util.h                    |   71 +
 be/src/kudu/util/hdr_histogram-test.cc          |  116 +
 be/src/kudu/util/hdr_histogram.cc               |  501 ++++
 be/src/kudu/util/hdr_histogram.h                |  351 +++
 be/src/kudu/util/hexdump.cc                     |   85 +
 be/src/kudu/util/hexdump.h                      |   34 +
 be/src/kudu/util/high_water_mark.h              |   85 +
 be/src/kudu/util/histogram.proto                |   48 +
 be/src/kudu/util/init.cc                        |   89 +
 be/src/kudu/util/init.h                         |   33 +
 be/src/kudu/util/inline_slice-test.cc           |   88 +
 be/src/kudu/util/inline_slice.h                 |  181 ++
 be/src/kudu/util/int128-test.cc                 |   69 +
 be/src/kudu/util/int128.h                       |   46 +
 be/src/kudu/util/int128_util.h                  |   39 +
 be/src/kudu/util/interval_tree-inl.h            |  444 ++++
 be/src/kudu/util/interval_tree-test.cc          |  353 +++
 be/src/kudu/util/interval_tree.h                |  158 ++
 be/src/kudu/util/jsonreader-test.cc             |  193 ++
 be/src/kudu/util/jsonreader.cc                  |  141 +
 be/src/kudu/util/jsonreader.h                   |   92 +
 be/src/kudu/util/jsonwriter-test.cc             |  216 ++
 be/src/kudu/util/jsonwriter.cc                  |  352 +++
 be/src/kudu/util/jsonwriter.h                   |  102 +
 be/src/kudu/util/jsonwriter_test.proto          |   79 +
 be/src/kudu/util/kernel_stack_watchdog.cc       |  256 ++
 be/src/kudu/util/kernel_stack_watchdog.h        |  290 +++
 be/src/kudu/util/knapsack_solver-test.cc        |  172 ++
 be/src/kudu/util/knapsack_solver.h              |  269 ++
 be/src/kudu/util/locks.cc                       |   47 +
 be/src/kudu/util/locks.h                        |  294 +++
 be/src/kudu/util/logging-test.cc                |  249 ++
 be/src/kudu/util/logging.cc                     |  413 +++
 be/src/kudu/util/logging.h                      |  367 +++
 be/src/kudu/util/logging_callback.h             |   46 +
 be/src/kudu/util/logging_test_util.h            |   60 +
 be/src/kudu/util/maintenance_manager-test.cc    |  369 +++
 be/src/kudu/util/maintenance_manager.cc         |  550 ++++
 be/src/kudu/util/maintenance_manager.h          |  361 +++
 be/src/kudu/util/maintenance_manager.proto      |   54 +
 be/src/kudu/util/make_shared.h                  |   64 +
 be/src/kudu/util/malloc.cc                      |   35 +
 be/src/kudu/util/malloc.h                       |   32 +
 be/src/kudu/util/map-util-test.cc               |  116 +
 be/src/kudu/util/mem_tracker-test.cc            |  285 ++
 be/src/kudu/util/mem_tracker.cc                 |  296 +++
 be/src/kudu/util/mem_tracker.h                  |  272 ++
 be/src/kudu/util/memcmpable_varint-test.cc      |  220 ++
 be/src/kudu/util/memcmpable_varint.cc           |  257 ++
 be/src/kudu/util/memcmpable_varint.h            |   45 +
 be/src/kudu/util/memory/arena-test.cc           |  205 ++
 be/src/kudu/util/memory/arena.cc                |  167 ++
 be/src/kudu/util/memory/arena.h                 |  501 ++++
 be/src/kudu/util/memory/memory.cc               |  339 +++
 be/src/kudu/util/memory/memory.h                |  970 +++++++
 be/src/kudu/util/memory/overwrite.cc            |   42 +
 be/src/kudu/util/memory/overwrite.h             |   33 +
 be/src/kudu/util/metrics-test.cc                |  388 +++
 be/src/kudu/util/metrics.cc                     |  746 ++++++
 be/src/kudu/util/metrics.h                      | 1195 +++++++++
 be/src/kudu/util/minidump-test.cc               |  149 ++
 be/src/kudu/util/minidump.cc                    |  382 +++
 be/src/kudu/util/minidump.h                     |  104 +
 be/src/kudu/util/monotime-test.cc               |  424 +++
 be/src/kudu/util/monotime.cc                    |  334 +++
 be/src/kudu/util/monotime.h                     |  421 +++
 be/src/kudu/util/mt-hdr_histogram-test.cc       |  116 +
 be/src/kudu/util/mt-metrics-test.cc             |  128 +
 be/src/kudu/util/mt-threadlocal-test.cc         |  357 +++
 be/src/kudu/util/mutex.cc                       |  164 ++
 be/src/kudu/util/mutex.h                        |  142 +
 be/src/kudu/util/net/dns_resolver-test.cc       |   59 +
 be/src/kudu/util/net/dns_resolver.cc            |   65 +
 be/src/kudu/util/net/dns_resolver.h             |   62 +
 be/src/kudu/util/net/net_util-test.cc           |  170 ++
 be/src/kudu/util/net/net_util.cc                |  402 +++
 be/src/kudu/util/net/net_util.h                 |  166 ++
 be/src/kudu/util/net/sockaddr.cc                |  136 +
 be/src/kudu/util/net/sockaddr.h                 |   94 +
 be/src/kudu/util/net/socket-test.cc             |   89 +
 be/src/kudu/util/net/socket.cc                  |  590 +++++
 be/src/kudu/util/net/socket.h                   |  178 ++
 be/src/kudu/util/nvm_cache.cc                   |  577 +++++
 be/src/kudu/util/nvm_cache.h                    |   31 +
 be/src/kudu/util/object_pool-test.cc            |   86 +
 be/src/kudu/util/object_pool.h                  |  166 ++
 be/src/kudu/util/oid_generator-test.cc          |   52 +
 be/src/kudu/util/oid_generator.cc               |   65 +
 be/src/kudu/util/oid_generator.h                |   63 +
 be/src/kudu/util/once-test.cc                   |  113 +
 be/src/kudu/util/once.cc                        |   32 +
 be/src/kudu/util/once.h                         |  116 +
 be/src/kudu/util/os-util-test.cc                |   62 +
 be/src/kudu/util/os-util.cc                     |  185 ++
 be/src/kudu/util/os-util.h                      |   72 +
 be/src/kudu/util/path_util-test.cc              |   77 +
 be/src/kudu/util/path_util.cc                   |  122 +
 be/src/kudu/util/path_util.h                    |   63 +
 be/src/kudu/util/pb_util-internal.cc            |  105 +
 be/src/kudu/util/pb_util-internal.h             |  136 +
 be/src/kudu/util/pb_util-test.cc                |  661 +++++
 be/src/kudu/util/pb_util.cc                     | 1088 ++++++++
 be/src/kudu/util/pb_util.h                      |  513 ++++
 be/src/kudu/util/pb_util.proto                  |   45 +
 be/src/kudu/util/pb_util_test.proto             |   29 +
 be/src/kudu/util/process_memory-test.cc         |   75 +
 be/src/kudu/util/process_memory.cc              |  287 +++
 be/src/kudu/util/process_memory.h               |   62 +
 be/src/kudu/util/promise.h                      |   79 +
 be/src/kudu/util/proto_container_test.proto     |   25 +
 be/src/kudu/util/proto_container_test2.proto    |   29 +
 be/src/kudu/util/proto_container_test3.proto    |   33 +
 be/src/kudu/util/protobuf-annotations.h         |   33 +
 be/src/kudu/util/protobuf_util.h                |   39 +
 be/src/kudu/util/protoc-gen-insertions.cc       |   77 +
 be/src/kudu/util/pstack_watcher-test.cc         |  100 +
 be/src/kudu/util/pstack_watcher.cc              |  249 ++
 be/src/kudu/util/pstack_watcher.h               |  101 +
 be/src/kudu/util/random-test.cc                 |  171 ++
 be/src/kudu/util/random.h                       |  252 ++
 be/src/kudu/util/random_util-test.cc            |   75 +
 be/src/kudu/util/random_util.cc                 |   65 +
 be/src/kudu/util/random_util.h                  |   44 +
 be/src/kudu/util/rle-encoding.h                 |  523 ++++
 be/src/kudu/util/rle-test.cc                    |  546 ++++
 be/src/kudu/util/rolling_log-test.cc            |  147 ++
 be/src/kudu/util/rolling_log.cc                 |  285 ++
 be/src/kudu/util/rolling_log.h                  |  128 +
 be/src/kudu/util/rw_mutex-test.cc               |  185 ++
 be/src/kudu/util/rw_mutex.cc                    |  207 ++
 be/src/kudu/util/rw_mutex.h                     |  123 +
 be/src/kudu/util/rw_semaphore-test.cc           |   94 +
 be/src/kudu/util/rw_semaphore.h                 |  206 ++
 be/src/kudu/util/rwc_lock-test.cc               |  147 ++
 be/src/kudu/util/rwc_lock.cc                    |  136 +
 be/src/kudu/util/rwc_lock.h                     |  142 +
 be/src/kudu/util/safe_math-test.cc              |   56 +
 be/src/kudu/util/safe_math.h                    |   69 +
 be/src/kudu/util/scoped_cleanup-test.cc         |   56 +
 be/src/kudu/util/scoped_cleanup.h               |   67 +
 be/src/kudu/util/semaphore.cc                   |  105 +
 be/src/kudu/util/semaphore.h                    |   77 +
 be/src/kudu/util/semaphore_macosx.cc            |   75 +
 be/src/kudu/util/signal.cc                      |   47 +
 be/src/kudu/util/signal.h                       |   42 +
 be/src/kudu/util/slice-test.cc                  |   61 +
 be/src/kudu/util/slice.cc                       |   97 +
 be/src/kudu/util/slice.h                        |  332 +++
 .../util/sorted_disjoint_interval_list-test.cc  |   98 +
 .../kudu/util/sorted_disjoint_interval_list.h   |   95 +
 be/src/kudu/util/spinlock_profiling-test.cc     |   81 +
 be/src/kudu/util/spinlock_profiling.cc          |  308 +++
 be/src/kudu/util/spinlock_profiling.h           |   72 +
 be/src/kudu/util/stack_watchdog-test.cc         |  152 ++
 be/src/kudu/util/status-test.cc                 |  119 +
 be/src/kudu/util/status.cc                      |  170 ++
 be/src/kudu/util/status.h                       |  493 ++++
 be/src/kudu/util/status_callback.cc             |   41 +
 be/src/kudu/util/status_callback.h              |   54 +
 be/src/kudu/util/stopwatch.h                    |  364 +++
 be/src/kudu/util/string_case-test.cc            |   65 +
 be/src/kudu/util/string_case.cc                 |   76 +
 be/src/kudu/util/string_case.h                  |   48 +
 be/src/kudu/util/striped64-test.cc              |  163 ++
 be/src/kudu/util/striped64.cc                   |  191 ++
 be/src/kudu/util/striped64.h                    |  168 ++
 be/src/kudu/util/subprocess-test.cc             |  381 +++
 be/src/kudu/util/subprocess.cc                  |  815 ++++++
 be/src/kudu/util/subprocess.h                   |  219 ++
 be/src/kudu/util/test_graph.cc                  |  121 +
 be/src/kudu/util/test_graph.h                   |   90 +
 be/src/kudu/util/test_macros.h                  |  123 +
 be/src/kudu/util/test_main.cc                   |  109 +
 be/src/kudu/util/test_util.cc                   |  446 ++++
 be/src/kudu/util/test_util.h                    |  146 ++
 be/src/kudu/util/test_util_prod.cc              |   28 +
 be/src/kudu/util/test_util_prod.h               |   32 +
 be/src/kudu/util/thread-test.cc                 |  160 ++
 be/src/kudu/util/thread.cc                      |  628 +++++
 be/src/kudu/util/thread.h                       |  373 +++
 be/src/kudu/util/thread_restrictions.cc         |   87 +
 be/src/kudu/util/thread_restrictions.h          |  121 +
 be/src/kudu/util/threadlocal.cc                 |   89 +
 be/src/kudu/util/threadlocal.h                  |  128 +
 be/src/kudu/util/threadlocal_cache.h            |  110 +
 be/src/kudu/util/threadpool-test.cc             |  941 +++++++
 be/src/kudu/util/threadpool.cc                  |  766 ++++++
 be/src/kudu/util/threadpool.h                   |  505 ++++
 be/src/kudu/util/throttler-test.cc              |   76 +
 be/src/kudu/util/throttler.cc                   |   67 +
 be/src/kudu/util/throttler.h                    |   62 +
 be/src/kudu/util/trace-test.cc                  |  891 +++++++
 be/src/kudu/util/trace.cc                       |  259 ++
 be/src/kudu/util/trace.h                        |  292 +++
 be/src/kudu/util/trace_metrics.cc               |   74 +
 be/src/kudu/util/trace_metrics.h                |   89 +
 be/src/kudu/util/url-coding-test.cc             |  112 +
 be/src/kudu/util/url-coding.cc                  |  208 ++
 be/src/kudu/util/url-coding.h                   |   69 +
 be/src/kudu/util/user-test.cc                   |   44 +
 be/src/kudu/util/user.cc                        |   90 +
 be/src/kudu/util/user.h                         |   32 +
 be/src/kudu/util/version_info.cc                |   84 +
 be/src/kudu/util/version_info.h                 |   51 +
 be/src/kudu/util/version_info.proto             |   32 +
 be/src/kudu/util/version_util-test.cc           |   66 +
 be/src/kudu/util/version_util.cc                |   83 +
 be/src/kudu/util/version_util.h                 |   58 +
 be/src/kudu/util/web_callback_registry.h        |  129 +
 be/src/kudu/util/website_util.cc                |   43 +
 be/src/kudu/util/website_util.h                 |   35 +
 be/src/kudu/util/zlib.cc                        |  127 +
 be/src/kudu/util/zlib.h                         |   39 +
 450 files changed, 99139 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/CMakeLists.txt b/be/src/kudu/rpc/CMakeLists.txt
new file mode 100644
index 0000000..f8cdb02
--- /dev/null
+++ b/be/src/kudu/rpc/CMakeLists.txt
@@ -0,0 +1,138 @@
+# 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.
+
+#### Global header protobufs
+PROTOBUF_GENERATE_CPP(
+  RPC_HEADER_PROTO_SRCS RPC_HEADER_PROTO_HDRS RPC_HEADER_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rpc_header.proto)
+ADD_EXPORTABLE_LIBRARY(rpc_header_proto
+  SRCS ${RPC_HEADER_PROTO_SRCS}
+  DEPS protobuf pb_util_proto token_proto
+  NONLINK_DEPS ${RPC_HEADER_PROTO_TGTS})
+
+PROTOBUF_GENERATE_CPP(
+  RPC_INTROSPECTION_PROTO_SRCS RPC_INTROSPECTION_PROTO_HDRS RPC_INTROSPECTION_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rpc_introspection.proto)
+set(RPC_INTROSPECTION_PROTO_LIBS
+  rpc_header_proto
+  protobuf)
+ADD_EXPORTABLE_LIBRARY(rpc_introspection_proto
+  SRCS ${RPC_INTROSPECTION_PROTO_SRCS}
+  DEPS ${RPC_INTROSPECTION_PROTO_LIBS}
+  NONLINK_DEPS ${RPC_INTROSPECTION_PROTO_TGTS})
+
+### RPC library
+set(KRPC_SRCS
+    acceptor_pool.cc
+    blocking_ops.cc
+    client_negotiation.cc
+    connection.cc
+    connection_id.cc
+    constants.cc
+    inbound_call.cc
+    messenger.cc
+    negotiation.cc
+    outbound_call.cc
+    periodic.cc
+    proxy.cc
+    reactor.cc
+    remote_method.cc
+    remote_user.cc
+    request_tracker.cc
+    result_tracker.cc
+    rpc.cc
+    rpc_context.cc
+    rpc_controller.cc
+    rpc_sidecar.cc
+    rpcz_store.cc
+    sasl_common.cc
+    sasl_helper.cc
+    serialization.cc
+    server_negotiation.cc
+    service_if.cc
+    service_pool.cc
+    service_queue.cc
+    user_credentials.cc
+    transfer.cc
+)
+
+set(KRPC_LIBS
+  cyrus_sasl
+  gssapi_krb5
+  gutil
+  kudu_util
+  libev
+  rpc_header_proto
+  rpc_introspection_proto
+  security)
+
+ADD_EXPORTABLE_LIBRARY(krpc
+  SRCS ${KRPC_SRCS}
+  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
+    kudu_util)
+
+#### RPC test
+PROTOBUF_GENERATE_CPP(
+  RPC_TEST_DIFF_PACKAGE_SRCS RPC_TEST_DIFF_PACKAGE_HDRS RPC_TEST_DIFF_PACKAGE_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rtest_diff_package.proto)
+add_library(rtest_diff_package_proto ${RPC_TEST_DIFF_PACKAGE_SRCS} ${RPC_TEST_DIFF_PACKAGE_HDRS})
+target_link_libraries(rtest_diff_package_proto rpc_header_proto)
+
+KRPC_GENERATE(
+  RTEST_KRPC_SRCS RTEST_KRPC_HDRS RTEST_KRPC_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rtest.proto)
+add_library(rtest_krpc ${RTEST_KRPC_SRCS} ${RTEST_KRPC_HDRS})
+target_link_libraries(rtest_krpc
+  krpc
+  rpc_header_proto
+  rtest_diff_package_proto)
+
+# Tests
+set(KUDU_TEST_LINK_LIBS
+  krpc
+  mini_kdc
+  rpc_header_proto
+  rtest_krpc
+  security_test_util
+  ${KUDU_MIN_TEST_LIBS})
+ADD_KUDU_TEST(exactly_once_rpc-test PROCESSORS 10)
+ADD_KUDU_TEST(mt-rpc-test RUN_SERIAL true)
+ADD_KUDU_TEST(negotiation-test)
+ADD_KUDU_TEST(periodic-test)
+ADD_KUDU_TEST(reactor-test)
+ADD_KUDU_TEST(request_tracker-test)
+ADD_KUDU_TEST(rpc-bench RUN_SERIAL true)
+ADD_KUDU_TEST(rpc-test)
+ADD_KUDU_TEST(rpc_stub-test)
+ADD_KUDU_TEST(service_queue-test RUN_SERIAL true)

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/acceptor_pool.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/acceptor_pool.cc b/be/src/kudu/rpc/acceptor_pool.cc
new file mode 100644
index 0000000..e4bcbd1
--- /dev/null
+++ b/be/src/kudu/rpc/acceptor_pool.cc
@@ -0,0 +1,175 @@
+// 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 "kudu/rpc/acceptor_pool.h"
+
+#include <string>
+#include <ostream>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+namespace google {
+namespace protobuf {
+
+class Message;
+
+}
+}
+
+using google::protobuf::Message;
+using std::string;
+
+METRIC_DEFINE_counter(server, rpc_connections_accepted,
+                      "RPC Connections Accepted",
+                      kudu::MetricUnit::kConnections,
+                      "Number of incoming TCP connections made to the RPC server");
+
+DEFINE_int32(rpc_acceptor_listen_backlog, 128,
+             "Socket backlog parameter used when listening for RPC connections. "
+             "This defines the maximum length to which the queue of pending "
+             "TCP connections inbound to the RPC server may grow. If a connection "
+             "request arrives when the queue is full, the client may receive "
+             "an error. Higher values may help the server ride over bursts of "
+             "new inbound connection requests.");
+TAG_FLAG(rpc_acceptor_listen_backlog, advanced);
+
+namespace kudu {
+namespace rpc {
+
+AcceptorPool::AcceptorPool(Messenger* messenger, Socket* socket,
+                           Sockaddr bind_address)
+    : messenger_(messenger),
+      socket_(socket->Release()),
+      bind_address_(bind_address),
+      rpc_connections_accepted_(METRIC_rpc_connections_accepted.Instantiate(
+          messenger->metric_entity())),
+      closing_(false) {}
+
+AcceptorPool::~AcceptorPool() {
+  Shutdown();
+}
+
+Status AcceptorPool::Start(int num_threads) {
+  RETURN_NOT_OK(socket_.Listen(FLAGS_rpc_acceptor_listen_backlog));
+
+  for (int i = 0; i < num_threads; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    Status s = kudu::Thread::Create("acceptor pool", "acceptor",
+        &AcceptorPool::RunThread, this, &new_thread);
+    if (!s.ok()) {
+      Shutdown();
+      return s;
+    }
+    threads_.push_back(new_thread);
+  }
+  return Status::OK();
+}
+
+void AcceptorPool::Shutdown() {
+  if (Acquire_CompareAndSwap(&closing_, false, true) != false) {
+    VLOG(2) << "Acceptor Pool on " << bind_address_.ToString()
+            << " already shut down";
+    return;
+  }
+
+#if defined(__linux__)
+  // Closing the socket will break us out of accept() if we're in it, and
+  // prevent future accepts.
+  WARN_NOT_OK(socket_.Shutdown(true, true),
+              strings::Substitute("Could not shut down acceptor socket on $0",
+                                  bind_address_.ToString()));
+#else
+  // Calling shutdown on an accepting (non-connected) socket is illegal on most
+  // platforms (but not Linux). Instead, the accepting threads are interrupted
+  // forcefully.
+  for (const scoped_refptr<kudu::Thread>& thread : threads_) {
+    pthread_cancel(thread.get()->pthread_id());
+  }
+#endif
+
+  for (const scoped_refptr<kudu::Thread>& thread : threads_) {
+    CHECK_OK(ThreadJoiner(thread.get()).Join());
+  }
+  threads_.clear();
+
+  // Close the socket: keeping the descriptor open and, possibly, receiving late
+  // not-to-be-read messages from the peer does not make much sense. The
+  // Socket::Close() method is called upon destruction of the aggregated socket_
+  // object as well. However, the typical ownership pattern of an AcceptorPool
+  // object includes two references wrapped via a shared_ptr smart pointer: one
+  // is held by Messenger, another by RpcServer. If not calling Socket::Close()
+  // here, it would  necessary to wait until Messenger::Shutdown() is called for
+  // the corresponding messenger object to close this socket.
+  ignore_result(socket_.Close());
+}
+
+Sockaddr AcceptorPool::bind_address() const {
+  return bind_address_;
+}
+
+Status AcceptorPool::GetBoundAddress(Sockaddr* addr) const {
+  return socket_.GetSocketAddress(addr);
+}
+
+int64_t AcceptorPool::num_rpc_connections_accepted() const {
+  return rpc_connections_accepted_->value();
+}
+
+void AcceptorPool::RunThread() {
+  while (true) {
+    Socket new_sock;
+    Sockaddr remote;
+    VLOG(2) << "calling accept() on socket " << socket_.GetFd()
+            << " listening on " << bind_address_.ToString();
+    Status s = socket_.Accept(&new_sock, &remote, Socket::FLAG_NONBLOCKING);
+    if (!s.ok()) {
+      if (Release_Load(&closing_)) {
+        break;
+      }
+      KLOG_EVERY_N_SECS(WARNING, 1) << "AcceptorPool: accept failed: " << s.ToString()
+                                    << THROTTLE_MSG;
+      continue;
+    }
+    s = new_sock.SetNoDelay(true);
+    if (!s.ok()) {
+      KLOG_EVERY_N_SECS(WARNING, 1) << "Acceptor with remote = " << remote.ToString()
+          << " failed to set TCP_NODELAY on a newly accepted socket: "
+          << s.ToString() << THROTTLE_MSG;
+      continue;
+    }
+    rpc_connections_accepted_->Increment();
+    messenger_->RegisterInboundSocket(&new_sock, remote);
+  }
+  VLOG(1) << "AcceptorPool shutting down.";
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/acceptor_pool.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/acceptor_pool.h b/be/src/kudu/rpc/acceptor_pool.h
new file mode 100644
index 0000000..ba1996a
--- /dev/null
+++ b/be/src/kudu/rpc/acceptor_pool.h
@@ -0,0 +1,84 @@
+// 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.
+
+#ifndef KUDU_RPC_ACCEPTOR_POOL_H
+#define KUDU_RPC_ACCEPTOR_POOL_H
+
+#include <stdint.h>
+#include <vector>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Counter;
+class Thread;
+
+namespace rpc {
+
+class Messenger;
+
+// A pool of threads calling accept() to create new connections.
+// Acceptor pool threads terminate when they notice that the messenger has been
+// shut down, if Shutdown() is called, or if the pool object is destructed.
+class AcceptorPool {
+ public:
+  // Create a new acceptor pool.  Calls socket::Release to take ownership of the
+  // socket.
+  // 'socket' must be already bound, but should not yet be listening.
+  AcceptorPool(Messenger *messenger, Socket *socket, Sockaddr bind_address);
+  ~AcceptorPool();
+
+  // Start listening and accepting connections.
+  Status Start(int num_threads);
+  void Shutdown();
+
+  // Return the address that the pool is bound to. If the port is specified as
+  // 0, then this will always return port 0.
+  Sockaddr bind_address() const;
+
+  // Return the address that the pool is bound to. This only works while the
+  // socket is open, and if the specified port is 0 then this will return the
+  // actual port that was bound.
+  Status GetBoundAddress(Sockaddr* addr) const;
+
+  // Return the number of connections accepted by this messenger. Thread-safe.
+  int64_t num_rpc_connections_accepted() const;
+
+ private:
+  void RunThread();
+
+  Messenger *messenger_;
+  Socket socket_;
+  Sockaddr bind_address_;
+  std::vector<scoped_refptr<kudu::Thread> > threads_;
+
+  scoped_refptr<Counter> rpc_connections_accepted_;
+
+  Atomic32 closing_;
+
+  DISALLOW_COPY_AND_ASSIGN(AcceptorPool);
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/blocking_ops.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/blocking_ops.cc b/be/src/kudu/rpc/blocking_ops.cc
new file mode 100644
index 0000000..f5cd644
--- /dev/null
+++ b/be/src/kudu/rpc/blocking_ops.cc
@@ -0,0 +1,126 @@
+// 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 "kudu/rpc/blocking_ops.h"
+
+#include <cstdint>
+#include <cstring>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <google/protobuf/message_lite.h>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace rpc {
+
+using google::protobuf::MessageLite;
+
+const char kHTTPHeader[] = "HTTP";
+
+Status CheckInBlockingMode(const Socket* sock) {
+  bool is_nonblocking;
+  RETURN_NOT_OK(sock->IsNonBlocking(&is_nonblocking));
+  if (is_nonblocking) {
+    static const char* const kErrMsg = "socket is not in blocking mode";
+    LOG(DFATAL) << kErrMsg;
+    return Status::IllegalState(kErrMsg);
+  }
+  return Status::OK();
+}
+
+Status SendFramedMessageBlocking(Socket* sock, const MessageLite& header, const MessageLite& msg,
+    const MonoTime& deadline) {
+  DCHECK(sock != nullptr);
+  DCHECK(header.IsInitialized()) << "header protobuf must be initialized";
+  DCHECK(msg.IsInitialized()) << "msg protobuf must be initialized";
+
+  // Ensure we are in blocking mode.
+  // These blocking calls are typically not in the fast path, so doing this for all build types.
+  RETURN_NOT_OK(CheckInBlockingMode(sock));
+
+  // Serialize message
+  faststring param_buf;
+  serialization::SerializeMessage(msg, &param_buf);
+
+  // Serialize header and initial length
+  faststring header_buf;
+  serialization::SerializeHeader(header, param_buf.size(), &header_buf);
+
+  // Write header & param to stream
+  size_t nsent;
+  RETURN_NOT_OK(sock->BlockingWrite(header_buf.data(), header_buf.size(), &nsent, deadline));
+  RETURN_NOT_OK(sock->BlockingWrite(param_buf.data(), param_buf.size(), &nsent, deadline));
+
+  return Status::OK();
+}
+
+Status ReceiveFramedMessageBlocking(Socket* sock, faststring* recv_buf,
+    MessageLite* header, Slice* param_buf, const MonoTime& deadline) {
+  DCHECK(sock != nullptr);
+  DCHECK(recv_buf != nullptr);
+  DCHECK(header != nullptr);
+  DCHECK(param_buf != nullptr);
+
+  RETURN_NOT_OK(CheckInBlockingMode(sock));
+
+  // Read the message prefix, which specifies the length of the payload.
+  recv_buf->clear();
+  recv_buf->resize(kMsgLengthPrefixLength);
+  size_t recvd = 0;
+  RETURN_NOT_OK(sock->BlockingRecv(recv_buf->data(), kMsgLengthPrefixLength, &recvd, deadline));
+  uint32_t payload_len = NetworkByteOrder::Load32(recv_buf->data());
+
+  // Verify that the payload size isn't out of bounds.
+  // This can happen because of network corruption, or a naughty client.
+  if (PREDICT_FALSE(payload_len > FLAGS_rpc_max_message_size)) {
+    // A common user mistake is to try to speak the Kudu RPC protocol to an
+    // HTTP endpoint, or vice versa.
+    if (memcmp(recv_buf->data(), kHTTPHeader, strlen(kHTTPHeader)) == 0) {
+      return Status::IOError(
+          "received invalid RPC message which appears to be an HTTP response. "
+          "Verify that you have specified a valid RPC port and not an HTTP port.");
+    }
+
+    return Status::IOError(
+        strings::Substitute(
+            "received invalid message of size $0 which exceeds"
+            " the rpc_max_message_size of $1 bytes",
+            payload_len, FLAGS_rpc_max_message_size));
+  }
+
+  // Read the message payload.
+  recvd = 0;
+  recv_buf->resize(payload_len + kMsgLengthPrefixLength);
+  RETURN_NOT_OK(sock->BlockingRecv(recv_buf->data() + kMsgLengthPrefixLength,
+                payload_len, &recvd, deadline));
+  RETURN_NOT_OK(serialization::ParseMessage(Slice(*recv_buf), header, param_buf));
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/blocking_ops.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/blocking_ops.h b/be/src/kudu/rpc/blocking_ops.h
new file mode 100644
index 0000000..b305ba7
--- /dev/null
+++ b/be/src/kudu/rpc/blocking_ops.h
@@ -0,0 +1,58 @@
+// 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.
+
+#ifndef KUDU_RPC_BLOCKING_OPS_H
+#define KUDU_RPC_BLOCKING_OPS_H
+
+namespace google {
+namespace protobuf {
+class MessageLite;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class faststring;
+class MonoTime;
+class Slice;
+class Socket;
+class Status;
+
+namespace rpc {
+
+// Returns OK if socket is in blocking mode. Otherwise, returns an error.
+Status CheckInBlockingMode(const Socket* sock);
+
+// Encode and send a message over a socket.
+// header: Request or Response header protobuf.
+// msg: Protobuf message to send. This message must be fully initialized.
+// deadline: Latest time allowed for receive to complete before timeout.
+Status SendFramedMessageBlocking(Socket* sock, const google::protobuf::MessageLite& header,
+    const google::protobuf::MessageLite& msg, const MonoTime& deadline);
+
+// Receive a full message frame from the server.
+// recv_buf: buffer to use for reading the data from the socket.
+// header: Request or Response header protobuf.
+// param_buf: Slice into recv_buf containing unparsed RPC param protobuf data.
+// deadline: Latest time allowed for receive to complete before timeout.
+Status ReceiveFramedMessageBlocking(Socket* sock, faststring* recv_buf,
+    google::protobuf::MessageLite* header, Slice* param_buf, const MonoTime& deadline);
+
+} // namespace rpc
+} // namespace kudu
+
+#endif  // KUDU_RPC_BLOCKING_OPS_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/client_negotiation.cc b/be/src/kudu/rpc/client_negotiation.cc
new file mode 100644
index 0000000..02175f6
--- /dev/null
+++ b/be/src/kudu/rpc/client_negotiation.cc
@@ -0,0 +1,853 @@
+// 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 "kudu/rpc/client_negotiation.h"
+
+#include <cstdint>
+#include <cstring>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gssapi/gssapi.h>
+#include <gssapi/gssapi_krb5.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/blocking_ops.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/sasl_helper.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/trace.h"
+
+using std::map;
+using std::set;
+using std::string;
+using std::unique_ptr;
+
+using strings::Substitute;
+
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
+namespace kudu {
+namespace rpc {
+
+static int ClientNegotiationGetoptCb(ClientNegotiation* client_negotiation,
+                                     const char* plugin_name,
+                                     const char* option,
+                                     const char** result,
+                                     unsigned* len) {
+  return client_negotiation->GetOptionCb(plugin_name, option, result, len);
+}
+
+static int ClientNegotiationSimpleCb(ClientNegotiation* client_negotiation,
+                                     int id,
+                                     const char** result,
+                                     unsigned* len) {
+  return client_negotiation->SimpleCb(id, result, len);
+}
+
+static int ClientNegotiationSecretCb(sasl_conn_t* conn,
+                                     ClientNegotiation* client_negotiation,
+                                     int id,
+                                     sasl_secret_t** psecret) {
+  return client_negotiation->SecretCb(conn, id, psecret);
+}
+
+// Return an appropriately-typed Status object based on an ErrorStatusPB returned
+// from an Error RPC.
+// In case there is no relevant Status type, return a RuntimeError.
+static Status StatusFromRpcError(const ErrorStatusPB& error) {
+  DCHECK(error.IsInitialized()) << "Error status PB must be initialized";
+  if (PREDICT_FALSE(!error.has_code())) {
+    return Status::RuntimeError(error.message());
+  }
+  const string code_name = ErrorStatusPB::RpcErrorCodePB_Name(error.code());
+  switch (error.code()) {
+    case ErrorStatusPB_RpcErrorCodePB_FATAL_UNAUTHORIZED: // fall-through
+    case ErrorStatusPB_RpcErrorCodePB_FATAL_INVALID_AUTHENTICATION_TOKEN:
+      return Status::NotAuthorized(code_name, error.message());
+    case ErrorStatusPB_RpcErrorCodePB_ERROR_UNAVAILABLE:
+      return Status::ServiceUnavailable(code_name, error.message());
+    default:
+      return Status::RuntimeError(code_name, error.message());
+  }
+}
+
+ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
+                                     const security::TlsContext* tls_context,
+                                     boost::optional<security::SignedTokenPB> authn_token,
+                                     RpcEncryption encryption,
+                                     std::string sasl_proto_name)
+    : socket_(std::move(socket)),
+      helper_(SaslHelper::CLIENT),
+      tls_context_(tls_context),
+      encryption_(encryption),
+      tls_negotiated_(false),
+      authn_token_(std::move(authn_token)),
+      psecret_(nullptr, std::free),
+      negotiated_authn_(AuthenticationType::INVALID),
+      negotiated_mech_(SaslMechanism::INVALID),
+      sasl_proto_name_(std::move(sasl_proto_name)),
+      deadline_(MonoTime::Max()) {
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
+      reinterpret_cast<int (*)()>(&ClientNegotiationGetoptCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_AUTHNAME,
+      reinterpret_cast<int (*)()>(&ClientNegotiationSimpleCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_PASS,
+      reinterpret_cast<int (*)()>(&ClientNegotiationSecretCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
+  DCHECK(socket_);
+  DCHECK(tls_context_);
+}
+
+Status ClientNegotiation::EnablePlain(const string& user, const string& pass) {
+  RETURN_NOT_OK(helper_.EnablePlain());
+  plain_auth_user_ = user;
+  plain_pass_ = pass;
+  return Status::OK();
+}
+
+Status ClientNegotiation::EnableGSSAPI() {
+  return helper_.EnableGSSAPI();
+}
+
+SaslMechanism::Type ClientNegotiation::negotiated_mechanism() const {
+  return negotiated_mech_;
+}
+
+void ClientNegotiation::set_server_fqdn(const string& domain_name) {
+  helper_.set_server_fqdn(domain_name);
+}
+
+void ClientNegotiation::set_deadline(const MonoTime& deadline) {
+  deadline_ = deadline;
+}
+
+Status ClientNegotiation::Negotiate(unique_ptr<ErrorStatusPB>* rpc_error) {
+  TRACE("Beginning negotiation");
+
+  // Ensure we can use blocking calls on the socket during negotiation.
+  RETURN_NOT_OK(CheckInBlockingMode(socket_.get()));
+
+  // Step 1: send the connection header.
+  RETURN_NOT_OK(SendConnectionHeader());
+
+  faststring recv_buf;
+
+  { // Step 2: send and receive the NEGOTIATE step messages.
+    RETURN_NOT_OK(SendNegotiate());
+    NegotiatePB response;
+    RETURN_NOT_OK(RecvNegotiatePB(&response, &recv_buf, rpc_error));
+    RETURN_NOT_OK(HandleNegotiate(response));
+    TRACE("Negotiated authn=$0", AuthenticationTypeToString(negotiated_authn_));
+  }
+
+  // Step 3: if both ends support TLS, do a TLS handshake.
+  // TODO(KUDU-1921): allow the client to require TLS.
+  if (encryption_ != RpcEncryption::DISABLED &&
+      ContainsKey(server_features_, TLS)) {
+    RETURN_NOT_OK(tls_context_->InitiateHandshake(security::TlsHandshakeType::CLIENT,
+                                                  &tls_handshake_));
+
+    if (negotiated_authn_ == AuthenticationType::SASL) {
+      // When using SASL authentication, verifying the server's certificate is
+      // not necessary. This allows the client to still use TLS encryption for
+      // connections to servers which only have a self-signed certificate.
+      tls_handshake_.set_verification_mode(security::TlsVerificationMode::VERIFY_NONE);
+    }
+
+    // To initiate the TLS handshake, we pretend as if the server sent us an
+    // empty TLS_HANDSHAKE token.
+    NegotiatePB initial;
+    initial.set_step(NegotiatePB::TLS_HANDSHAKE);
+    initial.set_tls_handshake("");
+    Status s = HandleTlsHandshake(initial);
+
+    while (s.IsIncomplete()) {
+      NegotiatePB response;
+      RETURN_NOT_OK(RecvNegotiatePB(&response, &recv_buf, rpc_error));
+      s = HandleTlsHandshake(response);
+    }
+    RETURN_NOT_OK(s);
+    tls_negotiated_ = true;
+  }
+
+  // Step 4: Authentication
+  switch (negotiated_authn_) {
+    case AuthenticationType::SASL:
+      RETURN_NOT_OK(AuthenticateBySasl(&recv_buf, rpc_error));
+      break;
+    case AuthenticationType::TOKEN:
+      RETURN_NOT_OK(AuthenticateByToken(&recv_buf, rpc_error));
+      break;
+    case AuthenticationType::CERTIFICATE:
+      // The TLS handshake has already authenticated the server.
+      break;
+    case AuthenticationType::INVALID: LOG(FATAL) << "unreachable";
+  }
+
+  // Step 5: Send connection context.
+  RETURN_NOT_OK(SendConnectionContext());
+
+  TRACE("Negotiation successful");
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendNegotiatePB(const NegotiatePB& msg) {
+  RequestHeader header;
+  header.set_call_id(kNegotiateCallId);
+
+  DCHECK(socket_);
+  DCHECK(msg.IsInitialized()) << "message must be initialized";
+  DCHECK(msg.has_step()) << "message must have a step";
+
+  TRACE("Sending $0 NegotiatePB request", NegotiatePB::NegotiateStep_Name(msg.step()));
+  return SendFramedMessageBlocking(socket(), header, msg, deadline_);
+}
+
+Status ClientNegotiation::RecvNegotiatePB(NegotiatePB* msg,
+                                          faststring* buffer,
+                                          unique_ptr<ErrorStatusPB>* rpc_error) {
+  ResponseHeader header;
+  Slice param_buf;
+  RETURN_NOT_OK(ReceiveFramedMessageBlocking(socket(), buffer, &header, &param_buf, deadline_));
+  RETURN_NOT_OK(helper_.CheckNegotiateCallId(header.call_id()));
+
+  if (header.is_error()) {
+    return ParseError(param_buf, rpc_error);
+  }
+
+  RETURN_NOT_OK(helper_.ParseNegotiatePB(param_buf, msg));
+  TRACE("Received $0 NegotiatePB response", NegotiatePB::NegotiateStep_Name(msg->step()));
+  return Status::OK();
+}
+
+Status ClientNegotiation::ParseError(const Slice& err_data,
+                                     unique_ptr<ErrorStatusPB>* rpc_error) {
+  unique_ptr<ErrorStatusPB> error(new ErrorStatusPB);
+  if (!error->ParseFromArray(err_data.data(), err_data.size())) {
+    return Status::IOError("invalid error response, missing fields",
+                           error->InitializationErrorString());
+  }
+  Status s = StatusFromRpcError(*error);
+  TRACE("Received error response from server: $0", s.ToString());
+
+  if (rpc_error) {
+    rpc_error->swap(error);
+  }
+  return s;
+}
+
+Status ClientNegotiation::SendConnectionHeader() {
+  const uint8_t buflen = kMagicNumberLength + kHeaderFlagsLength;
+  uint8_t buf[buflen];
+  serialization::SerializeConnHeader(buf);
+  size_t nsent;
+  return socket()->BlockingWrite(buf, buflen, &nsent, deadline_);
+}
+
+Status ClientNegotiation::InitSaslClient() {
+  // TODO(KUDU-1922): consider setting SASL_SUCCESS_DATA
+  unsigned flags = 0;
+
+  sasl_conn_t* sasl_conn = nullptr;
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(nullptr /* no conn */, [&]() {
+      return sasl_client_new(
+          sasl_proto_name_.c_str(),     // Registered name of the service using SASL. Required.
+          helper_.server_fqdn(),        // The fully qualified domain name of the remote server.
+          nullptr,                      // Local and remote IP address strings. (we don't use
+          nullptr,                      // any mechanisms which require this info.)
+          &callbacks_[0],               // Connection-specific callbacks.
+          flags,
+          &sasl_conn);
+    }), Substitute("unable to create new SASL $0 client", sasl_proto_name_));
+  sasl_conn_.reset(sasl_conn);
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendNegotiate() {
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::NEGOTIATE);
+
+  // Advertise our supported features.
+  client_features_ = kSupportedClientRpcFeatureFlags;
+
+  if (encryption_ != RpcEncryption::DISABLED) {
+    client_features_.insert(TLS);
+    // If the remote peer is local, then we allow using TLS for authentication
+    // without encryption or integrity.
+    if (socket_->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections) {
+      client_features_.insert(TLS_AUTHENTICATION_ONLY);
+    }
+  }
+
+  for (RpcFeatureFlag feature : client_features_) {
+    msg.add_supported_features(feature);
+  }
+
+  if (!helper_.EnabledMechs().empty()) {
+    msg.add_authn_types()->mutable_sasl();
+  }
+  if (tls_context_->has_signed_cert() && !tls_context_->is_external_cert()) {
+    // We only provide authenticated TLS if the certificates are generated
+    // by the internal CA.
+    msg.add_authn_types()->mutable_certificate();
+  }
+  if (authn_token_ && tls_context_->has_trusted_cert()) {
+    // TODO(KUDU-1924): check that the authn token is not expired. Can this be done
+    // reliably on clients?
+    msg.add_authn_types()->mutable_token();
+  }
+
+  if (PREDICT_FALSE(msg.authn_types().empty())) {
+    return Status::NotAuthorized("client is not configured with an authentication type");
+  }
+
+  RETURN_NOT_OK(SendNegotiatePB(msg));
+  return Status::OK();
+}
+
+Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::NEGOTIATE)) {
+    return Status::NotAuthorized("expected NEGOTIATE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received NEGOTIATE response from server");
+
+  // Fill in the set of features supported by the server.
+  for (int flag : response.supported_features()) {
+    // We only add the features that our local build knows about.
+    RpcFeatureFlag feature_flag = RpcFeatureFlag_IsValid(flag) ?
+                                  static_cast<RpcFeatureFlag>(flag) : UNKNOWN;
+    if (feature_flag != UNKNOWN) {
+      server_features_.insert(feature_flag);
+    }
+  }
+
+  if (encryption_ == RpcEncryption::REQUIRED &&
+      !ContainsKey(server_features_, RpcFeatureFlag::TLS)) {
+    return Status::NotAuthorized("server does not support required TLS encryption");
+  }
+
+  // Get the authentication type which the server would like to use.
+  DCHECK_LE(response.authn_types().size(), 1);
+  if (response.authn_types().empty()) {
+    // If the server doesn't send back an authentication type, default to SASL
+    // in order to maintain backwards compatibility.
+    negotiated_authn_ = AuthenticationType::SASL;
+  } else {
+    const auto& authn_type = response.authn_types(0);
+    switch (authn_type.type_case()) {
+      case AuthenticationTypePB::kSasl:
+        negotiated_authn_ = AuthenticationType::SASL;
+        break;
+      case AuthenticationTypePB::kToken:
+        // TODO(todd): we should also be checking tls_context_->has_trusted_cert()
+        // here to match the original logic we used to advertise TOKEN support,
+        // or perhaps just check explicitly whether we advertised TOKEN.
+        if (!authn_token_) {
+          return Status::RuntimeError(
+              "server chose token authentication, but client has no token");
+        }
+        negotiated_authn_ = AuthenticationType::TOKEN;
+        return Status::OK();
+      case AuthenticationTypePB::kCertificate:
+        if (!tls_context_->has_signed_cert()) {
+          return Status::RuntimeError(
+              "server chose certificate authentication, but client has no certificate");
+        }
+        negotiated_authn_ = AuthenticationType::CERTIFICATE;
+        return Status::OK();
+      case AuthenticationTypePB::TYPE_NOT_SET:
+        return Status::RuntimeError("server chose an unknown authentication type");
+    }
+  }
+
+  DCHECK_EQ(negotiated_authn_, AuthenticationType::SASL);
+
+  // Build a map of the SASL mechanisms offered by the server.
+  set<SaslMechanism::Type> client_mechs(helper_.EnabledMechs());
+  set<SaslMechanism::Type> server_mechs;
+  for (const NegotiatePB::SaslMechanism& sasl_mech : response.sasl_mechanisms()) {
+    auto mech = SaslMechanism::value_of(sasl_mech.mechanism());
+    if (mech == SaslMechanism::INVALID) {
+      continue;
+    }
+    server_mechs.insert(mech);
+  }
+
+  // Determine which SASL mechanism to use for authenticating the connection.
+  // We pick the most preferred mechanism which is supported by both parties.
+  // The preference list in order of most to least preferred:
+  //  * GSSAPI
+  //  * PLAIN
+  //
+  // TODO(KUDU-1921): allow the client to require authentication.
+  if (ContainsKey(client_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(server_mechs, SaslMechanism::GSSAPI)) {
+
+    // Check that the client has local Kerberos credentials, and if not fall
+    // back to an alternate mechanism.
+    Status s = CheckGSSAPI();
+    if (s.ok()) {
+      negotiated_mech_ = SaslMechanism::GSSAPI;
+      return Status::OK();
+    }
+
+    TRACE("Kerberos authentication credentials are not available: $0", s.ToString());
+    client_mechs.erase(SaslMechanism::GSSAPI);
+  }
+
+  if (ContainsKey(client_mechs, SaslMechanism::PLAIN) &&
+      ContainsKey(server_mechs, SaslMechanism::PLAIN)) {
+    negotiated_mech_ = SaslMechanism::PLAIN;
+    return Status::OK();
+  }
+
+  // There are no mechanisms in common.
+  if (ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      !ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("server requires authentication, "
+                                  "but client does not have Kerberos credentials available");
+  }
+  if (!ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("client requires authentication, "
+                                  "but server does not have Kerberos enabled");
+  }
+  string msg = Substitute("client/server supported SASL mechanism mismatch; "
+                          "client mechanisms: [$0], server mechanisms: [$1]",
+                          JoinMapped(client_mechs, SaslMechanism::name_of, ", "),
+                          JoinMapped(server_mechs, SaslMechanism::name_of, ", "));
+
+  // For now, there should never be a SASL mechanism mismatch that isn't due
+  // to one of the sides requiring Kerberos and the other not having it, so
+  // lets sanity check that.
+  DCHECK(STLSetIntersection(client_mechs, server_mechs).empty()) << msg;
+  return Status::NotAuthorized(msg);
+}
+
+Status ClientNegotiation::SendTlsHandshake(string tls_token) {
+  TRACE("Sending TLS_HANDSHAKE message to server");
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::TLS_HANDSHAKE);
+  msg.mutable_tls_handshake()->swap(tls_token);
+  return SendNegotiatePB(msg);
+}
+
+Status ClientNegotiation::HandleTlsHandshake(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::TLS_HANDSHAKE)) {
+    return Status::NotAuthorized("expected TLS_HANDSHAKE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received TLS_HANDSHAKE response from server");
+
+  if (PREDICT_FALSE(!response.has_tls_handshake())) {
+    return Status::NotAuthorized("No TLS handshake token in TLS_HANDSHAKE response from server");
+  }
+
+  string token;
+  Status s = tls_handshake_.Continue(response.tls_handshake(), &token);
+  if (s.IsIncomplete()) {
+    // Another roundtrip is required to complete the handshake.
+    RETURN_NOT_OK(SendTlsHandshake(std::move(token)));
+  }
+
+  // Check that the handshake step didn't produce an error. Will also propagate
+  // an Incomplete status.
+  RETURN_NOT_OK(s);
+
+  // TLS handshake is finished.
+  if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
+      ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    TRACE("Negotiated auth-only $0 with cipher $1",
+          tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+    return tls_handshake_.FinishNoWrap(*socket_);
+  }
+
+  TRACE("Negotiated $0 with cipher $1",
+        tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+  return tls_handshake_.Finish(&socket_);
+}
+
+Status ClientNegotiation::AuthenticateBySasl(faststring* recv_buf,
+                                             unique_ptr<ErrorStatusPB>* rpc_error) {
+  RETURN_NOT_OK(InitSaslClient());
+  Status s = SendSaslInitiate();
+
+  // HandleSasl[Initiate, Challenge] return incomplete if an additional
+  // challenge step is required, or OK if a SASL_SUCCESS message is expected.
+  while (s.IsIncomplete()) {
+    NegotiatePB challenge;
+    RETURN_NOT_OK(RecvNegotiatePB(&challenge, recv_buf, rpc_error));
+    s = HandleSaslChallenge(challenge);
+  }
+
+  // Propagate failure from SendSaslInitiate or HandleSaslChallenge.
+  RETURN_NOT_OK(s);
+
+  // Server challenges are over; we now expect the success message.
+  NegotiatePB success;
+  RETURN_NOT_OK(RecvNegotiatePB(&success, recv_buf, rpc_error));
+  return HandleSaslSuccess(success);
+}
+
+Status ClientNegotiation::AuthenticateByToken(faststring* recv_buf,
+                                              unique_ptr<ErrorStatusPB>* rpc_error) {
+  // Sanity check that TLS has been negotiated. Sending the token on an
+  // unencrypted channel is a big no-no.
+  CHECK(tls_negotiated_);
+
+  // Send the token to the server.
+  NegotiatePB pb;
+  pb.set_step(NegotiatePB::TOKEN_EXCHANGE);
+  *pb.mutable_authn_token() = std::move(*authn_token_);
+  RETURN_NOT_OK(SendNegotiatePB(pb));
+  pb.Clear();
+
+  // Check that the server responds with a non-error TOKEN_EXCHANGE message.
+  RETURN_NOT_OK(RecvNegotiatePB(&pb, recv_buf, rpc_error));
+  if (pb.step() != NegotiatePB::TOKEN_EXCHANGE) {
+    return Status::NotAuthorized("expected TOKEN_EXCHANGE step",
+                                 NegotiatePB::NegotiateStep_Name(pb.step()));
+  }
+
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendSaslInitiate() {
+  TRACE("Initiating SASL $0 handshake", SaslMechanism::name_of(negotiated_mech_));
+
+  // At this point we've already chosen the SASL mechanism to use
+  // (negotiated_mech_), but we need to let the SASL library know. SASL likes to
+  // choose the mechanism from among a list of possible options, so we simply
+  // provide it one option, and then check that it picks that option.
+
+  const char* init_msg = nullptr;
+  unsigned init_msg_len = 0;
+  const char* negotiated_mech = nullptr;
+
+  /* select a mechanism for a connection
+   *  mechlist      -- mechanisms server has available (punctuation ignored)
+   * output:
+   *  prompt_need   -- on SASL_INTERACT, list of prompts needed to continue
+   *  clientout     -- the initial client response to send to the server
+   *  mech          -- set to mechanism name
+   *
+   * Returns:
+   *  SASL_OK       -- success
+   *  SASL_CONTINUE -- negotiation required
+   *  SASL_NOMEM    -- not enough memory
+   *  SASL_NOMECH   -- no mechanism meets requested properties
+   *  SASL_INTERACT -- user interaction needed to fill in prompt_need list
+   */
+  TRACE("Calling sasl_client_start()");
+  const Status s = WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_client_start(
+          sasl_conn_.get(),                         // The SASL connection context created by init()
+          SaslMechanism::name_of(negotiated_mech_), // The list of mechanisms to negotiate.
+          nullptr,                                  // Disables INTERACT return if NULL.
+          &init_msg,                                // Filled in on success.
+          &init_msg_len,                            // Filled in on success.
+          &negotiated_mech);                        // Filled in on success.
+  });
+
+  if (PREDICT_FALSE(!s.IsIncomplete() && !s.ok())) {
+    return s;
+  }
+
+  // Check that the SASL library is using the mechanism that we picked.
+  DCHECK_EQ(SaslMechanism::value_of(negotiated_mech), negotiated_mech_);
+
+  // If the negotiated mechanism is GSSAPI (Kerberos), configure SASL to use
+  // integrity protection so that the channel bindings and nonce can be
+  // verified.
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    RETURN_NOT_OK(EnableProtection(sasl_conn_.get(), SaslProtection::kIntegrity));
+  }
+
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::SASL_INITIATE);
+  msg.mutable_token()->assign(init_msg, init_msg_len);
+  msg.add_sasl_mechanisms()->set_mechanism(negotiated_mech);
+  RETURN_NOT_OK(SendNegotiatePB(msg));
+  return s;
+}
+
+Status ClientNegotiation::SendSaslResponse(const char* resp_msg, unsigned resp_msg_len) {
+  NegotiatePB reply;
+  reply.set_step(NegotiatePB::SASL_RESPONSE);
+  reply.mutable_token()->assign(resp_msg, resp_msg_len);
+  return SendNegotiatePB(reply);
+}
+
+Status ClientNegotiation::HandleSaslChallenge(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::SASL_CHALLENGE)) {
+    return Status::NotAuthorized("expected SASL_CHALLENGE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received SASL_CHALLENGE response from server");
+  if (PREDICT_FALSE(!response.has_token())) {
+    return Status::NotAuthorized("no token in SASL_CHALLENGE response from server");
+  }
+
+  const char* out = nullptr;
+  unsigned out_len = 0;
+  const Status s = DoSaslStep(response.token(), &out, &out_len);
+  if (PREDICT_FALSE(!s.IsIncomplete() && !s.ok())) {
+    return s;
+  }
+
+  RETURN_NOT_OK(SendSaslResponse(out, out_len));
+  return s;
+}
+
+Status ClientNegotiation::HandleSaslSuccess(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::SASL_SUCCESS)) {
+    return Status::NotAuthorized("expected SASL_SUCCESS step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received SASL_SUCCESS response from server");
+
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    if (response.has_nonce()) {
+      // Grab the nonce from the server, if it has sent one. We'll send it back
+      // later with SASL integrity protection as part of the connection context.
+      nonce_ = response.nonce();
+    }
+
+    if (tls_negotiated_) {
+      // Check the channel bindings provided by the server against the expected channel bindings.
+      if (!response.has_channel_bindings()) {
+        return Status::NotAuthorized("no channel bindings provided by server");
+      }
+
+      security::Cert cert;
+      RETURN_NOT_OK(tls_handshake_.GetRemoteCert(&cert));
+
+      string expected_channel_bindings;
+      RETURN_NOT_OK_PREPEND(cert.GetServerEndPointChannelBindings(&expected_channel_bindings),
+                            "failed to generate channel bindings");
+
+      Slice received_channel_bindings;
+      RETURN_NOT_OK_PREPEND(SaslDecode(sasl_conn_.get(),
+                                       response.channel_bindings(),
+                                       &received_channel_bindings),
+                            "failed to decode channel bindings");
+
+      if (expected_channel_bindings != received_channel_bindings) {
+        Sockaddr addr;
+        ignore_result(socket_->GetPeerAddress(&addr));
+
+        LOG(WARNING) << "Received invalid channel bindings from server "
+                    << addr.ToString()
+                    << ", this could indicate an active network man-in-the-middle";
+        return Status::NotAuthorized("channel bindings do not match");
+      }
+    }
+  }
+
+  return Status::OK();
+}
+
+Status ClientNegotiation::DoSaslStep(const string& in, const char** out, unsigned* out_len) {
+  TRACE("Calling sasl_client_step()");
+
+  return WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_client_step(sasl_conn_.get(), in.c_str(), in.length(), nullptr, out, out_len);
+  });
+}
+
+Status ClientNegotiation::SendConnectionContext() {
+  TRACE("Sending connection context");
+  RequestHeader header;
+  header.set_call_id(kConnectionContextCallId);
+
+  ConnectionContextPB conn_context;
+  // This field is deprecated but used by servers <Kudu 1.1. Newer server versions ignore
+  // this and use the SASL-provided username instead.
+  conn_context.mutable_deprecated_user_info()->set_real_user(
+      plain_auth_user_.empty() ? "cpp-client" : plain_auth_user_);
+
+  if (nonce_) {
+    // Reply with the SASL-protected nonce. We only set the nonce when using SASL GSSAPI.
+    Slice ciphertext;
+    RETURN_NOT_OK(SaslEncode(sasl_conn_.get(), *nonce_, &ciphertext));
+    *conn_context.mutable_encoded_nonce() = ciphertext.ToString();
+  }
+
+  return SendFramedMessageBlocking(socket(), header, conn_context, deadline_);
+}
+
+int ClientNegotiation::GetOptionCb(const char* plugin_name, const char* option,
+                            const char** result, unsigned* len) {
+  return helper_.GetOptionCb(plugin_name, option, result, len);
+}
+
+// Used for PLAIN.
+// SASL callback for SASL_CB_USER, SASL_CB_AUTHNAME, SASL_CB_LANGUAGE
+int ClientNegotiation::SimpleCb(int id, const char** result, unsigned* len) {
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "Simple callback called, but PLAIN auth is not enabled";
+    return SASL_FAIL;
+  }
+  if (PREDICT_FALSE(result == nullptr)) {
+    LOG(DFATAL) << "result outparam is NULL";
+    return SASL_BADPARAM;
+  }
+  switch (id) {
+    // TODO(unknown): Support impersonation?
+    // For impersonation, USER is the impersonated user, AUTHNAME is the "sudoer".
+    case SASL_CB_USER:
+      TRACE("callback for SASL_CB_USER");
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
+      break;
+    case SASL_CB_AUTHNAME:
+      TRACE("callback for SASL_CB_AUTHNAME");
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
+      break;
+    case SASL_CB_LANGUAGE:
+      LOG(DFATAL) << "Unable to handle SASL callback type SASL_CB_LANGUAGE"
+        << "(" << id << ")";
+      return SASL_BADPARAM;
+    default:
+      LOG(DFATAL) << "Unexpected SASL callback type: " << id;
+      return SASL_BADPARAM;
+  }
+
+  return SASL_OK;
+}
+
+// Used for PLAIN.
+// SASL callback for SASL_CB_PASS: User password.
+int ClientNegotiation::SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret) {
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "Plain secret callback called, but PLAIN auth is not enabled";
+    return SASL_FAIL;
+  }
+  switch (id) {
+    case SASL_CB_PASS: {
+      if (!conn || !psecret) return SASL_BADPARAM;
+
+      size_t len = plain_pass_.length();
+      *psecret = reinterpret_cast<sasl_secret_t*>(malloc(sizeof(sasl_secret_t) + len));
+      if (!*psecret) {
+        return SASL_NOMEM;
+      }
+      psecret_.reset(*psecret);  // Ensure that we free() this structure later.
+      (*psecret)->len = len;
+      memcpy((*psecret)->data, plain_pass_.c_str(), len + 1);
+      break;
+    }
+    default:
+      LOG(DFATAL) << "Unexpected SASL callback type: " << id;
+      return SASL_BADPARAM;
+  }
+
+  return SASL_OK;
+}
+
+namespace {
+// Retrieve the GSSAPI error description for an error code and type.
+string gss_error_description(OM_uint32 code, int type) {
+  string description;
+  OM_uint32 message_context = 0;
+
+  do {
+    if (!description.empty()) {
+      description.append(": ");
+    }
+    OM_uint32 minor = 0;
+    gss_buffer_desc buf;
+    gss_display_status(&minor, code, type, GSS_C_NULL_OID, &message_context, &buf);
+    description.append(static_cast<const char*>(buf.value), buf.length);
+    gss_release_buffer(&minor, &buf);
+  } while (message_context != 0);
+
+  return description;
+}
+
+// Transforms a GSSAPI major and minor error code into a Kudu Status.
+Status check_gss_error(OM_uint32 major, OM_uint32 minor) {
+    if (GSS_ERROR(major)) {
+      return Status::NotAuthorized(gss_error_description(major, GSS_C_GSS_CODE),
+                                   gss_error_description(minor, GSS_C_MECH_CODE));
+    }
+    return Status::OK();
+}
+} // anonymous namespace
+
+Status ClientNegotiation::CheckGSSAPI() {
+  OM_uint32 major, minor;
+  gss_cred_id_t cred = GSS_C_NO_CREDENTIAL;
+
+  // Acquire the Kerberos credential. This will fail if the client does not have
+  // a Kerberos tgt ticket. In theory it should be sufficient to call
+  // gss_inquire_cred_by_mech, but that causes a memory leak on RHEL 7.
+  major = gss_acquire_cred(&minor,
+                           GSS_C_NO_NAME,
+                           GSS_C_INDEFINITE,
+                           const_cast<gss_OID_set>(gss_mech_set_krb5),
+                           GSS_C_INITIATE,
+                           &cred,
+                           nullptr,
+                           nullptr);
+  Status s = check_gss_error(major, minor);
+
+  // Inspect the Kerberos credential to determine if it is expired. The lifetime
+  // returned from gss_acquire_cred in the RHEL 6 version of krb5 is always 0,
+  // so it has to be done with a separate call to gss_inquire_cred. The lifetime
+  // holds the remaining validity of the tgt in seconds.
+  OM_uint32 lifetime;
+  if (s.ok()) {
+    major = gss_inquire_cred(&minor, cred, nullptr, &lifetime, nullptr, nullptr);
+    s = check_gss_error(major, minor);
+  }
+
+  // Release the credential even if gss_inquire_cred fails.
+  gss_release_cred(&minor, &cred);
+  RETURN_NOT_OK(s);
+
+  if (lifetime == 0) {
+    return Status::NotAuthorized("Kerberos ticket expired");
+  }
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu


[23/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env-test.cc b/be/src/kudu/util/env-test.cc
new file mode 100644
index 0000000..1c7f899
--- /dev/null
+++ b/be/src/kudu/util/env-test.cc
@@ -0,0 +1,1173 @@
+// 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.
+
+#if !defined(__APPLE__)
+#include <linux/falloc.h>
+#endif  // !defined(__APPLE__)
+// Copied from falloc.h. Useful for older kernels that lack support for
+// hole punching; fallocate(2) will return EOPNOTSUPP.
+#ifndef FALLOC_FL_KEEP_SIZE
+#define FALLOC_FL_KEEP_SIZE 0x01 /* default is extend size */
+#endif
+#ifndef FALLOC_FL_PUNCH_HOLE
+#define FALLOC_FL_PUNCH_HOLE  0x02 /* de-allocates range */
+#endif
+
+#include "kudu/util/env.h"
+
+#include <fcntl.h>
+#include <sys/stat.h>
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdio>
+#include <cstdlib>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <glog/stl_logging.h> // IWYU pragma: keep
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/array_view.h" // IWYU pragma: keep
+#include "kudu/util/env_util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_bool(never_fsync);
+DECLARE_bool(crash_on_eio);
+DECLARE_double(env_inject_eio);
+DECLARE_int32(env_inject_short_read_bytes);
+DECLARE_int32(env_inject_short_write_bytes);
+DECLARE_string(env_inject_eio_globs);
+
+namespace kudu {
+
+using std::pair;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::unordered_set;
+using std::vector;
+using strings::Substitute;
+
+static const uint64_t kOneMb = 1024 * 1024;
+static const uint64_t kTwoMb = 2 * kOneMb;
+
+class TestEnv : public KuduTest {
+ public:
+  virtual void SetUp() OVERRIDE {
+    KuduTest::SetUp();
+    CheckFallocateSupport();
+  }
+
+  // Verify that fallocate() is supported in the test directory.
+  // Some local file systems like ext3 do not support it, and we don't
+  // want to fail tests on those systems.
+  //
+  // Sets fallocate_supported_ based on the result.
+  void CheckFallocateSupport() {
+    static bool checked = false;
+    if (checked) return;
+
+#if defined(__linux__)
+    int fd;
+    RETRY_ON_EINTR(fd, creat(GetTestPath("check-fallocate").c_str(), S_IWUSR));
+    CHECK_ERR(fd);
+    int err;
+    RETRY_ON_EINTR(err, fallocate(fd, 0, 0, 4096));
+    if (err != 0) {
+      PCHECK(errno == ENOTSUP);
+    } else {
+      fallocate_supported_ = true;
+
+      RETRY_ON_EINTR(err, fallocate(fd, FALLOC_FL_KEEP_SIZE | FALLOC_FL_PUNCH_HOLE,
+                                    1024, 1024));
+      if (err != 0) {
+        PCHECK(errno == ENOTSUP);
+      } else {
+        fallocate_punch_hole_supported_ = true;
+      }
+    }
+
+    RETRY_ON_EINTR(err, close(fd));
+#endif
+
+    checked = true;
+  }
+
+ protected:
+
+  void VerifyTestData(const Slice& read_data, size_t offset) {
+    for (int i = 0; i < read_data.size(); i++) {
+      size_t file_offset = offset + i;
+      ASSERT_EQ((file_offset * 31) & 0xff, read_data[i]) << "failed at " << i;
+    }
+  }
+
+  void MakeVectors(int num_slices, int slice_size, int num_iterations,
+                   unique_ptr<faststring[]>* data, vector<vector<Slice > >* vec) {
+    data->reset(new faststring[num_iterations * num_slices]);
+    vec->resize(num_iterations);
+
+    int data_idx = 0;
+    int byte_idx = 0;
+    for (int vec_idx = 0; vec_idx < num_iterations; vec_idx++) {
+      vector<Slice>& iter_vec = vec->at(vec_idx);
+      iter_vec.resize(num_slices);
+      for (int i = 0; i < num_slices; i++) {
+        (*data)[data_idx].resize(slice_size);
+        for (int j = 0; j < slice_size; j++) {
+          (*data)[data_idx][j] = (byte_idx * 31) & 0xff;
+          ++byte_idx;
+        }
+        iter_vec[i]= Slice((*data)[data_idx]);
+        ++data_idx;
+      }
+    }
+  }
+
+  void ReadAndVerifyTestData(RandomAccessFile* raf, size_t offset, size_t n) {
+    unique_ptr<uint8_t[]> scratch(new uint8_t[n]);
+    Slice s(scratch.get(), n);
+    ASSERT_OK(raf->Read(offset, s));
+    ASSERT_NO_FATAL_FAILURE(VerifyTestData(s, offset));
+  }
+
+  void TestAppendV(size_t num_slices, size_t slice_size, size_t iterations,
+                   bool fast, bool pre_allocate,
+                   const WritableFileOptions &opts) {
+    const string kTestPath = GetTestPath("test_env_appendvec_read_append");
+    shared_ptr<WritableFile> file;
+    ASSERT_OK(env_util::OpenFileForWrite(opts, env_, kTestPath, &file));
+
+    if (pre_allocate) {
+      ASSERT_OK(file->PreAllocate(num_slices * slice_size * iterations));
+      ASSERT_OK(file->Sync());
+    }
+
+    unique_ptr<faststring[]> data;
+    vector<vector<Slice> > input;
+
+    MakeVectors(num_slices, slice_size, iterations, &data, &input);
+
+    // Force short writes to half the slice length.
+    FLAGS_env_inject_short_write_bytes = slice_size / 2;
+
+    shared_ptr<RandomAccessFile> raf;
+
+    if (!fast) {
+      ASSERT_OK(env_util::OpenFileForRandom(env_, kTestPath, &raf));
+    }
+
+    srand(123);
+
+    const string test_descr = Substitute(
+        "appending a vector of slices(number of slices=$0,size of slice=$1 b) $2 times",
+        num_slices, slice_size, iterations);
+    LOG_TIMING(INFO, test_descr)  {
+      for (int i = 0; i < iterations; i++) {
+        if (fast || random() % 2) {
+          ASSERT_OK(file->AppendV(input[i]));
+        } else {
+          for (const Slice& slice : input[i]) {
+            ASSERT_OK(file->Append(slice));
+          }
+        }
+        if (!fast) {
+          // Verify as write. Note: this requires that file is pre-allocated, otherwise
+          // the Read() fails with EINVAL.
+          ASSERT_NO_FATAL_FAILURE(ReadAndVerifyTestData(raf.get(), num_slices * slice_size * i,
+                                                        num_slices * slice_size));
+        }
+      }
+    }
+
+    // Verify the entire file
+    ASSERT_OK(file->Close());
+
+    if (fast) {
+      ASSERT_OK(env_util::OpenFileForRandom(env_, kTestPath, &raf));
+    }
+    for (int i = 0; i < iterations; i++) {
+      ASSERT_NO_FATAL_FAILURE(ReadAndVerifyTestData(raf.get(), num_slices * slice_size * i,
+                                                    num_slices * slice_size));
+    }
+  }
+
+  static bool fallocate_supported_;
+  static bool fallocate_punch_hole_supported_;
+};
+
+bool TestEnv::fallocate_supported_ = false;
+bool TestEnv::fallocate_punch_hole_supported_ = false;
+
+TEST_F(TestEnv, TestPreallocate) {
+  if (!fallocate_supported_) {
+    LOG(INFO) << "fallocate not supported, skipping test";
+    return;
+  }
+  LOG(INFO) << "Testing PreAllocate()";
+  string test_path = GetTestPath("test_env_wf");
+  shared_ptr<WritableFile> file;
+  ASSERT_OK(env_util::OpenFileForWrite(env_, test_path, &file));
+
+  // pre-allocate 1 MB
+  ASSERT_OK(file->PreAllocate(kOneMb));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should report 0
+  ASSERT_EQ(file->Size(), 0);
+  // but the real size of the file on disk should report 1MB
+  uint64_t size;
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(size, kOneMb);
+
+  // write 1 MB
+  uint8_t scratch[kOneMb];
+  Slice slice(scratch, kOneMb);
+  ASSERT_OK(file->Append(slice));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should now report 1 MB
+  ASSERT_EQ(file->Size(), kOneMb);
+  ASSERT_OK(file->Close());
+  // and the real size for the file on disk should match ony the
+  // written size
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(kOneMb, size);
+}
+
+// To test consecutive pre-allocations we need higher pre-allocations since the
+// mmapped regions grow in size until 2MBs (so smaller pre-allocations will easily
+// be smaller than the mmapped regions size).
+TEST_F(TestEnv, TestConsecutivePreallocate) {
+  if (!fallocate_supported_) {
+    LOG(INFO) << "fallocate not supported, skipping test";
+    return;
+  }
+  LOG(INFO) << "Testing consecutive PreAllocate()";
+  string test_path = GetTestPath("test_env_wf");
+  shared_ptr<WritableFile> file;
+  ASSERT_OK(env_util::OpenFileForWrite(env_, test_path, &file));
+
+  // pre-allocate 64 MB
+  ASSERT_OK(file->PreAllocate(64 * kOneMb));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should report 0
+  ASSERT_EQ(file->Size(), 0);
+  // but the real size of the file on disk should report 64 MBs
+  uint64_t size;
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(size, 64 * kOneMb);
+
+  // write 1 MB
+  uint8_t scratch[kOneMb];
+  Slice slice(scratch, kOneMb);
+  ASSERT_OK(file->Append(slice));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should now report 1 MB
+  ASSERT_EQ(kOneMb, file->Size());
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(64 * kOneMb, size);
+
+  // pre-allocate 64 additional MBs
+  ASSERT_OK(file->PreAllocate(64 * kOneMb));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should now report 1 MB
+  ASSERT_EQ(kOneMb, file->Size());
+  // while the real file size should report 128 MB's
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(128 * kOneMb, size);
+
+  // write another MB
+  ASSERT_OK(file->Append(slice));
+  ASSERT_OK(file->Sync());
+
+  // the writable file size should now report 2 MB
+  ASSERT_EQ(file->Size(), 2 * kOneMb);
+  // while the real file size should reamin at 128 MBs
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(128 * kOneMb, size);
+
+  // close the file (which ftruncates it to the real size)
+  ASSERT_OK(file->Close());
+  // and the real size for the file on disk should match only the written size
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(2* kOneMb, size);
+
+}
+
+TEST_F(TestEnv, TestHolePunch) {
+  if (!fallocate_punch_hole_supported_) {
+    LOG(INFO) << "hole punching not supported, skipping test";
+    return;
+  }
+  string test_path = GetTestPath("test_env_wf");
+  unique_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(test_path, &file));
+
+  // Write 1 MB. The size and size-on-disk both agree.
+  uint8_t scratch[kOneMb];
+  Slice slice(scratch, kOneMb);
+  ASSERT_OK(file->Write(0, slice));
+  ASSERT_OK(file->Sync());
+  uint64_t sz;
+  ASSERT_OK(file->Size(&sz));
+  ASSERT_EQ(kOneMb, sz);
+  uint64_t size_on_disk;
+  ASSERT_OK(env_->GetFileSizeOnDisk(test_path, &size_on_disk));
+  // Some kernels and filesystems (e.g. Centos 6.6 with XFS) aggressively
+  // preallocate file disk space when writing to files, so the disk space may be
+  // greater than 1MiB.
+  ASSERT_LE(kOneMb, size_on_disk);
+
+  // Punch some data out at byte marker 4096. Now the two sizes diverge.
+  uint64_t punch_amount = 4096 * 4;
+  uint64_t new_size_on_disk;
+  ASSERT_OK(file->PunchHole(4096, punch_amount));
+  ASSERT_OK(file->Size(&sz));
+  ASSERT_EQ(kOneMb, sz);
+  ASSERT_OK(env_->GetFileSizeOnDisk(test_path, &new_size_on_disk));
+  ASSERT_EQ(size_on_disk - punch_amount, new_size_on_disk);
+}
+
+TEST_F(TestEnv, TestHolePunchBenchmark) {
+  const int kFileSize = 1 * 1024 * 1024 * 1024;
+  const int kHoleSize = 10 * kOneMb;
+  const int kNumRuns = 1000;
+  if (!fallocate_punch_hole_supported_) {
+    LOG(INFO) << "hole punching not supported, skipping test";
+    return;
+  }
+  Random r(SeedRandom());
+
+  string test_path = GetTestPath("test");
+  unique_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(test_path, &file));
+
+  // Initialize a scratch buffer with random data.
+  uint8_t scratch[kOneMb];
+  RandomString(&scratch, kOneMb, &r);
+
+  // Fill the file with sequences of the random data.
+  LOG_TIMING(INFO, Substitute("writing $0 bytes to file", kFileSize)) {
+    Slice slice(scratch, kOneMb);
+    for (int i = 0; i < kFileSize; i += kOneMb) {
+      ASSERT_OK(file->Write(i, slice));
+    }
+  }
+  LOG_TIMING(INFO, "syncing file") {
+    ASSERT_OK(file->Sync());
+  }
+
+  // Punch the first hole.
+  LOG_TIMING(INFO, Substitute("punching first hole of size $0", kHoleSize)) {
+    ASSERT_OK(file->PunchHole(0, kHoleSize));
+  }
+  LOG_TIMING(INFO, "syncing file") {
+    ASSERT_OK(file->Sync());
+  }
+
+  // Run the benchmark.
+  LOG_TIMING(INFO, Substitute("repunching $0 holes of size $1",
+                              kNumRuns, kHoleSize)) {
+    for (int i = 0; i < kNumRuns; i++) {
+      ASSERT_OK(file->PunchHole(0, kHoleSize));
+    }
+  }
+}
+
+TEST_F(TestEnv, TestTruncate) {
+  LOG(INFO) << "Testing Truncate()";
+  string test_path = GetTestPath("test_env_wf");
+  unique_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(test_path, &file));
+  uint64_t size;
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(0, size);
+
+  // Truncate to 2 MB (up).
+  ASSERT_OK(file->Truncate(kTwoMb));
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(kTwoMb, size);
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(kTwoMb, size);
+
+  // Truncate to 1 MB (down).
+  ASSERT_OK(file->Truncate(kOneMb));
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(kOneMb, size);
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(kOneMb, size);
+
+  ASSERT_OK(file->Close());
+
+  // Read the whole file. Ensure it is all zeroes.
+  unique_ptr<RandomAccessFile> raf;
+  ASSERT_OK(env_->NewRandomAccessFile(test_path, &raf));
+  unique_ptr<uint8_t[]> scratch(new uint8_t[size]);
+  Slice s(scratch.get(), size);
+  ASSERT_OK(raf->Read(0, s));
+  const uint8_t* data = s.data();
+  for (int i = 0; i < size; i++) {
+    ASSERT_EQ(0, data[i]) << "Not null at position " << i;
+  }
+}
+
+// Write 'size' bytes of data to a file, with a simple pattern stored in it.
+static void WriteTestFile(Env* env, const string& path, size_t size) {
+  shared_ptr<WritableFile> wf;
+  ASSERT_OK(env_util::OpenFileForWrite(env, path, &wf));
+  faststring data;
+  data.resize(size);
+  for (int i = 0; i < data.size(); i++) {
+    data[i] = (i * 31) & 0xff;
+  }
+  ASSERT_OK(wf->Append(Slice(data)));
+  ASSERT_OK(wf->Close());
+}
+
+TEST_F(TestEnv, TestReadFully) {
+  SeedRandom();
+  const string kTestPath = GetTestPath("test");
+  const int kFileSize = 64 * 1024;
+  Env* env = Env::Default();
+
+  WriteTestFile(env, kTestPath, kFileSize);
+  ASSERT_NO_FATAL_FAILURE();
+
+  // Reopen for read
+  shared_ptr<RandomAccessFile> raf;
+  ASSERT_OK(env_util::OpenFileForRandom(env, kTestPath, &raf));
+
+  const int kReadLength = 10000;
+  unique_ptr<uint8_t[]> scratch(new uint8_t[kReadLength]);
+  Slice s(scratch.get(), kReadLength);
+
+  // Force a short read to half the data length
+  FLAGS_env_inject_short_read_bytes = kReadLength / 2;
+
+  // Verify that Read fully reads the whole requested data.
+  ASSERT_OK(raf->Read(0, s));
+  VerifyTestData(s, 0);
+
+  // Turn short reads off again
+  FLAGS_env_inject_short_read_bytes = 0;
+
+  // Verify that Read fails with an EndOfFile error EOF.
+  Slice s2(scratch.get(), 200);
+  Status status = raf->Read(kFileSize - 100, s2);
+  ASSERT_FALSE(status.ok());
+  ASSERT_TRUE(status.IsEndOfFile());
+  ASSERT_STR_CONTAINS(status.ToString(), "EOF");
+}
+
+TEST_F(TestEnv, TestReadVFully) {
+  // Create the file.
+  unique_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(GetTestPath("foo"), &file));
+
+  // Append to it.
+  string kTestData = "abcde12345";
+  ASSERT_OK(file->Write(0, kTestData));
+
+  // Setup read parameters
+  size_t size1 = 5;
+  uint8_t scratch1[size1];
+  Slice result1(scratch1, size1);
+  size_t size2 = 5;
+  uint8_t scratch2[size2];
+  Slice result2(scratch2, size2);
+  vector<Slice> results = { result1, result2 };
+
+  // Force a short read
+  FLAGS_env_inject_short_read_bytes = 3;
+
+  // Verify that Read fully reads the whole requested data.
+  ASSERT_OK(file->ReadV(0, results));
+  ASSERT_EQ(result1, "abcde");
+  ASSERT_EQ(result2, "12345");
+
+  // Turn short reads off again
+  FLAGS_env_inject_short_read_bytes = 0;
+
+  // Verify that Read fails with an EndOfFile error at EOF.
+  Status status = file->ReadV(5, results);
+  ASSERT_FALSE(status.ok());
+  ASSERT_TRUE(status.IsEndOfFile());
+  ASSERT_STR_CONTAINS(status.ToString(), "EOF");
+}
+
+TEST_F(TestEnv, TestIOVMax) {
+  Env* env = Env::Default();
+  const string kTestPath = GetTestPath("test");
+
+  const size_t slice_count = IOV_MAX + 42;
+  const size_t slice_size = 5;
+  const size_t data_size = slice_count * slice_size;
+
+  NO_FATALS(WriteTestFile(env, kTestPath, data_size));
+
+  // Reopen for read
+  shared_ptr<RandomAccessFile> file;
+  ASSERT_OK(env_util::OpenFileForRandom(env, kTestPath, &file));
+
+  // Setup more results slices than IOV_MAX
+  uint8_t scratch[data_size];
+  vector<Slice> results;
+  for (size_t i = 0; i < slice_count; i++) {
+    size_t shift = slice_size * i;
+    results.emplace_back(scratch + shift, slice_size);
+  }
+
+  // Force a short read too
+  FLAGS_env_inject_short_read_bytes = 3;
+
+  // Verify all the data is read
+  ASSERT_OK(file->ReadV(0, results));
+  VerifyTestData(Slice(scratch, data_size), 0);
+}
+
+TEST_F(TestEnv, TestAppendV) {
+  WritableFileOptions opts;
+  LOG(INFO) << "Testing AppendV() only, NO pre-allocation";
+  ASSERT_NO_FATAL_FAILURE(TestAppendV(2000, 1024, 5, true, false, opts));
+
+  if (!fallocate_supported_) {
+    LOG(INFO) << "fallocate not supported, skipping preallocated runs";
+  } else {
+    LOG(INFO) << "Testing AppendV() only, WITH pre-allocation";
+    ASSERT_NO_FATAL_FAILURE(TestAppendV(2000, 1024, 5, true, true, opts));
+    LOG(INFO) << "Testing AppendV() together with Append() and Read(), WITH pre-allocation";
+    ASSERT_NO_FATAL_FAILURE(TestAppendV(128, 4096, 5, false, true, opts));
+  }
+}
+
+TEST_F(TestEnv, TestGetExecutablePath) {
+  string p;
+  ASSERT_OK(Env::Default()->GetExecutablePath(&p));
+  ASSERT_TRUE(HasSuffixString(p, "env-test")) << p;
+}
+
+TEST_F(TestEnv, TestOpenEmptyRandomAccessFile) {
+  Env* env = Env::Default();
+  string test_file = GetTestPath("test_file");
+  ASSERT_NO_FATAL_FAILURE(WriteTestFile(env, test_file, 0));
+  unique_ptr<RandomAccessFile> readable_file;
+  ASSERT_OK(env->NewRandomAccessFile(test_file, &readable_file));
+  uint64_t size;
+  ASSERT_OK(readable_file->Size(&size));
+  ASSERT_EQ(0, size);
+}
+
+TEST_F(TestEnv, TestOverwrite) {
+  string test_path = GetTestPath("test_env_wf");
+
+  // File does not exist, create it.
+  shared_ptr<WritableFile> writer;
+  ASSERT_OK(env_util::OpenFileForWrite(env_, test_path, &writer));
+
+  // File exists, overwrite it.
+  ASSERT_OK(env_util::OpenFileForWrite(env_, test_path, &writer));
+
+  // File exists, try to overwrite (and fail).
+  WritableFileOptions opts;
+  opts.mode = Env::CREATE_NON_EXISTING;
+  Status s = env_util::OpenFileForWrite(opts,
+                                        env_, test_path, &writer);
+  ASSERT_TRUE(s.IsAlreadyPresent());
+}
+
+TEST_F(TestEnv, TestReopen) {
+  LOG(INFO) << "Testing reopening behavior";
+  string test_path = GetTestPath("test_env_wf");
+  string first = "The quick brown fox";
+  string second = "jumps over the lazy dog";
+
+  // Create the file and write to it.
+  shared_ptr<WritableFile> writer;
+  ASSERT_OK(env_util::OpenFileForWrite(WritableFileOptions(),
+                                       env_, test_path, &writer));
+  ASSERT_OK(writer->Append(first));
+  ASSERT_EQ(first.length(), writer->Size());
+  ASSERT_OK(writer->Close());
+
+  // Reopen it and append to it.
+  WritableFileOptions reopen_opts;
+  reopen_opts.mode = Env::OPEN_EXISTING;
+  ASSERT_OK(env_util::OpenFileForWrite(reopen_opts,
+                                       env_, test_path, &writer));
+  ASSERT_EQ(first.length(), writer->Size());
+  ASSERT_OK(writer->Append(second));
+  ASSERT_EQ(first.length() + second.length(), writer->Size());
+  ASSERT_OK(writer->Close());
+
+  // Check that the file has both strings.
+  shared_ptr<RandomAccessFile> reader;
+  ASSERT_OK(env_util::OpenFileForRandom(env_, test_path, &reader));
+  uint64_t size;
+  ASSERT_OK(reader->Size(&size));
+  ASSERT_EQ(first.length() + second.length(), size);
+  uint8_t scratch[size];
+  Slice s(scratch, size);
+  ASSERT_OK(reader->Read(0, s));
+  ASSERT_EQ(first + second, s.ToString());
+}
+
+TEST_F(TestEnv, TestIsDirectory) {
+  string dir = GetTestPath("a_directory");
+  ASSERT_OK(env_->CreateDir(dir));
+  bool is_dir;
+  ASSERT_OK(env_->IsDirectory(dir, &is_dir));
+  ASSERT_TRUE(is_dir);
+
+  string not_dir = GetTestPath("not_a_directory");
+  unique_ptr<WritableFile> writer;
+  ASSERT_OK(env_->NewWritableFile(not_dir, &writer));
+  ASSERT_OK(env_->IsDirectory(not_dir, &is_dir));
+  ASSERT_FALSE(is_dir);
+}
+
+class ResourceLimitTypeTest : public TestEnv,
+                              public ::testing::WithParamInterface<Env::ResourceLimitType> {};
+
+INSTANTIATE_TEST_CASE_P(ResourceLimitTypes,
+                        ResourceLimitTypeTest,
+                        ::testing::Values(Env::ResourceLimitType::OPEN_FILES_PER_PROCESS,
+                                          Env::ResourceLimitType::RUNNING_THREADS_PER_EUID));
+
+// Regression test for KUDU-1798.
+TEST_P(ResourceLimitTypeTest, TestIncreaseLimit) {
+  // Increase the resource limit. It should either increase or remain the same.
+  Env::ResourceLimitType t = GetParam();
+  int64_t limit_before = env_->GetResourceLimit(t);
+  env_->IncreaseResourceLimit(t);
+  int64_t limit_after = env_->GetResourceLimit(t);
+  ASSERT_GE(limit_after, limit_before);
+
+  // Try again. It should definitely be the same now.
+  env_->IncreaseResourceLimit(t);
+  int64_t limit_after_again = env_->GetResourceLimit(t);
+  ASSERT_EQ(limit_after, limit_after_again);
+}
+
+static Status TestWalkCb(unordered_set<string>* actual,
+                         Env::FileType type,
+                         const string& dirname, const string& basename) {
+  VLOG(1) << type << ":" << dirname << ":" << basename;
+  InsertOrDie(actual, (JoinPathSegments(dirname, basename)));
+  return Status::OK();
+}
+
+static Status NoopTestWalkCb(Env::FileType /*type*/,
+                             const string& /*dirname*/,
+                             const string& /*basename*/) {
+  return Status::OK();
+}
+
+TEST_F(TestEnv, TestWalk) {
+  // We test with this tree:
+  //
+  // /root/
+  // /root/file_1
+  // /root/file_2
+  // /root/dir_a/file_1
+  // /root/dir_a/file_2
+  // /root/dir_b/file_1
+  // /root/dir_b/file_2
+  // /root/dir_b/dir_c/file_1
+  // /root/dir_b/dir_c/file_2
+  unordered_set<string> expected;
+  auto create_dir = [&](const string& name) {
+    ASSERT_OK(env_->CreateDir(name));
+    InsertOrDie(&expected, name);
+  };
+  auto create_file = [&](const string& name) {
+    unique_ptr<WritableFile> writer;
+    ASSERT_OK(env_->NewWritableFile(name, &writer));
+    InsertOrDie(&expected, writer->filename());
+  };
+  string root = GetTestPath("root");
+  string subdir_a = JoinPathSegments(root, "dir_a");
+  string subdir_b = JoinPathSegments(root, "dir_b");
+  string subdir_c = JoinPathSegments(subdir_b, "dir_c");
+  string file_one = "file_1";
+  string file_two = "file_2";
+  NO_FATALS(create_dir(root));
+  NO_FATALS(create_file(JoinPathSegments(root, file_one)));
+  NO_FATALS(create_file(JoinPathSegments(root, file_two)));
+  NO_FATALS(create_dir(subdir_a));
+  NO_FATALS(create_file(JoinPathSegments(subdir_a, file_one)));
+  NO_FATALS(create_file(JoinPathSegments(subdir_a, file_two)));
+  NO_FATALS(create_dir(subdir_b));
+  NO_FATALS(create_file(JoinPathSegments(subdir_b, file_one)));
+  NO_FATALS(create_file(JoinPathSegments(subdir_b, file_two)));
+  NO_FATALS(create_dir(subdir_c));
+  NO_FATALS(create_file(JoinPathSegments(subdir_c, file_one)));
+  NO_FATALS(create_file(JoinPathSegments(subdir_c, file_two)));
+
+  // Do the walk.
+  unordered_set<string> actual;
+  ASSERT_OK(env_->Walk(root, Env::PRE_ORDER, Bind(&TestWalkCb, &actual)));
+  ASSERT_EQ(expected, actual);
+}
+
+TEST_F(TestEnv, TestWalkNonExistentPath) {
+  // A walk on a non-existent path should fail.
+  Status s = env_->Walk("/not/a/real/path", Env::PRE_ORDER, Bind(&NoopTestWalkCb));
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "One or more errors occurred");
+}
+
+TEST_F(TestEnv, TestWalkBadPermissions) {
+  // Create a directory with mode of 0000.
+  const string kTestPath = GetTestPath("asdf");
+  ASSERT_OK(env_->CreateDir(kTestPath));
+  struct stat stat_buf;
+  PCHECK(stat(kTestPath.c_str(), &stat_buf) == 0);
+  PCHECK(chmod(kTestPath.c_str(), 0000) == 0);
+  SCOPED_CLEANUP({
+    // Restore the old permissions so the path can be successfully deleted.
+    PCHECK(chmod(kTestPath.c_str(), stat_buf.st_mode) == 0);
+  });
+
+  // A walk on a directory without execute permission should fail.
+  Status s = env_->Walk(kTestPath, Env::PRE_ORDER, Bind(&NoopTestWalkCb));
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "One or more errors occurred");
+}
+
+static Status TestWalkErrorCb(int* num_calls,
+                              Env::FileType /*type*/,
+                              const string& /*dirname*/,
+                              const string& /*basename*/) {
+  (*num_calls)++;
+  return Status::Aborted("Returning abort status");
+}
+
+TEST_F(TestEnv, TestWalkCbReturnsError) {
+  string new_dir = GetTestPath("foo");
+  string new_file = "myfile";
+  ASSERT_OK(env_->CreateDir(new_dir));
+  unique_ptr<WritableFile> writer;
+  ASSERT_OK(env_->NewWritableFile(JoinPathSegments(new_dir, new_file), &writer));
+  int num_calls = 0;
+  ASSERT_TRUE(env_->Walk(new_dir, Env::PRE_ORDER,
+                         Bind(&TestWalkErrorCb, &num_calls)).IsIOError());
+
+  // Once for the directory and once for the file inside it.
+  ASSERT_EQ(2, num_calls);
+}
+
+TEST_F(TestEnv, TestGlob) {
+  string dir = GetTestPath("glob");
+  ASSERT_OK(env_->CreateDir(dir));
+
+  vector<string> filenames = { "fuzz", "fuzzy", "fuzzyiest", "buzz" };
+  vector<pair<string, size_t>> matchers = {
+    { "file", 0 },
+    { "fuzz", 1 },
+    { "fuzz*", 3 },
+    { "?uzz", 2 },
+  };
+
+  for (const auto& name : filenames) {
+    unique_ptr<WritableFile> file;
+    ASSERT_OK(env_->NewWritableFile(JoinPathSegments(dir, name), &file));
+  }
+
+  for (const auto& matcher : matchers) {
+    SCOPED_TRACE(Substitute("pattern: $0, expected matches: $1",
+                                     matcher.first, matcher.second));
+    vector<string> matches;
+    ASSERT_OK(env_->Glob(JoinPathSegments(dir, matcher.first), &matches));
+    ASSERT_EQ(matcher.second, matches.size());
+  }
+}
+
+// Test that the status returned when 'glob' fails with a permission
+// error is reasonable.
+TEST_F(TestEnv, TestGlobPermissionDenied) {
+  string dir = GetTestPath("glob");
+  ASSERT_OK(env_->CreateDir(dir));
+  chmod(dir.c_str(), 0000);
+  SCOPED_CLEANUP({
+      chmod(dir.c_str(), 0700);
+    });
+  vector<string> matches;
+  Status s = env_->Glob(JoinPathSegments(dir, "*"), &matches);
+  ASSERT_STR_MATCHES(s.ToString(), "IO error: glob failed for /.*: Permission denied");
+}
+
+TEST_F(TestEnv, TestGetBlockSize) {
+  uint64_t block_size;
+
+  // Does not exist.
+  ASSERT_TRUE(env_->GetBlockSize("does_not_exist", &block_size).IsNotFound());
+
+  // Try with a directory.
+  ASSERT_OK(env_->GetBlockSize(".", &block_size));
+  ASSERT_GT(block_size, 0);
+
+  // Try with a file.
+  string path = GetTestPath("foo");
+  unique_ptr<WritableFile> writer;
+  ASSERT_OK(env_->NewWritableFile(path, &writer));
+  ASSERT_OK(env_->GetBlockSize(path, &block_size));
+  ASSERT_GT(block_size, 0);
+}
+
+TEST_F(TestEnv, TestGetFileModifiedTime) {
+  string path = GetTestPath("mtime");
+  unique_ptr<WritableFile> writer;
+  ASSERT_OK(env_->NewWritableFile(path, &writer));
+
+  int64_t initial_time;
+  ASSERT_OK(env_->GetFileModifiedTime(writer->filename(), &initial_time));
+
+  // HFS has 1 second mtime granularity.
+  AssertEventually([&] {
+    int64_t after_time;
+    writer->Append(" ");
+    writer->Sync();
+    ASSERT_OK(env_->GetFileModifiedTime(writer->filename(), &after_time));
+    ASSERT_LT(initial_time, after_time);
+  }, MonoDelta::FromSeconds(5));
+  NO_PENDING_FATALS();
+}
+
+TEST_F(TestEnv, TestRWFile) {
+  // Create the file.
+  unique_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(GetTestPath("foo"), &file));
+
+  // Append to it.
+  string kTestData = "abcde";
+  ASSERT_OK(file->Write(0, kTestData));
+
+  // Read from it.
+  uint8_t scratch[kTestData.length()];
+  Slice result(scratch, kTestData.length());
+  ASSERT_OK(file->Read(0, result));
+  ASSERT_EQ(result, kTestData);
+  uint64_t sz;
+  ASSERT_OK(file->Size(&sz));
+  ASSERT_EQ(kTestData.length(), sz);
+
+  // Read into multiple buffers
+  size_t size1 = 3;
+  uint8_t scratch1[size1];
+  Slice result1(scratch1, size1);
+  size_t size2 = 2;
+  uint8_t scratch2[size2];
+  Slice result2(scratch2, size2);
+  vector<Slice> results = { result1, result2 };
+  ASSERT_OK(file->ReadV(0, results));
+  ASSERT_EQ(result1, "abc");
+  ASSERT_EQ(result2, "de");
+
+  // Write past the end of the file and rewrite some of the interior.
+  ASSERT_OK(file->Write(kTestData.length() * 2, kTestData));
+  ASSERT_OK(file->Write(kTestData.length(), kTestData));
+  ASSERT_OK(file->Write(1, kTestData));
+  string kNewTestData = "aabcdebcdeabcde";
+  uint8_t scratch3[kNewTestData.length()];
+  Slice result3(scratch3, kNewTestData.length());
+  ASSERT_OK(file->Read(0, result3));
+
+  // Retest.
+  ASSERT_EQ(result3, kNewTestData);
+  ASSERT_OK(file->Size(&sz));
+  ASSERT_EQ(kNewTestData.length(), sz);
+
+  // Make sure we can't overwrite it.
+  RWFileOptions opts;
+  opts.mode = Env::CREATE_NON_EXISTING;
+  ASSERT_TRUE(env_->NewRWFile(opts, GetTestPath("foo"), &file).IsAlreadyPresent());
+
+  // Reopen it without truncating the existing data.
+  opts.mode = Env::OPEN_EXISTING;
+  ASSERT_OK(env_->NewRWFile(opts, GetTestPath("foo"), &file));
+  uint8_t scratch4[kNewTestData.length()];
+  Slice result4(scratch4, kNewTestData.length());
+  ASSERT_OK(file->Read(0, result4));
+  ASSERT_EQ(result4, kNewTestData);
+}
+
+TEST_F(TestEnv, TestCanonicalize) {
+  vector<string> synonyms = { GetTestPath("."), GetTestPath("./."), GetTestPath(".//./") };
+  for (const string& synonym : synonyms) {
+    string result;
+    ASSERT_OK(env_->Canonicalize(synonym, &result));
+    ASSERT_EQ(test_dir_, result);
+  }
+
+  string dir = GetTestPath("some_dir");
+  ASSERT_OK(env_->CreateDir(dir));
+  string result;
+  ASSERT_OK(env_->Canonicalize(dir + "/", &result));
+  ASSERT_EQ(dir, result);
+
+  ASSERT_TRUE(env_->Canonicalize(dir + "/bar", nullptr).IsNotFound());
+}
+
+TEST_F(TestEnv, TestGetTotalRAMBytes) {
+  int64_t ram = 0;
+  ASSERT_OK(env_->GetTotalRAMBytes(&ram));
+
+  // Can't test much about it.
+  ASSERT_GT(ram, 0);
+}
+
+// Test that CopyFile() copies all the bytes properly.
+TEST_F(TestEnv, TestCopyFile) {
+  string orig_path = GetTestPath("test");
+  string copy_path = orig_path + ".copy";
+  const int kFileSize = 1024 * 1024 + 11; // Some odd number of bytes.
+
+  Env* env = Env::Default();
+  NO_FATALS(WriteTestFile(env, orig_path, kFileSize));
+  ASSERT_OK(env_util::CopyFile(env, orig_path, copy_path, WritableFileOptions()));
+  unique_ptr<RandomAccessFile> copy;
+  ASSERT_OK(env->NewRandomAccessFile(copy_path, &copy));
+  NO_FATALS(ReadAndVerifyTestData(copy.get(), 0, kFileSize));
+}
+
+// Simple regression test for NewTempRWFile().
+TEST_F(TestEnv, TestTempRWFile) {
+  string tmpl = "foo.XXXXXX";
+  string path;
+  unique_ptr<RWFile> file;
+
+  ASSERT_OK(env_->NewTempRWFile(RWFileOptions(), tmpl, &path, &file));
+  ASSERT_NE(path, tmpl);
+  ASSERT_EQ(0, path.find("foo."));
+  ASSERT_OK(file->Close());
+  ASSERT_OK(env_->DeleteFile(path));
+}
+
+// Test that when we write data to disk we see SpaceInfo.free_bytes go down.
+TEST_F(TestEnv, TestGetSpaceInfoFreeBytes) {
+  const string kDataDir = GetTestPath("parent");
+  const string kTestFilePath = JoinPathSegments(kDataDir, "testfile");
+  const int kFileSizeBytes = 256;
+  ASSERT_OK(env_->CreateDir(kDataDir));
+
+  // Loop in case there are concurrent tests running that are modifying the
+  // filesystem.
+  ASSERT_EVENTUALLY([&] {
+    if (env_->FileExists(kTestFilePath)) {
+      ASSERT_OK(env_->DeleteFile(kTestFilePath)); // Clean up the previous iteration.
+    }
+    SpaceInfo before_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &before_space_info));
+
+    NO_FATALS(WriteTestFile(env_, kTestFilePath, kFileSizeBytes));
+
+    SpaceInfo after_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &after_space_info));
+    ASSERT_GE(before_space_info.free_bytes - after_space_info.free_bytes, kFileSizeBytes);
+  });
+}
+
+// Basic sanity check for GetSpaceInfo().
+TEST_F(TestEnv, TestGetSpaceInfoBasicInvariants) {
+  string path = GetTestDataDirectory();
+  SpaceInfo space_info;
+  ASSERT_OK(env_->GetSpaceInfo(path, &space_info));
+  ASSERT_GT(space_info.capacity_bytes, 0);
+  ASSERT_LE(space_info.free_bytes, space_info.capacity_bytes);
+  VLOG(1) << "Path " << path << " has capacity "
+          << HumanReadableNumBytes::ToString(space_info.capacity_bytes)
+          << " (" << HumanReadableNumBytes::ToString(space_info.free_bytes) << " free)";
+}
+
+TEST_F(TestEnv, TestChangeDir) {
+  string orig_dir;
+  ASSERT_OK(env_->GetCurrentWorkingDir(&orig_dir));
+
+  string cwd;
+  ASSERT_OK(env_->ChangeDir("/"));
+  ASSERT_OK(env_->GetCurrentWorkingDir(&cwd));
+  ASSERT_EQ("/", cwd);
+
+  ASSERT_OK(env_->ChangeDir(test_dir_));
+  ASSERT_OK(env_->GetCurrentWorkingDir(&cwd));
+  ASSERT_EQ(test_dir_, cwd);
+
+  ASSERT_OK(env_->ChangeDir(orig_dir));
+  ASSERT_OK(env_->GetCurrentWorkingDir(&cwd));
+  ASSERT_EQ(orig_dir, cwd);
+}
+
+TEST_F(TestEnv, TestGetExtentMap) {
+  // In order to force filesystems that use delayed allocation to write out the
+  // extents, we must Sync() after the file is done growing, and that should
+  // trigger a real fsync() to the filesystem.
+  FLAGS_never_fsync = false;
+
+  const string kTestFilePath = GetTestPath("foo");
+  const int kFileSizeBytes = 1024*1024;
+
+  // Create a test file of a particular size.
+  unique_ptr<RWFile> f;
+  ASSERT_OK(env_->NewRWFile(kTestFilePath, &f));
+  ASSERT_OK(f->PreAllocate(0, kFileSizeBytes, RWFile::CHANGE_FILE_SIZE));
+  ASSERT_OK(f->Sync());
+
+  // The number and distribution of extents differs depending on the
+  // filesystem; this just provides coverage of the code path.
+  RWFile::ExtentMap extents;
+  Status s = f->GetExtentMap(&extents);
+  if (s.IsNotSupported()) {
+    LOG(INFO) << "GetExtentMap() not supported, skipping test";
+    return;
+  }
+  ASSERT_OK(s);
+  SCOPED_TRACE(extents);
+  int num_extents = extents.size();
+  ASSERT_GT(num_extents, 0) <<
+      "There should have been at least one extent in the file";
+
+  uint64_t fs_block_size;
+  ASSERT_OK(env_->GetBlockSize(kTestFilePath, &fs_block_size));
+
+  // Look for an extent to punch. We want an extent that's at least three times
+  // the block size so that we can punch out the "middle" fs block and thus
+  // split the extent in half.
+  uint64_t found_offset = 0;
+  for (const auto& e : extents) {
+    if (e.second >= (fs_block_size * 3)) {
+      found_offset = e.first + fs_block_size;
+      break;
+    }
+  }
+  ASSERT_GT(found_offset, 0) << "Couldn't find extent to split";
+
+  // Punch out a hole and split the extent.
+  s = f->PunchHole(found_offset, fs_block_size);
+  if (s.IsNotSupported()) {
+    LOG(INFO) << "PunchHole() not supported, skipping this part of the test";
+    return;
+  }
+  ASSERT_OK(s);
+  ASSERT_OK(f->Sync());
+
+  // Test the extent map; there should be one more extent.
+  ASSERT_OK(f->GetExtentMap(&extents));
+  ASSERT_EQ(num_extents + 1, extents.size()) <<
+      "Punching a hole should have increased the number of extents by one";
+}
+
+TEST_F(TestEnv, TestInjectEIO) {
+  // Use two files to fail with.
+  FLAGS_crash_on_eio = false;
+  const string kTestRWPath1 = GetTestPath("test_env_rw_file1");
+  unique_ptr<RWFile> rw1;
+  ASSERT_OK(env_->NewRWFile(kTestRWPath1, &rw1));
+
+  const string kTestRWPath2 = GetTestPath("test_env_rw_file2");
+  unique_ptr<RWFile> rw2;
+  ASSERT_OK(env_->NewRWFile(kTestRWPath2, &rw2));
+
+  // Inject EIOs to all operations that might result in an EIO, without
+  // specifying a glob pattern (not specifying the glob pattern will inject
+  // EIOs wherever possible by default).
+  FLAGS_env_inject_eio = 1.0;
+  uint64_t size;
+  Status s = rw1->Size(&size);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+  s = rw2->Size(&size);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+
+  // Specify and verify that both files should fail by matching glob patterns
+  // to of each's literal paths.
+  FLAGS_env_inject_eio_globs = Substitute("$0,$1", kTestRWPath1, kTestRWPath2);
+  Slice result;
+  s = rw1->Read(0, result);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+  s = rw2->Size(&size);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+
+  // Inject EIOs to all operations that might result in an EIO across paths,
+  // specified with a glob pattern.
+  FLAGS_env_inject_eio_globs = "*";
+  Slice data("data");
+  s = rw1->Write(0, data);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+  s = rw2->Size(&size);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+
+  // Specify and verify that one of the files should fail by matching a glob
+  // pattern of one of the literal paths.
+  FLAGS_env_inject_eio_globs = kTestRWPath1;
+  s = rw1->Size(&size);
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+  ASSERT_OK(rw2->Write(0, data));
+
+  // Specify the directory of one of the files and ensure that fails.
+  FLAGS_env_inject_eio_globs = JoinPathSegments(DirName(kTestRWPath2), "**");
+  s = rw2->Sync();
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+
+  // Specify a directory and check that failed directory operations are caught.
+  FLAGS_env_inject_eio_globs = DirName(kTestRWPath2);
+  s = env_->SyncDir(DirName(kTestRWPath2));
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_STR_CONTAINS(s.ToString(), "INJECTED FAILURE");
+
+  // Specify that neither file fails.
+  FLAGS_env_inject_eio_globs = "neither_path";
+  ASSERT_OK(rw1->Close());
+  ASSERT_OK(rw2->Close());
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env.cc b/be/src/kudu/util/env.cc
new file mode 100644
index 0000000..90755e0
--- /dev/null
+++ b/be/src/kudu/util/env.cc
@@ -0,0 +1,93 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "kudu/util/env.h"
+
+#include <memory>
+
+#include <glog/logging.h>
+
+#include "kudu/util/faststring.h"
+#include "kudu/util/slice.h"
+
+using std::unique_ptr;
+
+namespace kudu {
+
+Env::~Env() {
+}
+
+SequentialFile::~SequentialFile() {
+}
+
+RandomAccessFile::~RandomAccessFile() {
+}
+
+WritableFile::~WritableFile() {
+}
+
+RWFile::~RWFile() {
+}
+
+FileLock::~FileLock() {
+}
+
+static Status DoWriteStringToFile(Env* env, const Slice& data,
+                                  const std::string& fname,
+                                  bool should_sync) {
+  unique_ptr<WritableFile> file;
+  Status s = env->NewWritableFile(fname, &file);
+  if (!s.ok()) {
+    return s;
+  }
+  s = file->Append(data);
+  if (s.ok() && should_sync) {
+    s = file->Sync();
+  }
+  if (s.ok()) {
+    s = file->Close();
+  }
+  file.reset();  // Will auto-close if we did not close above
+  if (!s.ok()) {
+    WARN_NOT_OK(env->DeleteFile(fname),
+                "Failed to delete partially-written file " + fname);
+  }
+  return s;
+}
+
+// TODO: move these utils into env_util
+Status WriteStringToFile(Env* env, const Slice& data,
+                         const std::string& fname) {
+  return DoWriteStringToFile(env, data, fname, false);
+}
+
+Status WriteStringToFileSync(Env* env, const Slice& data,
+                             const std::string& fname) {
+  return DoWriteStringToFile(env, data, fname, true);
+}
+
+Status ReadFileToString(Env* env, const std::string& fname, faststring* data) {
+  data->clear();
+  unique_ptr<SequentialFile> file;
+  Status s = env->NewSequentialFile(fname, &file);
+  if (!s.ok()) {
+    return s;
+  }
+  static const int kBufferSize = 8192;
+  unique_ptr<uint8_t[]> scratch(new uint8_t[kBufferSize]);
+  while (true) {
+    Slice fragment(scratch.get(), kBufferSize);
+    s = file->Read(&fragment);
+    if (!s.ok()) {
+      break;
+    }
+    data->append(fragment.data(), fragment.size());
+    if (fragment.empty()) {
+      break;
+    }
+  }
+  return s;
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env.h b/be/src/kudu/util/env.h
new file mode 100644
index 0000000..2822994
--- /dev/null
+++ b/be/src/kudu/util/env.h
@@ -0,0 +1,681 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// An Env is an interface used by the kudu implementation to access
+// operating system functionality like the filesystem etc.  Callers
+// may wish to provide a custom Env object when opening a database to
+// get fine gain control; e.g., to rate limit file system operations.
+//
+// All Env implementations are safe for concurrent access from
+// multiple threads without any external synchronization.
+
+#ifndef STORAGE_LEVELDB_INCLUDE_ENV_H_
+#define STORAGE_LEVELDB_INCLUDE_ENV_H_
+
+#include <cstddef>
+#include <cstdint>
+#include <iosfwd>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "kudu/gutil/callback_forward.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class faststring;
+class FileLock;
+class RandomAccessFile;
+class RWFile;
+class SequentialFile;
+class Slice;
+class WritableFile;
+
+struct RandomAccessFileOptions;
+struct RWFileOptions;
+struct WritableFileOptions;
+
+template <typename T>
+class ArrayView;
+
+// Returned by Env::GetSpaceInfo().
+struct SpaceInfo {
+  int64_t capacity_bytes; // Capacity of a filesystem, in bytes.
+  int64_t free_bytes;     // Bytes available to non-privileged processes.
+};
+
+class Env {
+ public:
+  // Governs if/how the file is created.
+  //
+  // enum value                      | file exists       | file does not exist
+  // --------------------------------+-------------------+--------------------
+  // CREATE_IF_NON_EXISTING_TRUNCATE | opens + truncates | creates
+  // CREATE_NON_EXISTING             | fails             | creates
+  // OPEN_EXISTING                   | opens             | fails
+  enum CreateMode {
+    CREATE_IF_NON_EXISTING_TRUNCATE,
+    CREATE_NON_EXISTING,
+    OPEN_EXISTING
+  };
+
+  Env() { }
+  virtual ~Env();
+
+  // Return a default environment suitable for the current operating
+  // system.  Sophisticated users may wish to provide their own Env
+  // implementation instead of relying on this default environment.
+  //
+  // The result of Default() belongs to kudu and must never be deleted.
+  static Env* Default();
+
+  // Create a brand new sequentially-readable file with the specified name.
+  // On success, stores a pointer to the new file in *result and returns OK.
+  // On failure stores NULL in *result and returns non-OK.  If the file does
+  // not exist, returns a non-OK status.
+  //
+  // The returned file will only be accessed by one thread at a time.
+  virtual Status NewSequentialFile(const std::string& fname,
+                                   std::unique_ptr<SequentialFile>* result) = 0;
+
+  // Create a brand new random access read-only file with the
+  // specified name.  On success, stores a pointer to the new file in
+  // *result and returns OK.  On failure stores NULL in *result and
+  // returns non-OK.  If the file does not exist, returns a non-OK
+  // status.
+  //
+  // The returned file may be concurrently accessed by multiple threads.
+  virtual Status NewRandomAccessFile(const std::string& fname,
+                                     std::unique_ptr<RandomAccessFile>* result) = 0;
+
+  // Like the previous NewRandomAccessFile, but allows options to be specified.
+  virtual Status NewRandomAccessFile(const RandomAccessFileOptions& opts,
+                                     const std::string& fname,
+                                     std::unique_ptr<RandomAccessFile>* result) = 0;
+
+  // Create an object that writes to a new file with the specified
+  // name.  Deletes any existing file with the same name and creates a
+  // new file.  On success, stores a pointer to the new file in
+  // *result and returns OK.  On failure stores NULL in *result and
+  // returns non-OK.
+  //
+  // The returned file will only be accessed by one thread at a time.
+  virtual Status NewWritableFile(const std::string& fname,
+                                 std::unique_ptr<WritableFile>* result) = 0;
+
+
+  // Like the previous NewWritableFile, but allows options to be
+  // specified.
+  virtual Status NewWritableFile(const WritableFileOptions& opts,
+                                 const std::string& fname,
+                                 std::unique_ptr<WritableFile>* result) = 0;
+
+  // Creates a new WritableFile provided the name_template parameter.
+  // The last six characters of name_template must be "XXXXXX" and these are
+  // replaced with a string that makes the filename unique.
+  // The resulting created filename, if successful, will be stored in the
+  // created_filename out parameter.
+  // The file is created with permissions 0600, that is, read plus write for
+  // owner only. The implementation will create the file in a secure manner,
+  // and will return an error Status if it is unable to open the file.
+  virtual Status NewTempWritableFile(const WritableFileOptions& opts,
+                                     const std::string& name_template,
+                                     std::string* created_filename,
+                                     std::unique_ptr<WritableFile>* result) = 0;
+
+  // Creates a new readable and writable file. If a file with the same name
+  // already exists on disk, it is deleted.
+  //
+  // Some of the methods of the new file may be accessed concurrently,
+  // while others are only safe for access by one thread at a time.
+  virtual Status NewRWFile(const std::string& fname,
+                           std::unique_ptr<RWFile>* result) = 0;
+
+  // Like the previous NewRWFile, but allows options to be specified.
+  virtual Status NewRWFile(const RWFileOptions& opts,
+                           const std::string& fname,
+                           std::unique_ptr<RWFile>* result) = 0;
+
+  // Same as abovoe for NewTempWritableFile(), but for an RWFile.
+  virtual Status NewTempRWFile(const RWFileOptions& opts,
+                               const std::string& name_template,
+                               std::string* created_filename,
+                               std::unique_ptr<RWFile>* res) = 0;
+
+  // Returns true iff the named file exists.
+  virtual bool FileExists(const std::string& fname) = 0;
+
+  // Store in *result the names of the children of the specified directory.
+  // The names are relative to "dir".
+  // Original contents of *results are dropped.
+  virtual Status GetChildren(const std::string& dir,
+                             std::vector<std::string>* result) = 0;
+
+  // Delete the named file.
+  virtual Status DeleteFile(const std::string& fname) = 0;
+
+  // Create the specified directory.
+  virtual Status CreateDir(const std::string& dirname) = 0;
+
+  // Delete the specified directory.
+  virtual Status DeleteDir(const std::string& dirname) = 0;
+
+  // Return the current working directory.
+  virtual Status GetCurrentWorkingDir(std::string* cwd) const = 0;
+
+  // Change the current working directory.
+  virtual Status ChangeDir(const std::string& dest) = 0;
+
+  // Synchronize the entry for a specific directory.
+  virtual Status SyncDir(const std::string& dirname) = 0;
+
+  // Recursively delete the specified directory.
+  // This should operate safely, not following any symlinks, etc.
+  virtual Status DeleteRecursively(const std::string &dirname) = 0;
+
+  // Store the logical size of fname in *file_size.
+  virtual Status GetFileSize(const std::string& fname, uint64_t* file_size) = 0;
+
+  // Store the physical size of fname in *file_size.
+  //
+  // This differs from GetFileSize() in that it returns the actual amount
+  // of space consumed by the file, not the user-facing file size.
+  virtual Status GetFileSizeOnDisk(const std::string& fname, uint64_t* file_size) = 0;
+
+  // Walk 'root' recursively, looking up the amount of space used by each file
+  // as reported by GetFileSizeOnDisk(), storing the grand total in 'bytes_used'.
+  virtual Status GetFileSizeOnDiskRecursively(const std::string& root, uint64_t* bytes_used) = 0;
+
+  // Returns the modified time of the file in microseconds.
+  //
+  // The timestamp is a 'system' timestamp, and is not guaranteed to be
+  // monotonic, or have any other consistency properties. The granularity of the
+  // timestamp is not guaranteed, and may be as high as 1 second on some
+  // platforms. The timestamp is not guaranteed to be anchored to any particular
+  // epoch.
+  virtual Status GetFileModifiedTime(const std::string& fname, int64_t* timestamp) = 0;
+
+  // Store the block size of the filesystem where fname resides in
+  // *block_size. fname must exist but it may be a file or a directory.
+  virtual Status GetBlockSize(const std::string& fname, uint64_t* block_size) = 0;
+
+  // Determine the capacity and number of bytes free on the filesystem
+  // specified by 'path'. "Free space" accounting on the underlying filesystem
+  // may be more coarse than single bytes.
+  virtual Status GetSpaceInfo(const std::string& path, SpaceInfo* space_info) = 0;
+
+  // Rename file src to target.
+  virtual Status RenameFile(const std::string& src,
+                            const std::string& target) = 0;
+
+  // Lock the specified file.  Used to prevent concurrent access to
+  // the same db by multiple processes.  On failure, stores NULL in
+  // *lock and returns non-OK.
+  //
+  // On success, stores a pointer to the object that represents the
+  // acquired lock in *lock and returns OK.  The caller should call
+  // UnlockFile(*lock) to release the lock.  If the process exits,
+  // the lock will be automatically released.
+  //
+  // If somebody else already holds the lock, finishes immediately
+  // with a failure.  I.e., this call does not wait for existing locks
+  // to go away.
+  //
+  // May create the named file if it does not already exist.
+  virtual Status LockFile(const std::string& fname, FileLock** lock) = 0;
+
+  // Release the lock acquired by a previous successful call to LockFile.
+  // REQUIRES: lock was returned by a successful LockFile() call
+  // REQUIRES: lock has not already been unlocked.
+  virtual Status UnlockFile(FileLock* lock) = 0;
+
+  // *path is set to a temporary directory that can be used for testing. It may
+  // or many not have just been created. The directory may or may not differ
+  // between runs of the same process, but subsequent calls will return the
+  // same directory.
+  virtual Status GetTestDirectory(std::string* path) = 0;
+
+  // Returns the number of micro-seconds since some fixed point in time. Only
+  // useful for computing deltas of time.
+  virtual uint64_t NowMicros() = 0;
+
+  // Sleep/delay the thread for the perscribed number of micro-seconds.
+  virtual void SleepForMicroseconds(int micros) = 0;
+
+  // Get caller's thread id.
+  virtual uint64_t gettid() = 0;
+
+  // Return the full path of the currently running executable.
+  virtual Status GetExecutablePath(std::string* path) = 0;
+
+  // Checks if the file is a directory. Returns an error if it doesn't
+  // exist, otherwise writes true or false into 'is_dir' appropriately.
+  virtual Status IsDirectory(const std::string& path, bool* is_dir) = 0;
+
+  // The kind of file found during a walk. Note that symbolic links are
+  // reported as FILE_TYPE.
+  enum FileType {
+    DIRECTORY_TYPE,
+    FILE_TYPE,
+  };
+
+  // Called for each file/directory in the walk.
+  //
+  // The first argument is the type of file.
+  // The second is the dirname of the file.
+  // The third is the basename of the file.
+  //
+  // Returning an error won't halt the walk, but it will cause it to return
+  // with an error status when it's done.
+  typedef Callback<Status(FileType, const std::string&, const std::string&)> WalkCallback;
+
+  // Whether to walk directories in pre-order or post-order.
+  enum DirectoryOrder {
+    PRE_ORDER,
+    POST_ORDER,
+  };
+
+  // Walk the filesystem subtree from 'root' down, invoking 'cb' for each
+  // file or directory found, including 'root'.
+  //
+  // The walk will not cross filesystem boundaries. It won't change the
+  // working directory, nor will it follow symbolic links.
+  virtual Status Walk(const std::string& root,
+                      DirectoryOrder order,
+                      const WalkCallback& cb) = 0;
+
+  // Finds paths on the filesystem matching a pattern.
+  //
+  // The found pathnames are added to the 'paths' vector. If no pathnames are
+  // found matching the pattern, no paths are added to the vector and an OK
+  // status is returned.
+  virtual Status Glob(const std::string& path_pattern, std::vector<std::string>* paths) = 0;
+
+  // Canonicalize 'path' by applying the following conversions:
+  // - Converts a relative path into an absolute one using the cwd.
+  // - Converts '.' and '..' references.
+  // - Resolves all symbolic links.
+  //
+  // All directory entries in 'path' must exist on the filesystem.
+  virtual Status Canonicalize(const std::string& path, std::string* result) = 0;
+
+  // Gets the total amount of RAM installed on this machine.
+  virtual Status GetTotalRAMBytes(int64_t* ram) = 0;
+
+  enum class ResourceLimitType {
+    // The maximum number of file descriptors that this process can have open
+    // at any given time.
+    //
+    // Corresponds to RLIMIT_NOFILE on UNIX platforms.
+    OPEN_FILES_PER_PROCESS,
+
+    // The maximum number of threads (or processes) that this process's
+    // effective user ID may have spawned and running at any given time.
+    //
+    // Corresponds to RLIMIT_NPROC on UNIX platforms.
+    RUNNING_THREADS_PER_EUID,
+  };
+
+  // Gets the process' current limit for the given resource type.
+  //
+  // On UNIX platforms, this is equivalent to the resource's soft limit.
+  virtual uint64_t GetResourceLimit(ResourceLimitType t) = 0;
+
+  // Increases the resource limit by as much as possible.
+  //
+  // On UNIX platforms, this means increasing the resource's soft limit (the
+  // limit actually enforced by the kernel) to be equal to the hard limit.
+  virtual void IncreaseResourceLimit(ResourceLimitType t) = 0;
+
+  // Checks whether the given path resides on an ext2, ext3, or ext4
+  // filesystem.
+  //
+  // On success, 'result' contains the answer. On failure, 'result' is unset.
+  virtual Status IsOnExtFilesystem(const std::string& path, bool* result) = 0;
+
+  // Checks whether the given path resides on an xfs filesystem.
+  //
+  // On success, 'result' contains the answer. On failure, 'result' is unset.
+  virtual Status IsOnXfsFilesystem(const std::string& path, bool* result) = 0;
+
+  // Gets the kernel release string for this machine.
+  virtual std::string GetKernelRelease() = 0;
+
+  // Ensure that the file with the given path has permissions which adhere
+  // to the current configured umask (from flags.h). If the permissions are
+  // wider than the current umask, then a warning is logged and the permissions
+  // are fixed.
+  //
+  // Returns a bad Status if the file does not exist or the permissions cannot
+  // be changed.
+  virtual Status EnsureFileModeAdheresToUmask(const std::string& path) = 0;
+
+  // Checks whether the given path has world-readable permissions.
+  //
+  // On success, 'result' contains the answer. On failure, 'result' is unset.
+  virtual Status IsFileWorldReadable(const std::string& path, bool* result) = 0;
+
+  // Special string injected into file-growing operations' random failures
+  // (if enabled).
+  //
+  // Only useful for tests.
+  static const char* const kInjectedFailureStatusMsg;
+
+ private:
+  // No copying allowed
+  Env(const Env&);
+  void operator=(const Env&);
+};
+
+// A file abstraction for reading sequentially through a file
+class SequentialFile {
+ public:
+  SequentialFile() { }
+  virtual ~SequentialFile();
+
+  // Read up to "result.size" bytes from the file.
+  // Sets "result.data" to the data that was read.
+  //
+  // If an error was encountered, returns a non-OK status
+  // and the contents of "result" are invalid.
+  //
+  // REQUIRES: External synchronization
+  virtual Status Read(Slice* result) = 0;
+
+  // Skip "n" bytes from the file. This is guaranteed to be no
+  // slower that reading the same data, but may be faster.
+  //
+  // If end of file is reached, skipping will stop at the end of the
+  // file, and Skip will return OK.
+  //
+  // REQUIRES: External synchronization
+  virtual Status Skip(uint64_t n) = 0;
+
+  // Returns the filename provided when the SequentialFile was constructed.
+  virtual const std::string& filename() const = 0;
+};
+
+// A file abstraction for randomly reading the contents of a file.
+class RandomAccessFile {
+ public:
+  RandomAccessFile() { }
+  virtual ~RandomAccessFile();
+
+  // Read "result.size" bytes from the file starting at "offset".
+  // Copies the resulting data into "result.data".
+  //
+  // If an error was encountered, returns a non-OK status.
+  //
+  // This method will internally retry on EINTR and "short reads" in order to
+  // fully read the requested number of bytes. In the event that it is not
+  // possible to read exactly 'length' bytes, an IOError is returned.
+  //
+  // Safe for concurrent use by multiple threads.
+  virtual Status Read(uint64_t offset, Slice result) const = 0;
+
+  // Reads up to the "results" aggregate size, based on each Slice's "size",
+  // from the file starting at 'offset'. The Slices must point to already-allocated
+  // buffers for the data to be written to.
+  //
+  // If an error was encountered, returns a non-OK status.
+  //
+  // This method will internally retry on EINTR and "short reads" in order to
+  // fully read the requested number of bytes. In the event that it is not
+  // possible to read exactly 'length' bytes, an IOError is returned.
+  //
+  // Safe for concurrent use by multiple threads.
+  virtual Status ReadV(uint64_t offset, ArrayView<Slice> results) const = 0;
+
+  // Returns the size of the file
+  virtual Status Size(uint64_t *size) const = 0;
+
+  // Returns the filename provided when the RandomAccessFile was constructed.
+  virtual const std::string& filename() const = 0;
+
+  // Returns the approximate memory usage of this RandomAccessFile including
+  // the object itself.
+  virtual size_t memory_footprint() const = 0;
+};
+
+// Creation-time options for WritableFile
+struct WritableFileOptions {
+  // Call Sync() during Close().
+  bool sync_on_close;
+
+  // See CreateMode for details.
+  Env::CreateMode mode;
+
+  WritableFileOptions()
+    : sync_on_close(false),
+      mode(Env::CREATE_IF_NON_EXISTING_TRUNCATE) { }
+};
+
+// Options specified when a file is opened for random access.
+struct RandomAccessFileOptions {
+  RandomAccessFileOptions() {}
+};
+
+// A file abstraction for sequential writing.  The implementation
+// must provide buffering since callers may append small fragments
+// at a time to the file.
+class WritableFile {
+ public:
+  enum FlushMode {
+    FLUSH_SYNC,
+    FLUSH_ASYNC
+  };
+
+  WritableFile() { }
+  virtual ~WritableFile();
+
+  virtual Status Append(const Slice& data) = 0;
+
+  // If possible, uses scatter-gather I/O to efficiently append
+  // multiple buffers to a file. Otherwise, falls back to regular I/O.
+  //
+  // For implementation specific quirks and details, see comments in
+  // implementation source code (e.g., env_posix.cc)
+  virtual Status AppendV(ArrayView<const Slice> data) = 0;
+
+  // Pre-allocates 'size' bytes for the file in the underlying filesystem.
+  // size bytes are added to the current pre-allocated size or to the current
+  // offset, whichever is bigger. In no case is the file truncated by this
+  // operation.
+  //
+  // On some implementations, preallocation is done without initializing the
+  // contents of the data blocks (as opposed to writing zeroes), requiring no
+  // IO to the data blocks.
+  //
+  // In no case is the file truncated by this operation.
+  virtual Status PreAllocate(uint64_t size) = 0;
+
+  virtual Status Close() = 0;
+
+  // Flush all dirty data (not metadata) to disk.
+  //
+  // If the flush mode is synchronous, will wait for flush to finish and
+  // return a meaningful status.
+  virtual Status Flush(FlushMode mode) = 0;
+
+  virtual Status Sync() = 0;
+
+  virtual uint64_t Size() const = 0;
+
+  // Returns the filename provided when the WritableFile was constructed.
+  virtual const std::string& filename() const = 0;
+
+ private:
+  // No copying allowed
+  WritableFile(const WritableFile&);
+  void operator=(const WritableFile&);
+};
+
+// Creation-time options for RWFile
+struct RWFileOptions {
+  // Call Sync() during Close().
+  bool sync_on_close;
+
+  // See CreateMode for details.
+  Env::CreateMode mode;
+
+  RWFileOptions()
+    : sync_on_close(false),
+      mode(Env::CREATE_IF_NON_EXISTING_TRUNCATE) { }
+};
+
+// A file abstraction for both reading and writing. No notion of a built-in
+// file offset is ever used; instead, all operations must provide an
+// explicit offset.
+//
+// All operations are safe for concurrent use by multiple threads (unless
+// noted otherwise) bearing in mind the usual filesystem coherency guarantees
+// (e.g. two threads that write concurrently to the same file offset will
+// probably yield garbage).
+class RWFile {
+ public:
+  enum FlushMode {
+    FLUSH_SYNC,
+    FLUSH_ASYNC
+  };
+
+  RWFile() {
+  }
+
+  virtual ~RWFile();
+
+  // Read "result.size" bytes from the file starting at "offset".
+  // Copies the resulting data into "result.data".
+  // If an error was encountered, returns a non-OK status.
+  //
+  // This method will internally retry on EINTR and "short reads" in order to
+  // fully read the requested number of bytes. In the event that it is not
+  // possible to read exactly 'length' bytes, an IOError is returned.
+  //
+  // Safe for concurrent use by multiple threads.
+  virtual Status Read(uint64_t offset, Slice result) const = 0;
+
+  // Reads up to the "results" aggregate size, based on each Slice's "size",
+  // from the file starting at 'offset'. The Slices must point to already-allocated
+  // buffers for the data to be written to.
+  //
+  // If an error was encountered, returns a non-OK status.
+  //
+  // This method will internally retry on EINTR and "short reads" in order to
+  // fully read the requested number of bytes. In the event that it is not
+  // possible to read exactly 'length' bytes, an IOError is returned.
+  //
+  // Safe for concurrent use by multiple threads.
+  virtual Status ReadV(uint64_t offset, ArrayView<Slice> results) const = 0;
+
+  // Writes 'data' to the file position given by 'offset'.
+  virtual Status Write(uint64_t offset, const Slice& data) = 0;
+
+  // Writes the 'data' slices to the file position given by 'offset'.
+  virtual Status WriteV(uint64_t offset, ArrayView<const Slice> data) = 0;
+
+  // Preallocates 'length' bytes for the file in the underlying filesystem
+  // beginning at 'offset'. It is safe to preallocate the same range
+  // repeatedly; this is an idempotent operation.
+  //
+  // On some implementations, preallocation is done without initializing the
+  // contents of the data blocks (as opposed to writing zeroes), requiring no
+  // IO to the data blocks. On such implementations, this is much faster than
+  // using Truncate() to increase the file size.
+  //
+  // In no case is the file truncated by this operation.
+  //
+  // 'mode' controls whether the file's logical size grows to include the
+  // preallocated space, or whether it remains the same.
+  enum PreAllocateMode {
+    CHANGE_FILE_SIZE,
+    DONT_CHANGE_FILE_SIZE
+  };
+  virtual Status PreAllocate(uint64_t offset,
+                             size_t length,
+                             PreAllocateMode mode) = 0;
+
+  // Truncate the file. If 'new_size' is less than the previous file size, the
+  // extra data will be lost. If 'new_size' is greater than the previous file
+  // size, the file length is extended, and the extended portion will contain
+  // null bytes ('\0').
+  virtual Status Truncate(uint64_t length) = 0;
+
+  // Deallocates space given by 'offset' and length' from the file,
+  // effectively "punching a hole" in it. The space will be reclaimed by
+  // the filesystem and reads to that range will return zeroes. Useful
+  // for making whole files sparse.
+  //
+  // Filesystems that don't implement this will return an error.
+  virtual Status PunchHole(uint64_t offset, size_t length) = 0;
+
+  // Flushes the range of dirty data (not metadata) given by 'offset' and
+  // 'length' to disk. If length is 0, all bytes from 'offset' to the end
+  // of the file are flushed.
+  //
+  // If the flush mode is synchronous, will wait for flush to finish and
+  // return a meaningful status.
+  virtual Status Flush(FlushMode mode, uint64_t offset, size_t length) = 0;
+
+  // Synchronously flushes all dirty file data and metadata to disk. Upon
+  // returning successfully, all previously issued file changes have been
+  // made durable.
+  virtual Status Sync() = 0;
+
+  // Closes the file, optionally calling Sync() on it if the file was
+  // created with the sync_on_close option enabled.
+  //
+  // Not thread-safe.
+  virtual Status Close() = 0;
+
+  // Retrieves the file's size.
+  virtual Status Size(uint64_t* size) const = 0;
+
+  // Retrieve a map of the file's live extents.
+  //
+  // Each map entry is an offset and size representing a section of live file
+  // data. Any byte offset not contained in a map entry implicitly belongs to a
+  // "hole" in the (sparse) file.
+  //
+  // If the underlying filesystem does not support extents, map entries
+  // represent runs of adjacent fixed-size filesystem blocks instead. If the
+  // platform doesn't support fetching extents at all, a NotSupported status
+  // will be returned.
+  typedef std::map<uint64_t, uint64_t> ExtentMap;
+  virtual Status GetExtentMap(ExtentMap* out) const = 0;
+
+  // Returns the filename provided when the RWFile was constructed.
+  virtual const std::string& filename() const = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(RWFile);
+};
+
+// Identifies a locked file.
+class FileLock {
+ public:
+  FileLock() { }
+  virtual ~FileLock();
+ private:
+  // No copying allowed
+  FileLock(const FileLock&);
+  void operator=(const FileLock&);
+};
+
+// A utility routine: write "data" to the named file.
+extern Status WriteStringToFile(Env* env, const Slice& data,
+                                const std::string& fname);
+
+// A utility routine: read contents of named file into *data
+extern Status ReadFileToString(Env* env, const std::string& fname,
+                               faststring* data);
+
+// Overloaded operator for printing Env::ResourceLimitType.
+std::ostream& operator<<(std::ostream& o, Env::ResourceLimitType t);
+
+}  // namespace kudu
+
+#endif  // STORAGE_LEVELDB_INCLUDE_ENV_H_


[10/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/os-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/os-util.h b/be/src/kudu/util/os-util.h
new file mode 100644
index 0000000..7e1bbb6
--- /dev/null
+++ b/be/src/kudu/util/os-util.h
@@ -0,0 +1,72 @@
+// 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.
+//
+// Imported from Impala. Changes include:
+// - Namespace + imports.
+// - Fixes for cpplint.
+// - Fixed parsing when thread names have spaces.
+
+#ifndef KUDU_UTIL_OS_UTIL_H
+#define KUDU_UTIL_OS_UTIL_H
+
+#include <cstdint>
+#include <string>
+#include <type_traits> // IWYU pragma: keep
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Utility methods to read interesting values from /proc.
+// TODO: Get stats for parent process.
+
+// Container struct for statistics read from the /proc filesystem for a thread.
+struct ThreadStats {
+  int64_t user_ns;
+  int64_t kernel_ns;
+  int64_t iowait_ns;
+
+  // Default constructor zeroes all members in case structure can't be filled by
+  // GetThreadStats.
+  ThreadStats() : user_ns(0), kernel_ns(0), iowait_ns(0) { }
+};
+
+// Populates ThreadStats object using a given buffer. The buffer is expected to
+// conform to /proc/<pid>/task/<tid>/stat layout; an error will be returned otherwise.
+//
+// If 'name' is supplied, the extracted thread name will be written to it.
+Status ParseStat(const std::string&buffer, std::string* name, ThreadStats* stats);
+
+// Populates ThreadStats object for a given thread by reading from
+// /proc/<pid>/task/<tid>/stat. Returns OK unless the file cannot be read or is in an
+// unrecognised format, or if the kernel version is not modern enough.
+Status GetThreadStats(int64_t tid, ThreadStats* stats);
+
+// Disable core dumps for this process.
+//
+// This is useful particularly in tests where we have injected failures and don't
+// want to generate a core dump from an "expected" crash.
+void DisableCoreDumps();
+
+// Return true if this process appears to be running under a debugger or strace.
+//
+// This may return false on unsupported (non-Linux) platforms.
+bool IsBeingDebugged();
+
+} // namespace kudu
+
+#endif /* KUDU_UTIL_OS_UTIL_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/path_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/path_util-test.cc b/be/src/kudu/util/path_util-test.cc
new file mode 100644
index 0000000..0d617fc
--- /dev/null
+++ b/be/src/kudu/util/path_util-test.cc
@@ -0,0 +1,77 @@
+// 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 <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/path_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+TEST(TestPathUtil, BaseNameTest) {
+  ASSERT_EQ(".", BaseName(""));
+  ASSERT_EQ(".", BaseName("."));
+  ASSERT_EQ("..", BaseName(".."));
+  ASSERT_EQ("/", BaseName("/"));
+  ASSERT_EQ("/", BaseName("//"));
+  ASSERT_EQ("a", BaseName("a"));
+  ASSERT_EQ("ab", BaseName("ab"));
+  ASSERT_EQ("ab", BaseName("ab/"));
+  ASSERT_EQ("cd", BaseName("ab/cd"));
+  ASSERT_EQ("ab", BaseName("/ab"));
+  ASSERT_EQ("ab", BaseName("/ab///"));
+  ASSERT_EQ("cd", BaseName("/ab/cd"));
+}
+
+TEST(TestPathUtil, DirNameTest) {
+  ASSERT_EQ(".", DirName(""));
+  ASSERT_EQ(".", DirName("."));
+  ASSERT_EQ(".", DirName(".."));
+  ASSERT_EQ("/", DirName("/"));
+#if defined(__linux__)
+  // On OS X this test case returns "/", while Linux returns "//". On both
+  // platforms dirname(1) returns "/". The difference is unlikely to matter in
+  // practice.
+  ASSERT_EQ("//", DirName("//"));
+#else
+  ASSERT_EQ("/", DirName("//"));
+#endif // defined(__linux__)
+  ASSERT_EQ(".", DirName("a"));
+  ASSERT_EQ(".", DirName("ab"));
+  ASSERT_EQ(".", DirName("ab/"));
+  ASSERT_EQ("ab", DirName("ab/cd"));
+  ASSERT_EQ("/", DirName("/ab"));
+  ASSERT_EQ("/", DirName("/ab///"));
+  ASSERT_EQ("/ab", DirName("/ab/cd"));
+}
+
+TEST(TestPathUtil, SplitPathTest) {
+  typedef vector<string> Vec;
+  ASSERT_EQ(Vec({"/"}), SplitPath("/"));
+  ASSERT_EQ(Vec({"/", "a", "b"}), SplitPath("/a/b"));
+  ASSERT_EQ(Vec({"/", "a", "b"}), SplitPath("/a/b/"));
+  ASSERT_EQ(Vec({"a", "b"}), SplitPath("a/b"));
+  ASSERT_EQ(Vec({"."}), SplitPath("."));
+  ASSERT_EQ(Vec(), SplitPath(""));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/path_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/path_util.cc b/be/src/kudu/util/path_util.cc
new file mode 100644
index 0000000..6d1c4a5
--- /dev/null
+++ b/be/src/kudu/util/path_util.cc
@@ -0,0 +1,122 @@
+// 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 "kudu/util/path_util.h"
+
+// Use the POSIX version of dirname(3).
+#include <libgen.h>
+
+#include <cstring>
+#if defined(__APPLE__)
+#include <mutex>
+#endif // defined(__APPLE__)
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/util/env.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+
+
+using std::string;
+using std::vector;
+using strings::SkipEmpty;
+using strings::Split;
+
+namespace kudu {
+
+const char kTmpInfix[] = ".kudutmp";
+const char kOldTmpInfix[] = ".tmp";
+
+string JoinPathSegments(const string& a, const string& b) {
+  CHECK(!a.empty()) << "empty first component: " << a;
+  CHECK(!b.empty() && b[0] != '/')
+    << "second path component must be non-empty and relative: "
+    << b;
+  if (a[a.size() - 1] == '/') {
+    return a + b;
+  } else {
+    return a + "/" + b;
+  }
+}
+
+vector<string> JoinPathSegmentsV(const vector<string>& v, const string& s) {
+  vector<string> out;
+  for (const string& path : v) {
+    out.emplace_back(JoinPathSegments(path, s));
+  }
+  return out;
+}
+
+vector<string> SplitPath(const string& path) {
+  if (path.empty()) return {};
+  vector<string> segments;
+  if (path[0] == '/') segments.emplace_back("/");
+  vector<StringPiece> pieces = Split(path, "/", SkipEmpty());
+  for (const StringPiece& piece : pieces) {
+    segments.emplace_back(piece.data(), piece.size());
+  }
+  return segments;
+}
+
+string DirName(const string& path) {
+  gscoped_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
+#if defined(__APPLE__)
+  static std::mutex lock;
+  std::lock_guard<std::mutex> l(lock);
+#endif // defined(__APPLE__)
+  return ::dirname(path_copy.get());
+}
+
+string BaseName(const string& path) {
+  gscoped_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
+  return basename(path_copy.get());
+}
+
+Status FindExecutable(const string& binary,
+                      const vector<string>& search,
+                      string* path) {
+  string p;
+
+  // First, check specified locations. This is necessary to check first so that
+  // the system binaries won't be found before the specified search locations.
+  for (const auto& location : search) {
+    p = JoinPathSegments(location, binary);
+    if (Env::Default()->FileExists(p)) {
+      *path = p;
+      return Status::OK();
+    }
+  }
+
+  // Next check if the binary is on the PATH.
+  Status s = Subprocess::Call({ "which", binary }, "", &p);
+  if (s.ok()) {
+    StripTrailingNewline(&p);
+    *path = p;
+    return Status::OK();
+  }
+
+  return Status::NotFound("Unable to find binary", binary);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/path_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/path_util.h b/be/src/kudu/util/path_util.h
new file mode 100644
index 0000000..58211a9
--- /dev/null
+++ b/be/src/kudu/util/path_util.h
@@ -0,0 +1,63 @@
+// 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.
+//
+// Utility methods for dealing with file paths.
+#ifndef KUDU_UTIL_PATH_UTIL_H
+#define KUDU_UTIL_PATH_UTIL_H
+
+#include <string>
+#include <vector>
+
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+class Status;
+
+// Common tmp infix
+extern const char kTmpInfix[];
+// Infix from versions of Kudu prior to 1.2.
+extern const char kOldTmpInfix[];
+
+// Join two path segments with the appropriate path separator,
+// if necessary.
+std::string JoinPathSegments(const std::string& a,
+                             const std::string& b);
+
+// Join each path segment in a list with a common suffix segment.
+std::vector<std::string> JoinPathSegmentsV(const std::vector<std::string>& v,
+                                           const std::string& s);
+
+// Split a path into segments with the appropriate path separator.
+std::vector<std::string> SplitPath(const std::string& path);
+
+// Return the enclosing directory of path.
+// This is like dirname(3) but for C++ strings.
+std::string DirName(const std::string& path);
+
+// Return the terminal component of a path.
+// This is like basename(3) but for C++ strings.
+std::string BaseName(const std::string& path);
+
+// Attempts to find the path to the executable, searching the provided locations
+// as well as the $PATH environment variable.
+Status FindExecutable(const std::string& binary,
+                      const std::vector<std::string>& search,
+                      std::string* path) WARN_UNUSED_RESULT;
+
+} // namespace kudu
+#endif /* KUDU_UTIL_PATH_UTIL_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util-internal.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util-internal.cc b/be/src/kudu/util/pb_util-internal.cc
new file mode 100644
index 0000000..380072c
--- /dev/null
+++ b/be/src/kudu/util/pb_util-internal.cc
@@ -0,0 +1,105 @@
+// 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 "kudu/util/pb_util-internal.h"
+
+#include <ostream>
+#include <string>
+
+namespace kudu {
+namespace pb_util {
+namespace internal {
+
+////////////////////////////////////////////
+// SequentialFileFileInputStream
+////////////////////////////////////////////
+
+bool SequentialFileFileInputStream::Next(const void **data, int *size) {
+  if (PREDICT_FALSE(!status_.ok())) {
+    LOG(WARNING) << "Already failed on a previous read: " << status_.ToString();
+    return false;
+  }
+
+  size_t available = (buffer_used_ - buffer_offset_);
+  if (available > 0) {
+    *data = buffer_.get() + buffer_offset_;
+    *size = available;
+    buffer_offset_ += available;
+    total_read_ += available;
+    return true;
+  }
+
+  Slice result(buffer_.get(), buffer_size_);
+  status_ = rfile_->Read(&result);
+  if (!status_.ok()) {
+    LOG(WARNING) << "Read at " << buffer_offset_ << " failed: " << status_.ToString();
+    return false;
+  }
+
+  buffer_used_ = result.size();
+  buffer_offset_ = buffer_used_;
+  total_read_ += buffer_used_;
+  *data = buffer_.get();
+  *size = buffer_used_;
+  return buffer_used_ > 0;
+}
+
+bool SequentialFileFileInputStream::Skip(int count) {
+  CHECK_GT(count, 0);
+  int avail = (buffer_used_ - buffer_offset_);
+  if (avail > count) {
+    buffer_offset_ += count;
+    total_read_ += count;
+  } else {
+    buffer_used_ = 0;
+    buffer_offset_ = 0;
+    status_ = rfile_->Skip(count - avail);
+    total_read_ += count - avail;
+  }
+  return status_.ok();
+}
+
+////////////////////////////////////////////
+// WritableFileOutputStream
+////////////////////////////////////////////
+
+bool WritableFileOutputStream::Next(void **data, int *size) {
+  if (PREDICT_FALSE(!status_.ok())) {
+    LOG(WARNING) << "Already failed on a previous write: " << status_.ToString();
+    return false;
+  }
+
+  size_t available = (buffer_size_ - buffer_offset_);
+  if (available > 0) {
+    *data = buffer_.get() + buffer_offset_;
+    *size = available;
+    buffer_offset_ += available;
+    return true;
+  }
+
+  if (!Flush()) {
+    return false;
+  }
+
+  buffer_offset_ = buffer_size_;
+  *data = buffer_.get();
+  *size = buffer_size_;
+  return true;
+}
+
+} // namespace internal
+} // namespace pb_util
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util-internal.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util-internal.h b/be/src/kudu/util/pb_util-internal.h
new file mode 100644
index 0000000..48a501d
--- /dev/null
+++ b/be/src/kudu/util/pb_util-internal.h
@@ -0,0 +1,136 @@
+// 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.
+//
+// Classes used internally by pb_util.h.
+// This header should not be included by anything but pb_util and its tests.
+#ifndef KUDU_UTIL_PB_UTIL_INTERNAL_H
+#define KUDU_UTIL_PB_UTIL_INTERNAL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+
+#include <glog/logging.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/env.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace pb_util {
+namespace internal {
+
+// Input Stream used by ParseFromSequentialFile()
+class SequentialFileFileInputStream : public google::protobuf::io::ZeroCopyInputStream {
+ public:
+  explicit SequentialFileFileInputStream(SequentialFile *rfile,
+                                         size_t buffer_size = kDefaultBufferSize)
+    : buffer_used_(0), buffer_offset_(0),
+      buffer_size_(buffer_size), buffer_(new uint8[buffer_size_]),
+      total_read_(0), rfile_(rfile) {
+    CHECK_GT(buffer_size, 0);
+  }
+
+  ~SequentialFileFileInputStream() {
+  }
+
+  bool Next(const void **data, int *size) OVERRIDE;
+  bool Skip(int count) OVERRIDE;
+
+  void BackUp(int count) OVERRIDE {
+    CHECK_GE(count, 0);
+    CHECK_LE(count, buffer_offset_);
+    buffer_offset_ -= count;
+    total_read_ -= count;
+  }
+
+  int64_t ByteCount() const OVERRIDE {
+    return total_read_;
+  }
+
+  Status status() const {
+    return status_;
+  }
+
+ private:
+  static const size_t kDefaultBufferSize = 8192;
+
+  Status status_;
+
+  size_t buffer_used_;
+  size_t buffer_offset_;
+  const size_t buffer_size_;
+  std::unique_ptr<uint8_t[]> buffer_;
+
+  size_t total_read_;
+  SequentialFile *rfile_;
+};
+
+// Output Stream used by SerializeToWritableFile()
+class WritableFileOutputStream : public google::protobuf::io::ZeroCopyOutputStream {
+ public:
+  explicit WritableFileOutputStream(WritableFile *wfile, size_t buffer_size = kDefaultBufferSize)
+    : buffer_offset_(0), buffer_size_(buffer_size), buffer_(new uint8[buffer_size_]),
+      flushed_(0), wfile_(wfile) {
+    CHECK_GT(buffer_size, 0);
+  }
+
+  ~WritableFileOutputStream() {
+  }
+
+  bool Flush() {
+    if (buffer_offset_ > 0) {
+      Slice data(buffer_.get(), buffer_offset_);
+      status_ = wfile_->Append(data);
+      flushed_ += buffer_offset_;
+      buffer_offset_ = 0;
+    }
+    return status_.ok();
+  }
+
+  bool Next(void **data, int *size) OVERRIDE;
+
+  void BackUp(int count) OVERRIDE {
+    CHECK_GE(count, 0);
+    CHECK_LE(count, buffer_offset_);
+    buffer_offset_ -= count;
+  }
+
+  int64_t ByteCount() const OVERRIDE {
+    return flushed_ + buffer_offset_;
+  }
+
+ private:
+  static const size_t kDefaultBufferSize = 8192;
+
+  Status status_;
+
+  size_t buffer_offset_;
+  const size_t buffer_size_;
+  std::unique_ptr<uint8_t[]> buffer_;
+
+  size_t flushed_;
+  WritableFile *wfile_;
+};
+
+} // namespace internal
+} // namespace pb_util
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util-test.cc b/be/src/kudu/util/pb_util-test.cc
new file mode 100644
index 0000000..a942e9a
--- /dev/null
+++ b/be/src/kudu/util/pb_util-test.cc
@@ -0,0 +1,661 @@
+// 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 <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/pb_util-internal.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/pb_util_test.pb.h"
+#include "kudu/util/proto_container_test.pb.h"
+#include "kudu/util/proto_container_test2.pb.h"
+#include "kudu/util/proto_container_test3.pb.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+namespace pb_util {
+
+using google::protobuf::FileDescriptorSet;
+using internal::WritableFileOutputStream;
+using std::ostringstream;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+static const char* kTestFileName = "pb_container.meta";
+static const char* kTestKeyvalName = "my-key";
+static const int kTestKeyvalValue = 1;
+static const int kUseDefaultVersion = 0; // Use the default container version (don't set it).
+
+class TestPBUtil : public KuduTest {
+ public:
+  virtual void SetUp() OVERRIDE {
+    KuduTest::SetUp();
+    path_ = GetTestPath(kTestFileName);
+  }
+
+ protected:
+  // Create a container file with expected values.
+  // Since this is a unit test class, and we want it to be fast, we do not
+  // fsync by default.
+  Status CreateKnownGoodContainerFile(CreateMode create = OVERWRITE,
+                                      SyncMode sync = NO_SYNC);
+
+  // Create a new Protobuf Container File Writer.
+  // Set version to kUseDefaultVersion to use the default version.
+  Status NewPBCWriter(int version, RWFileOptions opts,
+                      unique_ptr<WritablePBContainerFile>* pb_writer);
+
+  // Same as CreateKnownGoodContainerFile(), but with settable file version.
+  // Set version to kUseDefaultVersion to use the default version.
+  Status CreateKnownGoodContainerFileWithVersion(int version,
+                                                 CreateMode create = OVERWRITE,
+                                                 SyncMode sync = NO_SYNC);
+
+  // XORs the data in the specified range of the file at the given path.
+  Status BitFlipFileByteRange(const string& path, uint64_t offset, uint64_t length);
+
+  void DumpPBCToString(const string& path, ReadablePBContainerFile::Format format, string* ret);
+
+  // Truncate the specified file to the specified length.
+  Status TruncateFile(const string& path, uint64_t size);
+
+  // Output file name for most unit tests.
+  string path_;
+};
+
+// Parameterized test class for running tests across various versions of PB
+// container files.
+class TestPBContainerVersions : public TestPBUtil,
+                                public ::testing::WithParamInterface<int> {
+ public:
+  TestPBContainerVersions()
+      : version_(GetParam()) {
+  }
+
+ protected:
+  const int version_; // The parameterized container version we are testing.
+};
+
+INSTANTIATE_TEST_CASE_P(SupportedVersions, TestPBContainerVersions,
+                        ::testing::Values(1, 2, kUseDefaultVersion));
+
+Status TestPBUtil::CreateKnownGoodContainerFile(CreateMode create, SyncMode sync) {
+  ProtoContainerTestPB test_pb;
+  test_pb.set_name(kTestKeyvalName);
+  test_pb.set_value(kTestKeyvalValue);
+  return WritePBContainerToPath(env_, path_, test_pb, create, sync);
+}
+
+Status TestPBUtil::NewPBCWriter(int version, RWFileOptions opts,
+                                unique_ptr<WritablePBContainerFile>* pb_writer) {
+  unique_ptr<RWFile> writer;
+  RETURN_NOT_OK(env_->NewRWFile(opts, path_, &writer));
+  pb_writer->reset(new WritablePBContainerFile(std::move(writer)));
+  if (version != kUseDefaultVersion) {
+    (*pb_writer)->SetVersionForTests(version);
+  }
+  return Status::OK();
+}
+
+Status TestPBUtil::CreateKnownGoodContainerFileWithVersion(int version,
+                                                           CreateMode create,
+                                                           SyncMode sync) {
+  ProtoContainerTestPB test_pb;
+  test_pb.set_name(kTestKeyvalName);
+  test_pb.set_value(kTestKeyvalValue);
+
+  unique_ptr<WritablePBContainerFile> pb_writer;
+  RETURN_NOT_OK(NewPBCWriter(version, RWFileOptions(), &pb_writer));
+  RETURN_NOT_OK(pb_writer->CreateNew(test_pb));
+  RETURN_NOT_OK(pb_writer->Append(test_pb));
+  RETURN_NOT_OK(pb_writer->Close());
+  return Status::OK();
+}
+
+Status TestPBUtil::BitFlipFileByteRange(const string& path, uint64_t offset, uint64_t length) {
+  faststring buf;
+  // Read the data from disk.
+  {
+    unique_ptr<RandomAccessFile> file;
+    RETURN_NOT_OK(env_->NewRandomAccessFile(path, &file));
+    uint64_t size;
+    RETURN_NOT_OK(file->Size(&size));
+    faststring scratch;
+    scratch.resize(size);
+    Slice slice(scratch.data(), size);
+    RETURN_NOT_OK(file->Read(0, slice));
+    buf.append(slice.data(), slice.size());
+  }
+
+  // Flip the bits.
+  for (uint64_t i = 0; i < length; i++) {
+    uint8_t* addr = buf.data() + offset + i;
+    *addr = ~*addr;
+  }
+
+  // Write the data back to disk.
+  unique_ptr<WritableFile> file;
+  RETURN_NOT_OK(env_->NewWritableFile(path, &file));
+  RETURN_NOT_OK(file->Append(buf));
+  RETURN_NOT_OK(file->Close());
+
+  return Status::OK();
+}
+
+Status TestPBUtil::TruncateFile(const string& path, uint64_t size) {
+  unique_ptr<RWFile> file;
+  RWFileOptions opts;
+  opts.mode = Env::OPEN_EXISTING;
+  RETURN_NOT_OK(env_->NewRWFile(opts, path, &file));
+  RETURN_NOT_OK(file->Truncate(size));
+  return Status::OK();
+}
+
+TEST_F(TestPBUtil, TestWritableFileOutputStream) {
+  shared_ptr<WritableFile> file;
+  string path = GetTestPath("test.out");
+  ASSERT_OK(env_util::OpenFileForWrite(env_, path, &file));
+
+  WritableFileOutputStream stream(file.get(), 4096);
+
+  void* buf;
+  int size;
+
+  // First call should yield the whole buffer.
+  ASSERT_TRUE(stream.Next(&buf, &size));
+  ASSERT_EQ(4096, size);
+  ASSERT_EQ(4096, stream.ByteCount());
+
+  // Backup 1000 and the next call should yield 1000
+  stream.BackUp(1000);
+  ASSERT_EQ(3096, stream.ByteCount());
+
+  ASSERT_TRUE(stream.Next(&buf, &size));
+  ASSERT_EQ(1000, size);
+
+  // Another call should flush and yield a new buffer of 4096
+  ASSERT_TRUE(stream.Next(&buf, &size));
+  ASSERT_EQ(4096, size);
+  ASSERT_EQ(8192, stream.ByteCount());
+
+  // Should be able to backup to 7192
+  stream.BackUp(1000);
+  ASSERT_EQ(7192, stream.ByteCount());
+
+  // Flushing shouldn't change written count.
+  ASSERT_TRUE(stream.Flush());
+  ASSERT_EQ(7192, stream.ByteCount());
+
+  // Since we just flushed, we should get another full buffer.
+  ASSERT_TRUE(stream.Next(&buf, &size));
+  ASSERT_EQ(4096, size);
+  ASSERT_EQ(7192 + 4096, stream.ByteCount());
+
+  ASSERT_TRUE(stream.Flush());
+
+  ASSERT_EQ(stream.ByteCount(), file->Size());
+}
+
+// Basic read/write test.
+TEST_F(TestPBUtil, TestPBContainerSimple) {
+  // Exercise both the SYNC and NO_SYNC codepaths, despite the fact that we
+  // aren't able to observe a difference in the test.
+  vector<SyncMode> modes = { SYNC, NO_SYNC };
+  for (SyncMode mode : modes) {
+
+    // Write the file.
+    ASSERT_OK(CreateKnownGoodContainerFile(NO_OVERWRITE, mode));
+
+    // Read it back, should validate and contain the expected values.
+    ProtoContainerTestPB test_pb;
+    ASSERT_OK(ReadPBContainerFromPath(env_, path_, &test_pb));
+    ASSERT_EQ(kTestKeyvalName, test_pb.name());
+    ASSERT_EQ(kTestKeyvalValue, test_pb.value());
+
+    // Delete the file.
+    ASSERT_OK(env_->DeleteFile(path_));
+  }
+}
+
+// Corruption / various failure mode test.
+TEST_P(TestPBContainerVersions, TestCorruption) {
+  // Test that we indicate when the file does not exist.
+  ProtoContainerTestPB test_pb;
+  Status s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsNotFound()) << "Should not be found: " << path_ << ": " << s.ToString();
+
+  // Test that an empty file looks like corruption.
+  {
+    // Create the empty file.
+    unique_ptr<WritableFile> file;
+    ASSERT_OK(env_->NewWritableFile(path_, &file));
+    ASSERT_OK(file->Close());
+  }
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsIncomplete()) << "Should be zero length: " << path_ << ": " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+
+  // Test truncated file.
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  uint64_t known_good_size = 0;
+  ASSERT_OK(env_->GetFileSize(path_, &known_good_size));
+  ASSERT_OK(TruncateFile(path_, known_good_size - 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  if (version_ == 1) {
+    ASSERT_TRUE(s.IsCorruption()) << "Should be incorrect size: " << path_ << ": " << s.ToString();
+  } else {
+    ASSERT_TRUE(s.IsIncomplete()) << "Should be incorrect size: " << path_ << ": " << s.ToString();
+  }
+  ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+
+  // Test corrupted magic.
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(BitFlipFileByteRange(path_, 0, 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsCorruption()) << "Should have invalid magic: " << path_ << ": " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "Invalid magic number");
+
+  // Test corrupted version.
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(BitFlipFileByteRange(path_, 8, 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsNotSupported()) << "Should have unsupported version number: " << path_ << ": "
+                                  << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), " Protobuf container has unsupported version");
+
+  // Test corrupted magic+version checksum (only exists in the V2+ format).
+  if (version_ >= 2) {
+    ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+    ASSERT_OK(BitFlipFileByteRange(path_, 12, 2));
+    s = ReadPBContainerFromPath(env_, path_, &test_pb);
+    ASSERT_TRUE(s.IsCorruption()) << "Should have corrupted file header checksum: " << path_ << ": "
+                                    << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "File header checksum does not match");
+  }
+
+  // Test record corruption below.
+  const int kFirstRecordOffset = (version_ == 1) ? 12 : 16;
+
+  // Test corrupted data length.
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(BitFlipFileByteRange(path_, kFirstRecordOffset, 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  if (version_ == 1) {
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+  } else {
+    ASSERT_TRUE(s.IsCorruption()) << "Should be invalid data length checksum: "
+                                  << path_ << ": " << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "Incorrect checksum");
+  }
+
+  // Test corrupted data (looks like bad checksum).
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(BitFlipFileByteRange(path_, kFirstRecordOffset + 4, 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsCorruption()) << "Should be incorrect checksum: " << path_ << ": "
+                                << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "Incorrect checksum");
+
+  // Test corrupted checksum.
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(BitFlipFileByteRange(path_, known_good_size - 4, 2));
+  s = ReadPBContainerFromPath(env_, path_, &test_pb);
+  ASSERT_TRUE(s.IsCorruption()) << "Should be incorrect checksum: " << path_ << ": "
+                                << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "Incorrect checksum");
+}
+
+// Test partial record at end of file.
+TEST_P(TestPBContainerVersions, TestPartialRecord) {
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  uint64_t known_good_size;
+  ASSERT_OK(env_->GetFileSize(path_, &known_good_size));
+  ASSERT_OK(TruncateFile(path_, known_good_size - 2));
+
+  unique_ptr<RandomAccessFile> file;
+  ASSERT_OK(env_->NewRandomAccessFile(path_, &file));
+  ReadablePBContainerFile pb_file(std::move(file));
+  ASSERT_OK(pb_file.Open());
+  ProtoContainerTestPB test_pb;
+  Status s = pb_file.ReadNextPB(&test_pb);
+  // Loop to verify that the same response is repeatably returned.
+  for (int i = 0; i < 2; i++) {
+    if (version_ == 1) {
+      ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    } else {
+      ASSERT_TRUE(s.IsIncomplete()) << s.ToString();
+    }
+    ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+  }
+  ASSERT_OK(pb_file.Close());
+}
+
+// KUDU-2260: Test handling extra null bytes at the end of file. This can
+// occur, for example, on ext4 in default data=ordered mode when a write
+// increases the filesize but the system crashes before the actual data is
+// persisted.
+TEST_P(TestPBContainerVersions, TestExtraNullBytes) {
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  uint64_t known_good_size;
+  ASSERT_OK(env_->GetFileSize(path_, &known_good_size));
+  for (const auto extra_bytes : {1, 8, 128}) {
+    ASSERT_OK(TruncateFile(path_, known_good_size + extra_bytes));
+
+    unique_ptr<RandomAccessFile> file;
+    ASSERT_OK(env_->NewRandomAccessFile(path_, &file));
+    ReadablePBContainerFile pb_file(std::move(file));
+    ASSERT_OK(pb_file.Open());
+    ProtoContainerTestPB test_pb;
+    // Read the first good PB. Trouble starts at the second.
+    ASSERT_OK(pb_file.ReadNextPB(&test_pb));
+    Status s = pb_file.ReadNextPB(&test_pb);
+    // Loop to verify that the same response is repeatably returned.
+    for (int i = 0; i < 2; i++) {
+      ASSERT_TRUE(version_ == 1 ? s.IsCorruption() : s.IsIncomplete()) << s.ToString();
+      if (extra_bytes < 8) {
+        ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+      } else if (version_ == 1) {
+        ASSERT_STR_CONTAINS(s.ToString(), "Length and data checksum does not match");
+      } else {
+        ASSERT_STR_CONTAINS(s.ToString(), "rest of file is NULL bytes");
+      }
+    }
+    ASSERT_OK(pb_file.Close());
+  }
+}
+
+// Test that it is possible to append after a partial write if we truncate the
+// partial record. This is only fully supported in V2+.
+TEST_P(TestPBContainerVersions, TestAppendAfterPartialWrite) {
+  uint64_t known_good_size;
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ASSERT_OK(env_->GetFileSize(path_, &known_good_size));
+
+  unique_ptr<WritablePBContainerFile> writer;
+  RWFileOptions opts;
+  opts.mode = Env::OPEN_EXISTING;
+  ASSERT_OK(NewPBCWriter(version_, opts, &writer));
+  ASSERT_OK(writer->OpenExisting());
+
+  ASSERT_OK(TruncateFile(path_, known_good_size - 2));
+
+  unique_ptr<RandomAccessFile> file;
+  ASSERT_OK(env_->NewRandomAccessFile(path_, &file));
+  ReadablePBContainerFile reader(std::move(file));
+  ASSERT_OK(reader.Open());
+  ProtoContainerTestPB test_pb;
+  Status s = reader.ReadNextPB(&test_pb);
+  ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid");
+  if (version_ == 1) {
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    return; // The rest of the test does not apply to version 1.
+  }
+  ASSERT_TRUE(s.IsIncomplete()) << s.ToString();
+
+  // Now truncate cleanly.
+  ASSERT_OK(TruncateFile(path_, reader.offset()));
+  s = reader.ReadNextPB(&test_pb);
+  ASSERT_TRUE(s.IsEndOfFile()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "Reached end of file");
+
+  // Reopen the writer to allow appending more records.
+  // Append a record and read it back.
+  ASSERT_OK(NewPBCWriter(version_, opts, &writer));
+  ASSERT_OK(writer->OpenExisting());
+  test_pb.set_name("hello");
+  test_pb.set_value(1);
+  ASSERT_OK(writer->Append(test_pb));
+  test_pb.Clear();
+  ASSERT_OK(reader.ReadNextPB(&test_pb));
+  ASSERT_EQ("hello", test_pb.name());
+  ASSERT_EQ(1, test_pb.value());
+}
+
+// Simple test for all versions.
+TEST_P(TestPBContainerVersions, TestSingleMessage) {
+  ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_));
+  ProtoContainerTestPB test_pb;
+  ASSERT_OK(ReadPBContainerFromPath(env_, path_, &test_pb));
+  ASSERT_EQ(kTestKeyvalName, test_pb.name());
+  ASSERT_EQ(kTestKeyvalValue, test_pb.value());
+}
+
+TEST_P(TestPBContainerVersions, TestMultipleMessages) {
+  ProtoContainerTestPB pb;
+  pb.set_name("foo");
+  pb.set_note("bar");
+
+  unique_ptr<WritablePBContainerFile> pb_writer;
+  ASSERT_OK(NewPBCWriter(version_, RWFileOptions(), &pb_writer));
+  ASSERT_OK(pb_writer->CreateNew(pb));
+
+  for (int i = 0; i < 10; i++) {
+    pb.set_value(i);
+    ASSERT_OK(pb_writer->Append(pb));
+  }
+  ASSERT_OK(pb_writer->Close());
+
+  int pbs_read = 0;
+  unique_ptr<RandomAccessFile> reader;
+  ASSERT_OK(env_->NewRandomAccessFile(path_, &reader));
+  ReadablePBContainerFile pb_reader(std::move(reader));
+  ASSERT_OK(pb_reader.Open());
+  for (int i = 0;; i++) {
+    ProtoContainerTestPB read_pb;
+    Status s = pb_reader.ReadNextPB(&read_pb);
+    if (s.IsEndOfFile()) {
+      break;
+    }
+    ASSERT_OK(s);
+    ASSERT_EQ(pb.name(), read_pb.name());
+    ASSERT_EQ(read_pb.value(), i);
+    ASSERT_EQ(pb.note(), read_pb.note());
+    pbs_read++;
+  }
+  ASSERT_EQ(10, pbs_read);
+  ASSERT_OK(pb_reader.Close());
+}
+
+TEST_P(TestPBContainerVersions, TestInterleavedReadWrite) {
+  ProtoContainerTestPB pb;
+  pb.set_name("foo");
+  pb.set_note("bar");
+
+  // Open the file for writing and reading.
+  unique_ptr<WritablePBContainerFile> pb_writer;
+  ASSERT_OK(NewPBCWriter(version_, RWFileOptions(), &pb_writer));
+  unique_ptr<RandomAccessFile> reader;
+  ASSERT_OK(env_->NewRandomAccessFile(path_, &reader));
+  ReadablePBContainerFile pb_reader(std::move(reader));
+
+  // Write the header (writer) and validate it (reader).
+  ASSERT_OK(pb_writer->CreateNew(pb));
+  ASSERT_OK(pb_reader.Open());
+
+  for (int i = 0; i < 10; i++) {
+    SCOPED_TRACE(i);
+    // Write a message and read it back.
+    pb.set_value(i);
+    ASSERT_OK(pb_writer->Append(pb));
+    ProtoContainerTestPB read_pb;
+    ASSERT_OK(pb_reader.ReadNextPB(&read_pb));
+    ASSERT_EQ(pb.name(), read_pb.name());
+    ASSERT_EQ(read_pb.value(), i);
+    ASSERT_EQ(pb.note(), read_pb.note());
+  }
+
+  // After closing the writer, the reader should be out of data.
+  ASSERT_OK(pb_writer->Close());
+  ASSERT_TRUE(pb_reader.ReadNextPB(nullptr).IsEndOfFile());
+  ASSERT_OK(pb_reader.Close());
+}
+
+TEST_F(TestPBUtil, TestPopulateDescriptorSet) {
+  {
+    // No dependencies --> just one proto.
+    ProtoContainerTestPB pb;
+    FileDescriptorSet protos;
+    WritablePBContainerFile::PopulateDescriptorSet(
+        pb.GetDescriptor()->file(), &protos);
+    ASSERT_EQ(1, protos.file_size());
+  }
+  {
+    // One direct dependency --> two protos.
+    ProtoContainerTest2PB pb;
+    FileDescriptorSet protos;
+    WritablePBContainerFile::PopulateDescriptorSet(
+        pb.GetDescriptor()->file(), &protos);
+    ASSERT_EQ(2, protos.file_size());
+  }
+  {
+    // One direct and one indirect dependency --> three protos.
+    ProtoContainerTest3PB pb;
+    FileDescriptorSet protos;
+    WritablePBContainerFile::PopulateDescriptorSet(
+        pb.GetDescriptor()->file(), &protos);
+    ASSERT_EQ(3, protos.file_size());
+  }
+}
+
+void TestPBUtil::DumpPBCToString(const string& path,
+                                 ReadablePBContainerFile::Format format,
+                                 string* ret) {
+  unique_ptr<RandomAccessFile> reader;
+  ASSERT_OK(env_->NewRandomAccessFile(path, &reader));
+  ReadablePBContainerFile pb_reader(std::move(reader));
+  ASSERT_OK(pb_reader.Open());
+  ostringstream oss;
+  ASSERT_OK(pb_reader.Dump(&oss, format));
+  ASSERT_OK(pb_reader.Close());
+  *ret = oss.str();
+}
+
+TEST_P(TestPBContainerVersions, TestDumpPBContainer) {
+  const char* kExpectedOutput =
+      "Message 0\n"
+      "-------\n"
+      "record_one {\n"
+      "  name: \"foo\"\n"
+      "  value: 0\n"
+      "}\n"
+      "record_two {\n"
+      "  record {\n"
+      "    name: \"foo\"\n"
+      "    value: 0\n"
+      "  }\n"
+      "}\n"
+      "\n"
+      "Message 1\n"
+      "-------\n"
+      "record_one {\n"
+      "  name: \"foo\"\n"
+      "  value: 1\n"
+      "}\n"
+      "record_two {\n"
+      "  record {\n"
+      "    name: \"foo\"\n"
+      "    value: 2\n"
+      "  }\n"
+      "}\n\n";
+
+  const char* kExpectedOutputShort =
+    "0\trecord_one { name: \"foo\" value: 0 } record_two { record { name: \"foo\" value: 0 } }\n"
+    "1\trecord_one { name: \"foo\" value: 1 } record_two { record { name: \"foo\" value: 2 } }\n";
+
+  const char* kExpectedOutputJson =
+      "{\"recordOne\":{\"name\":\"foo\",\"value\":0},\"recordTwo\":{\"record\":{\"name\":\"foo\",\"value\":0}}}\n" // NOLINT
+      "{\"recordOne\":{\"name\":\"foo\",\"value\":1},\"recordTwo\":{\"record\":{\"name\":\"foo\",\"value\":2}}}\n"; // NOLINT
+
+  ProtoContainerTest3PB pb;
+  pb.mutable_record_one()->set_name("foo");
+  pb.mutable_record_two()->mutable_record()->set_name("foo");
+
+  unique_ptr<WritablePBContainerFile> pb_writer;
+  ASSERT_OK(NewPBCWriter(version_, RWFileOptions(), &pb_writer));
+  ASSERT_OK(pb_writer->CreateNew(pb));
+
+  for (int i = 0; i < 2; i++) {
+    pb.mutable_record_one()->set_value(i);
+    pb.mutable_record_two()->mutable_record()->set_value(i*2);
+    ASSERT_OK(pb_writer->Append(pb));
+  }
+  ASSERT_OK(pb_writer->Close());
+
+  string output;
+  NO_FATALS(DumpPBCToString(path_, ReadablePBContainerFile::Format::DEFAULT, &output));
+  ASSERT_STREQ(kExpectedOutput, output.c_str());
+
+  NO_FATALS(DumpPBCToString(path_, ReadablePBContainerFile::Format::ONELINE, &output));
+  ASSERT_STREQ(kExpectedOutputShort, output.c_str());
+
+  NO_FATALS(DumpPBCToString(path_, ReadablePBContainerFile::Format::JSON, &output));
+  ASSERT_STREQ(kExpectedOutputJson, output.c_str());
+}
+
+TEST_F(TestPBUtil, TestOverwriteExistingPB) {
+  ASSERT_OK(CreateKnownGoodContainerFile(NO_OVERWRITE));
+  ASSERT_TRUE(CreateKnownGoodContainerFile(NO_OVERWRITE).IsAlreadyPresent());
+  ASSERT_OK(CreateKnownGoodContainerFile(OVERWRITE));
+  ASSERT_OK(CreateKnownGoodContainerFile(OVERWRITE));
+}
+
+TEST_F(TestPBUtil, TestRedaction) {
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", "log"));
+  TestSecurePrintingPB pb;
+
+  pb.set_insecure1("public 1");
+  pb.set_insecure2("public 2");
+  pb.set_secure1("private 1");
+  pb.set_secure2("private 2");
+  pb.add_repeated_secure("private 3");
+  pb.add_repeated_secure("private 4");
+  pb.set_insecure3("public 3");
+
+  for (auto s : {SecureDebugString(pb), SecureShortDebugString(pb)}) {
+    ASSERT_EQ(string::npos, s.find("private"));
+    ASSERT_STR_CONTAINS(s, "<redacted>");
+    ASSERT_STR_CONTAINS(s, "public 1");
+    ASSERT_STR_CONTAINS(s, "public 2");
+    ASSERT_STR_CONTAINS(s, "public 3");
+  }
+
+  // If we disable redaction, we should see the private fields.
+  ASSERT_NE("", gflags::SetCommandLineOption("redact", ""));
+  ASSERT_STR_CONTAINS(SecureDebugString(pb), "private");
+}
+
+} // namespace pb_util
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util.cc b/be/src/kudu/util/pb_util.cc
new file mode 100644
index 0000000..edf1222
--- /dev/null
+++ b/be/src/kudu/util/pb_util.cc
@@ -0,0 +1,1088 @@
+// 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.
+//
+// Some portions copyright (C) 2008, Google, inc.
+//
+// Utilities for working with protobufs.
+// Some of this code is cribbed from the protobuf source,
+// but modified to work with kudu's 'faststring' instead of STL strings.
+
+#include "kudu/util/pb_util.h"
+
+#include <algorithm>
+#include <cstddef>
+#include <deque>
+#include <initializer_list>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
+#include <google/protobuf/descriptor_database.h>
+#include <google/protobuf/dynamic_message.h>
+#include <google/protobuf/io/coded_stream.h>
+#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
+#include <google/protobuf/message.h>
+#include <google/protobuf/message_lite.h>
+#include <google/protobuf/stubs/status.h>
+#include <google/protobuf/text_format.h>
+#include <google/protobuf/util/json_util.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/escaping.h"
+#include "kudu/gutil/strings/fastmem.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/coding.h"
+#include "kudu/util/coding-inl.h"
+#include "kudu/util/crc.h"
+#include "kudu/util/debug/sanitizer_scopes.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/pb_util-internal.h"
+#include "kudu/util/pb_util.pb.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+using google::protobuf::Descriptor;
+using google::protobuf::DescriptorPool;
+using google::protobuf::DynamicMessageFactory;
+using google::protobuf::FieldDescriptor;
+using google::protobuf::FileDescriptor;
+using google::protobuf::FileDescriptorProto;
+using google::protobuf::FileDescriptorSet;
+using google::protobuf::io::ArrayInputStream;
+using google::protobuf::io::CodedInputStream;
+using google::protobuf::Message;
+using google::protobuf::MessageLite;
+using google::protobuf::Reflection;
+using google::protobuf::SimpleDescriptorDatabase;
+using google::protobuf::TextFormat;
+using kudu::crc::Crc;
+using kudu::pb_util::internal::SequentialFileFileInputStream;
+using kudu::pb_util::internal::WritableFileOutputStream;
+using std::deque;
+using std::endl;
+using std::initializer_list;
+using std::ostream;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::unordered_set;
+using std::vector;
+using strings::Substitute;
+using strings::Utf8SafeCEscape;
+
+namespace std {
+
+// Allow the use of FileState with DCHECK_EQ.
+std::ostream& operator<< (std::ostream& os, const kudu::pb_util::FileState& state) {
+  os << static_cast<int>(state);
+  return os;
+}
+
+} // namespace std
+
+namespace kudu {
+namespace pb_util {
+
+static const char* const kTmpTemplateSuffix = ".XXXXXX";
+
+// Protobuf container constants.
+static const uint32_t kPBContainerInvalidVersion = 0;
+static const uint32_t kPBContainerDefaultVersion = 2;
+static const int kPBContainerChecksumLen = sizeof(uint32_t);
+static const char kPBContainerMagic[] = "kuducntr";
+static const int kPBContainerMagicLen = 8;
+static const int kPBContainerV1HeaderLen =
+    kPBContainerMagicLen + sizeof(uint32_t); // Magic number + version.
+static const int kPBContainerV2HeaderLen =
+    kPBContainerV1HeaderLen + kPBContainerChecksumLen; // Same as V1 plus a checksum.
+
+const int kPBContainerMinimumValidLength = kPBContainerV1HeaderLen;
+
+static_assert(arraysize(kPBContainerMagic) - 1 == kPBContainerMagicLen,
+              "kPBContainerMagic does not match expected length");
+
+namespace {
+
+// When serializing, we first compute the byte size, then serialize the message.
+// If serialization produces a different number of bytes than expected, we
+// call this function, which crashes.  The problem could be due to a bug in the
+// protobuf implementation but is more likely caused by concurrent modification
+// of the message.  This function attempts to distinguish between the two and
+// provide a useful error message.
+void ByteSizeConsistencyError(int byte_size_before_serialization,
+                              int byte_size_after_serialization,
+                              int bytes_produced_by_serialization) {
+  CHECK_EQ(byte_size_before_serialization, byte_size_after_serialization)
+      << "Protocol message was modified concurrently during serialization.";
+  CHECK_EQ(bytes_produced_by_serialization, byte_size_before_serialization)
+      << "Byte size calculation and serialization were inconsistent.  This "
+         "may indicate a bug in protocol buffers or it may be caused by "
+         "concurrent modification of the message.";
+  LOG(FATAL) << "This shouldn't be called if all the sizes are equal.";
+}
+
+string InitializationErrorMessage(const char* action,
+                                  const MessageLite& message) {
+  // Note:  We want to avoid depending on strutil in the lite library, otherwise
+  //   we'd use:
+  //
+  // return strings::Substitute(
+  //   "Can't $0 message of type \"$1\" because it is missing required "
+  //   "fields: $2",
+  //   action, message.GetTypeName(),
+  //   message.InitializationErrorString());
+
+  string result;
+  result += "Can't ";
+  result += action;
+  result += " message of type \"";
+  result += message.GetTypeName();
+  result += "\" because it is missing required fields: ";
+  result += message.InitializationErrorString();
+  return result;
+}
+
+// Returns true iff the specified protobuf container file version is supported
+// by this implementation.
+bool IsSupportedContainerVersion(uint32_t version) {
+  if (version == 1 || version == 2) {
+    return true;
+  }
+  return false;
+}
+
+// Reads exactly 'length' bytes from the container file into 'result',
+// validating that there is sufficient data in the file to read this length
+// before attempting to do so, and validating that it has read that length
+// after performing the read.
+//
+// If the file size is less than the requested size of the read, returns
+// Status::Incomplete.
+// If there is an unexpected short read, returns Status::Corruption.
+//
+// NOTE: the data in 'result' may be modified even in the case of a failed read.
+template<typename ReadableFileType>
+Status ValidateAndReadData(ReadableFileType* reader, uint64_t file_size,
+                           uint64_t* offset, uint64_t length,
+                           faststring* result) {
+  // Validate the read length using the file size.
+  if (*offset + length > file_size) {
+    return Status::Incomplete("File size not large enough to be valid",
+                              Substitute("Proto container file $0: "
+                                  "Tried to read $1 bytes at offset "
+                                  "$2 but file size is only $3 bytes",
+                                  reader->filename(), length,
+                                  *offset, file_size));
+  }
+
+  // Perform the read.
+  result->resize(length);
+  RETURN_NOT_OK(reader->Read(*offset, Slice(*result)));
+  *offset += length;
+  return Status::OK();
+}
+
+// Helper macro for use with ParseAndCompareChecksum(). Example usage:
+// RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(checksum.data(), { data }),
+//    CHECKSUM_ERR_MSG("Data checksum does not match", filename, offset));
+#define CHECKSUM_ERR_MSG(prefix, filename, cksum_offset) \
+  Substitute("$0: Incorrect checksum in file $1 at offset $2", prefix, filename, cksum_offset)
+
+// Parses a checksum from the specified buffer and compares it to the bytes
+// given in 'slices' by calculating a rolling CRC32 checksum of the bytes in
+// the 'slices'.
+// If they match, returns OK. Otherwise, returns Status::Corruption.
+Status ParseAndCompareChecksum(const uint8_t* checksum_buf,
+                               const initializer_list<Slice>& slices) {
+  uint32_t written_checksum = DecodeFixed32(checksum_buf);
+  uint64_t actual_checksum = 0;
+  Crc* crc32c = crc::GetCrc32cInstance();
+  for (Slice s : slices) {
+    crc32c->Compute(s.data(), s.size(), &actual_checksum);
+  }
+  if (PREDICT_FALSE(actual_checksum != written_checksum)) {
+    return Status::Corruption(Substitute("Checksum does not match. Expected: $0. Actual: $1",
+                                         written_checksum, actual_checksum));
+  }
+  return Status::OK();
+}
+
+// If necessary, get the size of the file opened by 'reader' in 'cached_file_size'.
+// If 'cached_file_size' already has a value, this is a no-op.
+template<typename ReadableFileType>
+Status CacheFileSize(ReadableFileType* reader,
+                     boost::optional<uint64_t>* cached_file_size) {
+  if (*cached_file_size) {
+    return Status::OK();
+  }
+
+  uint64_t file_size;
+  RETURN_NOT_OK(reader->Size(&file_size));
+  *cached_file_size = file_size;
+  return Status::OK();
+}
+
+template<typename ReadableFileType>
+Status RestOfFileIsAllZeros(ReadableFileType* reader,
+                            uint64_t filesize,
+                            uint64_t offset,
+                            bool* all_zeros) {
+  DCHECK(reader);
+  DCHECK_GE(filesize, offset);
+  DCHECK(all_zeros);
+  constexpr uint64_t max_to_read = 4 * 1024 * 1024; // 4 MiB.
+  faststring buf;
+  while (true) {
+    uint64_t to_read = std::min(max_to_read, filesize - offset);
+    if (to_read == 0) {
+      break;
+    }
+    buf.resize(to_read);
+    RETURN_NOT_OK(reader->Read(offset, Slice(buf)));
+    offset += to_read;
+    if (!IsAllZeros(buf)) {
+      *all_zeros = false;
+      return Status::OK();
+    }
+  }
+  *all_zeros = true;
+  return Status::OK();
+}
+
+// Read and parse a message of the specified format at the given offset in the
+// format documented in pb_util.h. 'offset' is an in-out parameter and will be
+// updated with the new offset on success. On failure, 'offset' is not modified.
+template<typename ReadableFileType>
+Status ReadPBStartingAt(ReadableFileType* reader, int version,
+                        boost::optional<uint64_t>* cached_file_size,
+                        uint64_t* offset, Message* msg) {
+  uint64_t tmp_offset = *offset;
+  VLOG(1) << "Reading PB with version " << version << " starting at offset " << *offset;
+
+  RETURN_NOT_OK(CacheFileSize(reader, cached_file_size));
+  uint64_t file_size = cached_file_size->get();
+
+  if (tmp_offset == *cached_file_size) {
+    return Status::EndOfFile("Reached end of file");
+  }
+
+  // Read the data length from the file.
+  // Version 2+ includes a checksum for the length field.
+  uint64_t length_buflen = (version == 1) ? sizeof(uint32_t)
+                                          : sizeof(uint32_t) + kPBContainerChecksumLen;
+  faststring length_and_cksum_buf;
+  RETURN_NOT_OK_PREPEND(ValidateAndReadData(reader, file_size, &tmp_offset, length_buflen,
+                                            &length_and_cksum_buf),
+                        Substitute("Could not read data length from proto container file $0 "
+                                   "at offset $1", reader->filename(), *offset));
+  Slice length(length_and_cksum_buf.data(), sizeof(uint32_t));
+
+  // Versions >= 2 have an individual checksum for the data length.
+  if (version >= 2) {
+    // KUDU-2260: If the length and checksum data are all 0's, and the rest of
+    // the file is all 0's, then it's an incomplete record, not corruption.
+    // This can happen e.g. on ext4 in the default data=ordered mode, when the
+    // filesize metadata is updated but the new data is not persisted.
+    // See https://plus.google.com/+KentonVarda/posts/JDwHfAiLGNQ.
+    if (IsAllZeros(length_and_cksum_buf)) {
+      bool all_zeros;
+      RETURN_NOT_OK(RestOfFileIsAllZeros(reader, file_size, tmp_offset, &all_zeros));
+      if (all_zeros) {
+        return Status::Incomplete("incomplete write of PB: rest of file is NULL bytes");
+      }
+    }
+    Slice length_checksum(length_and_cksum_buf.data() + sizeof(uint32_t), kPBContainerChecksumLen);
+    RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(length_checksum.data(), { length }),
+        CHECKSUM_ERR_MSG("Data length checksum does not match",
+                         reader->filename(), tmp_offset - kPBContainerChecksumLen));
+  }
+  uint32_t data_length = DecodeFixed32(length.data());
+
+  // Read body and checksum into buffer for checksum & parsing.
+  uint64_t data_and_cksum_buflen = data_length + kPBContainerChecksumLen;
+  faststring body_and_cksum_buf;
+  RETURN_NOT_OK_PREPEND(ValidateAndReadData(reader, file_size, &tmp_offset, data_and_cksum_buflen,
+                                            &body_and_cksum_buf),
+                        Substitute("Could not read PB message data from proto container file $0 "
+                                   "at offset $1",
+                                   reader->filename(), tmp_offset));
+  Slice body(body_and_cksum_buf.data(), data_length);
+  Slice record_checksum(body_and_cksum_buf.data() + data_length, kPBContainerChecksumLen);
+
+  // Version 1 has a single checksum for length, body.
+  // Version 2+ has individual checksums for length and body, respectively.
+  if (version == 1) {
+    RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(record_checksum.data(), { length, body }),
+        CHECKSUM_ERR_MSG("Length and data checksum does not match",
+                         reader->filename(), tmp_offset - kPBContainerChecksumLen));
+  } else {
+    RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(record_checksum.data(), { body }),
+        CHECKSUM_ERR_MSG("Data checksum does not match",
+                         reader->filename(), tmp_offset - kPBContainerChecksumLen));
+  }
+
+  // The checksum is correct. Time to decode the body.
+  //
+  // We could compare pb_type_ against msg.GetTypeName(), but:
+  // 1. pb_type_ is not available when reading the supplemental header,
+  // 2. ParseFromArray() should fail if the data cannot be parsed into the
+  //    provided message type.
+
+  // To permit parsing of very large PB messages, we must use parse through a
+  // CodedInputStream and bump the byte limit. The SetTotalBytesLimit() docs
+  // say that 512MB is the shortest theoretical message length that may produce
+  // integer overflow warnings, so that's what we'll use.
+  ArrayInputStream ais(body.data(), body.size());
+  CodedInputStream cis(&ais);
+  cis.SetTotalBytesLimit(512 * 1024 * 1024, -1);
+  if (PREDICT_FALSE(!msg->ParseFromCodedStream(&cis))) {
+    return Status::IOError("Unable to parse PB from path", reader->filename());
+  }
+
+  *offset = tmp_offset;
+  return Status::OK();
+}
+
+// Wrapper around ReadPBStartingAt() to enforce that we don't return
+// Status::Incomplete() for V1 format files.
+template<typename ReadableFileType>
+Status ReadFullPB(ReadableFileType* reader, int version,
+                  boost::optional<uint64_t>* cached_file_size,
+                  uint64_t* offset, Message* msg) {
+  bool had_cached_size = *cached_file_size != boost::none;
+  Status s = ReadPBStartingAt(reader, version, cached_file_size, offset, msg);
+  if (PREDICT_FALSE(s.IsIncomplete() && version == 1)) {
+    return Status::Corruption("Unrecoverable incomplete record", s.ToString());
+  }
+  // If we hit EOF, but we were using a cached view of the file size, then it might be
+  // that the file has been extended. Invalidate the cache and try again.
+  if (had_cached_size && (s.IsIncomplete() || s.IsEndOfFile())) {
+    *cached_file_size = boost::none;
+    return ReadFullPB(reader, version, cached_file_size, offset, msg);
+  }
+  return s;
+}
+
+// Read and parse the protobuf container file-level header documented in pb_util.h.
+template<typename ReadableFileType>
+Status ParsePBFileHeader(ReadableFileType* reader, boost::optional<uint64_t>* cached_file_size,
+                         uint64_t* offset, int* version) {
+  RETURN_NOT_OK(CacheFileSize(reader, cached_file_size));
+  uint64_t file_size = cached_file_size->get();
+
+  // We initially read enough data for a V2+ file header. This optimizes for
+  // V2+ and is valid on a V1 file because we don't consider these files valid
+  // unless they contain a record in addition to the file header. The
+  // additional 4 bytes required by a V2+ header (vs V1) is still less than the
+  // minimum number of bytes required for a V1 format data record.
+  uint64_t tmp_offset = *offset;
+  faststring header;
+  RETURN_NOT_OK_PREPEND(ValidateAndReadData(reader, file_size, &tmp_offset, kPBContainerV2HeaderLen,
+                                            &header),
+                        Substitute("Could not read header for proto container file $0",
+                                   reader->filename()));
+  Slice magic_and_version(header.data(), kPBContainerMagicLen + sizeof(uint32_t));
+  Slice checksum(header.data() + kPBContainerMagicLen + sizeof(uint32_t), kPBContainerChecksumLen);
+
+  // Validate magic number.
+  if (PREDICT_FALSE(!strings::memeq(kPBContainerMagic, header.data(), kPBContainerMagicLen))) {
+    string file_magic(reinterpret_cast<const char*>(header.data()), kPBContainerMagicLen);
+    return Status::Corruption("Invalid magic number",
+                              Substitute("Expected: $0, found: $1",
+                                         Utf8SafeCEscape(kPBContainerMagic),
+                                         Utf8SafeCEscape(file_magic)));
+  }
+
+  // Validate container file version.
+  uint32_t tmp_version = DecodeFixed32(header.data() + kPBContainerMagicLen);
+  if (PREDICT_FALSE(!IsSupportedContainerVersion(tmp_version))) {
+    return Status::NotSupported(
+        Substitute("Protobuf container has unsupported version: $0. Default version: $1",
+                   tmp_version, kPBContainerDefaultVersion));
+  }
+
+  // Versions >= 2 have a checksum after the magic number and encoded version
+  // to ensure the integrity of these fields.
+  if (tmp_version >= 2) {
+    RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(checksum.data(), { magic_and_version }),
+        CHECKSUM_ERR_MSG("File header checksum does not match",
+                         reader->filename(), tmp_offset - kPBContainerChecksumLen));
+  } else {
+    // Version 1 doesn't have a header checksum. Rewind our read offset so this
+    // data will be read again when we next attempt to read a data record.
+    tmp_offset -= kPBContainerChecksumLen;
+  }
+
+  *offset = tmp_offset;
+  *version = tmp_version;
+  return Status::OK();
+}
+
+// Read and parse the supplemental header from the container file.
+template<typename ReadableFileType>
+Status ReadSupplementalHeader(ReadableFileType* reader, int version,
+                              boost::optional<uint64_t>* cached_file_size,
+                              uint64_t* offset,
+                              ContainerSupHeaderPB* sup_header) {
+  RETURN_NOT_OK_PREPEND(ReadFullPB(reader, version, cached_file_size, offset, sup_header),
+      Substitute("Could not read supplemental header from proto container file $0 "
+                 "with version $1 at offset $2",
+                 reader->filename(), version, *offset));
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+void AppendToString(const MessageLite &msg, faststring *output) {
+  DCHECK(msg.IsInitialized()) << InitializationErrorMessage("serialize", msg);
+  AppendPartialToString(msg, output);
+}
+
+void AppendPartialToString(const MessageLite &msg, faststring* output) {
+  size_t old_size = output->size();
+  int byte_size = msg.ByteSize();
+  // Messages >2G cannot be serialized due to overflow computing ByteSize.
+  DCHECK_GE(byte_size, 0) << "Error computing ByteSize";
+
+  output->resize(old_size + static_cast<size_t>(byte_size));
+
+  uint8* start = &((*output)[old_size]);
+  uint8* end = msg.SerializeWithCachedSizesToArray(start);
+  if (end - start != byte_size) {
+    ByteSizeConsistencyError(byte_size, msg.ByteSize(), end - start);
+  }
+}
+
+void SerializeToString(const MessageLite &msg, faststring *output) {
+  output->clear();
+  AppendToString(msg, output);
+}
+
+Status ParseFromSequentialFile(MessageLite *msg, SequentialFile *rfile) {
+  SequentialFileFileInputStream input(rfile);
+  if (!msg->ParseFromZeroCopyStream(&input)) {
+    RETURN_NOT_OK(input.status());
+
+    // If it's not a file IO error then it's a parsing error.
+    // Probably, we read wrong or damaged data here.
+    return Status::Corruption("Error parsing msg", InitializationErrorMessage("parse", *msg));
+  }
+  return Status::OK();
+}
+
+Status ParseFromArray(MessageLite* msg, const uint8_t* data, uint32_t length) {
+  if (!msg->ParseFromArray(data, length)) {
+    return Status::Corruption("Error parsing msg", InitializationErrorMessage("parse", *msg));
+  }
+  return Status::OK();
+}
+
+Status WritePBToPath(Env* env, const std::string& path,
+                     const MessageLite& msg,
+                     SyncMode sync) {
+  const string tmp_template = path + kTmpInfix + kTmpTemplateSuffix;
+  string tmp_path;
+
+  unique_ptr<WritableFile> file;
+  RETURN_NOT_OK(env->NewTempWritableFile(WritableFileOptions(), tmp_template, &tmp_path, &file));
+  auto tmp_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_path), "Could not delete file " + tmp_path);
+  });
+
+  WritableFileOutputStream output(file.get());
+  bool res = msg.SerializeToZeroCopyStream(&output);
+  if (!res || !output.Flush()) {
+    return Status::IOError("Unable to serialize PB to file");
+  }
+
+  if (sync == pb_util::SYNC) {
+    RETURN_NOT_OK_PREPEND(file->Sync(), "Failed to Sync() " + tmp_path);
+  }
+  RETURN_NOT_OK_PREPEND(file->Close(), "Failed to Close() " + tmp_path);
+  RETURN_NOT_OK_PREPEND(env->RenameFile(tmp_path, path), "Failed to rename tmp file to " + path);
+  tmp_deleter.cancel();
+  if (sync == pb_util::SYNC) {
+    RETURN_NOT_OK_PREPEND(env->SyncDir(DirName(path)), "Failed to SyncDir() parent of " + path);
+  }
+  return Status::OK();
+}
+
+Status ReadPBFromPath(Env* env, const std::string& path, MessageLite* msg) {
+  shared_ptr<SequentialFile> rfile;
+  RETURN_NOT_OK(env_util::OpenFileForSequential(env, path, &rfile));
+  RETURN_NOT_OK(ParseFromSequentialFile(msg, rfile.get()));
+  return Status::OK();
+}
+
+static void TruncateString(string* s, int max_len) {
+  if (s->size() > max_len) {
+    s->resize(max_len);
+    s->append("<truncated>");
+  }
+}
+
+void TruncateFields(Message* message, int max_len) {
+  const Reflection* reflection = message->GetReflection();
+  vector<const FieldDescriptor*> fields;
+  reflection->ListFields(*message, &fields);
+  for (const FieldDescriptor* field : fields) {
+    if (field->is_repeated()) {
+      for (int i = 0; i < reflection->FieldSize(*message, field); i++) {
+        switch (field->cpp_type()) {
+          case FieldDescriptor::CPPTYPE_STRING: {
+            const string& s_const = reflection->GetRepeatedStringReference(*message, field, i,
+                                                                           nullptr);
+            TruncateString(const_cast<string*>(&s_const), max_len);
+            break;
+          }
+          case FieldDescriptor::CPPTYPE_MESSAGE: {
+            TruncateFields(reflection->MutableRepeatedMessage(message, field, i), max_len);
+            break;
+          }
+          default:
+            break;
+        }
+      }
+    } else {
+      switch (field->cpp_type()) {
+        case FieldDescriptor::CPPTYPE_STRING: {
+          const string& s_const = reflection->GetStringReference(*message, field, nullptr);
+          TruncateString(const_cast<string*>(&s_const), max_len);
+          break;
+        }
+        case FieldDescriptor::CPPTYPE_MESSAGE: {
+          TruncateFields(reflection->MutableMessage(message, field), max_len);
+          break;
+        }
+        default:
+          break;
+      }
+    }
+  }
+}
+
+namespace {
+class SecureFieldPrinter : public TextFormat::FieldValuePrinter {
+ public:
+  using super = TextFormat::FieldValuePrinter;
+
+  string PrintFieldName(const Message& message,
+                        const Reflection* reflection,
+                        const FieldDescriptor* field) const override {
+    hide_next_string_ = field->cpp_type() == FieldDescriptor::CPPTYPE_STRING &&
+        field->options().GetExtension(REDACT);
+    return super::PrintFieldName(message, reflection, field);
+  }
+
+  string PrintString(const string& val) const override {
+    if (hide_next_string_) {
+      hide_next_string_ = false;
+      return KUDU_REDACT(super::PrintString(val));
+    }
+    return super::PrintString(val);
+  }
+  string PrintBytes(const string& val) const override {
+    if (hide_next_string_) {
+      hide_next_string_ = false;
+      return KUDU_REDACT(super::PrintBytes(val));
+    }
+    return super::PrintBytes(val);
+  }
+
+  mutable bool hide_next_string_ = false;
+};
+} // anonymous namespace
+
+string SecureDebugString(const Message& msg) {
+  string debug_string;
+  TextFormat::Printer printer;
+  printer.SetDefaultFieldValuePrinter(new SecureFieldPrinter());
+  printer.PrintToString(msg, &debug_string);
+  return debug_string;
+}
+
+string SecureShortDebugString(const Message& msg) {
+  string debug_string;
+
+  TextFormat::Printer printer;
+  printer.SetSingleLineMode(true);
+  printer.SetDefaultFieldValuePrinter(new SecureFieldPrinter());
+
+  printer.PrintToString(msg, &debug_string);
+  // Single line mode currently might have an extra space at the end.
+  if (!debug_string.empty() &&
+      debug_string[debug_string.size() - 1] == ' ') {
+    debug_string.resize(debug_string.size() - 1);
+  }
+
+  return debug_string;
+}
+
+
+WritablePBContainerFile::WritablePBContainerFile(shared_ptr<RWFile> writer)
+  : state_(FileState::NOT_INITIALIZED),
+    offset_(0),
+    version_(kPBContainerDefaultVersion),
+    writer_(std::move(writer)) {
+}
+
+WritablePBContainerFile::~WritablePBContainerFile() {
+  WARN_NOT_OK(Close(), "Could not Close() when destroying file");
+}
+
+Status WritablePBContainerFile::SetVersionForTests(int version) {
+  DCHECK_EQ(FileState::NOT_INITIALIZED, state_);
+  if (!IsSupportedContainerVersion(version)) {
+    return Status::NotSupported(Substitute("Version $0 is not supported", version));
+  }
+  version_ = version;
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::CreateNew(const Message& msg) {
+  DCHECK_EQ(FileState::NOT_INITIALIZED, state_);
+
+  const uint64_t kHeaderLen = (version_ == 1) ? kPBContainerV1HeaderLen
+                                              : kPBContainerV1HeaderLen + kPBContainerChecksumLen;
+
+  faststring buf;
+  buf.resize(kHeaderLen);
+
+  // Serialize the magic.
+  strings::memcpy_inlined(buf.data(), kPBContainerMagic, kPBContainerMagicLen);
+  uint64_t offset = kPBContainerMagicLen;
+
+  // Serialize the version.
+  InlineEncodeFixed32(buf.data() + offset, version_);
+  offset += sizeof(uint32_t);
+  DCHECK_EQ(kPBContainerV1HeaderLen, offset)
+    << "Serialized unexpected number of total bytes";
+
+  // Versions >= 2: Checksum the magic and version.
+  if (version_ >= 2) {
+    uint32_t header_checksum = crc::Crc32c(buf.data(), offset);
+    InlineEncodeFixed32(buf.data() + offset, header_checksum);
+    offset += sizeof(uint32_t);
+  }
+  DCHECK_EQ(offset, kHeaderLen);
+
+  // Serialize the supplemental header.
+  ContainerSupHeaderPB sup_header;
+  PopulateDescriptorSet(msg.GetDescriptor()->file(),
+                        sup_header.mutable_protos());
+  sup_header.set_pb_type(msg.GetTypeName());
+  RETURN_NOT_OK_PREPEND(AppendMsgToBuffer(sup_header, &buf),
+                        "Failed to prepare supplemental header for writing");
+
+  // Write the serialized buffer to the file.
+  RETURN_NOT_OK_PREPEND(AppendBytes(buf),
+                        "Failed to append header to file");
+  state_ = FileState::OPEN;
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::OpenExisting() {
+  DCHECK_EQ(FileState::NOT_INITIALIZED, state_);
+  boost::optional<uint64_t> size;
+  RETURN_NOT_OK(ParsePBFileHeader(writer_.get(), &size, &offset_, &version_));
+  ContainerSupHeaderPB sup_header;
+  RETURN_NOT_OK(ReadSupplementalHeader(writer_.get(), version_, &size,
+                                       &offset_, &sup_header));
+  offset_ = size.get(); // Reset the write offset to the end of the file.
+  state_ = FileState::OPEN;
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::AppendBytes(const Slice& data) {
+  std::lock_guard<Mutex> l(offset_lock_);
+  RETURN_NOT_OK(writer_->Write(offset_, data));
+  offset_ += data.size();
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::Append(const Message& msg) {
+  DCHECK_EQ(FileState::OPEN, state_);
+
+  faststring buf;
+  RETURN_NOT_OK_PREPEND(AppendMsgToBuffer(msg, &buf),
+                        "Failed to prepare buffer for writing");
+  RETURN_NOT_OK_PREPEND(AppendBytes(buf), "Failed to append data to file");
+
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::Flush() {
+  DCHECK_EQ(FileState::OPEN, state_);
+
+  // TODO: Flush just the dirty bytes.
+  RETURN_NOT_OK_PREPEND(writer_->Flush(RWFile::FLUSH_ASYNC, 0, 0), "Failed to Flush() file");
+
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::Sync() {
+  DCHECK_EQ(FileState::OPEN, state_);
+
+  RETURN_NOT_OK_PREPEND(writer_->Sync(), "Failed to Sync() file");
+
+  return Status::OK();
+}
+
+Status WritablePBContainerFile::Close() {
+  if (state_ != FileState::CLOSED) {
+    state_ = FileState::CLOSED;
+    Status s = writer_->Close();
+    writer_.reset();
+    RETURN_NOT_OK_PREPEND(s, "Failed to Close() file");
+  }
+  return Status::OK();
+}
+
+const string& WritablePBContainerFile::filename() const {
+  return writer_->filename();
+}
+
+Status WritablePBContainerFile::AppendMsgToBuffer(const Message& msg, faststring* buf) {
+  DCHECK(msg.IsInitialized()) << InitializationErrorMessage("serialize", msg);
+  int data_len = msg.ByteSize();
+  // Messages >2G cannot be serialized due to overflow computing ByteSize.
+  DCHECK_GE(data_len, 0) << "Error computing ByteSize";
+  uint64_t record_buflen =  sizeof(uint32_t) + data_len + sizeof(uint32_t);
+  if (version_ >= 2) {
+    record_buflen += sizeof(uint32_t); // Additional checksum just for the length.
+  }
+
+  // Grow the buffer to hold the new data.
+  uint64_t record_offset = buf->size();
+  buf->resize(record_offset + record_buflen);
+  uint8_t* dst = buf->data() + record_offset;
+
+  // Serialize the data length.
+  size_t cur_offset = 0;
+  InlineEncodeFixed32(dst + cur_offset, static_cast<uint32_t>(data_len));
+  cur_offset += sizeof(uint32_t);
+
+  // For version >= 2: Serialize the checksum of the data length.
+  if (version_ >= 2) {
+    uint32_t length_checksum = crc::Crc32c(&data_len, sizeof(data_len));
+    InlineEncodeFixed32(dst + cur_offset, length_checksum);
+    cur_offset += sizeof(uint32_t);
+  }
+
+  // Serialize the data.
+  uint64_t data_offset = cur_offset;
+  if (PREDICT_FALSE(!msg.SerializeWithCachedSizesToArray(dst + cur_offset))) {
+    return Status::IOError("Failed to serialize PB to array");
+  }
+  cur_offset += data_len;
+
+  // Calculate and serialize the data checksum.
+  // For version 1, this is the checksum of the len + data.
+  // For version >= 2, this is only the checksum of the data.
+  uint32_t data_checksum;
+  if (version_ == 1) {
+    data_checksum = crc::Crc32c(dst, cur_offset);
+  } else {
+    data_checksum = crc::Crc32c(dst + data_offset, data_len);
+  }
+  InlineEncodeFixed32(dst + cur_offset, data_checksum);
+  cur_offset += sizeof(uint32_t);
+
+  DCHECK_EQ(record_buflen, cur_offset) << "Serialized unexpected number of total bytes";
+  return Status::OK();
+}
+
+void WritablePBContainerFile::PopulateDescriptorSet(
+    const FileDescriptor* desc, FileDescriptorSet* output) {
+  // Because we don't compile protobuf with TSAN enabled, copying the
+  // static PB descriptors in this function ends up triggering a lot of
+  // race reports. We suppress the reports, but TSAN still has to walk
+  // the stack, etc, and this function becomes very slow. So, we ignore
+  // TSAN here.
+  debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan;
+
+  FileDescriptorSet all_descs;
+
+  // Tracks all schemas that have been added to 'unemitted' at one point
+  // or another. Is a superset of 'unemitted' and only ever grows.
+  unordered_set<const FileDescriptor*> processed;
+
+  // Tracks all remaining unemitted schemas.
+  deque<const FileDescriptor*> unemitted;
+
+  InsertOrDie(&processed, desc);
+  unemitted.push_front(desc);
+  while (!unemitted.empty()) {
+    const FileDescriptor* proto = unemitted.front();
+
+    // The current schema is emitted iff we've processed (i.e. emitted) all
+    // of its dependencies.
+    bool emit = true;
+    for (int i = 0; i < proto->dependency_count(); i++) {
+      const FileDescriptor* dep = proto->dependency(i);
+      if (InsertIfNotPresent(&processed, dep)) {
+        unemitted.push_front(dep);
+        emit = false;
+      }
+    }
+    if (emit) {
+      unemitted.pop_front();
+      proto->CopyTo(all_descs.mutable_file()->Add());
+    }
+  }
+  all_descs.Swap(output);
+}
+
+ReadablePBContainerFile::ReadablePBContainerFile(shared_ptr<RandomAccessFile> reader)
+  : state_(FileState::NOT_INITIALIZED),
+    version_(kPBContainerInvalidVersion),
+    offset_(0),
+    reader_(std::move(reader)) {
+}
+
+ReadablePBContainerFile::~ReadablePBContainerFile() {
+  Close();
+}
+
+Status ReadablePBContainerFile::Open() {
+  DCHECK_EQ(FileState::NOT_INITIALIZED, state_);
+  RETURN_NOT_OK(ParsePBFileHeader(reader_.get(), &cached_file_size_, &offset_, &version_));
+  ContainerSupHeaderPB sup_header;
+  RETURN_NOT_OK(ReadSupplementalHeader(reader_.get(), version_, &cached_file_size_,
+                                       &offset_, &sup_header));
+  protos_.reset(sup_header.release_protos());
+  pb_type_ = sup_header.pb_type();
+  state_ = FileState::OPEN;
+  return Status::OK();
+}
+
+Status ReadablePBContainerFile::ReadNextPB(Message* msg) {
+  DCHECK_EQ(FileState::OPEN, state_);
+  return ReadFullPB(reader_.get(), version_, &cached_file_size_, &offset_, msg);
+}
+
+Status ReadablePBContainerFile::GetPrototype(const Message** prototype) {
+  if (!prototype_) {
+    // Loading the schemas into a DescriptorDatabase (and not directly into
+    // a DescriptorPool) defers resolution until FindMessageTypeByName()
+    // below, allowing for schemas to be loaded in any order.
+    unique_ptr<SimpleDescriptorDatabase> db(new SimpleDescriptorDatabase());
+    for (int i = 0; i < protos()->file_size(); i++) {
+      if (!db->Add(protos()->file(i))) {
+        return Status::Corruption("Descriptor not loaded", Substitute(
+            "Could not load descriptor for PB type $0 referenced in container file",
+            pb_type()));
+      }
+    }
+    unique_ptr<DescriptorPool> pool(new DescriptorPool(db.get()));
+    const Descriptor* desc = pool->FindMessageTypeByName(pb_type());
+    if (!desc) {
+      return Status::NotFound("Descriptor not found", Substitute(
+          "Could not find descriptor for PB type $0 referenced in container file",
+          pb_type()));
+    }
+
+    unique_ptr<DynamicMessageFactory> factory(new DynamicMessageFactory());
+    const Message* p = factory->GetPrototype(desc);
+    if (!p) {
+      return Status::NotSupported("Descriptor not supported", Substitute(
+          "Descriptor $0 referenced in container file not supported",
+          pb_type()));
+    }
+
+    db_ = std::move(db);
+    descriptor_pool_ = std::move(pool);
+    message_factory_ = std::move(factory);
+    prototype_ = p;
+  }
+  *prototype = prototype_;
+  return Status::OK();
+}
+
+Status ReadablePBContainerFile::Dump(ostream* os, ReadablePBContainerFile::Format format) {
+  DCHECK_EQ(FileState::OPEN, state_);
+
+  // Since we use the protobuf library support for dumping JSON, there isn't any easy
+  // way to hook in our redaction support. Since this is only used by CLI tools,
+  // just refuse to dump JSON if redaction is enabled.
+  if (format == Format::JSON && KUDU_SHOULD_REDACT()) {
+    return Status::NotSupported("cannot dump PBC file in JSON format if redaction is enabled");
+  }
+
+  const char* const kDashes = "-------";
+
+  if (format == Format::DEBUG) {
+    *os << "File header" << endl;
+    *os << kDashes << endl;
+    *os << "Protobuf container version: " << version_ << endl;
+    *os << "Total container file size: " << *cached_file_size_ << endl;
+    *os << "Entry PB type: " << pb_type_ << endl;
+    *os << endl;
+  }
+
+  // Use the embedded protobuf information from the container file to
+  // create the appropriate kind of protobuf Message.
+  const Message* prototype;
+  RETURN_NOT_OK(GetPrototype(&prototype));
+  unique_ptr<Message> msg(prototype_->New());
+
+  // Dump each message in the container file.
+  int count = 0;
+  uint64_t prev_offset = offset_;
+  Status s;
+  string buf;
+  for (s = ReadNextPB(msg.get());
+      s.ok();
+      s = ReadNextPB(msg.get())) {
+    switch (format) {
+      case Format::ONELINE:
+        *os << count << "\t" << SecureShortDebugString(*msg) << endl;
+        break;
+      case Format::DEFAULT:
+      case Format::DEBUG:
+        *os << "Message " << count << endl;
+        if (format == Format::DEBUG) {
+          *os << "offset: " << prev_offset << endl;
+          *os << "length: " << (offset_ - prev_offset) << endl;
+        }
+        *os << kDashes << endl;
+        *os << SecureDebugString(*msg) << endl;
+        break;
+      case Format::JSON:
+        buf.clear();
+        const auto& google_status = google::protobuf::util::MessageToJsonString(
+            *msg, &buf, google::protobuf::util::JsonPrintOptions());
+        if (!google_status.ok()) {
+          return Status::RuntimeError("could not convert PB to JSON", google_status.ToString());
+        }
+        *os << buf << endl;
+        break;
+    }
+
+    prev_offset = offset_;
+    count++;
+  }
+  if (format == Format::DEBUG && !s.IsEndOfFile()) {
+    *os << "Message " << count << endl;
+    *os << "error: failed to parse protobuf message" << endl;
+    *os << "offset: " << prev_offset << endl;
+    *os << "remaining file length: " << (*cached_file_size_ - prev_offset) << endl;
+    *os << kDashes << endl;
+  }
+  return s.IsEndOfFile() ? Status::OK() : s;
+}
+
+Status ReadablePBContainerFile::Close() {
+  state_ = FileState::CLOSED;
+  reader_.reset();
+  return Status::OK();
+}
+
+int ReadablePBContainerFile::version() const {
+  DCHECK_EQ(FileState::OPEN, state_);
+  return version_;
+}
+
+uint64_t ReadablePBContainerFile::offset() const {
+  DCHECK_EQ(FileState::OPEN, state_);
+  return offset_;
+}
+
+Status ReadPBContainerFromPath(Env* env, const std::string& path, Message* msg) {
+  unique_ptr<RandomAccessFile> file;
+  RETURN_NOT_OK(env->NewRandomAccessFile(path, &file));
+
+  ReadablePBContainerFile pb_file(std::move(file));
+  RETURN_NOT_OK(pb_file.Open());
+  RETURN_NOT_OK(pb_file.ReadNextPB(msg));
+  return pb_file.Close();
+}
+
+Status WritePBContainerToPath(Env* env, const std::string& path,
+                              const Message& msg,
+                              CreateMode create,
+                              SyncMode sync) {
+  TRACE_EVENT2("io", "WritePBContainerToPath",
+               "path", path,
+               "msg_type", msg.GetTypeName());
+
+  if (create == NO_OVERWRITE && env->FileExists(path)) {
+    return Status::AlreadyPresent(Substitute("File $0 already exists", path));
+  }
+
+  const string tmp_template = path + kTmpInfix + kTmpTemplateSuffix;
+  string tmp_path;
+
+  unique_ptr<RWFile> file;
+  RETURN_NOT_OK(env->NewTempRWFile(RWFileOptions(), tmp_template, &tmp_path, &file));
+  auto tmp_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_path), "Could not delete file " + tmp_path);
+  });
+
+  WritablePBContainerFile pb_file(std::move(file));
+  RETURN_NOT_OK(pb_file.CreateNew(msg));
+  RETURN_NOT_OK(pb_file.Append(msg));
+  if (sync == pb_util::SYNC) {
+    RETURN_NOT_OK(pb_file.Sync());
+  }
+  RETURN_NOT_OK(pb_file.Close());
+  RETURN_NOT_OK_PREPEND(env->RenameFile(tmp_path, path),
+                        "Failed to rename tmp file to " + path);
+  tmp_deleter.cancel();
+  if (sync == pb_util::SYNC) {
+    RETURN_NOT_OK_PREPEND(env->SyncDir(DirName(path)),
+                          "Failed to SyncDir() parent of " + path);
+  }
+  return Status::OK();
+}
+
+
+scoped_refptr<debug::ConvertableToTraceFormat> PbTracer::TracePb(const Message& msg) {
+  return make_scoped_refptr(new PbTracer(msg));
+}
+
+PbTracer::PbTracer(const Message& msg) : msg_(msg.New()) {
+  msg_->CopyFrom(msg);
+}
+
+void PbTracer::AppendAsTraceFormat(std::string* out) const {
+  pb_util::TruncateFields(msg_.get(), kMaxFieldLengthToTrace);
+  std::ostringstream ss;
+  JsonWriter jw(&ss, JsonWriter::COMPACT);
+  jw.Protobuf(*msg_);
+  out->append(ss.str());
+}
+
+} // namespace pb_util
+} // namespace kudu


[13/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/metrics.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/metrics.h b/be/src/kudu/util/metrics.h
new file mode 100644
index 0000000..d49afac
--- /dev/null
+++ b/be/src/kudu/util/metrics.h
@@ -0,0 +1,1195 @@
+// 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.
+#ifndef KUDU_UTIL_METRICS_H
+#define KUDU_UTIL_METRICS_H
+
+/////////////////////////////////////////////////////
+// Kudu Metrics
+/////////////////////////////////////////////////////
+//
+// Summary
+// ------------------------------------------------------------
+//
+// This API provides a basic set of metrics primitives along the lines of the Coda Hale's
+// metrics library along with JSON formatted output of running metrics.
+//
+// The metrics system has a few main concepts in its data model:
+//
+// Metric Prototypes
+// -----------------
+// Every metric that may be emitted is constructed from a prototype. The prototype defines
+// the name of the metric, the entity it is attached to, its type, its units, and a description.
+//
+// Metric prototypes are defined statically using the METRIC_DEFINE_*(...) macros. This
+// allows us to easily enumerate a full list of every metric that might be emitted from a
+// server, thus allowing auto-generation of metric metadata for integration with
+// monitoring systems such as Cloudera Manager.
+//
+// Metric Entity Prototypes
+// ------------------------
+// The other main type in the data model is the Metric Entity. The most basic entity is the
+// "server" entity -- metrics such as memory usage, RPC rates, etc, are typically associated
+// with the server as a whole.
+//
+// Users of the metrics framework can define more entity types using the
+// METRIC_DEFINE_entity(...) macro.
+//
+// MetricEntity instances
+// -----------------------
+// Each defined Metric Entity Type serves as a prototype allowing instantiation of a
+// MetricEntity object. Each instance then has its own unique set of metrics. For
+// example, in the case of Kudu, we define a Metric Entity Type called 'tablet', and the
+// Tablet Server instantiates one MetricEntity instance per tablet that it hosts.
+//
+// MetricEntity instances are instantiated within a MetricRegistry, and each instance is
+// expected to have a unique string identifier within that registry. To continue the
+// example above, a tablet entity uses its tablet ID as its unique identifier. These
+// identifiers are exposed to the operator and surfaced in monitoring tools.
+//
+// MetricEntity instances may also carry a key-value map of string attributes. These
+// attributes are directly exposed to monitoring systems via the JSON output. Monitoring
+// systems may use this information to allow hierarchical aggregation between entities,
+// display them to the user, etc.
+//
+// Metric instances
+// ----------------
+// Given a MetricEntity instance and a Metric Prototype, one can instantiate a Metric
+// instance. For example, the Kudu Tablet Server instantiates one MetricEntity instance
+// for each tablet, and then instantiates the 'tablet_rows_inserted' prototype within that
+// entity. Thus, each tablet then has a separate instance of the metric, allowing the end
+// operator to track the metric on a per-tablet basis.
+//
+//
+// Types of metrics
+// ------------------------------------------------------------
+// Gauge: Set or get a point-in-time value.
+//  - string: Gauge for a string value.
+//  - Primitive types (bool, int64_t/uint64_t, double): Lock-free gauges.
+// Counter: Get, reset, increment or decrement an int64_t value.
+// Histogram: Increment buckets of values segmented by configurable max and precision.
+//
+// Gauge vs. Counter
+// ------------------------------------------------------------
+//
+// A Counter is a metric we expect to only monotonically increase. A
+// Gauge is a metric that can decrease and increase. Use a Gauge to
+// reflect a sample, e.g., the number of transaction in-flight at a
+// given time; use a Counter when considering a metric over time,
+// e.g., exposing the number of transactions processed since start to
+// produce a metric for the number of transactions processed over some
+// time period.
+//
+// The one exception to this rule is that occasionally it may be more convenient to
+// implement a metric as a Gauge, even when it is logically a counter, due to Gauge's
+// support for fetching metric values via a bound function. In that case, you can
+// use the 'EXPOSE_AS_COUNTER' flag when defining the gauge prototype. For example:
+//
+// METRIC_DEFINE_gauge_uint64(server, threads_started,
+//                            "Threads Started",
+//                            kudu::MetricUnit::kThreads,
+//                            "Total number of threads started on this server",
+//                            kudu::EXPOSE_AS_COUNTER);
+//
+//
+// Metrics ownership
+// ------------------------------------------------------------
+//
+// Metrics are reference-counted, and one of the references is always held by a metrics
+// entity itself. Users of metrics should typically hold a scoped_refptr to their metrics
+// within class instances, so that they also hold a reference. The one exception to this
+// is FunctionGauges: see the class documentation below for a typical Gauge ownership pattern.
+//
+// Because the metrics entity holds a reference to the metric, this means that metrics will
+// not be immediately destructed when your class instance publishing them is destructed.
+// This is on purpose: metrics are retained for a configurable time interval even after they
+// are no longer being published. The purpose of this is to allow monitoring systems, which
+// only poll metrics infrequently (eg once a minute) to see the last value of a metric whose
+// owner was destructed in between two polls.
+//
+//
+// Example usage for server-level metrics
+// ------------------------------------------------------------
+//
+// 1) In your server class, define the top-level registry and the server entity:
+//
+//   MetricRegistry metric_registry_;
+//   scoped_refptr<MetricEntity> metric_entity_;
+//
+// 2) In your server constructor/initialization, construct metric_entity_. This instance
+//    will be plumbed through into other subsystems that want to register server-level
+//    metrics.
+//
+//   metric_entity_ = METRIC_ENTITY_server.Instantiate(&registry_, "some server identifier)");
+//
+// 3) At the top of your .cc file where you want to emit a metric, define the metric prototype:
+//
+//   METRIC_DEFINE_counter(server, ping_requests, "Ping Requests", kudu::MetricUnit::kRequests,
+//       "Number of Ping() RPC requests this server has handled since start");
+//
+// 4) In your class where you want to emit metrics, define the metric instance itself:
+//   scoped_refptr<Counter> ping_counter_;
+//
+// 5) In your class constructor, instantiate the metric based on the MetricEntity plumbed in:
+//
+//   MyClass(..., const scoped_refptr<MetricEntity>& metric_entity) :
+//     ping_counter_(METRIC_ping_requests.Instantiate(metric_entity)) {
+//   }
+//
+// 6) Where you want to change the metric value, just use the instance variable:
+//
+//   ping_counter_->IncrementBy(100);
+//
+//
+// Example usage for custom entity metrics
+// ------------------------------------------------------------
+// Follow the same pattern as above, but also define a metric entity somewhere. For example:
+//
+// At the top of your CC file:
+//
+//   METRIC_DEFINE_entity(my_entity);
+//   METRIC_DEFINE_counter(my_entity, ping_requests, "Ping Requests", kudu::MetricUnit::kRequests,
+//       "Number of Ping() RPC requests this particular entity has handled since start");
+//
+// In whatever class represents the entity:
+//
+//   entity_ = METRIC_ENTITY_my_entity.Instantiate(&registry_, my_entity_id);
+//
+// In whatever classes emit metrics:
+//
+//   scoped_refptr<Counter> ping_requests_ = METRIC_ping_requests.Instantiate(entity);
+//   ping_requests_->Increment();
+//
+// NOTE: at runtime, the metrics system prevents you from instantiating a metric in the
+// wrong entity type. This ensures that the metadata can fully describe the set of metric-entity
+// relationships.
+//
+// Plumbing of MetricEntity and MetricRegistry objects
+// ------------------------------------------------------------
+// Generally, the rule of thumb to follow when plumbing through entities and registries is
+// this: if you're creating new entities or you need to dump the registry contents
+// (e.g. path handlers), pass in the registry. Otherwise, pass in the entity.
+//
+// ===========
+// JSON output
+// ===========
+//
+// The first-class output format for metrics is pretty-printed JSON.
+// Such a format is relatively easy for humans and machines to read.
+//
+// The top level JSON object is an array, which contains one element per
+// entity. Each entity is an object which has its type, id, and an array
+// of metrics. Each metric contains its type, name, unit, description, value,
+// etc.
+// TODO: Output to HTML.
+//
+// Example JSON output:
+//
+// [
+//     {
+//         "type": "tablet",
+//         "id": "e95e57ba8d4d48458e7c7d35020d4a46",
+//         "attributes": {
+//           "table_id": "12345",
+//           "table_name": "my_table"
+//         },
+//         "metrics": [
+//             {
+//                 "type": "counter",
+//                 "name": "log_reader_bytes_read",
+//                 "label": "Log Reader Bytes Read",
+//                 "unit": "bytes",
+//                 "description": "Number of bytes read since tablet start",
+//                 "value": 0
+//             },
+//             ...
+//           ]
+//      },
+//      ...
+// ]
+//
+/////////////////////////////////////////////////////
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <limits>
+#include <mutex>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/casts.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/jsonwriter.h" // IWYU pragma: keep
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/striped64.h"
+
+// Define a new entity type.
+//
+// The metrics subsystem itself defines the entity type 'server', but other
+// entity types can be registered using this macro.
+#define METRIC_DEFINE_entity(name)                               \
+  ::kudu::MetricEntityPrototype METRIC_ENTITY_##name(#name)
+
+// Convenience macros to define metric prototypes.
+// See the documentation at the top of this file for example usage.
+#define METRIC_DEFINE_counter(entity, name, label, unit, desc)   \
+  ::kudu::CounterPrototype METRIC_##name(                        \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc))
+
+#define METRIC_DEFINE_gauge_string(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<std::string> METRIC_##name(                 \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_bool(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<bool> METRIC_##  name(                    \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_int32(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<int32_t> METRIC_##name(                   \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_uint32(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<uint32_t> METRIC_##name(                    \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_int64(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<int64_t> METRIC_##name(                   \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_uint64(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<uint64_t> METRIC_##name(                    \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DEFINE_gauge_double(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<double> METRIC_##name(                      \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+
+#define METRIC_DEFINE_histogram(entity, name, label, unit, desc, max_val, num_sig_digits) \
+  ::kudu::HistogramPrototype METRIC_##name(                                       \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc), \
+    max_val, num_sig_digits)
+
+// The following macros act as forward declarations for entity types and metric prototypes.
+#define METRIC_DECLARE_entity(name) \
+  extern ::kudu::MetricEntityPrototype METRIC_ENTITY_##name
+#define METRIC_DECLARE_counter(name)                             \
+  extern ::kudu::CounterPrototype METRIC_##name
+#define METRIC_DECLARE_gauge_string(name) \
+  extern ::kudu::GaugePrototype<std::string> METRIC_##name
+#define METRIC_DECLARE_gauge_bool(name) \
+  extern ::kudu::GaugePrototype<bool> METRIC_##name
+#define METRIC_DECLARE_gauge_int32(name) \
+  extern ::kudu::GaugePrototype<int32_t> METRIC_##name
+#define METRIC_DECLARE_gauge_uint32(name) \
+  extern ::kudu::GaugePrototype<uint32_t> METRIC_##name
+#define METRIC_DECLARE_gauge_int64(name) \
+  extern ::kudu::GaugePrototype<int64_t> METRIC_##name
+#define METRIC_DECLARE_gauge_uint64(name) \
+  extern ::kudu::GaugePrototype<uint64_t> METRIC_##name
+#define METRIC_DECLARE_gauge_double(name) \
+  extern ::kudu::GaugePrototype<double> METRIC_##name
+#define METRIC_DECLARE_histogram(name) \
+  extern ::kudu::HistogramPrototype METRIC_##name
+
+#if defined(__APPLE__)
+#define METRIC_DEFINE_gauge_size(entity, name, label, unit, desc, ...) \
+  ::kudu::GaugePrototype<size_t> METRIC_##name(                    \
+      ::kudu::MetricPrototype::CtorArgs(#entity, #name, label, unit, desc, ## __VA_ARGS__))
+#define METRIC_DECLARE_gauge_size(name) \
+  extern ::kudu::GaugePrototype<size_t> METRIC_##name
+#else
+#define METRIC_DEFINE_gauge_size METRIC_DEFINE_gauge_uint64
+#define METRIC_DECLARE_gauge_size METRIC_DECLARE_gauge_uint64
+#endif
+
+template <typename Type> class Singleton;
+
+namespace kudu {
+
+class Counter;
+class CounterPrototype;
+
+template<typename T>
+class AtomicGauge;
+template <typename Sig>
+class Callback;
+template<typename T>
+class FunctionGauge;
+template<typename T>
+class GaugePrototype;
+
+class Metric;
+class MetricEntityPrototype;
+class MetricPrototype;
+class MetricRegistry;
+
+class Histogram;
+class HistogramPrototype;
+class HistogramSnapshotPB;
+
+class MetricEntity;
+
+} // namespace kudu
+
+// Forward-declare the generic 'server' entity type.
+// We have to do this here below the forward declarations, but not
+// in the kudu namespace.
+METRIC_DECLARE_entity(server);
+
+namespace kudu {
+
+// Unit types to be used with metrics.
+// As additional units are required, add them to this enum and also to Name().
+struct MetricUnit {
+  enum Type {
+    kCacheHits,
+    kCacheQueries,
+    kBytes,
+    kRequests,
+    kEntries,
+    kRows,
+    kCells,
+    kConnections,
+    kOperations,
+    kProbes,
+    kNanoseconds,
+    kMicroseconds,
+    kMilliseconds,
+    kSeconds,
+    kThreads,
+    kTransactions,
+    kUnits,
+    kScanners,
+    kMaintenanceOperations,
+    kBlocks,
+    kHoles,
+    kLogBlockContainers,
+    kTasks,
+    kMessages,
+    kContextSwitches,
+    kDataDirectories,
+    kState,
+    kSessions,
+    kTablets,
+  };
+  static const char* Name(Type unit);
+};
+
+class MetricType {
+ public:
+  enum Type { kGauge, kCounter, kHistogram };
+  static const char* Name(Type t);
+ private:
+  static const char* const kGaugeType;
+  static const char* const kCounterType;
+  static const char* const kHistogramType;
+};
+
+struct MetricJsonOptions {
+  MetricJsonOptions() :
+    include_raw_histograms(false),
+    include_schema_info(false) {
+  }
+
+  // Include the raw histogram values and counts in the JSON output.
+  // This allows consumers to do cross-server aggregation or window
+  // data over time.
+  // Default: false
+  bool include_raw_histograms;
+
+  // Include the metrics "schema" information (i.e description, label,
+  // unit, etc).
+  // Default: false
+  bool include_schema_info;
+
+  // Try to skip any metrics which have not been modified since before
+  // the given epoch. The current epoch can be fetched using
+  // Metric::current_epoch() and incremented using Metric::IncrementEpoch().
+  //
+  // Note that this is an inclusive bound.
+  int64_t only_modified_in_or_after_epoch = 0;
+
+  // Whether to include metrics which have had no data recorded and thus have
+  // a value of 0. Note that some metrics with the value 0 may still be included:
+  // notably, gauges may be non-zero and then reset to zero, so seeing that
+  // they are currently zero does not indicate they are "untouched".
+  bool include_untouched_metrics = true;
+
+  // Whether to include the attributes of each entity.
+  bool include_entity_attributes = true;
+};
+
+class MetricEntityPrototype {
+ public:
+  explicit MetricEntityPrototype(const char* name);
+  ~MetricEntityPrototype();
+
+  const char* name() const { return name_; }
+
+  // Find or create an entity with the given ID within the provided 'registry'.
+  scoped_refptr<MetricEntity> Instantiate(
+      MetricRegistry* registry,
+      const std::string& id) const {
+    return Instantiate(registry, id, std::unordered_map<std::string, std::string>());
+  }
+
+  // If the entity already exists, then 'initial_attrs' will replace all existing
+  // attributes.
+  scoped_refptr<MetricEntity> Instantiate(
+      MetricRegistry* registry,
+      const std::string& id,
+      const std::unordered_map<std::string, std::string>& initial_attrs) const;
+
+ private:
+  const char* const name_;
+
+  DISALLOW_COPY_AND_ASSIGN(MetricEntityPrototype);
+};
+
+class MetricEntity : public RefCountedThreadSafe<MetricEntity> {
+ public:
+  typedef std::unordered_map<const MetricPrototype*, scoped_refptr<Metric> > MetricMap;
+  typedef std::unordered_map<std::string, std::string> AttributeMap;
+
+  scoped_refptr<Counter> FindOrCreateCounter(const CounterPrototype* proto);
+  scoped_refptr<Histogram> FindOrCreateHistogram(const HistogramPrototype* proto);
+
+  template<typename T>
+  scoped_refptr<AtomicGauge<T> > FindOrCreateGauge(const GaugePrototype<T>* proto,
+                                                   const T& initial_value);
+
+  template<typename T>
+  scoped_refptr<FunctionGauge<T> > FindOrCreateFunctionGauge(const GaugePrototype<T>* proto,
+                                                             const Callback<T()>& function);
+
+  // Return the metric instantiated from the given prototype, or NULL if none has been
+  // instantiated. Primarily used by tests trying to read metric values.
+  scoped_refptr<Metric> FindOrNull(const MetricPrototype& prototype) const;
+
+  const std::string& id() const { return id_; }
+
+  // See MetricRegistry::WriteAsJson()
+  Status WriteAsJson(JsonWriter* writer,
+                     const std::vector<std::string>& requested_metrics,
+                     const MetricJsonOptions& opts) const;
+
+  const MetricMap& UnsafeMetricsMapForTests() const { return metric_map_; }
+
+  // Mark that the given metric should never be retired until the metric
+  // registry itself destructs. This is useful for system metrics such as
+  // tcmalloc, etc, which should live as long as the process itself.
+  void NeverRetire(const scoped_refptr<Metric>& metric);
+
+  // Scan the metrics map for metrics needing retirement, removing them as necessary.
+  //
+  // Metrics are retired when they are no longer referenced outside of the metrics system
+  // itself. Additionally, we only retire a metric that has been in this state for
+  // at least FLAGS_metrics_retirement_age_ms milliseconds.
+  void RetireOldMetrics();
+
+  // Replaces all attributes for this entity.
+  // Any attributes currently set, but not in 'attrs', are removed.
+  void SetAttributes(const AttributeMap& attrs);
+
+  // Set a particular attribute. Replaces any current value.
+  void SetAttribute(const std::string& key, const std::string& val);
+
+  int num_metrics() const {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return metric_map_.size();
+  }
+
+  // Mark this entity as unpublished. This will cause the registry to retire its metrics
+  // and unregister it.
+  void Unpublish() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    published_ = false;
+  }
+
+  bool published() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return published_;
+  }
+
+ private:
+  friend class MetricRegistry;
+  friend class RefCountedThreadSafe<MetricEntity>;
+
+  MetricEntity(const MetricEntityPrototype* prototype, std::string id,
+               AttributeMap attributes);
+  ~MetricEntity();
+
+  // Ensure that the given metric prototype is allowed to be instantiated
+  // within this entity. This entity's type must match the expected entity
+  // type defined within the metric prototype.
+  void CheckInstantiation(const MetricPrototype* proto) const;
+
+  const MetricEntityPrototype* const prototype_;
+  const std::string id_;
+
+  mutable simple_spinlock lock_;
+
+  // Map from metric name to Metric object. Protected by lock_.
+  MetricMap metric_map_;
+
+  // The key/value attributes. Protected by lock_.
+  AttributeMap attributes_;
+
+  // The set of metrics which should never be retired. Protected by lock_.
+  std::vector<scoped_refptr<Metric> > never_retire_metrics_;
+
+  // Whether this entity is published. Protected by lock_.
+  bool published_;
+};
+
+// Base class to allow for putting all metrics into a single container.
+// See documentation at the top of this file for information on metrics ownership.
+class Metric : public RefCountedThreadSafe<Metric> {
+ public:
+  // All metrics must be able to render themselves as JSON.
+  virtual Status WriteAsJson(JsonWriter* writer,
+                             const MetricJsonOptions& opts) const = 0;
+
+  const MetricPrototype* prototype() const { return prototype_; }
+
+  // Return true if this metric has never been touched.
+  virtual bool IsUntouched() const = 0;
+
+  // Return true if this metric has changed in or after the given metrics epoch.
+  bool ModifiedInOrAfterEpoch(int64_t epoch) {
+    return m_epoch_ >= epoch;
+  }
+
+  // Return the current epoch for tracking modification of metrics.
+  // This can be passed as 'MetricJsonOptions::only_modified_since_epoch' to
+  // get a diff of metrics between two points in time.
+  static int64_t current_epoch() {
+    return g_epoch_;
+  }
+
+  // Advance to the next epoch for metrics.
+  // This is cheap for the calling thread but causes some extra work on the paths
+  // of hot metric updaters, so should only be done rarely (eg before dumping
+  // metrics).
+  static void IncrementEpoch();
+
+ protected:
+  explicit Metric(const MetricPrototype* prototype);
+  virtual ~Metric();
+
+  const MetricPrototype* const prototype_;
+
+  void UpdateModificationEpoch() {
+    // If we have some upper bound, we need to invalidate it. We use a 'test-and-set'
+    // here to avoid contending on writes to this cacheline.
+    if (m_epoch_ < current_epoch()) {
+      // Out-of-line the uncommon case which requires a bit more code.
+      UpdateModificationEpochSlowPath();
+    }
+  }
+
+  // The last metrics epoch in which this metric was modified.
+  // We use epochs instead of timestamps since we can ensure that epochs
+  // only change rarely. Thus this member is read-mostly and doesn't cause
+  // cacheline bouncing between metrics writers. We also don't need to read
+  // the system clock, which is more expensive compared to reading 'g_epoch_'.
+  std::atomic<int64_t> m_epoch_;
+
+ private:
+  void UpdateModificationEpochSlowPath();
+
+  friend class MetricEntity;
+  friend class RefCountedThreadSafe<Metric>;
+
+  // The time at which we should retire this metric if it is still un-referenced outside
+  // of the metrics subsystem. If this metric is not due for retirement, this member is
+  // uninitialized.
+  MonoTime retire_time_;
+
+  // See 'current_epoch()'.
+  static std::atomic<int64_t> g_epoch_;
+
+  DISALLOW_COPY_AND_ASSIGN(Metric);
+};
+
+// Registry of all the metrics for a server.
+//
+// This aggregates the MetricEntity objects associated with the server.
+class MetricRegistry {
+ public:
+  MetricRegistry();
+  ~MetricRegistry();
+
+  scoped_refptr<MetricEntity> FindOrCreateEntity(const MetricEntityPrototype* prototype,
+                                                 const std::string& id,
+                                                 const MetricEntity::AttributeMap& initial_attrs);
+
+  // Writes metrics in this registry to 'writer'.
+  //
+  // 'requested_metrics' is a set of substrings to match metric names against,
+  // where '*' matches all metrics.
+  //
+  // The string matching can either match an entity ID or a metric name.
+  // If it matches an entity ID, then all metrics for that entity will be printed.
+  //
+  // See the MetricJsonOptions struct definition above for options changing the
+  // output of this function.
+  Status WriteAsJson(JsonWriter* writer,
+                     const std::vector<std::string>& requested_metrics,
+                     const MetricJsonOptions& opts) const;
+
+  // For each registered entity, retires orphaned metrics. If an entity has no more
+  // metrics and there are no external references, entities are removed as well.
+  //
+  // See MetricEntity::RetireOldMetrics().
+  void RetireOldMetrics();
+
+  // Return the number of entities in this registry.
+  int num_entities() const {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return entities_.size();
+  }
+
+ private:
+  typedef std::unordered_map<std::string, scoped_refptr<MetricEntity> > EntityMap;
+  EntityMap entities_;
+
+  mutable simple_spinlock lock_;
+  DISALLOW_COPY_AND_ASSIGN(MetricRegistry);
+};
+
+// Registry of all of the metric and entity prototypes that have been
+// defined.
+//
+// Prototypes are typically defined as static variables in different compilation
+// units, and their constructors register themselves here. The registry is then
+// used in order to dump metrics metadata to generate a Cloudera Manager MDL
+// file.
+//
+// This class is thread-safe.
+class MetricPrototypeRegistry {
+ public:
+  // Get the singleton instance.
+  static MetricPrototypeRegistry* get();
+
+  // Dump a JSON document including all of the registered entity and metric
+  // prototypes.
+  void WriteAsJson(JsonWriter* writer) const;
+
+  // Convenience wrapper around WriteAsJson(...). This dumps the JSON information
+  // to stdout.
+  void WriteAsJson() const;
+ private:
+  friend class Singleton<MetricPrototypeRegistry>;
+  friend class MetricPrototype;
+  friend class MetricEntityPrototype;
+  MetricPrototypeRegistry() {}
+  ~MetricPrototypeRegistry() {}
+
+  // Register a metric prototype in the registry.
+  void AddMetric(const MetricPrototype* prototype);
+
+  // Register a metric entity prototype in the registry.
+  void AddEntity(const MetricEntityPrototype* prototype);
+
+  mutable simple_spinlock lock_;
+  std::vector<const MetricPrototype*> metrics_;
+  std::vector<const MetricEntityPrototype*> entities_;
+
+  DISALLOW_COPY_AND_ASSIGN(MetricPrototypeRegistry);
+};
+
+enum PrototypeFlags {
+  // Flag which causes a Gauge prototype to expose itself as if it
+  // were a counter.
+  EXPOSE_AS_COUNTER = 1 << 0
+};
+
+class MetricPrototype {
+ public:
+  // Simple struct to aggregate the arguments common to all prototypes.
+  // This makes constructor chaining a little less tedious.
+  struct CtorArgs {
+    CtorArgs(const char* entity_type,
+             const char* name,
+             const char* label,
+             MetricUnit::Type unit,
+             const char* description,
+             uint32_t flags = 0)
+      : entity_type_(entity_type),
+        name_(name),
+        label_(label),
+        unit_(unit),
+        description_(description),
+        flags_(flags) {
+    }
+
+    const char* const entity_type_;
+    const char* const name_;
+    const char* const label_;
+    const MetricUnit::Type unit_;
+    const char* const description_;
+    const uint32_t flags_;
+  };
+
+  const char* entity_type() const { return args_.entity_type_; }
+  const char* name() const { return args_.name_; }
+  const char* label() const { return args_.label_; }
+  MetricUnit::Type unit() const { return args_.unit_; }
+  const char* description() const { return args_.description_; }
+  virtual MetricType::Type type() const = 0;
+
+  // Writes the fields of this prototype to the given JSON writer.
+  void WriteFields(JsonWriter* writer,
+                   const MetricJsonOptions& opts) const;
+
+ protected:
+  explicit MetricPrototype(CtorArgs args);
+  virtual ~MetricPrototype() {
+  }
+
+  const CtorArgs args_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(MetricPrototype);
+};
+
+// A description of a Gauge.
+template<typename T>
+class GaugePrototype : public MetricPrototype {
+ public:
+  explicit GaugePrototype(const MetricPrototype::CtorArgs& args)
+    : MetricPrototype(args) {
+  }
+
+  // Instantiate a "manual" gauge.
+  scoped_refptr<AtomicGauge<T> > Instantiate(
+      const scoped_refptr<MetricEntity>& entity,
+      const T& initial_value) const {
+    return entity->FindOrCreateGauge(this, initial_value);
+  }
+
+  // Instantiate a gauge that is backed by the given callback.
+  scoped_refptr<FunctionGauge<T> > InstantiateFunctionGauge(
+      const scoped_refptr<MetricEntity>& entity,
+      const Callback<T()>& function) const {
+    return entity->FindOrCreateFunctionGauge(this, function);
+  }
+
+  virtual MetricType::Type type() const OVERRIDE {
+    if (args_.flags_ & EXPOSE_AS_COUNTER) {
+      return MetricType::kCounter;
+    } else {
+      return MetricType::kGauge;
+    }
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(GaugePrototype);
+};
+
+// Abstract base class to provide point-in-time metric values.
+class Gauge : public Metric {
+ public:
+  explicit Gauge(const MetricPrototype* prototype)
+    : Metric(prototype) {
+  }
+  virtual ~Gauge() {}
+  virtual Status WriteAsJson(JsonWriter* w,
+                             const MetricJsonOptions& opts) const OVERRIDE;
+
+ protected:
+  virtual void WriteValue(JsonWriter* writer) const = 0;
+ private:
+  DISALLOW_COPY_AND_ASSIGN(Gauge);
+};
+
+// Gauge implementation for string that uses locks to ensure thread safety.
+class StringGauge : public Gauge {
+ public:
+  StringGauge(const GaugePrototype<std::string>* proto,
+              std::string initial_value);
+  std::string value() const;
+  void set_value(const std::string& value);
+  virtual bool IsUntouched() const override {
+    return false;
+  }
+
+ protected:
+  virtual void WriteValue(JsonWriter* writer) const OVERRIDE;
+ private:
+  std::string value_;
+  mutable simple_spinlock lock_;  // Guards value_
+  DISALLOW_COPY_AND_ASSIGN(StringGauge);
+};
+
+// Lock-free implementation for types that are convertible to/from int64_t.
+template <typename T>
+class AtomicGauge : public Gauge {
+ public:
+  AtomicGauge(const GaugePrototype<T>* proto, T initial_value)
+    : Gauge(proto),
+      value_(initial_value) {
+  }
+  T value() const {
+    return static_cast<T>(value_.Load(kMemOrderRelease));
+  }
+  virtual void set_value(const T& value) {
+    value_.Store(static_cast<int64_t>(value), kMemOrderNoBarrier);
+  }
+  void Increment() {
+    UpdateModificationEpoch();
+    value_.IncrementBy(1, kMemOrderNoBarrier);
+  }
+  virtual void IncrementBy(int64_t amount) {
+    UpdateModificationEpoch();
+    value_.IncrementBy(amount, kMemOrderNoBarrier);
+  }
+  void Decrement() {
+    IncrementBy(-1);
+  }
+  void DecrementBy(int64_t amount) {
+    IncrementBy(-amount);
+  }
+  virtual bool IsUntouched() const override {
+    return false;
+  }
+ protected:
+  virtual void WriteValue(JsonWriter* writer) const OVERRIDE {
+    writer->Value(value());
+  }
+  AtomicInt<int64_t> value_;
+ private:
+  DISALLOW_COPY_AND_ASSIGN(AtomicGauge);
+};
+
+// Utility class to automatically detach FunctionGauges when a class destructs.
+//
+// Because FunctionGauges typically access class instance state, it's important to ensure
+// that they are detached before the class destructs. One approach is to make all
+// FunctionGauge instances be members of the class, and then call gauge_->Detach() in your
+// class's destructor. However, it's easy to forget to do this, which would lead to
+// heap-use-after-free bugs. This type of bug is easy to miss in unit tests because the
+// tests don't always poll metrics. Using a FunctionGaugeDetacher member instead makes
+// the detaching automatic and thus less error-prone.
+//
+// Example usage:
+//
+// METRIC_define_gauge_int64(my_metric, MetricUnit::kOperations, "My metric docs");
+// class MyClassWithMetrics {
+//  public:
+//   MyClassWithMetrics(const scoped_refptr<MetricEntity>& entity) {
+//     METRIC_my_metric.InstantiateFunctionGauge(entity,
+//       Bind(&MyClassWithMetrics::ComputeMyMetric, Unretained(this)))
+//       ->AutoDetach(&metric_detacher_);
+//   }
+//   ~MyClassWithMetrics() {
+//   }
+//
+//   private:
+//    int64_t ComputeMyMetric() {
+//      // Compute some metric based on instance state.
+//    }
+//    FunctionGaugeDetacher metric_detacher_;
+// };
+class FunctionGaugeDetacher {
+ public:
+  FunctionGaugeDetacher();
+  ~FunctionGaugeDetacher();
+
+ private:
+  template<typename T>
+  friend class FunctionGauge;
+
+  void OnDestructor(const Closure& c) {
+    callbacks_.push_back(c);
+  }
+
+  std::vector<Closure> callbacks_;
+
+  DISALLOW_COPY_AND_ASSIGN(FunctionGaugeDetacher);
+};
+
+
+// A Gauge that calls back to a function to get its value.
+//
+// This metric type should be used in cases where it is difficult to keep a running
+// measure of a metric, but instead would like to compute the metric value whenever it is
+// requested by a user.
+//
+// The lifecycle should be carefully considered when using a FunctionGauge. In particular,
+// the bound function needs to always be safe to run -- so if it references a particular
+// non-singleton class instance, the instance must out-live the function. Typically,
+// the easiest way to ensure this is to use a FunctionGaugeDetacher (see above).
+template <typename T>
+class FunctionGauge : public Gauge {
+ public:
+  T value() const {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return function_.Run();
+  }
+
+  virtual void WriteValue(JsonWriter* writer) const OVERRIDE {
+    writer->Value(value());
+  }
+
+  // Reset this FunctionGauge to return a specific value.
+  // This should be used during destruction. If you want a settable
+  // Gauge, use a normal Gauge instead of a FunctionGauge.
+  void DetachToConstant(T v) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    function_ = Bind(&FunctionGauge::Return, v);
+  }
+
+  // Get the current value of the gauge, and detach so that it continues to return this
+  // value in perpetuity.
+  void DetachToCurrentValue() {
+    T last_value = value();
+    DetachToConstant(last_value);
+  }
+
+  // Automatically detach this gauge when the given 'detacher' destructs.
+  // After detaching, the metric will return 'value' in perpetuity.
+  void AutoDetach(FunctionGaugeDetacher* detacher, T value = T()) {
+    detacher->OnDestructor(Bind(&FunctionGauge<T>::DetachToConstant,
+                                this, value));
+  }
+
+  // Automatically detach this gauge when the given 'detacher' destructs.
+  // After detaching, the metric will return whatever its value was at the
+  // time of detaching.
+  //
+  // Note that, when using this method, you should be sure that the FunctionGaugeDetacher
+  // is destructed before any objects which are required by the gauge implementation.
+  // In typical usage (see the FunctionGaugeDetacher class documentation) this means you
+  // should declare the detacher member after all other class members that might be
+  // accessed by the gauge function implementation.
+  void AutoDetachToLastValue(FunctionGaugeDetacher* detacher) {
+    detacher->OnDestructor(Bind(&FunctionGauge<T>::DetachToCurrentValue,
+                                this));
+  }
+
+  virtual bool IsUntouched() const override {
+    return false;
+  }
+
+ private:
+  friend class MetricEntity;
+
+  FunctionGauge(const GaugePrototype<T>* proto, Callback<T()> function)
+      : Gauge(proto), function_(std::move(function)) {
+    // Override the modification epoch to the maximum, since we don't have any idea
+    // when the bound function changes value.
+    m_epoch_ = std::numeric_limits<decltype(m_epoch_.load())>::max();
+  }
+
+  static T Return(T v) {
+    return v;
+  }
+
+  mutable simple_spinlock lock_;
+  Callback<T()> function_;
+  DISALLOW_COPY_AND_ASSIGN(FunctionGauge);
+};
+
+// Prototype for a counter.
+class CounterPrototype : public MetricPrototype {
+ public:
+  explicit CounterPrototype(const MetricPrototype::CtorArgs& args)
+    : MetricPrototype(args) {
+  }
+  scoped_refptr<Counter> Instantiate(const scoped_refptr<MetricEntity>& entity);
+
+  virtual MetricType::Type type() const OVERRIDE { return MetricType::kCounter; }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CounterPrototype);
+};
+
+// Simple incrementing 64-bit integer.
+// Only use Counters in cases that we expect the count to only increase. For example,
+// a counter is appropriate for "number of transactions processed by the server",
+// but not for "number of transactions currently in flight". Monitoring software
+// knows that counters only increase and thus can compute rates over time, rates
+// across multiple servers, etc, which aren't appropriate in the case of gauges.
+class Counter : public Metric {
+ public:
+  int64_t value() const;
+  void Increment();
+  void IncrementBy(int64_t amount);
+  virtual Status WriteAsJson(JsonWriter* w,
+                             const MetricJsonOptions& opts) const OVERRIDE;
+
+  virtual bool IsUntouched() const override {
+    return value() == 0;
+  }
+
+ private:
+  FRIEND_TEST(MetricsTest, SimpleCounterTest);
+  FRIEND_TEST(MultiThreadedMetricsTest, CounterIncrementTest);
+  friend class MetricEntity;
+
+  explicit Counter(const CounterPrototype* proto);
+
+  LongAdder value_;
+  DISALLOW_COPY_AND_ASSIGN(Counter);
+};
+
+class HistogramPrototype : public MetricPrototype {
+ public:
+  HistogramPrototype(const MetricPrototype::CtorArgs& args,
+                     uint64_t max_trackable_value, int num_sig_digits);
+  scoped_refptr<Histogram> Instantiate(const scoped_refptr<MetricEntity>& entity);
+
+  uint64_t max_trackable_value() const { return max_trackable_value_; }
+  int num_sig_digits() const { return num_sig_digits_; }
+  virtual MetricType::Type type() const OVERRIDE { return MetricType::kHistogram; }
+
+ private:
+  const uint64_t max_trackable_value_;
+  const int num_sig_digits_;
+  DISALLOW_COPY_AND_ASSIGN(HistogramPrototype);
+};
+
+class Histogram : public Metric {
+ public:
+  // Increment the histogram for the given value.
+  // 'value' must be non-negative.
+  void Increment(int64_t value);
+
+  // Increment the histogram for the given value by the given amount.
+  // 'value' and 'amount' must be non-negative.
+  void IncrementBy(int64_t value, int64_t amount);
+
+  // Return the total number of values added to the histogram (via Increment()
+  // or IncrementBy()).
+  uint64_t TotalCount() const;
+
+  virtual Status WriteAsJson(JsonWriter* w,
+                             const MetricJsonOptions& opts) const OVERRIDE;
+
+  // Returns a snapshot of this histogram including the bucketed values and counts.
+  Status GetHistogramSnapshotPB(HistogramSnapshotPB* snapshot_pb,
+                                const MetricJsonOptions& opts) const;
+
+  // Returns a pointer to the underlying histogram. The implementation of HdrHistogram
+  // is thread safe.
+  const HdrHistogram* histogram() const { return histogram_.get(); }
+
+  uint64_t CountInBucketForValueForTests(uint64_t value) const;
+  uint64_t MinValueForTests() const;
+  uint64_t MaxValueForTests() const;
+  double MeanValueForTests() const;
+
+  virtual bool IsUntouched() const override {
+    return TotalCount() == 0;
+  }
+
+ private:
+  FRIEND_TEST(MetricsTest, SimpleHistogramTest);
+  friend class MetricEntity;
+  explicit Histogram(const HistogramPrototype* proto);
+
+  const gscoped_ptr<HdrHistogram> histogram_;
+  DISALLOW_COPY_AND_ASSIGN(Histogram);
+};
+
+// Measures a duration while in scope. Adds this duration to specified histogram on destruction.
+class ScopedLatencyMetric {
+ public:
+  // NOTE: the given histogram must live as long as this object.
+  // If 'latency_hist' is NULL, this turns into a no-op.
+  explicit ScopedLatencyMetric(Histogram* latency_hist);
+  ~ScopedLatencyMetric();
+
+ private:
+  Histogram* latency_hist_;
+  MonoTime time_started_;
+};
+
+#define SCOPED_LATENCY_METRIC(_mtx, _h) \
+  ScopedLatencyMetric _h##_metric((_mtx) ? (_mtx)->_h.get() : NULL)
+
+
+////////////////////////////////////////////////////////////
+// Inline implementations of template methods
+////////////////////////////////////////////////////////////
+
+inline scoped_refptr<Counter> MetricEntity::FindOrCreateCounter(
+    const CounterPrototype* proto) {
+  CheckInstantiation(proto);
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<Counter> m = down_cast<Counter*>(FindPtrOrNull(metric_map_, proto).get());
+  if (!m) {
+    m = new Counter(proto);
+    InsertOrDie(&metric_map_, proto, m);
+  }
+  return m;
+}
+
+inline scoped_refptr<Histogram> MetricEntity::FindOrCreateHistogram(
+    const HistogramPrototype* proto) {
+  CheckInstantiation(proto);
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<Histogram> m = down_cast<Histogram*>(FindPtrOrNull(metric_map_, proto).get());
+  if (!m) {
+    m = new Histogram(proto);
+    InsertOrDie(&metric_map_, proto, m);
+  }
+  return m;
+}
+
+template<typename T>
+inline scoped_refptr<AtomicGauge<T> > MetricEntity::FindOrCreateGauge(
+    const GaugePrototype<T>* proto,
+    const T& initial_value) {
+  CheckInstantiation(proto);
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<AtomicGauge<T> > m = down_cast<AtomicGauge<T>*>(
+      FindPtrOrNull(metric_map_, proto).get());
+  if (!m) {
+    m = new AtomicGauge<T>(proto, initial_value);
+    InsertOrDie(&metric_map_, proto, m);
+  }
+  return m;
+}
+
+template<typename T>
+inline scoped_refptr<FunctionGauge<T> > MetricEntity::FindOrCreateFunctionGauge(
+    const GaugePrototype<T>* proto,
+    const Callback<T()>& function) {
+  CheckInstantiation(proto);
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<FunctionGauge<T> > m = down_cast<FunctionGauge<T>*>(
+      FindPtrOrNull(metric_map_, proto).get());
+  if (!m) {
+    m = new FunctionGauge<T>(proto, function);
+    InsertOrDie(&metric_map_, proto, m);
+  }
+  return m;
+}
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_METRICS_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/minidump-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/minidump-test.cc b/be/src/kudu/util/minidump-test.cc
new file mode 100644
index 0000000..5c756b6
--- /dev/null
+++ b/be/src/kudu/util/minidump-test.cc
@@ -0,0 +1,149 @@
+// 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 <unistd.h>
+
+#include <csignal>
+#include <cstdlib>
+#include <cstring>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/minidump.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/env.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+DECLARE_bool(enable_minidumps);
+DECLARE_int32(max_minidumps);
+DECLARE_string(minidump_path);
+
+namespace kudu {
+
+class MinidumpDeathTest : public KuduTest {
+ protected:
+  void WaitForMinidumps(int expected, const string& dir);
+};
+
+void MinidumpDeathTest::WaitForMinidumps(int expected, const string& dir) {
+  ASSERT_EVENTUALLY([&] {
+    vector<string> matches;
+    ASSERT_OK(env_->Glob(JoinPathSegments(dir, "*.dmp"), &matches));
+    ASSERT_EQ(expected, matches.size());
+  });
+}
+
+// Test that registering the minidump exception handler results in creation of
+// minidump files on crash. Also test that deleting excess minidump files works
+// as expected.
+TEST_F(MinidumpDeathTest, TestRegisterAndDelete) {
+  FLAGS_enable_minidumps = true;
+  FLAGS_minidump_path = JoinPathSegments(test_dir_, "minidumps");
+  MinidumpExceptionHandler minidump_handler;
+  ASSERT_DEATH({
+    abort();
+  },
+  // Ensure that a stack trace is produced.
+  "kudu::MinidumpDeathTest_TestRegisterAndDelete_Test::TestBody()");
+
+  // Ensure that a minidump is produced.
+  string minidump_dir = minidump_handler.minidump_dir();
+  NO_FATALS(WaitForMinidumps(1, minidump_dir));
+
+  // Now create more minidumps so we can clean them up.
+  for (int num_dumps : {2, 3}) {
+    kill(getpid(), SIGUSR1);
+    NO_FATALS(WaitForMinidumps(num_dumps, minidump_dir));
+  }
+
+  FLAGS_max_minidumps = 2;
+  ASSERT_OK(minidump_handler.DeleteExcessMinidumpFiles(env_));
+  NO_FATALS(WaitForMinidumps(2, minidump_dir));
+}
+
+// Test that a CHECK() failure produces a stack trace and a minidump.
+TEST_F(MinidumpDeathTest, TestCheckStackTraceAndMinidump) {
+  FLAGS_enable_minidumps = true;
+  FLAGS_minidump_path = JoinPathSegments(test_dir_, "minidumps");
+  MinidumpExceptionHandler minidump_handler;
+  ASSERT_DEATH({
+    CHECK_EQ(1, 0);
+  },
+  // Ensure that a stack trace is produced.
+  "kudu::MinidumpDeathTest_TestCheckStackTraceAndMinidump_Test::TestBody()");
+
+  // Ensure that a minidump is produced.
+  string minidump_dir = minidump_handler.minidump_dir();
+  NO_FATALS(WaitForMinidumps(1, minidump_dir));
+}
+
+class MinidumpSignalDeathTest : public MinidumpDeathTest,
+                                public ::testing::WithParamInterface<int> {
+};
+
+// Test that we get both a minidump and a stack trace for each supported signal.
+TEST_P(MinidumpSignalDeathTest, TestHaveMinidumpAndStackTrace) {
+  FLAGS_enable_minidumps = true;
+  int signal = GetParam();
+
+#if defined(ADDRESS_SANITIZER)
+  // ASAN appears to catch SIGBUS, SIGSEGV, and SIGFPE and the process is not killed.
+  if (signal == SIGBUS || signal == SIGSEGV || signal == SIGFPE) {
+    return;
+  }
+#endif
+
+#if defined(THREAD_SANITIZER)
+  // TSAN appears to catch SIGTERM and the process is not killed.
+  if (signal == SIGTERM) {
+    return;
+  }
+#endif
+
+  LOG(INFO) << "Testing signal: " << strsignal(signal);
+
+  FLAGS_minidump_path = JoinPathSegments(test_dir_, "minidumps");
+  MinidumpExceptionHandler minidump_handler;
+  ASSERT_DEATH({
+    kill(getpid(), signal);
+  },
+  // Ensure that a stack trace is produced.
+  "kudu::MinidumpSignalDeathTest_TestHaveMinidumpAndStackTrace_Test::TestBody()");
+
+  // Ensure that a mindump is produced, unless it's SIGTERM, which does not
+  // create a minidump.
+  int num_expected_minidumps = 1;
+  if (signal == SIGTERM) {
+    num_expected_minidumps = 0;
+  }
+  NO_FATALS(WaitForMinidumps(num_expected_minidumps, minidump_handler.minidump_dir()));
+}
+
+INSTANTIATE_TEST_CASE_P(DeadlySignals, MinidumpSignalDeathTest,
+    ::testing::Values(SIGABRT, SIGBUS, SIGSEGV, SIGILL, SIGFPE, SIGTERM));
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/minidump.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/minidump.cc b/be/src/kudu/util/minidump.cc
new file mode 100644
index 0000000..d02dc5d
--- /dev/null
+++ b/be/src/kudu/util/minidump.cc
@@ -0,0 +1,382 @@
+// 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 "kudu/util/minidump.h"
+
+#include <unistd.h>
+
+#include <atomic>
+#include <csignal>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <ostream>
+#include <string>
+
+#if defined(__linux__)
+#include <breakpad/client/linux/handler/exception_handler.h>
+#include <breakpad/common/linux/linux_libc_support.h>
+#endif // defined(__linux__)
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/linux_syscall_support.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+using kudu::env_util::CreateDirIfMissing;
+using std::string;
+
+#if defined(__linux__)
+static constexpr bool kMinidumpPlatformSupported = true;
+#else
+static constexpr bool kMinidumpPlatformSupported = false;
+#endif // defined(__linux__)
+
+DECLARE_string(log_dir);
+
+DEFINE_bool(enable_minidumps, kMinidumpPlatformSupported,
+            "Whether to enable minidump generation upon process crash or SIGUSR1. "
+            "Currently only supported on Linux systems.");
+TAG_FLAG(enable_minidumps, advanced);
+TAG_FLAG(enable_minidumps, evolving);
+static bool ValidateMinidumpEnabled(const char* /*flagname*/, bool value) {
+  if (value && !kMinidumpPlatformSupported) {
+    return false; // NOLINT(*)
+  }
+  return true;
+}
+DEFINE_validator(enable_minidumps, &ValidateMinidumpEnabled);
+
+DEFINE_string(minidump_path, "minidumps", "Directory to write minidump files to. This "
+    "can be either an absolute path or a path relative to --log_dir. Each daemon will "
+    "create an additional sub-directory to prevent naming conflicts and to make it "
+    "easier to identify a crashing daemon. Minidump files contain crash-related "
+    "information in a compressed format. Minidumps will be written when a daemon exits "
+    "unexpectedly, for example on an unhandled exception or signal, or when a "
+    "SIGUSR1 signal is sent to the process. Cannot be set to an empty value.");
+TAG_FLAG(minidump_path, evolving);
+// The minidump path cannot be empty.
+static bool ValidateMinidumpPath(const char* /*flagname*/, const string& value) {
+  return !value.empty();
+}
+DEFINE_validator(minidump_path, &ValidateMinidumpPath);
+
+DEFINE_int32(max_minidumps, 9, "Maximum number of minidump files to keep per daemon. "
+    "Older files are removed first. Set to 0 to keep all minidump files.");
+TAG_FLAG(max_minidumps, evolving);
+
+DEFINE_int32(minidump_size_limit_hint_kb, 20480, "Size limit hint for minidump files in "
+    "KB. If a minidump exceeds this value, then breakpad will reduce the stack memory it "
+    "collects for each thread from 8KB to 2KB. However it will always include the full "
+    "stack memory for the first 20 threads, including the thread that crashed.");
+TAG_FLAG(minidump_size_limit_hint_kb, advanced);
+TAG_FLAG(minidump_size_limit_hint_kb, evolving);
+
+#if !defined(__linux__)
+namespace google_breakpad {
+// Define this as an empty class to avoid an undefined symbol error on Mac.
+class ExceptionHandler {
+ public:
+  ExceptionHandler() {}
+  ~ExceptionHandler() {}
+};
+} // namespace google_breakpad
+#endif // !defined(__linux__)
+
+namespace kudu {
+
+static sigset_t GetSigset(int signo) {
+  sigset_t signals;
+  CHECK_EQ(0, sigemptyset(&signals));
+  CHECK_EQ(0, sigaddset(&signals, signo));
+  return signals;
+}
+
+#if defined(__linux__)
+
+// Called by the exception handler before minidump is produced.
+// Minidump is only written if this returns true.
+static bool FilterCallback(void* /*context*/) {
+  return true;
+}
+
+// Write two null-terminated strings and a newline to both stdout and stderr.
+static void WriteLineStdoutStderr(const char* msg1, const char* msg2) {
+  // We use Breakpad's reimplementation of strlen(), called my_strlen(), from
+  // linux_libc_support.h to avoid calling into libc.
+  // A comment from linux_libc_support.h is reproduced here:
+  // "This header provides replacements for libc functions that we need. If we
+  // call the libc functions directly we risk crashing in the dynamic linker as
+  // it tries to resolve uncached PLT entries."
+  int msg1_len = my_strlen(msg1);
+  int msg2_len = my_strlen(msg2);
+
+  // We use sys_write() from linux_syscall_support.h here per the
+  // recommendation of the breakpad docs for the same reasons as above.
+  for (int fd : {STDOUT_FILENO, STDERR_FILENO}) {
+    sys_write(fd, msg1, msg1_len);
+    sys_write(fd, msg2, msg2_len);
+    sys_write(fd, "\n", 1);
+  }
+}
+
+// Callback for breakpad. It is called whenever a minidump file has been
+// written and should not be called directly. It logs the event before breakpad
+// crashes the process. Due to the process being in a failed state we write to
+// stdout/stderr and let the surrounding redirection make sure the output gets
+// logged. The calls might still fail in unknown scenarios as the process is in
+// a broken state. However we don't rely on them as the minidump file has been
+// written already.
+static bool DumpCallback(const google_breakpad::MinidumpDescriptor& descriptor,
+                         void* context, bool succeeded) {
+
+  // Indicate whether a minidump file was written successfully. Write message
+  // to stdout/stderr, which will usually be captured in the INFO/ERROR log.
+  if (succeeded) {
+    WriteLineStdoutStderr("Wrote minidump to ", descriptor.path());
+  } else {
+    WriteLineStdoutStderr("Failed to write minidump to ", descriptor.path());
+  }
+
+  // If invoked by a user signal, return the actual success or failure of
+  // writing the minidump file so that we can print a user-friendly error
+  // message if writing the minidump fails.
+  bool is_user_signal = context != nullptr && *reinterpret_cast<bool*>(context);
+  if (is_user_signal) {
+    return succeeded;
+  }
+
+  // For crash signals. If we didn't want to invoke the previously-installed
+  // signal handler from glog, we would return the value received in
+  // 'succeeded' as described in the breakpad documentation. If this callback
+  // function returned true, breakpad would not invoke the previously-installed
+  // signal handler; instead, it would invoke the default signal handler, which
+  // would cause the process to crash immediately after writing the minidump.
+  //
+  // We make this callback always return false so that breakpad will invoke any
+  // previously-installed signal handler afterward. We want that to happen
+  // because the glog signal handlers print a helpful stacktrace on crash.
+  // That's convenient to have, because unlike a minidump, it doesn't need to
+  // be decoded to be useful for debugging.
+  return false;
+}
+
+// Failure function that simply calls abort().
+static void AbortFailureFunction() {
+  abort();
+}
+
+bool MinidumpExceptionHandler::WriteMinidump() {
+  bool user_signal = true;
+  return google_breakpad::ExceptionHandler::WriteMinidump(minidump_dir(),
+                                                          &DumpCallback,
+                                                          &user_signal);
+}
+
+Status MinidumpExceptionHandler::InitMinidumpExceptionHandler() {
+  minidump_dir_ = FLAGS_minidump_path;
+  if (minidump_dir_[0] != '/') {
+    minidump_dir_ = JoinPathSegments(FLAGS_log_dir, minidump_dir_);
+  }
+
+  // Create the first-level minidump directory.
+  Env* env = Env::Default();
+  RETURN_NOT_OK_PREPEND(CreateDirIfMissing(env, minidump_dir_),
+                        "Error creating top-level minidump directory");
+
+  // Add the executable name to the path where minidumps will be written. This makes
+  // identification easier and prevents name collisions between the files.
+  // This is also consistent with how Impala organizes its minidump files.
+  const char* exe_name = gflags::ProgramInvocationShortName();
+  minidump_dir_ = JoinPathSegments(minidump_dir_, exe_name);
+
+  // Create the directory if it is not there. The minidump doesn't get written if there is
+  // no directory.
+  RETURN_NOT_OK_PREPEND(CreateDirIfMissing(env, minidump_dir_),
+                        "Error creating minidump directory");
+
+  // Verify that the minidump directory really is a directory. We canonicalize
+  // in case it's a symlink to a directory.
+  string canonical_minidump_path;
+  RETURN_NOT_OK(env->Canonicalize(minidump_dir_, &canonical_minidump_path));
+  bool is_dir;
+  RETURN_NOT_OK(env->IsDirectory(canonical_minidump_path, &is_dir));
+  if (!is_dir) {
+    return Status::IOError("Unable to create minidump directory", canonical_minidump_path);
+  }
+
+  google_breakpad::MinidumpDescriptor desc(minidump_dir_);
+
+  // Limit filesize if configured.
+  if (FLAGS_minidump_size_limit_hint_kb > 0) {
+    size_t size_limit = 1024 * static_cast<int64_t>(FLAGS_minidump_size_limit_hint_kb);
+    LOG(INFO) << "Setting minidump size limit to "
+              << HumanReadableNumBytes::ToStringWithoutRounding(size_limit);
+    desc.set_size_limit(size_limit);
+  }
+
+  // If we don't uninstall the glog failure function when minidumps are enabled
+  // then we get two (2) stack traces printed from a LOG(FATAL) or CHECK(): one
+  // from the glog failure function and one from the glog signal handler. That
+  // is because we always return false in DumpCallback() in the non-user signal
+  // case.
+  google::InstallFailureFunction(&AbortFailureFunction);
+
+  breakpad_handler_.reset(
+      new google_breakpad::ExceptionHandler(desc,           // Path to minidump directory.
+                                            FilterCallback, // Indicates whether to write the dump.
+                                            DumpCallback,   // Output a log message when dumping.
+                                            nullptr,        // Optional context for callbacks.
+                                            true,           // Whether to install a crash handler.
+                                            -1));           // -1: Use in-process dump generation.
+
+  return Status::OK();
+}
+
+Status MinidumpExceptionHandler::RegisterMinidumpExceptionHandler() {
+  if (!FLAGS_enable_minidumps) return Status::OK();
+
+  // Ensure only one active instance is alive per process at any given time.
+  CHECK_EQ(0, MinidumpExceptionHandler::current_num_instances_.fetch_add(1));
+  RETURN_NOT_OK(InitMinidumpExceptionHandler());
+  RETURN_NOT_OK(StartUserSignalHandlerThread());
+  return Status::OK();
+}
+
+void MinidumpExceptionHandler::UnregisterMinidumpExceptionHandler() {
+  if (!FLAGS_enable_minidumps) return;
+
+  StopUserSignalHandlerThread();
+  CHECK_EQ(1, MinidumpExceptionHandler::current_num_instances_.fetch_sub(1));
+}
+
+Status MinidumpExceptionHandler::StartUserSignalHandlerThread() {
+  user_signal_handler_thread_running_.store(true, std::memory_order_relaxed);
+  return Thread::Create("minidump", "sigusr1-handler",
+                        &MinidumpExceptionHandler::RunUserSignalHandlerThread,
+                        this, &user_signal_handler_thread_);
+}
+
+void MinidumpExceptionHandler::StopUserSignalHandlerThread() {
+  user_signal_handler_thread_running_.store(false, std::memory_order_relaxed);
+  std::atomic_thread_fence(std::memory_order_release); // Store before signal.
+  // Send SIGUSR1 signal to thread, which will wake it up.
+  kill(getpid(), SIGUSR1);
+  user_signal_handler_thread_->Join();
+}
+
+void MinidumpExceptionHandler::RunUserSignalHandlerThread() {
+  sigset_t signals = GetSigset(SIGUSR1);
+  while (true) {
+    int signal;
+    int err = sigwait(&signals, &signal);
+    CHECK(err == 0) << "sigwait(): " << ErrnoToString(err) << ": " << err;
+    CHECK_EQ(SIGUSR1, signal);
+    if (!user_signal_handler_thread_running_.load(std::memory_order_relaxed)) {
+      // Exit thread if we are shutting down.
+      return;
+    }
+    if (!WriteMinidump()) {
+      LOG(WARNING) << "Received USR1 signal but failed to write minidump";
+    }
+  }
+}
+
+#else // defined(__linux__)
+
+// At the time of writing, we don't support breakpad on Mac so we just stub out
+// all the methods defined in the header file.
+
+Status MinidumpExceptionHandler::InitMinidumpExceptionHandler() {
+  return Status::OK();
+}
+
+// No-op on non-Linux platforms.
+Status MinidumpExceptionHandler::RegisterMinidumpExceptionHandler() {
+  return Status::OK();
+}
+
+void MinidumpExceptionHandler::UnregisterMinidumpExceptionHandler() {
+}
+
+bool MinidumpExceptionHandler::WriteMinidump() {
+  return true;
+}
+
+Status MinidumpExceptionHandler::StartUserSignalHandlerThread() {
+  return Status::OK();
+}
+
+void MinidumpExceptionHandler::StopUserSignalHandlerThread() {
+}
+
+void MinidumpExceptionHandler::RunUserSignalHandlerThread() {
+}
+
+#endif // defined(__linux__)
+
+std::atomic<int> MinidumpExceptionHandler::current_num_instances_;
+
+MinidumpExceptionHandler::MinidumpExceptionHandler() {
+  CHECK_OK(RegisterMinidumpExceptionHandler());
+}
+
+MinidumpExceptionHandler::~MinidumpExceptionHandler() {
+  UnregisterMinidumpExceptionHandler();
+}
+
+Status MinidumpExceptionHandler::DeleteExcessMinidumpFiles(Env* env) {
+  // Do not delete minidump files if minidumps are disabled.
+  if (!FLAGS_enable_minidumps) return Status::OK();
+
+  int32_t max_minidumps = FLAGS_max_minidumps;
+  // Disable rotation if set to 0 or less.
+  if (max_minidumps <= 0) return Status::OK();
+
+  // Minidump filenames are created by breakpad in the following format, for example:
+  // 7b57915b-ee6a-dbc5-21e59491-5c60a2cf.dmp.
+  string pattern = JoinPathSegments(minidump_dir(), "*.dmp");
+
+  // Use mtime to determine which minidumps to delete. While this could
+  // potentially be ambiguous if many minidumps were created in quick
+  // succession, users can always increase 'FLAGS_max_minidumps' if desired
+  // in order to work around the problem.
+  return env_util::DeleteExcessFilesByPattern(env, pattern, max_minidumps);
+}
+
+string MinidumpExceptionHandler::minidump_dir() const {
+  return minidump_dir_;
+}
+
+Status BlockSigUSR1() {
+  sigset_t signals = GetSigset(SIGUSR1);
+  int ret = pthread_sigmask(SIG_BLOCK, &signals, nullptr);
+  if (ret == 0) return Status::OK();
+  return Status::InvalidArgument("pthread_sigmask", ErrnoToString(ret), ret);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/minidump.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/minidump.h b/be/src/kudu/util/minidump.h
new file mode 100644
index 0000000..893440a
--- /dev/null
+++ b/be/src/kudu/util/minidump.h
@@ -0,0 +1,104 @@
+// 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.
+
+#pragma once
+
+#include <atomic>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/ref_counted.h"
+
+namespace google_breakpad {
+class ExceptionHandler;
+} // namespace google_breakpad
+
+namespace kudu {
+
+class Env;
+class Status;
+class Thread;
+
+// While an instance of this class is in scope, a Breakpad minidump handler
+// will generate a minidump for the current program if it crashes or if it
+// received a USR1 signal. This class must be instantiated after initializing
+// the gflags library. When used in conjuction with glog, or other signal
+// handling facilities, this class must be invoked after installing those
+// signal handlers.
+//
+// The BlockSigUSR1() function should be called before spawning any threads in
+// order to block the USR1 signal from crashing the process. This class relies
+// on that signal being blocked by all threads in order to safely generate
+// minidumps in response to the USR1 signal.
+//
+// Only one instance of this class may be instantiated at a time.
+//
+// For more information on Google Breakpad, see its documentation at:
+// http://chromium.googlesource.com/breakpad/breakpad/+/master/docs/getting_started_with_breakpad.md
+class MinidumpExceptionHandler {
+ public:
+  MinidumpExceptionHandler();
+  ~MinidumpExceptionHandler();
+
+  // Write a minidump immediately. Can be used to generate a minidump
+  // independently of a crash. Should not be called from a signal handler or a
+  // crash context because it uses the heap.
+  bool WriteMinidump();
+
+  // Deletes excess minidump files beyond the configured max of
+  // 'FLAGS_max_minidumps'. Uses the file's modified time to determine recency.
+  // Does nothing if 'FLAGS_enabled_minidumps' is false.
+  Status DeleteExcessMinidumpFiles(Env* env);
+
+  // Get the path to the directory that will be used for writing minidumps.
+  std::string minidump_dir() const;
+
+ private:
+  Status InitMinidumpExceptionHandler();
+  Status RegisterMinidumpExceptionHandler();
+  void UnregisterMinidumpExceptionHandler();
+
+  Status StartUserSignalHandlerThread();
+  void StopUserSignalHandlerThread();
+  void RunUserSignalHandlerThread();
+
+  // The number of instnaces of this class that are currently in existence.
+  // We keep this counter in order to force a crash if more than one is running
+  // at a time, as a sanity check.
+  static std::atomic<int> current_num_instances_;
+
+  #ifndef __APPLE__
+  std::atomic<bool> user_signal_handler_thread_running_;// Unused in macOS build.
+  #endif
+
+  scoped_refptr<Thread> user_signal_handler_thread_;
+
+  // Breakpad ExceptionHandler. It registers its own signal handlers to write
+  // minidump files during process crashes, but can also be used to write
+  // minidumps directly.
+  std::unique_ptr<google_breakpad::ExceptionHandler> breakpad_handler_;
+
+  // Directory in which we store our minidumps.
+  std::string minidump_dir_;
+};
+
+// Block SIGUSR1 from threads handling it.
+// This should be called by the process before spawning any threads so that a
+// USR1 signal will cause a minidump to be generated instead of a crash.
+Status BlockSigUSR1();
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/monotime-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/monotime-test.cc b/be/src/kudu/util/monotime-test.cc
new file mode 100644
index 0000000..755dac5
--- /dev/null
+++ b/be/src/kudu/util/monotime-test.cc
@@ -0,0 +1,424 @@
+// 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 "kudu/util/monotime.h"
+
+#include <sys/time.h>
+#include <unistd.h>
+
+#include <cstdint>
+#include <ctime>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+TEST(TestMonoTime, TestMonotonicity) {
+  alarm(360);
+  MonoTime prev(MonoTime::Now());
+  MonoTime next;
+
+  do {
+    next = MonoTime::Now();
+    //LOG(INFO) << " next = " << next.ToString();
+  } while (!prev.ComesBefore(next));
+  ASSERT_FALSE(next.ComesBefore(prev));
+  alarm(0);
+}
+
+TEST(TestMonoTime, TestComparison) {
+  MonoTime now(MonoTime::Now());
+  MonoTime future(now);
+  future.AddDelta(MonoDelta::FromNanoseconds(1L));
+
+  ASSERT_GT((future - now).ToNanoseconds(), 0);
+  ASSERT_LT((now - future).ToNanoseconds(), 0);
+  ASSERT_EQ((now - now).ToNanoseconds(), 0);
+
+  MonoDelta nano(MonoDelta::FromNanoseconds(1L));
+  MonoDelta mil(MonoDelta::FromMilliseconds(1L));
+  MonoDelta sec(MonoDelta::FromSeconds(1.0));
+
+  ASSERT_TRUE(nano.LessThan(mil));
+  ASSERT_TRUE(mil.LessThan(sec));
+  ASSERT_TRUE(mil.MoreThan(nano));
+  ASSERT_TRUE(sec.MoreThan(mil));
+}
+
+TEST(TestMonoTime, TestTimeVal) {
+  struct timeval tv;
+  tv.tv_sec = 0;
+  tv.tv_usec = 0;
+
+  // Normal conversion case.
+  MonoDelta one_sec_one_micro(MonoDelta::FromNanoseconds(1000001000L));
+  one_sec_one_micro.ToTimeVal(&tv);
+  ASSERT_EQ(1, tv.tv_sec);
+  ASSERT_EQ(1, tv.tv_usec);
+
+  // Case where we are still positive but sub-micro.
+  // Round up to nearest microsecond. This is to avoid infinite timeouts
+  // in APIs that take a struct timeval.
+  MonoDelta zero_sec_one_nano(MonoDelta::FromNanoseconds(1L));
+  zero_sec_one_nano.ToTimeVal(&tv);
+  ASSERT_EQ(0, tv.tv_sec);
+  ASSERT_EQ(1, tv.tv_usec); // Special case: 1ns rounds up to
+
+  // Negative conversion case. Ensure the timeval is normalized.
+  // That means sec is negative and usec is positive.
+  MonoDelta neg_micro(MonoDelta::FromMicroseconds(-1L));
+  ASSERT_EQ(-1000, neg_micro.ToNanoseconds());
+  neg_micro.ToTimeVal(&tv);
+  ASSERT_EQ(-1, tv.tv_sec);
+  ASSERT_EQ(999999, tv.tv_usec);
+
+  // Case where we are still negative but sub-micro.
+  // Round up to nearest microsecond. This is to avoid infinite timeouts
+  // in APIs that take a struct timeval and for consistency.
+  MonoDelta zero_sec_neg_one_nano(MonoDelta::FromNanoseconds(-1L));
+  zero_sec_neg_one_nano.ToTimeVal(&tv);
+  ASSERT_EQ(-1, tv.tv_sec);
+  ASSERT_EQ(999999, tv.tv_usec);
+}
+
+TEST(TestMonoTime, TestTimeSpec) {
+  MonoTime one_sec_one_nano_expected(1000000001L);
+  struct timespec ts;
+  ts.tv_sec = 1;
+  ts.tv_nsec = 1;
+  MonoTime one_sec_one_nano_actual(ts);
+  ASSERT_EQ(0, one_sec_one_nano_expected.GetDeltaSince(one_sec_one_nano_actual).ToNanoseconds());
+
+  MonoDelta zero_sec_two_nanos(MonoDelta::FromNanoseconds(2L));
+  zero_sec_two_nanos.ToTimeSpec(&ts);
+  ASSERT_EQ(0, ts.tv_sec);
+  ASSERT_EQ(2, ts.tv_nsec);
+
+  // Negative conversion case. Ensure the timespec is normalized.
+  // That means sec is negative and nsec is positive.
+  MonoDelta neg_nano(MonoDelta::FromNanoseconds(-1L));
+  ASSERT_EQ(-1, neg_nano.ToNanoseconds());
+  neg_nano.ToTimeSpec(&ts);
+  ASSERT_EQ(-1, ts.tv_sec);
+  ASSERT_EQ(999999999, ts.tv_nsec);
+
+}
+
+TEST(TestMonoTime, TestDeltas) {
+  alarm(360);
+  const MonoDelta max_delta(MonoDelta::FromSeconds(0.1));
+  MonoTime prev(MonoTime::Now());
+  MonoTime next;
+  MonoDelta cur_delta;
+  do {
+    next = MonoTime::Now();
+    cur_delta = next.GetDeltaSince(prev);
+  } while (cur_delta.LessThan(max_delta));
+  alarm(0);
+}
+
+TEST(TestMonoTime, TestDeltaConversions) {
+  // TODO: Reliably test MonoDelta::FromSeconds() considering floating-point rounding errors
+
+  MonoDelta mil(MonoDelta::FromMilliseconds(500));
+  ASSERT_EQ(500 * MonoTime::kNanosecondsPerMillisecond, mil.nano_delta_);
+
+  MonoDelta micro(MonoDelta::FromMicroseconds(500));
+  ASSERT_EQ(500 * MonoTime::kNanosecondsPerMicrosecond, micro.nano_delta_);
+
+  MonoDelta nano(MonoDelta::FromNanoseconds(500));
+  ASSERT_EQ(500, nano.nano_delta_);
+}
+
+static void DoTestMonoTimePerf() {
+  const MonoDelta max_delta(MonoDelta::FromMilliseconds(500));
+  uint64_t num_calls = 0;
+  MonoTime prev(MonoTime::Now());
+  MonoTime next;
+  MonoDelta cur_delta;
+  do {
+    next = MonoTime::Now();
+    cur_delta = next.GetDeltaSince(prev);
+    num_calls++;
+  } while (cur_delta.LessThan(max_delta));
+  LOG(INFO) << "DoTestMonoTimePerf():"
+        << num_calls << " in "
+        << max_delta.ToString() << " seconds.";
+}
+
+TEST(TestMonoTime, TestSleepFor) {
+  MonoTime start = MonoTime::Now();
+  MonoDelta sleep = MonoDelta::FromMilliseconds(100);
+  SleepFor(sleep);
+  MonoTime end = MonoTime::Now();
+  MonoDelta actualSleep = end.GetDeltaSince(start);
+  ASSERT_GE(actualSleep.ToNanoseconds(), sleep.ToNanoseconds());
+}
+
+TEST(TestMonoTime, TestSleepForOverflow) {
+  if (!AllowSlowTests()) {
+    LOG(INFO) << "Skipping test because it sleeps for ~4s";
+    return;
+  }
+
+  // This quantity (~4s sleep) overflows a 32-bit integer such that
+  // the value becomes 0.
+  MonoTime start = MonoTime::Now();
+  MonoDelta sleep = MonoDelta::FromNanoseconds(1L << 32);
+  SleepFor(sleep);
+  MonoTime end = MonoTime::Now();
+  MonoDelta actualSleep = end.GetDeltaSince(start);
+  ASSERT_GE(actualSleep.ToNanoseconds(), sleep.ToNanoseconds());
+}
+
+// Test functionality of the handy operators for MonoTime/MonoDelta objects.
+// The test assumes that the core functionality provided by the
+// MonoTime/MonoDelta objects are in place, and it tests that the operators
+// have the expected behavior expressed in terms of already existing,
+// semantically equivalent methods.
+TEST(TestMonoTime, TestOperators) {
+  // MonoTime& MonoTime::operator+=(const MonoDelta& delta);
+  {
+    MonoTime tmp = MonoTime::Now();
+    MonoTime start = tmp;
+    MonoDelta delta = MonoDelta::FromMilliseconds(100);
+    MonoTime o_end = start;
+    o_end += delta;
+    tmp.AddDelta(delta);
+    MonoTime m_end = tmp;
+    EXPECT_TRUE(m_end.Equals(o_end));
+  }
+
+  // MonoTime& MonoTime::operator-=(const MonoDelta& delta);
+  {
+    MonoTime tmp = MonoTime::Now();
+    MonoTime start = tmp;
+    MonoDelta delta = MonoDelta::FromMilliseconds(100);
+    MonoTime o_end = start;
+    o_end -= delta;
+    tmp.AddDelta(MonoDelta::FromNanoseconds(-delta.ToNanoseconds()));
+    MonoTime m_end = tmp;
+    EXPECT_TRUE(m_end.Equals(o_end));
+  }
+
+  // bool operator==(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta dn = MonoDelta::FromNanoseconds(0);
+    MonoDelta dm = MonoDelta::FromMicroseconds(0);
+    ASSERT_TRUE(dn.Equals(dm));
+    EXPECT_TRUE(dn == dm);
+    EXPECT_TRUE(dm == dn);
+  }
+
+  // bool operator!=(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta dn = MonoDelta::FromNanoseconds(1);
+    MonoDelta dm = MonoDelta::FromMicroseconds(1);
+    ASSERT_FALSE(dn.Equals(dm));
+    EXPECT_TRUE(dn != dm);
+    EXPECT_TRUE(dm != dn);
+  }
+
+  // bool operator<(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta d0 = MonoDelta::FromNanoseconds(0);
+    MonoDelta d1 = MonoDelta::FromNanoseconds(1);
+    ASSERT_TRUE(d0.LessThan(d1));
+    EXPECT_TRUE(d0 < d1);
+  }
+
+  // bool operator<=(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta d0 = MonoDelta::FromNanoseconds(0);
+    MonoDelta d1 = MonoDelta::FromNanoseconds(1);
+    ASSERT_TRUE(d0.LessThan(d1));
+    EXPECT_TRUE(d0 <= d1);
+
+    MonoDelta d20 = MonoDelta::FromNanoseconds(2);
+    MonoDelta d21 = MonoDelta::FromNanoseconds(2);
+    ASSERT_TRUE(d20.Equals(d21));
+    EXPECT_TRUE(d20 <= d21);
+  }
+
+  // bool operator>(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta d0 = MonoDelta::FromNanoseconds(0);
+    MonoDelta d1 = MonoDelta::FromNanoseconds(1);
+    ASSERT_TRUE(d1.MoreThan(d0));
+    EXPECT_TRUE(d1 > d0);
+  }
+
+  // bool operator>=(const MonoDelta& lhs, const MonoDelta& rhs);
+  {
+    MonoDelta d0 = MonoDelta::FromNanoseconds(0);
+    MonoDelta d1 = MonoDelta::FromNanoseconds(1);
+    ASSERT_TRUE(d1.MoreThan(d0));
+    EXPECT_TRUE(d1 >= d1);
+
+    MonoDelta d20 = MonoDelta::FromNanoseconds(2);
+    MonoDelta d21 = MonoDelta::FromNanoseconds(2);
+    ASSERT_TRUE(d20.Equals(d21));
+    EXPECT_TRUE(d21 >= d20);
+  }
+
+  // bool operator==(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t0 = MonoTime::Now();
+    MonoTime t1(t0);
+    ASSERT_TRUE(t0.Equals(t1));
+    ASSERT_TRUE(t1.Equals(t0));
+    EXPECT_TRUE(t0 == t1);
+    EXPECT_TRUE(t1 == t0);
+  }
+
+  // bool operator!=(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t0 = MonoTime::Now();
+    MonoTime t1(t0 + MonoDelta::FromMilliseconds(100));
+    ASSERT_TRUE(!t0.Equals(t1));
+    ASSERT_TRUE(!t1.Equals(t0));
+    EXPECT_TRUE(t0 != t1);
+    EXPECT_TRUE(t1 != t0);
+  }
+
+  // bool operator<(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t0 = MonoTime::Now();
+    MonoTime t1(t0 + MonoDelta::FromMilliseconds(100));
+    ASSERT_TRUE(t0.ComesBefore(t1));
+    ASSERT_FALSE(t1.ComesBefore(t0));
+    EXPECT_TRUE(t0 < t1);
+    EXPECT_FALSE(t1 < t0);
+  }
+
+  // bool operator<=(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t00 = MonoTime::Now();
+    MonoTime t01(t00);
+    ASSERT_TRUE(t00.Equals(t00));
+    ASSERT_TRUE(t00.Equals(t01));
+    ASSERT_TRUE(t01.Equals(t00));
+    ASSERT_TRUE(t01.Equals(t01));
+    EXPECT_TRUE(t00 <= t00);
+    EXPECT_TRUE(t00 <= t01);
+    EXPECT_TRUE(t01 <= t00);
+    EXPECT_TRUE(t01 <= t01);
+
+    MonoTime t1(t00 + MonoDelta::FromMilliseconds(100));
+    ASSERT_TRUE(t00.ComesBefore(t1));
+    ASSERT_TRUE(t01.ComesBefore(t1));
+    ASSERT_FALSE(t1.ComesBefore(t00));
+    ASSERT_FALSE(t1.ComesBefore(t01));
+    EXPECT_TRUE(t00 <= t1);
+    EXPECT_TRUE(t01 <= t1);
+    EXPECT_FALSE(t1 <= t00);
+    EXPECT_FALSE(t1 <= t01);
+  }
+
+  // bool operator>(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t0 = MonoTime::Now();
+    MonoTime t1(t0 + MonoDelta::FromMilliseconds(100));
+    ASSERT_TRUE(t0.ComesBefore(t1));
+    ASSERT_FALSE(t1.ComesBefore(t0));
+    EXPECT_TRUE(t0 < t1);
+    EXPECT_FALSE(t1 < t0);
+  }
+
+  // bool operator>=(const MonoTime& lhs, const MonoTime& rhs);
+  {
+    MonoTime t00 = MonoTime::Now();
+    MonoTime t01(t00);
+    ASSERT_TRUE(t00.Equals(t00));
+    ASSERT_TRUE(t00.Equals(t01));
+    ASSERT_TRUE(t01.Equals(t00));
+    ASSERT_TRUE(t01.Equals(t01));
+    EXPECT_TRUE(t00 >= t00);
+    EXPECT_TRUE(t00 >= t01);
+    EXPECT_TRUE(t01 >= t00);
+    EXPECT_TRUE(t01 >= t01);
+
+    MonoTime t1(t00 + MonoDelta::FromMilliseconds(100));
+    ASSERT_TRUE(t00.ComesBefore(t1));
+    ASSERT_TRUE(t01.ComesBefore(t1));
+    ASSERT_FALSE(t1.ComesBefore(t00));
+    ASSERT_FALSE(t1.ComesBefore(t01));
+    EXPECT_FALSE(t00 >= t1);
+    EXPECT_FALSE(t01 >= t1);
+    EXPECT_TRUE(t1 >= t00);
+    EXPECT_TRUE(t1 >= t01);
+  }
+
+  // MonoDelta operator-(const MonoTime& t0, const MonoTime& t1);
+  {
+    const int64_t deltas[] = { 100, -100 };
+
+    MonoTime tmp = MonoTime::Now();
+    for (auto d : deltas) {
+      MonoDelta delta = MonoDelta::FromMilliseconds(d);
+
+      MonoTime start = tmp;
+      tmp.AddDelta(delta);
+      MonoTime end = tmp;
+      MonoDelta delta_o = end - start;
+      EXPECT_TRUE(delta.Equals(delta_o));
+    }
+  }
+
+  // MonoTime operator+(const MonoTime& t, const MonoDelta& delta);
+  {
+    MonoTime start = MonoTime::Now();
+
+    MonoDelta delta_0 = MonoDelta::FromMilliseconds(0);
+    MonoTime end_0 = start + delta_0;
+    EXPECT_TRUE(end_0.Equals(start));
+
+    MonoDelta delta_1 = MonoDelta::FromMilliseconds(1);
+    MonoTime end_1 = start + delta_1;
+    EXPECT_TRUE(end_1 > end_0);
+    end_0.AddDelta(delta_1);
+    EXPECT_TRUE(end_0.Equals(end_1));
+  }
+
+  // MonoTime operator-(const MonoTime& t, const MonoDelta& delta);
+  {
+    MonoTime start = MonoTime::Now();
+
+    MonoDelta delta_0 = MonoDelta::FromMilliseconds(0);
+    MonoTime end_0 = start - delta_0;
+    EXPECT_TRUE(end_0.Equals(start));
+
+    MonoDelta delta_1 = MonoDelta::FromMilliseconds(1);
+    MonoTime end_1 = start - delta_1;
+    EXPECT_TRUE(end_1 < end_0);
+    end_1.AddDelta(delta_1);
+    EXPECT_TRUE(end_1.Equals(end_0));
+  }
+}
+
+TEST(TestMonoTimePerf, TestMonoTimePerf) {
+  alarm(360);
+  DoTestMonoTimePerf();
+  alarm(0);
+}
+
+} // namespace kudu


[39/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc-test-base.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc-test-base.h b/be/src/kudu/rpc/rpc-test-base.h
new file mode 100644
index 0000000..2fb742e
--- /dev/null
+++ b/be/src/kudu/rpc/rpc-test-base.h
@@ -0,0 +1,661 @@
+// 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.
+#ifndef KUDU_RPC_RPC_TEST_BASE_H
+#define KUDU_RPC_RPC_TEST_BASE_H
+
+#include <algorithm>
+#include <atomic>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/proxy.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/result_tracker.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/rtest.proxy.h"
+#include "kudu/rpc/rtest.service.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/service_pool.h"
+#include "kudu/security/security-test-util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/trace.h"
+
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
+namespace kudu {
+namespace rpc {
+
+using kudu::rpc_test::AddRequestPB;
+using kudu::rpc_test::AddResponsePB;
+using kudu::rpc_test::CalculatorError;
+using kudu::rpc_test::CalculatorServiceIf;
+using kudu::rpc_test::CalculatorServiceProxy;
+using kudu::rpc_test::EchoRequestPB;
+using kudu::rpc_test::EchoResponsePB;
+using kudu::rpc_test::ExactlyOnceRequestPB;
+using kudu::rpc_test::ExactlyOnceResponsePB;
+using kudu::rpc_test::FeatureFlags;
+using kudu::rpc_test::PanicRequestPB;
+using kudu::rpc_test::PanicResponsePB;
+using kudu::rpc_test::PushTwoStringsRequestPB;
+using kudu::rpc_test::PushTwoStringsResponsePB;
+using kudu::rpc_test::SendTwoStringsRequestPB;
+using kudu::rpc_test::SendTwoStringsResponsePB;
+using kudu::rpc_test::SleepRequestPB;
+using kudu::rpc_test::SleepResponsePB;
+using kudu::rpc_test::SleepWithSidecarRequestPB;
+using kudu::rpc_test::SleepWithSidecarResponsePB;
+using kudu::rpc_test::TestInvalidResponseRequestPB;
+using kudu::rpc_test::TestInvalidResponseResponsePB;
+using kudu::rpc_test::WhoAmIRequestPB;
+using kudu::rpc_test::WhoAmIResponsePB;
+using kudu::rpc_test_diff_package::ReqDiffPackagePB;
+using kudu::rpc_test_diff_package::RespDiffPackagePB;
+
+// Implementation of CalculatorService which just implements the generic
+// RPC handler (no generated code).
+class GenericCalculatorService : public ServiceIf {
+ public:
+  static const char *kFullServiceName;
+  static const char *kAddMethodName;
+  static const char *kSleepMethodName;
+  static const char *kSleepWithSidecarMethodName;
+  static const char *kPushTwoStringsMethodName;
+  static const char *kSendTwoStringsMethodName;
+  static const char *kAddExactlyOnce;
+
+  static const char* kFirstString;
+  static const char* kSecondString;
+
+  GenericCalculatorService() {
+  }
+
+  // To match the argument list of the generated CalculatorService.
+  explicit GenericCalculatorService(const scoped_refptr<MetricEntity>& entity,
+                                    const scoped_refptr<ResultTracker>& result_tracker) {
+    // this test doesn't generate metrics, so we ignore the argument.
+  }
+
+  void Handle(InboundCall *incoming) override {
+    if (incoming->remote_method().method_name() == kAddMethodName) {
+      DoAdd(incoming);
+    } else if (incoming->remote_method().method_name() == kSleepMethodName) {
+      DoSleep(incoming);
+    } else if (incoming->remote_method().method_name() == kSleepWithSidecarMethodName) {
+      DoSleepWithSidecar(incoming);
+    } else if (incoming->remote_method().method_name() == kSendTwoStringsMethodName) {
+      DoSendTwoStrings(incoming);
+    } else if (incoming->remote_method().method_name() == kPushTwoStringsMethodName) {
+      DoPushTwoStrings(incoming);
+    } else {
+      incoming->RespondFailure(ErrorStatusPB::ERROR_NO_SUCH_METHOD,
+                               Status::InvalidArgument("bad method"));
+    }
+  }
+
+  std::string service_name() const override { return kFullServiceName; }
+  static std::string static_service_name() { return kFullServiceName; }
+
+ private:
+  void DoAdd(InboundCall *incoming) {
+    Slice param(incoming->serialized_request());
+    AddRequestPB req;
+    if (!req.ParseFromArray(param.data(), param.size())) {
+      LOG(FATAL) << "couldn't parse: " << param.ToDebugString();
+    }
+
+    AddResponsePB resp;
+    resp.set_result(req.x() + req.y());
+    incoming->RespondSuccess(resp);
+  }
+
+  void DoSendTwoStrings(InboundCall* incoming) {
+    Slice param(incoming->serialized_request());
+    SendTwoStringsRequestPB req;
+    if (!req.ParseFromArray(param.data(), param.size())) {
+      LOG(FATAL) << "couldn't parse: " << param.ToDebugString();
+    }
+
+    std::unique_ptr<faststring> first(new faststring);
+    std::unique_ptr<faststring> second(new faststring);
+
+    Random r(req.random_seed());
+    first->resize(req.size1());
+    RandomString(first->data(), req.size1(), &r);
+
+    second->resize(req.size2());
+    RandomString(second->data(), req.size2(), &r);
+
+    SendTwoStringsResponsePB resp;
+    int idx1, idx2;
+    CHECK_OK(incoming->AddOutboundSidecar(
+            RpcSidecar::FromFaststring(std::move(first)), &idx1));
+    CHECK_OK(incoming->AddOutboundSidecar(
+            RpcSidecar::FromFaststring(std::move(second)), &idx2));
+    resp.set_sidecar1(idx1);
+    resp.set_sidecar2(idx2);
+
+    incoming->RespondSuccess(resp);
+  }
+
+  void DoPushTwoStrings(InboundCall* incoming) {
+    Slice param(incoming->serialized_request());
+    PushTwoStringsRequestPB req;
+    if (!req.ParseFromArray(param.data(), param.size())) {
+      LOG(FATAL) << "couldn't parse: " << param.ToDebugString();
+    }
+
+    Slice sidecar1;
+    CHECK_OK(incoming->GetInboundSidecar(req.sidecar1_idx(), &sidecar1));
+
+    Slice sidecar2;
+    CHECK_OK(incoming->GetInboundSidecar(req.sidecar2_idx(), &sidecar2));
+
+    // Check that reading non-existant sidecars doesn't work.
+    Slice tmp;
+    CHECK(!incoming->GetInboundSidecar(req.sidecar2_idx() + 2, &tmp).ok());
+
+    PushTwoStringsResponsePB resp;
+    resp.set_size1(sidecar1.size());
+    resp.set_data1(reinterpret_cast<const char*>(sidecar1.data()), sidecar1.size());
+    resp.set_size2(sidecar2.size());
+    resp.set_data2(reinterpret_cast<const char*>(sidecar2.data()), sidecar2.size());
+
+    // Drop the sidecars etc, just to confirm that it's safe to do so.
+    CHECK_GT(incoming->GetTransferSize(), 0);
+    incoming->DiscardTransfer();
+    CHECK_EQ(0, incoming->GetTransferSize());
+    incoming->RespondSuccess(resp);
+  }
+
+  void DoSleep(InboundCall *incoming) {
+    Slice param(incoming->serialized_request());
+    SleepRequestPB req;
+    if (!req.ParseFromArray(param.data(), param.size())) {
+      incoming->RespondFailure(ErrorStatusPB::ERROR_INVALID_REQUEST,
+        Status::InvalidArgument("Couldn't parse pb",
+                                req.InitializationErrorString()));
+      return;
+    }
+
+    LOG(INFO) << "got call: " << pb_util::SecureShortDebugString(req);
+    SleepFor(MonoDelta::FromMicroseconds(req.sleep_micros()));
+    MonoDelta duration(MonoTime::Now().GetDeltaSince(incoming->GetTimeReceived()));
+    CHECK_GE(duration.ToMicroseconds(), req.sleep_micros());
+    SleepResponsePB resp;
+    incoming->RespondSuccess(resp);
+  }
+
+  void DoSleepWithSidecar(InboundCall *incoming) {
+    Slice param(incoming->serialized_request());
+    SleepWithSidecarRequestPB req;
+    if (!req.ParseFromArray(param.data(), param.size())) {
+      incoming->RespondFailure(ErrorStatusPB::ERROR_INVALID_REQUEST,
+        Status::InvalidArgument("Couldn't parse pb",
+                                req.InitializationErrorString()));
+      return;
+    }
+
+    LOG(INFO) << "got call: " << pb_util::SecureShortDebugString(req);
+    SleepFor(MonoDelta::FromMicroseconds(req.sleep_micros()));
+
+    uint32_t pattern = req.pattern();
+    uint32_t num_repetitions = req.num_repetitions();
+    Slice sidecar;
+    CHECK_OK(incoming->GetInboundSidecar(req.sidecar_idx(), &sidecar));
+    CHECK_EQ(sidecar.size(), sizeof(uint32) * num_repetitions);
+    const uint32_t *data = reinterpret_cast<const uint32_t*>(sidecar.data());
+    for (int i = 0; i < num_repetitions; ++i) CHECK_EQ(data[i], pattern);
+
+    SleepResponsePB resp;
+    incoming->RespondSuccess(resp);
+  }
+};
+
+class CalculatorService : public CalculatorServiceIf {
+ public:
+  explicit CalculatorService(const scoped_refptr<MetricEntity>& entity,
+                             const scoped_refptr<ResultTracker> result_tracker)
+    : CalculatorServiceIf(entity, result_tracker),
+      exactly_once_test_val_(0) {
+  }
+
+  void Add(const AddRequestPB *req, AddResponsePB *resp, RpcContext *context) override {
+    CHECK_GT(context->GetTransferSize(), 0);
+    resp->set_result(req->x() + req->y());
+    context->RespondSuccess();
+  }
+
+  void Sleep(const SleepRequestPB *req, SleepResponsePB *resp, RpcContext *context) override {
+    if (req->return_app_error()) {
+      CalculatorError my_error;
+      my_error.set_extra_error_data("some application-specific error data");
+      context->RespondApplicationError(CalculatorError::app_error_ext.number(),
+                                       "Got some error", my_error);
+      return;
+    }
+
+    // Respond w/ error if the RPC specifies that the client deadline is set,
+    // but it isn't.
+    if (req->client_timeout_defined()) {
+      MonoTime deadline = context->GetClientDeadline();
+      if (deadline == MonoTime::Max()) {
+        CalculatorError my_error;
+        my_error.set_extra_error_data("Timeout not set");
+        context->RespondApplicationError(CalculatorError::app_error_ext.number(),
+                                        "Missing required timeout", my_error);
+        return;
+      }
+    }
+
+    if (req->deferred()) {
+      // Spawn a new thread which does the sleep and responds later.
+      scoped_refptr<Thread> thread;
+      CHECK_OK(Thread::Create("rpc-test", "deferred",
+                              &CalculatorService::DoSleep, this, req, context,
+                              &thread));
+      return;
+    }
+    DoSleep(req, context);
+  }
+
+  void Echo(const EchoRequestPB *req, EchoResponsePB *resp, RpcContext *context) override {
+    resp->set_data(req->data());
+    context->RespondSuccess();
+  }
+
+  void WhoAmI(const WhoAmIRequestPB* /*req*/,
+              WhoAmIResponsePB* resp,
+              RpcContext* context) override {
+    const RemoteUser& user = context->remote_user();
+    resp->mutable_credentials()->set_real_user(user.username());
+    resp->set_address(context->remote_address().ToString());
+    context->RespondSuccess();
+  }
+
+  void TestArgumentsInDiffPackage(const ReqDiffPackagePB *req,
+                                  RespDiffPackagePB *resp,
+                                  ::kudu::rpc::RpcContext *context) override {
+    context->RespondSuccess();
+  }
+
+  void Panic(const PanicRequestPB* req, PanicResponsePB* resp, RpcContext* context) override {
+    TRACE("Got panic request");
+    PANIC_RPC(context, "Test method panicking!");
+  }
+
+  void TestInvalidResponse(const TestInvalidResponseRequestPB* req,
+                           TestInvalidResponseResponsePB* resp,
+                           RpcContext* context) override {
+    switch (req->error_type()) {
+      case rpc_test::TestInvalidResponseRequestPB_ErrorType_MISSING_REQUIRED_FIELD:
+        // Respond without setting the 'resp->response' protobuf field, which is
+        // marked as required. This exercises the error path of invalid responses.
+        context->RespondSuccess();
+        break;
+      case rpc_test::TestInvalidResponseRequestPB_ErrorType_RESPONSE_TOO_LARGE:
+        resp->mutable_response()->resize(FLAGS_rpc_max_message_size + 1000);
+        context->RespondSuccess();
+        break;
+      default:
+        LOG(FATAL);
+    }
+  }
+
+  bool SupportsFeature(uint32_t feature) const override {
+    return feature == FeatureFlags::FOO;
+  }
+
+  void AddExactlyOnce(const ExactlyOnceRequestPB* req, ExactlyOnceResponsePB* resp,
+                      ::kudu::rpc::RpcContext* context) override {
+    if (req->sleep_for_ms() > 0) {
+      usleep(req->sleep_for_ms() * 1000);
+    }
+    // If failures are enabled, cause them some percentage of the time.
+    if (req->randomly_fail()) {
+      if (rand() % 10 < 3) {
+        context->RespondFailure(Status::ServiceUnavailable("Random injected failure."));
+        return;
+      }
+    }
+    int result = exactly_once_test_val_ += req->value_to_add();
+    resp->set_current_val(result);
+    resp->set_current_time_micros(GetCurrentTimeMicros());
+    context->RespondSuccess();
+  }
+
+  bool AuthorizeDisallowAlice(const google::protobuf::Message* /*req*/,
+                              google::protobuf::Message* /*resp*/,
+                              RpcContext* context) override {
+    if (context->remote_user().username() == "alice") {
+      context->RespondFailure(Status::NotAuthorized("alice is not allowed to call this method"));
+      return false;
+    }
+    return true;
+  }
+
+  bool AuthorizeDisallowBob(const google::protobuf::Message* /*req*/,
+                            google::protobuf::Message* /*resp*/,
+                            RpcContext* context) override {
+    if (context->remote_user().username() == "bob") {
+      context->RespondFailure(Status::NotAuthorized("bob is not allowed to call this method"));
+      return false;
+    }
+    return true;
+  }
+
+ private:
+  void DoSleep(const SleepRequestPB *req,
+               RpcContext *context) {
+    TRACE_COUNTER_INCREMENT("test_sleep_us", req->sleep_micros());
+    if (Trace::CurrentTrace()) {
+      scoped_refptr<Trace> child_trace(new Trace());
+      Trace::CurrentTrace()->AddChildTrace("test_child", child_trace.get());
+      ADOPT_TRACE(child_trace.get());
+      TRACE_COUNTER_INCREMENT("related_trace_metric", 1);
+    }
+
+    SleepFor(MonoDelta::FromMicroseconds(req->sleep_micros()));
+    context->RespondSuccess();
+  }
+
+  std::atomic_int exactly_once_test_val_;
+
+};
+
+const char *GenericCalculatorService::kFullServiceName = "kudu.rpc.GenericCalculatorService";
+const char *GenericCalculatorService::kAddMethodName = "Add";
+const char *GenericCalculatorService::kSleepMethodName = "Sleep";
+const char *GenericCalculatorService::kSleepWithSidecarMethodName = "SleepWithSidecar";
+const char *GenericCalculatorService::kPushTwoStringsMethodName = "PushTwoStrings";
+const char *GenericCalculatorService::kSendTwoStringsMethodName = "SendTwoStrings";
+const char *GenericCalculatorService::kAddExactlyOnce = "AddExactlyOnce";
+
+const char *GenericCalculatorService::kFirstString =
+    "1111111111111111111111111111111111111111111111111111111111";
+const char *GenericCalculatorService::kSecondString =
+    "2222222222222222222222222222222222222222222222222222222222222222222222";
+
+class RpcTestBase : public KuduTest {
+ public:
+  RpcTestBase()
+    : n_worker_threads_(3),
+      service_queue_length_(100),
+      n_server_reactor_threads_(3),
+      keepalive_time_ms_(1000),
+      metric_entity_(METRIC_ENTITY_server.Instantiate(&metric_registry_, "test.rpc_test")) {
+  }
+
+  void TearDown() override {
+    if (service_pool_) {
+      server_messenger_->UnregisterService(service_name_);
+      service_pool_->Shutdown();
+    }
+    if (server_messenger_) {
+      server_messenger_->Shutdown();
+    }
+    KuduTest::TearDown();
+  }
+
+ protected:
+  Status CreateMessenger(const std::string& name,
+                         std::shared_ptr<Messenger>* messenger,
+                         int n_reactors = 1,
+                         bool enable_ssl = false,
+                         const std::string& rpc_certificate_file = "",
+                         const std::string& rpc_private_key_file = "",
+                         const std::string& rpc_ca_certificate_file = "",
+                         const std::string& rpc_private_key_password_cmd = "") {
+    MessengerBuilder bld(name);
+
+    if (enable_ssl) {
+      FLAGS_rpc_encrypt_loopback_connections = true;
+      bld.set_epki_cert_key_files(rpc_certificate_file, rpc_private_key_file);
+      bld.set_epki_certificate_authority_file(rpc_ca_certificate_file);
+      bld.set_epki_private_password_key_cmd(rpc_private_key_password_cmd);
+      bld.set_rpc_encryption("required");
+      bld.enable_inbound_tls();
+    }
+
+    bld.set_num_reactors(n_reactors);
+    bld.set_connection_keepalive_time(MonoDelta::FromMilliseconds(keepalive_time_ms_));
+    if (keepalive_time_ms_ >= 0) {
+      // In order for the keepalive timing to be accurate, we need to scan connections
+      // significantly more frequently than the keepalive time. This "coarse timer"
+      // granularity determines this.
+      bld.set_coarse_timer_granularity(
+          MonoDelta::FromMilliseconds(std::min(keepalive_time_ms_ / 5, 100)));
+    }
+    bld.set_metric_entity(metric_entity_);
+    return bld.Build(messenger);
+  }
+
+  Status DoTestSyncCall(const Proxy &p, const char *method,
+                        CredentialsPolicy policy = CredentialsPolicy::ANY_CREDENTIALS) {
+    AddRequestPB req;
+    req.set_x(rand());
+    req.set_y(rand());
+    AddResponsePB resp;
+    RpcController controller;
+    controller.set_timeout(MonoDelta::FromMilliseconds(10000));
+    controller.set_credentials_policy(policy);
+    RETURN_NOT_OK(p.SyncRequest(method, req, &resp, &controller));
+
+    CHECK_EQ(req.x() + req.y(), resp.result());
+    return Status::OK();
+  }
+
+  void DoTestSidecar(const Proxy &p, int size1, int size2) {
+    const uint32_t kSeed = 12345;
+
+    SendTwoStringsRequestPB req;
+    req.set_size1(size1);
+    req.set_size2(size2);
+    req.set_random_seed(kSeed);
+
+    SendTwoStringsResponsePB resp;
+    RpcController controller;
+    controller.set_timeout(MonoDelta::FromMilliseconds(10000));
+    CHECK_OK(p.SyncRequest(GenericCalculatorService::kSendTwoStringsMethodName,
+                           req, &resp, &controller));
+
+    Slice first = GetSidecarPointer(controller, resp.sidecar1(), size1);
+    Slice second = GetSidecarPointer(controller, resp.sidecar2(), size2);
+    Random rng(kSeed);
+    faststring expected;
+
+    expected.resize(size1);
+    RandomString(expected.data(), size1, &rng);
+    CHECK_EQ(0, first.compare(Slice(expected)));
+
+    expected.resize(size2);
+    RandomString(expected.data(), size2, &rng);
+    CHECK_EQ(0, second.compare(Slice(expected)));
+  }
+
+  Status DoTestOutgoingSidecar(const Proxy &p, int size1, int size2) {
+    PushTwoStringsRequestPB request;
+    RpcController controller;
+
+    int idx1;
+    std::string s1(size1, 'a');
+    CHECK_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(s1)), &idx1));
+
+    int idx2;
+    std::string s2(size2, 'b');
+    CHECK_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(s2)), &idx2));
+
+    request.set_sidecar1_idx(idx1);
+    request.set_sidecar2_idx(idx2);
+
+    PushTwoStringsResponsePB resp;
+    KUDU_RETURN_NOT_OK(p.SyncRequest(GenericCalculatorService::kPushTwoStringsMethodName,
+                                     request, &resp, &controller));
+    CHECK_EQ(size1, resp.size1());
+    CHECK_EQ(resp.data1(), s1);
+    CHECK_EQ(size2, resp.size2());
+    CHECK_EQ(resp.data2(), s2);
+    return Status::OK();
+  }
+
+  void DoTestOutgoingSidecarExpectOK(const Proxy &p, int size1, int size2) {
+    CHECK_OK(DoTestOutgoingSidecar(p, size1, size2));
+  }
+
+  void DoTestExpectTimeout(const Proxy& p,
+                           const MonoDelta& timeout,
+                           bool* is_negotiaton_error = nullptr) {
+    SleepRequestPB req;
+    SleepResponsePB resp;
+    // Sleep for 500ms longer than the call timeout.
+    int sleep_micros = timeout.ToMicroseconds() + 500 * 1000;
+    req.set_sleep_micros(sleep_micros);
+
+    RpcController c;
+    c.set_timeout(timeout);
+    Stopwatch sw;
+    sw.start();
+    Status s = p.SyncRequest(GenericCalculatorService::kSleepMethodName, req, &resp, &c);
+    sw.stop();
+    ASSERT_FALSE(s.ok());
+    if (is_negotiaton_error != nullptr) {
+      *is_negotiaton_error = c.negotiation_failed();
+    }
+
+    int expected_millis = timeout.ToMilliseconds();
+    int elapsed_millis = sw.elapsed().wall_millis();
+
+    // We shouldn't timeout significantly faster than our configured timeout.
+    EXPECT_GE(elapsed_millis, expected_millis - 10);
+    // And we also shouldn't take the full time that we asked for
+    EXPECT_LT(elapsed_millis * 1000, sleep_micros);
+    EXPECT_TRUE(s.IsTimedOut());
+    LOG(INFO) << "status: " << s.ToString() << ", seconds elapsed: " << sw.elapsed().wall_seconds();
+  }
+
+  Status StartTestServer(Sockaddr *server_addr,
+                         bool enable_ssl = false,
+                         const std::string& rpc_certificate_file = "",
+                         const std::string& rpc_private_key_file = "",
+                         const std::string& rpc_ca_certificate_file = "",
+                         const std::string& rpc_private_key_password_cmd = "") {
+    return DoStartTestServer<GenericCalculatorService>(
+        server_addr, enable_ssl, rpc_certificate_file, rpc_private_key_file,
+        rpc_ca_certificate_file, rpc_private_key_password_cmd);
+  }
+
+  Status StartTestServerWithGeneratedCode(Sockaddr *server_addr, bool enable_ssl = false) {
+    return DoStartTestServer<CalculatorService>(server_addr, enable_ssl);
+  }
+
+  Status StartTestServerWithCustomMessenger(Sockaddr *server_addr,
+      const std::shared_ptr<Messenger>& messenger, bool enable_ssl = false) {
+    return DoStartTestServer<GenericCalculatorService>(
+        server_addr, enable_ssl, "", "", "", "", messenger);
+  }
+
+  // Start a simple socket listening on a local port, returning the address.
+  // This isn't an RPC server -- just a plain socket which can be helpful for testing.
+  Status StartFakeServer(Socket *listen_sock, Sockaddr *listen_addr) {
+    Sockaddr bind_addr;
+    bind_addr.set_port(0);
+    RETURN_NOT_OK(listen_sock->Init(0));
+    RETURN_NOT_OK(listen_sock->BindAndListen(bind_addr, 1));
+    RETURN_NOT_OK(listen_sock->GetSocketAddress(listen_addr));
+    LOG(INFO) << "Bound to: " << listen_addr->ToString();
+    return Status::OK();
+  }
+
+ private:
+
+  static Slice GetSidecarPointer(const RpcController& controller, int idx,
+                                 int expected_size) {
+    Slice sidecar;
+    CHECK_OK(controller.GetInboundSidecar(idx, &sidecar));
+    CHECK_EQ(expected_size, sidecar.size());
+    return Slice(sidecar.data(), expected_size);
+  }
+
+  template<class ServiceClass>
+  Status DoStartTestServer(Sockaddr *server_addr,
+                           bool enable_ssl = false,
+                           const std::string& rpc_certificate_file = "",
+                           const std::string& rpc_private_key_file = "",
+                           const std::string& rpc_ca_certificate_file = "",
+                           const std::string& rpc_private_key_password_cmd = "",
+                           const std::shared_ptr<Messenger>& messenger = nullptr) {
+    if (!messenger) {
+      RETURN_NOT_OK(CreateMessenger(
+          "TestServer", &server_messenger_, n_server_reactor_threads_, enable_ssl,
+          rpc_certificate_file, rpc_private_key_file, rpc_ca_certificate_file,
+          rpc_private_key_password_cmd));
+    } else {
+      server_messenger_ = messenger;
+    }
+    std::shared_ptr<AcceptorPool> pool;
+    RETURN_NOT_OK(server_messenger_->AddAcceptorPool(Sockaddr(), &pool));
+    RETURN_NOT_OK(pool->Start(2));
+    *server_addr = pool->bind_address();
+    mem_tracker_ = MemTracker::CreateTracker(-1, "result_tracker");
+    result_tracker_.reset(new ResultTracker(mem_tracker_));
+
+    gscoped_ptr<ServiceIf> service(new ServiceClass(metric_entity_, result_tracker_));
+    service_name_ = service->service_name();
+    scoped_refptr<MetricEntity> metric_entity = server_messenger_->metric_entity();
+    service_pool_ = new ServicePool(std::move(service), metric_entity, service_queue_length_);
+    server_messenger_->RegisterService(service_name_, service_pool_);
+    RETURN_NOT_OK(service_pool_->Init(n_worker_threads_));
+
+    return Status::OK();
+  }
+
+ protected:
+  std::string service_name_;
+  std::shared_ptr<Messenger> server_messenger_;
+  scoped_refptr<ServicePool> service_pool_;
+  std::shared_ptr<kudu::MemTracker> mem_tracker_;
+  scoped_refptr<ResultTracker> result_tracker_;
+  int n_worker_threads_;
+  int service_queue_length_;
+  int n_server_reactor_threads_;
+  int keepalive_time_ms_;
+
+  MetricRegistry metric_registry_;
+  scoped_refptr<MetricEntity> metric_entity_;
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc-test.cc b/be/src/kudu/rpc/rpc-test.cc
new file mode 100644
index 0000000..077b5a3
--- /dev/null
+++ b/be/src/kudu/rpc/rpc-test.cc
@@ -0,0 +1,1364 @@
+// 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 "kudu/rpc/rpc-test-base.h"
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
+#include <unistd.h>
+#include <unordered_map>
+#include <vector>
+
+#include <boost/bind.hpp>
+#include <boost/core/ref.hpp>
+#include <boost/function.hpp>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/casts.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/proxy.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/random.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+METRIC_DECLARE_histogram(handler_latency_kudu_rpc_test_CalculatorService_Sleep);
+METRIC_DECLARE_histogram(rpc_incoming_queue_time);
+
+DECLARE_bool(rpc_reopen_outbound_connections);
+DECLARE_int32(rpc_negotiation_inject_delay_ms);
+
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::unordered_map;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+class TestRpc : public RpcTestBase, public ::testing::WithParamInterface<bool> {
+};
+
+// This is used to run all parameterized tests with and without SSL.
+INSTANTIATE_TEST_CASE_P(OptionalSSL, TestRpc, testing::Values(false, true));
+
+TEST_F(TestRpc, TestSockaddr) {
+  Sockaddr addr1, addr2;
+  addr1.set_port(1000);
+  addr2.set_port(2000);
+  // port is ignored when comparing Sockaddr objects
+  ASSERT_FALSE(addr1 < addr2);
+  ASSERT_FALSE(addr2 < addr1);
+  ASSERT_EQ(1000, addr1.port());
+  ASSERT_EQ(2000, addr2.port());
+  ASSERT_EQ(string("0.0.0.0:1000"), addr1.ToString());
+  ASSERT_EQ(string("0.0.0.0:2000"), addr2.ToString());
+  Sockaddr addr3(addr1);
+  ASSERT_EQ(string("0.0.0.0:1000"), addr3.ToString());
+}
+
+TEST_P(TestRpc, TestMessengerCreateDestroy) {
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(CreateMessenger("TestCreateDestroy", &messenger, 1, GetParam()));
+  LOG(INFO) << "started messenger " << messenger->name();
+  messenger->Shutdown();
+}
+
+// Test starting and stopping a messenger. This is a regression
+// test for a segfault seen in early versions of the RPC code,
+// in which shutting down the acceptor would trigger an assert,
+// making our tests flaky.
+TEST_P(TestRpc, TestAcceptorPoolStartStop) {
+  int n_iters = AllowSlowTests() ? 100 : 5;
+  for (int i = 0; i < n_iters; i++) {
+    shared_ptr<Messenger> messenger;
+    ASSERT_OK(CreateMessenger("TestAcceptorPoolStartStop", &messenger, 1, GetParam()));
+    shared_ptr<AcceptorPool> pool;
+    ASSERT_OK(messenger->AddAcceptorPool(Sockaddr(), &pool));
+    Sockaddr bound_addr;
+    ASSERT_OK(pool->GetBoundAddress(&bound_addr));
+    ASSERT_NE(0, bound_addr.port());
+    ASSERT_OK(pool->Start(2));
+    messenger->Shutdown();
+  }
+}
+
+TEST_F(TestRpc, TestConnHeaderValidation) {
+  MessengerBuilder mb("TestRpc.TestConnHeaderValidation");
+  const int conn_hdr_len = kMagicNumberLength + kHeaderFlagsLength;
+  uint8_t buf[conn_hdr_len];
+  serialization::SerializeConnHeader(buf);
+  ASSERT_OK(serialization::ValidateConnHeader(Slice(buf, conn_hdr_len)));
+}
+
+// Regression test for KUDU-2041
+TEST_P(TestRpc, TestNegotiationDeadlock) {
+  bool enable_ssl = GetParam();
+
+  // The deadlock would manifest in cases where the number of concurrent connection
+  // requests >= the number of threads. 1 thread and 1 cnxn to ourself is just the easiest
+  // way to reproduce the issue, because the server negotiation task must get queued after
+  // the client negotiation task if they share the same thread pool.
+  MessengerBuilder mb("TestRpc.TestNegotiationDeadlock");
+  mb.set_min_negotiation_threads(1)
+      .set_max_negotiation_threads(1)
+      .set_metric_entity(metric_entity_);
+  if (enable_ssl) mb.enable_inbound_tls();
+
+  shared_ptr<Messenger> messenger;
+  CHECK_OK(mb.Build(&messenger));
+
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServerWithCustomMessenger(&server_addr, messenger, enable_ssl));
+
+  Proxy p(messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+}
+
+// Test making successful RPC calls.
+TEST_P(TestRpc, TestCall) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_STR_CONTAINS(p.ToString(), strings::Substitute("kudu.rpc.GenericCalculatorService@"
+                                                            "{remote=$0, user_credentials=",
+                                                        server_addr.ToString()));
+
+  for (int i = 0; i < 10; i++) {
+    ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+  }
+}
+
+// Test for KUDU-2091 and KUDU-2220.
+TEST_P(TestRpc, TestCallWithChainCertAndChainCA) {
+  bool enable_ssl = GetParam();
+  // We're only interested in running this test with TLS enabled.
+  if (!enable_ssl) return;
+
+  string rpc_certificate_file;
+  string rpc_private_key_file;
+  string rpc_ca_certificate_file;
+  ASSERT_OK(security::CreateTestSSLCertSignedByChain(GetTestDataDirectory(),
+                                                     &rpc_certificate_file,
+                                                     &rpc_private_key_file,
+                                                     &rpc_ca_certificate_file));
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  SCOPED_TRACE(strings::Substitute("Connecting to $0", server_addr.ToString()));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl,
+      rpc_certificate_file, rpc_private_key_file, rpc_ca_certificate_file));
+
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_STR_CONTAINS(p.ToString(), strings::Substitute("kudu.rpc.GenericCalculatorService@"
+                                                            "{remote=$0, user_credentials=",
+                                                        server_addr.ToString()));
+
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+}
+
+// Test for KUDU-2041.
+TEST_P(TestRpc, TestCallWithChainCertAndRootCA) {
+  bool enable_ssl = GetParam();
+  // We're only interested in running this test with TLS enabled.
+  if (!enable_ssl) return;
+
+  string rpc_certificate_file;
+  string rpc_private_key_file;
+  string rpc_ca_certificate_file;
+  ASSERT_OK(security::CreateTestSSLCertWithChainSignedByRoot(GetTestDataDirectory(),
+                                                             &rpc_certificate_file,
+                                                             &rpc_private_key_file,
+                                                             &rpc_ca_certificate_file));
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  SCOPED_TRACE(strings::Substitute("Connecting to $0", server_addr.ToString()));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl,
+      rpc_certificate_file, rpc_private_key_file, rpc_ca_certificate_file));
+
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_STR_CONTAINS(p.ToString(), strings::Substitute("kudu.rpc.GenericCalculatorService@"
+                                                            "{remote=$0, user_credentials=",
+                                                        server_addr.ToString()));
+
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+}
+
+// Test making successful RPC calls while using a TLS certificate with a password protected
+// private key.
+TEST_P(TestRpc, TestCallWithPasswordProtectedKey) {
+  bool enable_ssl = GetParam();
+  // We're only interested in running this test with TLS enabled.
+  if (!enable_ssl) return;
+
+  string rpc_certificate_file;
+  string rpc_private_key_file;
+  string rpc_ca_certificate_file;
+  string rpc_private_key_password_cmd;
+  string passwd;
+  ASSERT_OK(security::CreateTestSSLCertWithEncryptedKey(GetTestDataDirectory(),
+                                                        &rpc_certificate_file,
+                                                        &rpc_private_key_file,
+                                                        &passwd));
+  rpc_ca_certificate_file = rpc_certificate_file;
+  rpc_private_key_password_cmd = strings::Substitute("echo $0", passwd);
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  SCOPED_TRACE(strings::Substitute("Connecting to $0", server_addr.ToString()));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl,
+      rpc_certificate_file, rpc_private_key_file, rpc_ca_certificate_file,
+      rpc_private_key_password_cmd));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_STR_CONTAINS(p.ToString(), strings::Substitute("kudu.rpc.GenericCalculatorService@"
+                                                            "{remote=$0, user_credentials=",
+                                                        server_addr.ToString()));
+
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+}
+
+// Test that using a TLS certificate with a password protected private key and providing
+// the wrong password for that private key, causes a server startup failure.
+TEST_P(TestRpc, TestCallWithBadPasswordProtectedKey) {
+  bool enable_ssl = GetParam();
+  // We're only interested in running this test with TLS enabled.
+  if (!enable_ssl) return;
+
+  string rpc_certificate_file;
+  string rpc_private_key_file;
+  string rpc_ca_certificate_file;
+  string rpc_private_key_password_cmd;
+  string passwd;
+  ASSERT_OK(security::CreateTestSSLCertWithEncryptedKey(GetTestDataDirectory(),
+                                                        &rpc_certificate_file,
+                                                        &rpc_private_key_file,
+                                                        &passwd));
+  // Overwrite the password with an invalid one.
+  passwd = "badpassword";
+  rpc_ca_certificate_file = rpc_certificate_file;
+  rpc_private_key_password_cmd = strings::Substitute("echo $0", passwd);
+  // Verify that the server fails to start up.
+  Sockaddr server_addr;
+  Status s = StartTestServer(&server_addr, enable_ssl, rpc_certificate_file, rpc_private_key_file,
+      rpc_ca_certificate_file, rpc_private_key_password_cmd);
+  ASSERT_TRUE(s.IsRuntimeError());
+  ASSERT_STR_CONTAINS(s.ToString(), "failed to load private key file");
+}
+
+// Test that connecting to an invalid server properly throws an error.
+TEST_P(TestRpc, TestCallToBadServer) {
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, GetParam()));
+  Sockaddr addr;
+  addr.set_port(0);
+  Proxy p(client_messenger, addr, addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Loop a few calls to make sure that we properly set up and tear down
+  // the connections.
+  for (int i = 0; i < 5; i++) {
+    Status s = DoTestSyncCall(p, GenericCalculatorService::kAddMethodName);
+    LOG(INFO) << "Status: " << s.ToString();
+    ASSERT_TRUE(s.IsNetworkError()) << "unexpected status: " << s.ToString();
+  }
+}
+
+// Test that RPC calls can be failed with an error status on the server.
+TEST_P(TestRpc, TestInvalidMethodCall) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Call the method which fails.
+  Status s = DoTestSyncCall(p, "ThisMethodDoesNotExist");
+  ASSERT_TRUE(s.IsRemoteError()) << "unexpected status: " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "bad method");
+}
+
+// Test that the error message returned when connecting to the wrong service
+// is reasonable.
+TEST_P(TestRpc, TestWrongService) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client with the wrong service name.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, "localhost", "WrongServiceName");
+
+  // Call the method which fails.
+  Status s = DoTestSyncCall(p, "ThisMethodDoesNotExist");
+  ASSERT_TRUE(s.IsRemoteError()) << "unexpected status: " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(),
+                      "Service unavailable: service WrongServiceName "
+                      "not registered on TestServer");
+}
+
+// Test that we can still make RPC connections even if many fds are in use.
+// This is a regression test for KUDU-650.
+TEST_P(TestRpc, TestHighFDs) {
+  // This test can only run if ulimit is set high.
+  const int kNumFakeFiles = 3500;
+  const int kMinUlimit = kNumFakeFiles + 100;
+  if (env_->GetResourceLimit(Env::ResourceLimitType::OPEN_FILES_PER_PROCESS) < kMinUlimit) {
+    LOG(INFO) << "Test skipped: must increase ulimit -n to at least " << kMinUlimit;
+    return;
+  }
+
+  // Open a bunch of fds just to increase our fd count.
+  vector<RandomAccessFile*> fake_files;
+  ElementDeleter d(&fake_files);
+  for (int i = 0; i < kNumFakeFiles; i++) {
+    unique_ptr<RandomAccessFile> f;
+    CHECK_OK(Env::Default()->NewRandomAccessFile("/dev/zero", &f));
+    fake_files.push_back(f.release());
+  }
+
+  // Set up server and client, and verify we can make a successful call.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+}
+
+// Test that connections are kept alive between calls.
+TEST_P(TestRpc, TestConnectionKeepalive) {
+  // Only run one reactor per messenger, so we can grab the metrics from that
+  // one without having to check all.
+  n_server_reactor_threads_ = 1;
+  keepalive_time_ms_ = 500;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+
+  SleepFor(MonoDelta::FromMilliseconds(5));
+
+  ReactorMetrics metrics;
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(1, metrics.num_server_connections_) << "Server should have 1 server connection";
+  ASSERT_EQ(0, metrics.num_client_connections_) << "Server should have 0 client connections";
+
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.num_server_connections_) << "Client should have 0 server connections";
+  ASSERT_EQ(1, metrics.num_client_connections_) << "Client should have 1 client connections";
+
+  SleepFor(MonoDelta::FromMilliseconds(2 * keepalive_time_ms_));
+
+  // After sleeping, the keepalive timer should have closed both sides of
+  // the connection.
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.num_server_connections_) << "Server should have 0 server connections";
+  ASSERT_EQ(0, metrics.num_client_connections_) << "Server should have 0 client connections";
+
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.num_server_connections_) << "Client should have 0 server connections";
+  ASSERT_EQ(0, metrics.num_client_connections_) << "Client should have 0 client connections";
+}
+
+// Test that idle connection is kept alive when 'keepalive_time_ms_' is set to -1.
+TEST_P(TestRpc, TestConnectionAlwaysKeepalive) {
+  // Only run one reactor per messenger, so we can grab the metrics from that
+  // one without having to check all.
+  n_server_reactor_threads_ = 1;
+  keepalive_time_ms_ = -1;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+
+  ReactorMetrics metrics;
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(1, metrics.num_server_connections_) << "Server should have 1 server connection";
+  ASSERT_EQ(0, metrics.num_client_connections_) << "Server should have 0 client connections";
+
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.num_server_connections_) << "Client should have 0 server connections";
+  ASSERT_EQ(1, metrics.num_client_connections_) << "Client should have 1 client connections";
+
+  SleepFor(MonoDelta::FromSeconds(3));
+
+  // After sleeping, the connection should still be alive.
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(1, metrics.num_server_connections_) << "Server should have 1 server connections";
+  ASSERT_EQ(0, metrics.num_client_connections_) << "Server should have 0 client connections";
+
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.num_server_connections_) << "Client should have 0 server connections";
+  ASSERT_EQ(1, metrics.num_client_connections_) << "Client should have 1 client connections";
+}
+
+// Test that the metrics on a per connection level work accurately.
+TEST_P(TestRpc, TestClientConnectionMetrics) {
+  // Only run one reactor per messenger, so we can grab the metrics from that
+  // one without having to check all.
+  n_server_reactor_threads_ = 1;
+  keepalive_time_ms_ = -1;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Cause the reactor thread to be blocked for 2 seconds.
+  server_messenger_->ScheduleOnReactor(boost::bind(sleep, 2), MonoDelta::FromSeconds(0));
+
+  RpcController controller;
+  DumpRunningRpcsRequestPB dump_req;
+  DumpRunningRpcsResponsePB dump_resp;
+  dump_req.set_include_traces(false);
+
+  // We'll send several calls asynchronously to force RPC queueing on the sender side.
+  int n_calls = 1000;
+  AddRequestPB add_req;
+  add_req.set_x(rand());
+  add_req.set_y(rand());
+  AddResponsePB add_resp;
+
+  vector<unique_ptr<RpcController>> controllers;
+  CountDownLatch latch(n_calls);
+  for (int i = 0; i < n_calls; i++) {
+    controllers.emplace_back(new RpcController());
+    p.AsyncRequest(GenericCalculatorService::kAddMethodName, add_req, &add_resp,
+        controllers.back().get(), boost::bind(&CountDownLatch::CountDown, boost::ref(latch)));
+  }
+
+  // Since we blocked the only reactor thread for sometime, we should see RPCs queued on the
+  // OutboundTransfer queue, unless the main thread is very slow.
+  ASSERT_OK(client_messenger->DumpRunningRpcs(dump_req, &dump_resp));
+  ASSERT_EQ(1, dump_resp.outbound_connections_size());
+  ASSERT_GT(dump_resp.outbound_connections(0).outbound_queue_size(), 0);
+
+  // Wait for the calls to be marked finished.
+  latch.Wait();
+
+  // Verify that all the RPCs have finished.
+  for (const auto& controller : controllers) {
+    ASSERT_TRUE(controller->finished());
+  }
+}
+
+// Test that outbound connections to the same server are reopen upon every RPC
+// call when the 'rpc_reopen_outbound_connections' flag is set.
+TEST_P(TestRpc, TestReopenOutboundConnections) {
+  // Set the flag to enable special mode: close and reopen already established
+  // outbound connections.
+  FLAGS_rpc_reopen_outbound_connections = true;
+
+  // Only run one reactor per messenger, so we can grab the metrics from that
+  // one without having to check all.
+  n_server_reactor_threads_ = 1;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Verify the initial counters.
+  ReactorMetrics metrics;
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.total_client_connections_);
+  ASSERT_EQ(0, metrics.total_server_connections_);
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.total_client_connections_);
+  ASSERT_EQ(0, metrics.total_server_connections_);
+
+  // Run several iterations, just in case.
+  for (int i = 0; i < 32; ++i) {
+    ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+    ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+    ASSERT_EQ(0, metrics.total_client_connections_);
+    ASSERT_EQ(i + 1, metrics.total_server_connections_);
+    ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+    ASSERT_EQ(i + 1, metrics.total_client_connections_);
+    ASSERT_EQ(0, metrics.total_server_connections_);
+  }
+}
+
+// Test that an outbound connection is closed and a new one is open if going
+// from ANY_CREDENTIALS to PRIMARY_CREDENTIALS policy for RPC calls to the same
+// destination.
+// Test that changing from PRIMARY_CREDENTIALS policy to ANY_CREDENTIALS policy
+// re-uses the connection established with PRIMARY_CREDENTIALS policy.
+TEST_P(TestRpc, TestCredentialsPolicy) {
+  // Only run one reactor per messenger, so we can grab the metrics from that
+  // one without having to check all.
+  n_server_reactor_threads_ = 1;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Verify the initial counters.
+  ReactorMetrics metrics;
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.total_client_connections_);
+  ASSERT_EQ(0, metrics.total_server_connections_);
+  ASSERT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  ASSERT_EQ(0, metrics.total_client_connections_);
+  ASSERT_EQ(0, metrics.total_server_connections_);
+
+  // Make an RPC call with ANY_CREDENTIALS policy.
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(0, metrics.total_client_connections_);
+  EXPECT_EQ(1, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_server_connections_);
+  EXPECT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(1, metrics.total_client_connections_);
+  EXPECT_EQ(0, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_client_connections_);
+
+  // This is to allow all the data to be sent so the connection becomes idle.
+  SleepFor(MonoDelta::FromMilliseconds(5));
+
+  // Make an RPC call with PRIMARY_CREDENTIALS policy. Currently open connection
+  // with ANY_CREDENTIALS policy should be closed and a new one established
+  // with PRIMARY_CREDENTIALS policy.
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName,
+                           CredentialsPolicy::PRIMARY_CREDENTIALS));
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(0, metrics.total_client_connections_);
+  EXPECT_EQ(2, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_server_connections_);
+  EXPECT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(2, metrics.total_client_connections_);
+  EXPECT_EQ(0, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_client_connections_);
+
+  // Make another RPC call with ANY_CREDENTIALS policy. The already established
+  // connection with PRIMARY_CREDENTIALS policy should be re-used because
+  // the ANY_CREDENTIALS policy satisfies the PRIMARY_CREDENTIALS policy which
+  // the currently open connection has been established with.
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+  ASSERT_OK(server_messenger_->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(0, metrics.total_client_connections_);
+  EXPECT_EQ(2, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_server_connections_);
+  EXPECT_OK(client_messenger->reactors_[0]->GetMetrics(&metrics));
+  EXPECT_EQ(2, metrics.total_client_connections_);
+  EXPECT_EQ(0, metrics.total_server_connections_);
+  EXPECT_EQ(1, metrics.num_client_connections_);
+}
+
+// Test that a call which takes longer than the keepalive time
+// succeeds -- i.e that we don't consider a connection to be "idle" on the
+// server if there is a call outstanding on it.
+TEST_P(TestRpc, TestCallLongerThanKeepalive) {
+  // Set a short keepalive.
+  keepalive_time_ms_ = 1000;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Make a call which sleeps longer than the keepalive.
+  RpcController controller;
+  SleepRequestPB req;
+  req.set_sleep_micros(3 * 1000 * 1000); // 3 seconds.
+  req.set_deferred(true);
+  SleepResponsePB resp;
+  ASSERT_OK(p.SyncRequest(GenericCalculatorService::kSleepMethodName,
+                                 req, &resp, &controller));
+}
+
+// Test that the RpcSidecar transfers the expected messages.
+TEST_P(TestRpc, TestRpcSidecar) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, GetParam()));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Test a zero-length sidecar
+  DoTestSidecar(p, 0, 0);
+
+  // Test some small sidecars
+  DoTestSidecar(p, 123, 456);
+
+  // Test some larger sidecars to verify that we properly handle the case where
+  // we can't write the whole response to the socket in a single call.
+  DoTestSidecar(p, 3000 * 1024, 2000 * 1024);
+
+  DoTestOutgoingSidecarExpectOK(p, 0, 0);
+  DoTestOutgoingSidecarExpectOK(p, 123, 456);
+  DoTestOutgoingSidecarExpectOK(p, 3000 * 1024, 2000 * 1024);
+}
+
+TEST_P(TestRpc, TestRpcSidecarLimits) {
+  {
+    // Test that the limits on the number of sidecars is respected.
+    RpcController controller;
+    string s = "foo";
+    int idx;
+    for (int i = 0; i < TransferLimits::kMaxSidecars; ++i) {
+      ASSERT_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(s)), &idx));
+    }
+
+    ASSERT_TRUE(controller.AddOutboundSidecar(
+        RpcSidecar::FromSlice(Slice(s)), &idx).IsRuntimeError());
+  }
+
+  // Construct a string to use as a maximal payload in following tests
+  string max_string(TransferLimits::kMaxTotalSidecarBytes, 'a');
+
+  {
+    // Test that limit on the total size of sidecars is respected. The maximal payload
+    // reaches the limit exactly.
+    RpcController controller;
+    int idx;
+    ASSERT_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(max_string)), &idx));
+
+    // Trying to add another byte will fail.
+    int dummy = 0;
+    string s2(1, 'b');
+    Status max_sidecar_status =
+        controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(s2)), &dummy);
+    ASSERT_FALSE(max_sidecar_status.ok());
+    ASSERT_STR_MATCHES(max_sidecar_status.ToString(), "Total size of sidecars");
+  }
+
+  // Test two cases:
+  // 1) The RPC has maximal size and exceeds rpc_max_message_size. This tests the
+  //    functionality of rpc_max_message_size. The server will close the connection
+  //    immediately.
+  // 2) The RPC has maximal size, but rpc_max_message_size has been set to a higher
+  //    value. This tests the client's ability to send the maximal message.
+  //    The server will reject the message after it has been transferred.
+  //    This test is disabled for TSAN due to high memory requirements.
+  std::vector<int64_t> rpc_max_message_values;
+  rpc_max_message_values.push_back(FLAGS_rpc_max_message_size);
+#ifndef THREAD_SANITIZER
+  rpc_max_message_values.push_back(std::numeric_limits<int64_t>::max());
+#endif
+  for (int64_t rpc_max_message_size_val : rpc_max_message_values) {
+    // Set rpc_max_message_size
+    FLAGS_rpc_max_message_size = rpc_max_message_size_val;
+
+    // Set up server.
+    Sockaddr server_addr;
+    bool enable_ssl = GetParam();
+    ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+    // Set up client.
+    shared_ptr<Messenger> client_messenger;
+    ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, GetParam()));
+    Proxy p(client_messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
+
+    RpcController controller;
+    // KUDU-2305: Test with a maximal payload to verify that the implementation
+    // can handle the limits.
+    int idx;
+    ASSERT_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(Slice(max_string)), &idx));
+
+    PushTwoStringsRequestPB request;
+    request.set_sidecar1_idx(idx);
+    request.set_sidecar2_idx(idx);
+    PushTwoStringsResponsePB resp;
+    Status status = p.SyncRequest(GenericCalculatorService::kPushTwoStringsMethodName,
+        request, &resp, &controller);
+    ASSERT_TRUE(status.IsNetworkError()) << "Unexpected error: " << status.ToString();
+    // Remote responds to extra-large payloads by closing the connection.
+    ASSERT_STR_MATCHES(status.ToString(),
+                       // Linux
+                       "Connection reset by peer"
+                       // While reading from socket.
+                       "|recv got EOF from"
+                       // Linux, SSL enabled
+                       "|failed to read from TLS socket"
+                       // macOS, while writing to socket.
+                       "|Protocol wrong type for socket"
+                       // macOS, sendmsg(): the sum of the iov_len values overflows an ssize_t
+                       "|sendmsg error: Invalid argument");
+  }
+}
+
+// Test that timeouts are properly handled.
+TEST_P(TestRpc, TestCallTimeout) {
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Test a very short timeout - we expect this will time out while the
+  // call is still trying to connect, or in the send queue. This was triggering ASAN failures
+  // before.
+  ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromNanoseconds(1)));
+
+  // Test a longer timeout - expect this will time out after we send the request,
+  // but shorter than our threshold for two-stage timeout handling.
+  ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(200)));
+
+  // Test a longer timeout - expect this will trigger the "two-stage timeout"
+  // code path.
+  ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(1500)));
+}
+
+// Inject 500ms delay in negotiation, and send a call with a short timeout, followed by
+// one with a long timeout. The call with the long timeout should succeed even though
+// the previous one failed.
+//
+// This is a regression test against prior behavior where the connection negotiation
+// was assigned the timeout of the first call on that connection. So, if the first
+// call had a short timeout, the later call would also inherit the timed-out negotiation.
+TEST_P(TestRpc, TestCallTimeoutDoesntAffectNegotiation) {
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  FLAGS_rpc_negotiation_inject_delay_ms = 500;
+  ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(50)));
+  ASSERT_OK(DoTestSyncCall(p, GenericCalculatorService::kAddMethodName));
+
+  // Only the second call should have been received by the server, because we
+  // don't bother sending an already-timed-out call.
+  auto metric_map = server_messenger_->metric_entity()->UnsafeMetricsMapForTests();
+  auto* metric = FindOrDie(metric_map, &METRIC_rpc_incoming_queue_time).get();
+  ASSERT_EQ(1, down_cast<Histogram*>(metric)->TotalCount());
+}
+
+static void AcceptAndReadForever(Socket* listen_sock) {
+  // Accept the TCP connection.
+  Socket server_sock;
+  Sockaddr remote;
+  CHECK_OK(listen_sock->Accept(&server_sock, &remote, 0));
+
+  MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(10);
+
+  size_t nread;
+  uint8_t buf[1024];
+  while (server_sock.BlockingRecv(buf, sizeof(buf), &nread, deadline).ok()) {
+  }
+}
+
+// Starts a fake listening socket which never actually negotiates.
+// Ensures that the client gets a reasonable status code in this case.
+TEST_F(TestRpc, TestNegotiationTimeout) {
+  // Set up a simple socket server which accepts a connection.
+  Sockaddr server_addr;
+  Socket listen_sock;
+  ASSERT_OK(StartFakeServer(&listen_sock, &server_addr));
+
+  // Create another thread to accept the connection on the fake server.
+  scoped_refptr<Thread> acceptor_thread;
+  ASSERT_OK(Thread::Create("test", "acceptor",
+                           AcceptAndReadForever, &listen_sock,
+                           &acceptor_thread));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  bool is_negotiation_error = false;
+  ASSERT_NO_FATAL_FAILURE(DoTestExpectTimeout(
+      p, MonoDelta::FromMilliseconds(100), &is_negotiation_error));
+  EXPECT_TRUE(is_negotiation_error);
+
+  acceptor_thread->Join();
+}
+
+// Test that client calls get failed properly when the server they're connected to
+// shuts down.
+TEST_F(TestRpc, TestServerShutsDown) {
+  // Set up a simple socket server which accepts a connection.
+  Sockaddr server_addr;
+  Socket listen_sock;
+  ASSERT_OK(StartFakeServer(&listen_sock, &server_addr));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Send a call.
+  AddRequestPB req;
+  req.set_x(rand());
+  req.set_y(rand());
+  AddResponsePB resp;
+
+  vector<unique_ptr<RpcController>> controllers;
+
+  // We'll send several calls async, and ensure that they all
+  // get the error status when the connection drops.
+  int n_calls = 5;
+
+  CountDownLatch latch(n_calls);
+  for (int i = 0; i < n_calls; i++) {
+    controllers.emplace_back(new RpcController());
+    p.AsyncRequest(GenericCalculatorService::kAddMethodName, req, &resp, controllers.back().get(),
+                   boost::bind(&CountDownLatch::CountDown, boost::ref(latch)));
+  }
+
+  // Accept the TCP connection.
+  Socket server_sock;
+  Sockaddr remote;
+  ASSERT_OK(listen_sock.Accept(&server_sock, &remote, 0));
+
+  // The call is still in progress at this point.
+  for (const auto& controller : controllers) {
+    ASSERT_FALSE(controller->finished());
+  }
+
+  // Shut down the socket.
+  ASSERT_OK(listen_sock.Close());
+  ASSERT_OK(server_sock.Close());
+
+  // Wait for the call to be marked finished.
+  latch.Wait();
+
+  // Should get the appropriate error on the client for all calls;
+  for (const auto& controller : controllers) {
+    ASSERT_TRUE(controller->finished());
+    Status s = controller->status();
+    ASSERT_TRUE(s.IsNetworkError()) <<
+      "Unexpected status: " << s.ToString();
+
+    // Any of these errors could happen, depending on whether we were
+    // in the middle of sending a call while the connection died, or
+    // if we were already waiting for responses.
+    //
+    // ECONNREFUSED is possible because the sending of the calls is async.
+    // For example, the following interleaving:
+    // - Enqueue 3 calls
+    // - Reactor wakes up, creates connection, starts writing calls
+    // - Enqueue 2 more calls
+    // - Shut down socket
+    // - Reactor wakes up, tries to write more of the first 3 calls, gets error
+    // - Reactor shuts down connection
+    // - Reactor sees the 2 remaining calls, makes a new connection
+    // - Because the socket is shut down, gets ECONNREFUSED.
+    //
+    // EINVAL is possible if the controller socket had already disconnected by
+    // the time it trys to set the SO_SNDTIMEO socket option as part of the
+    // normal blocking SASL handshake.
+    ASSERT_TRUE(s.posix_code() == EPIPE ||
+                s.posix_code() == ECONNRESET ||
+                s.posix_code() == ESHUTDOWN ||
+                s.posix_code() == ECONNREFUSED ||
+                s.posix_code() == EINVAL)
+      << "Unexpected status: " << s.ToString();
+  }
+}
+
+// Test handler latency metric.
+TEST_P(TestRpc, TestRpcHandlerLatencyMetric) {
+
+  const uint64_t sleep_micros = 20 * 1000;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
+
+  RpcController controller;
+  SleepRequestPB req;
+  req.set_sleep_micros(sleep_micros);
+  req.set_deferred(true);
+  SleepResponsePB resp;
+  ASSERT_OK(p.SyncRequest("Sleep", req, &resp, &controller));
+
+  const unordered_map<const MetricPrototype*, scoped_refptr<Metric> > metric_map =
+    server_messenger_->metric_entity()->UnsafeMetricsMapForTests();
+
+  scoped_refptr<Histogram> latency_histogram = down_cast<Histogram *>(
+      FindOrDie(metric_map,
+                &METRIC_handler_latency_kudu_rpc_test_CalculatorService_Sleep).get());
+
+  LOG(INFO) << "Sleep() min lat: " << latency_histogram->MinValueForTests();
+  LOG(INFO) << "Sleep() mean lat: " << latency_histogram->MeanValueForTests();
+  LOG(INFO) << "Sleep() max lat: " << latency_histogram->MaxValueForTests();
+  LOG(INFO) << "Sleep() #calls: " << latency_histogram->TotalCount();
+
+  ASSERT_EQ(1, latency_histogram->TotalCount());
+  ASSERT_GE(latency_histogram->MaxValueForTests(), sleep_micros);
+  ASSERT_TRUE(latency_histogram->MinValueForTests() == latency_histogram->MaxValueForTests());
+
+  // TODO: Implement an incoming queue latency test.
+  // For now we just assert that the metric exists.
+  ASSERT_TRUE(FindOrDie(metric_map, &METRIC_rpc_incoming_queue_time));
+}
+
+static void DestroyMessengerCallback(shared_ptr<Messenger>* messenger,
+                                     CountDownLatch* latch) {
+  messenger->reset();
+  latch->CountDown();
+}
+
+TEST_P(TestRpc, TestRpcCallbackDestroysMessenger) {
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, GetParam()));
+  Sockaddr bad_addr;
+  CountDownLatch latch(1);
+
+  AddRequestPB req;
+  req.set_x(rand());
+  req.set_y(rand());
+  AddResponsePB resp;
+  RpcController controller;
+  controller.set_timeout(MonoDelta::FromMilliseconds(1));
+  {
+    Proxy p(client_messenger, bad_addr, "xxx-host", "xxx-service");
+    p.AsyncRequest("my-fake-method", req, &resp, &controller,
+                   boost::bind(&DestroyMessengerCallback, &client_messenger, &latch));
+  }
+  latch.Wait();
+}
+
+// Test that setting the client timeout / deadline gets propagated to RPC
+// services.
+TEST_P(TestRpc, TestRpcContextClientDeadline) {
+  const uint64_t sleep_micros = 20 * 1000;
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
+
+  SleepRequestPB req;
+  req.set_sleep_micros(sleep_micros);
+  req.set_client_timeout_defined(true);
+  SleepResponsePB resp;
+  RpcController controller;
+  Status s = p.SyncRequest("Sleep", req, &resp, &controller);
+  ASSERT_TRUE(s.IsRemoteError());
+  ASSERT_STR_CONTAINS(s.ToString(), "Missing required timeout");
+
+  controller.Reset();
+  controller.set_timeout(MonoDelta::FromMilliseconds(1000));
+  ASSERT_OK(p.SyncRequest("Sleep", req, &resp, &controller));
+}
+
+// Test that setting an call-level application feature flag to an unknown value
+// will make the server reject the call.
+TEST_P(TestRpc, TestApplicationFeatureFlag) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
+
+  { // Supported flag
+    AddRequestPB req;
+    req.set_x(1);
+    req.set_y(2);
+    AddResponsePB resp;
+    RpcController controller;
+    controller.RequireServerFeature(FeatureFlags::FOO);
+    Status s = p.SyncRequest("Add", req, &resp, &controller);
+    SCOPED_TRACE(strings::Substitute("supported response: $0", s.ToString()));
+    ASSERT_TRUE(s.ok());
+    ASSERT_EQ(resp.result(), 3);
+  }
+
+  { // Unsupported flag
+    AddRequestPB req;
+    req.set_x(1);
+    req.set_y(2);
+    AddResponsePB resp;
+    RpcController controller;
+    controller.RequireServerFeature(FeatureFlags::FOO);
+    controller.RequireServerFeature(99);
+    Status s = p.SyncRequest("Add", req, &resp, &controller);
+    SCOPED_TRACE(strings::Substitute("unsupported response: $0", s.ToString()));
+    ASSERT_TRUE(s.IsRemoteError());
+  }
+}
+
+TEST_P(TestRpc, TestApplicationFeatureFlagUnsupportedServer) {
+  auto savedFlags = kSupportedServerRpcFeatureFlags;
+  auto cleanup = MakeScopedCleanup([&] () { kSupportedServerRpcFeatureFlags = savedFlags; });
+  kSupportedServerRpcFeatureFlags = {};
+
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr, enable_ssl));
+
+  // Set up client.
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          CalculatorService::static_service_name());
+
+  { // Required flag
+    AddRequestPB req;
+    req.set_x(1);
+    req.set_y(2);
+    AddResponsePB resp;
+    RpcController controller;
+    controller.RequireServerFeature(FeatureFlags::FOO);
+    Status s = p.SyncRequest("Add", req, &resp, &controller);
+    SCOPED_TRACE(strings::Substitute("supported response: $0", s.ToString()));
+    ASSERT_TRUE(s.IsNotSupported());
+  }
+
+  { // No required flag
+    AddRequestPB req;
+    req.set_x(1);
+    req.set_y(2);
+    AddResponsePB resp;
+    RpcController controller;
+    Status s = p.SyncRequest("Add", req, &resp, &controller);
+    SCOPED_TRACE(strings::Substitute("supported response: $0", s.ToString()));
+    ASSERT_TRUE(s.ok());
+  }
+}
+
+TEST_P(TestRpc, TestCancellation) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  for (int i = OutboundCall::READY; i <= OutboundCall::FINISHED_SUCCESS; ++i) {
+    FLAGS_rpc_inject_cancellation_state = i;
+    switch (i) {
+      case OutboundCall::READY:
+      case OutboundCall::ON_OUTBOUND_QUEUE:
+      case OutboundCall::SENDING:
+      case OutboundCall::SENT:
+        ASSERT_TRUE(DoTestOutgoingSidecar(p, 0, 0).IsAborted());
+        ASSERT_TRUE(DoTestOutgoingSidecar(p, 123, 456).IsAborted());
+        ASSERT_TRUE(DoTestOutgoingSidecar(p, 3000 * 1024, 2000 * 1024).IsAborted());
+        break;
+      case OutboundCall::NEGOTIATION_TIMED_OUT:
+      case OutboundCall::TIMED_OUT:
+        DoTestExpectTimeout(p, MonoDelta::FromMilliseconds(1000));
+        break;
+      case OutboundCall::CANCELLED:
+        break;
+      case OutboundCall::FINISHED_NEGOTIATION_ERROR:
+      case OutboundCall::FINISHED_ERROR: {
+        AddRequestPB req;
+        req.set_x(1);
+        req.set_y(2);
+        AddResponsePB resp;
+        RpcController controller;
+        controller.RequireServerFeature(FeatureFlags::FOO);
+        controller.RequireServerFeature(99);
+        Status s = p.SyncRequest("Add", req, &resp, &controller);
+        ASSERT_TRUE(s.IsRemoteError());
+        break;
+      }
+      case OutboundCall::FINISHED_SUCCESS:
+        DoTestOutgoingSidecarExpectOK(p, 0, 0);
+        DoTestOutgoingSidecarExpectOK(p, 123, 456);
+        DoTestOutgoingSidecarExpectOK(p, 3000 * 1024, 2000 * 1024);
+        break;
+    }
+  }
+  client_messenger->Shutdown();
+}
+
+#define TEST_PAYLOAD_SIZE  (1 << 23)
+#define TEST_SLEEP_TIME_MS (500)
+
+static void SleepCallback(uint8_t* payload, CountDownLatch* latch) {
+  // Overwrites the payload which the sidecar is pointing to. The server
+  // checks if the payload matches the expected pattern to detect cases
+  // in which the payload is overwritten while it's being sent.
+  memset(payload, 0, TEST_PAYLOAD_SIZE);
+  latch->CountDown();
+}
+
+// Test to verify that sidecars aren't corrupted when cancelling an async RPC.
+TEST_P(TestRpc, TestCancellationAsync) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  RpcController controller;
+
+  // The payload to be used during the RPC.
+  gscoped_array<uint8_t> payload(new uint8_t[TEST_PAYLOAD_SIZE]);
+
+  // Used to generate sleep time between invoking RPC and requesting cancellation.
+  Random rand(SeedRandom());
+
+  for (int i = 0; i < 10; ++i) {
+    SleepWithSidecarRequestPB req;
+    SleepWithSidecarResponsePB resp;
+
+    // Initialize the payload with non-zero pattern.
+    memset(payload.get(), 0xff, TEST_PAYLOAD_SIZE);
+    req.set_sleep_micros(TEST_SLEEP_TIME_MS);
+    req.set_pattern(0xffffffff);
+    req.set_num_repetitions(TEST_PAYLOAD_SIZE / sizeof(uint32_t));
+
+    int idx;
+    Slice s(payload.get(), TEST_PAYLOAD_SIZE);
+    CHECK_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(s), &idx));
+    req.set_sidecar_idx(idx);
+
+    CountDownLatch latch(1);
+    p.AsyncRequest(GenericCalculatorService::kSleepWithSidecarMethodName,
+                   req, &resp, &controller,
+                   boost::bind(SleepCallback, payload.get(), &latch));
+    // Sleep for a while before cancelling the RPC.
+    if (i > 0) SleepFor(MonoDelta::FromMicroseconds(rand.Uniform64(i * 30)));
+    controller.Cancel();
+    latch.Wait();
+    ASSERT_TRUE(controller.status().IsAborted() || controller.status().ok());
+    controller.Reset();
+  }
+  client_messenger->Shutdown();
+}
+
+// This function loops for 40 iterations and for each iteration, sends an async RPC
+// and sleeps for some time between 1 to 100 microseconds before cancelling the RPC.
+// This serves as a helper function for TestCancellationMultiThreads() to exercise
+// cancellation when there are concurrent RPCs.
+static void SendAndCancelRpcs(Proxy* p, const Slice& slice) {
+  RpcController controller;
+
+  // Used to generate sleep time between invoking RPC and requesting cancellation.
+  Random rand(SeedRandom());
+
+  auto end_time = MonoTime::Now() + MonoDelta::FromSeconds(
+    AllowSlowTests() ? 15 : 3);
+
+  int i = 0;
+  while (MonoTime::Now() < end_time) {
+    controller.Reset();
+    PushTwoStringsRequestPB request;
+    PushTwoStringsResponsePB resp;
+    int idx;
+    CHECK_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(slice), &idx));
+    request.set_sidecar1_idx(idx);
+    CHECK_OK(controller.AddOutboundSidecar(RpcSidecar::FromSlice(slice), &idx));
+    request.set_sidecar2_idx(idx);
+
+    CountDownLatch latch(1);
+    p->AsyncRequest(GenericCalculatorService::kPushTwoStringsMethodName,
+                    request, &resp, &controller,
+                    boost::bind(&CountDownLatch::CountDown, boost::ref(latch)));
+
+    if ((i++ % 8) != 0) {
+      // Sleep for a while before cancelling the RPC.
+      SleepFor(MonoDelta::FromMicroseconds(rand.Uniform64(100)));
+      controller.Cancel();
+    }
+    latch.Wait();
+    CHECK(controller.status().IsAborted() || controller.status().IsServiceUnavailable() ||
+          controller.status().ok()) << controller.status().ToString();
+  }
+}
+
+// Test to exercise cancellation when there are multiple concurrent RPCs from the
+// same client to the same server.
+TEST_P(TestRpc, TestCancellationMultiThreads) {
+  // Set up server.
+  Sockaddr server_addr;
+  bool enable_ssl = GetParam();
+  ASSERT_OK(StartTestServer(&server_addr, enable_ssl));
+
+  // Set up client.
+  LOG(INFO) << "Connecting to " << server_addr.ToString();
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger, 1, enable_ssl));
+  Proxy p(client_messenger, server_addr, server_addr.host(),
+          GenericCalculatorService::static_service_name());
+
+  // Buffer used for sidecars by SendAndCancelRpcs().
+  string buf(16 * 1024 * 1024, 'a');
+  Slice slice(buf);
+
+  // Start a bunch of threads which invoke async RPC and cancellation.
+  std::vector<scoped_refptr<Thread>> threads;
+  for (int i = 0; i < 30; ++i) {
+    scoped_refptr<Thread> rpc_thread;
+    ASSERT_OK(Thread::Create("test", "rpc", SendAndCancelRpcs, &p, slice, &rpc_thread));
+    threads.push_back(rpc_thread);
+  }
+  // Wait for all threads to complete.
+  for (scoped_refptr<Thread>& rpc_thread : threads) {
+    rpc_thread->Join();
+  }
+  client_messenger->Shutdown();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc.cc b/be/src/kudu/rpc/rpc.cc
new file mode 100644
index 0000000..84ea892
--- /dev/null
+++ b/be/src/kudu/rpc/rpc.cc
@@ -0,0 +1,101 @@
+// 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 "kudu/rpc/rpc.h"
+
+#include <cstdlib>
+#include <string>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/function.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc_header.pb.h"
+
+using std::shared_ptr;
+using std::string;
+using strings::Substitute;
+using strings::SubstituteAndAppend;
+
+namespace kudu {
+
+namespace rpc {
+
+bool RpcRetrier::HandleResponse(Rpc* rpc, Status* out_status) {
+  DCHECK(rpc);
+  DCHECK(out_status);
+
+  // Always retry TOO_BUSY and UNAVAILABLE errors.
+  const Status controller_status = controller_.status();
+  if (controller_status.IsRemoteError()) {
+    const ErrorStatusPB* err = controller_.error_response();
+    if (err &&
+        err->has_code() &&
+        (err->code() == ErrorStatusPB::ERROR_SERVER_TOO_BUSY ||
+         err->code() == ErrorStatusPB::ERROR_UNAVAILABLE)) {
+      // The UNAVAILABLE code is a broader counterpart of the
+      // SERVER_TOO_BUSY. In both cases it's necessary to retry a bit later.
+      DelayedRetry(rpc, controller_status);
+      return true;
+    }
+  }
+
+  *out_status = controller_status;
+  return false;
+}
+
+void RpcRetrier::DelayedRetry(Rpc* rpc, const Status& why_status) {
+  if (!why_status.ok() && (last_error_.ok() || last_error_.IsTimedOut())) {
+    last_error_ = why_status;
+  }
+  // Add some jitter to the retry delay.
+  //
+  // If the delay causes us to miss our deadline, RetryCb will fail the
+  // RPC on our behalf.
+  int num_ms = ++attempt_num_ + ((rand() % 5));
+  messenger_->ScheduleOnReactor(boost::bind(&RpcRetrier::DelayedRetryCb,
+                                            this,
+                                            rpc, _1),
+                                MonoDelta::FromMilliseconds(num_ms));
+}
+
+void RpcRetrier::DelayedRetryCb(Rpc* rpc, const Status& status) {
+  Status new_status = status;
+  if (new_status.ok()) {
+    // Has this RPC timed out?
+    if (deadline_.Initialized()) {
+      if (MonoTime::Now() > deadline_) {
+        string err_str = Substitute("$0 passed its deadline", rpc->ToString());
+        if (!last_error_.ok()) {
+          SubstituteAndAppend(&err_str, ": $0", last_error_.ToString());
+        }
+        new_status = Status::TimedOut(err_str);
+      }
+    }
+  }
+  if (new_status.ok()) {
+    controller_.Reset();
+    rpc->SendRpc();
+  } else {
+    rpc->SendRpcCb(new_status);
+  }
+}
+
+} // namespace rpc
+} // namespace kudu


[26/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/leakcheck_disabler.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/leakcheck_disabler.h b/be/src/kudu/util/debug/leakcheck_disabler.h
new file mode 100644
index 0000000..815f818
--- /dev/null
+++ b/be/src/kudu/util/debug/leakcheck_disabler.h
@@ -0,0 +1,48 @@
+// 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.
+#ifndef KUDU_UTIL_DEBUG_LEAKCHECK_DISABLER_H_
+#define KUDU_UTIL_DEBUG_LEAKCHECK_DISABLER_H_
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/debug/leak_annotations.h"
+
+namespace kudu {
+namespace debug {
+
+// Scoped object that generically disables LSAN leak checking in a given scope.
+// While this object is alive, calls to "new" will not be checked for leaks.
+class ScopedLeakCheckDisabler {
+ public:
+  ScopedLeakCheckDisabler() {}
+
+ private:
+
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+#    if defined(__linux__)
+  ScopedLSANDisabler lsan_disabler;
+#    endif
+#  endif
+#endif
+
+  DISALLOW_COPY_AND_ASSIGN(ScopedLeakCheckDisabler);
+};
+
+} // namespace debug
+} // namespace kudu
+
+#endif // KUDU_UTIL_DEBUG_LEAKCHECK_DISABLER_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/sanitizer_scopes.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/sanitizer_scopes.h b/be/src/kudu/util/debug/sanitizer_scopes.h
new file mode 100644
index 0000000..2f8a557
--- /dev/null
+++ b/be/src/kudu/util/debug/sanitizer_scopes.h
@@ -0,0 +1,47 @@
+// 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.
+//
+// Wrappers around the annotations from gutil/dynamic_annotations.h,
+// provided as C++-style scope guards.
+#ifndef KUDU_UTIL_DEBUG_SANITIZER_SCOPES_H_
+#define KUDU_UTIL_DEBUG_SANITIZER_SCOPES_H_
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+namespace debug {
+
+// Scope guard which instructs TSAN to ignore all reads and writes
+// on the current thread as long as it is alive. These may be safely
+// nested.
+class ScopedTSANIgnoreReadsAndWrites {
+ public:
+  ScopedTSANIgnoreReadsAndWrites() {
+    ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN();
+  }
+  ~ScopedTSANIgnoreReadsAndWrites() {
+    ANNOTATE_IGNORE_READS_AND_WRITES_END();
+  }
+ private:
+  DISALLOW_COPY_AND_ASSIGN(ScopedTSANIgnoreReadsAndWrites);
+};
+
+} // namespace debug
+} // namespace kudu
+
+#endif  // KUDU_UTIL_DEBUG_SANITIZER_SCOPES_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event.h b/be/src/kudu/util/debug/trace_event.h
new file mode 100644
index 0000000..2c51309
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event.h
@@ -0,0 +1,1501 @@
+// Copyright (c) 2012 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+// This header file defines the set of trace_event macros without specifying
+// how the events actually get collected and stored. If you need to expose trace
+// events to some other universe, you can copy-and-paste this file as well as
+// trace_event.h, modifying the macros contained there as necessary for the
+// target platform. The end result is that multiple libraries can funnel events
+// through to a shared trace event collector.
+
+// Trace events are for tracking application performance and resource usage.
+// Macros are provided to track:
+//    Begin and end of function calls
+//    Counters
+//
+// Events are issued against categories. Whereas LOG's
+// categories are statically defined, TRACE categories are created
+// implicitly with a string. For example:
+//   TRACE_EVENT_INSTANT0("MY_SUBSYSTEM", "SomeImportantEvent",
+//                        TRACE_EVENT_SCOPE_THREAD)
+//
+// It is often the case that one trace may belong in multiple categories at the
+// same time. The first argument to the trace can be a comma-separated list of
+// categories, forming a category group, like:
+//
+// TRACE_EVENT_INSTANT0("input,views", "OnMouseOver", TRACE_EVENT_SCOPE_THREAD)
+//
+// We can enable/disable tracing of OnMouseOver by enabling/disabling either
+// category.
+//
+// Events can be INSTANT, or can be pairs of BEGIN and END in the same scope:
+//   TRACE_EVENT_BEGIN0("MY_SUBSYSTEM", "SomethingCostly")
+//   doSomethingCostly()
+//   TRACE_EVENT_END0("MY_SUBSYSTEM", "SomethingCostly")
+// Note: our tools can't always determine the correct BEGIN/END pairs unless
+// these are used in the same scope. Use ASYNC_BEGIN/ASYNC_END macros if you
+// need them to be in separate scopes.
+//
+// A common use case is to trace entire function scopes. This
+// issues a trace BEGIN and END automatically:
+//   void doSomethingCostly() {
+//     TRACE_EVENT0("MY_SUBSYSTEM", "doSomethingCostly");
+//     ...
+//   }
+//
+// Additional parameters can be associated with an event:
+//   void doSomethingCostly2(int howMuch) {
+//     TRACE_EVENT1("MY_SUBSYSTEM", "doSomethingCostly",
+//         "howMuch", howMuch);
+//     ...
+//   }
+//
+// The trace system will automatically add to this information the
+// current process id, thread id, and a timestamp in microseconds.
+//
+// To trace an asynchronous procedure such as an IPC send/receive, use
+// ASYNC_BEGIN and ASYNC_END:
+//   [single threaded sender code]
+//     static int send_count = 0;
+//     ++send_count;
+//     TRACE_EVENT_ASYNC_BEGIN0("ipc", "message", send_count);
+//     Send(new MyMessage(send_count));
+//   [receive code]
+//     void OnMyMessage(send_count) {
+//       TRACE_EVENT_ASYNC_END0("ipc", "message", send_count);
+//     }
+// The third parameter is a unique ID to match ASYNC_BEGIN/ASYNC_END pairs.
+// ASYNC_BEGIN and ASYNC_END can occur on any thread of any traced process.
+// Pointers can be used for the ID parameter, and they will be mangled
+// internally so that the same pointer on two different processes will not
+// match. For example:
+//   class MyTracedClass {
+//    public:
+//     MyTracedClass() {
+//       TRACE_EVENT_ASYNC_BEGIN0("category", "MyTracedClass", this);
+//     }
+//     ~MyTracedClass() {
+//       TRACE_EVENT_ASYNC_END0("category", "MyTracedClass", this);
+//     }
+//   }
+//
+// Trace event also supports counters, which is a way to track a quantity
+// as it varies over time. Counters are created with the following macro:
+//   TRACE_COUNTER1("MY_SUBSYSTEM", "myCounter", g_myCounterValue);
+//
+// Counters are process-specific. The macro itself can be issued from any
+// thread, however.
+//
+// Sometimes, you want to track two counters at once. You can do this with two
+// counter macros:
+//   TRACE_COUNTER1("MY_SUBSYSTEM", "myCounter0", g_myCounterValue[0]);
+//   TRACE_COUNTER1("MY_SUBSYSTEM", "myCounter1", g_myCounterValue[1]);
+// Or you can do it with a combined macro:
+//   TRACE_COUNTER2("MY_SUBSYSTEM", "myCounter",
+//       "bytesPinned", g_myCounterValue[0],
+//       "bytesAllocated", g_myCounterValue[1]);
+// This indicates to the tracing UI that these counters should be displayed
+// in a single graph, as a summed area chart.
+//
+// Since counters are in a global namespace, you may want to disambiguate with a
+// unique ID, by using the TRACE_COUNTER_ID* variations.
+//
+// By default, trace collection is compiled in, but turned off at runtime.
+// Collecting trace data is the responsibility of the embedding
+// application. In Chrome's case, navigating to about:tracing will turn on
+// tracing and display data collected across all active processes.
+//
+//
+// Memory scoping note:
+// Tracing copies the pointers, not the string content, of the strings passed
+// in for category_group, name, and arg_names.  Thus, the following code will
+// cause problems:
+//     char* str = strdup("importantName");
+//     TRACE_EVENT_INSTANT0("SUBSYSTEM", str);  // BAD!
+//     free(str);                   // Trace system now has dangling pointer
+//
+// To avoid this issue with the |name| and |arg_name| parameters, use the
+// TRACE_EVENT_COPY_XXX overloads of the macros at additional runtime overhead.
+// Notes: The category must always be in a long-lived char* (i.e. static const).
+//        The |arg_values|, when used, are always deep copied with the _COPY
+//        macros.
+//
+// When are string argument values copied:
+// const char* arg_values are only referenced by default:
+//     TRACE_EVENT1("category", "name",
+//                  "arg1", "literal string is only referenced");
+// Use TRACE_STR_COPY to force copying of a const char*:
+//     TRACE_EVENT1("category", "name",
+//                  "arg1", TRACE_STR_COPY("string will be copied"));
+// std::string arg_values are always copied:
+//     TRACE_EVENT1("category", "name",
+//                  "arg1", std::string("string will be copied"));
+//
+//
+// Convertable notes:
+// Converting a large data type to a string can be costly. To help with this,
+// the trace framework provides an interface ConvertableToTraceFormat. If you
+// inherit from it and implement the AppendAsTraceFormat method the trace
+// framework will call back to your object to convert a trace output time. This
+// means, if the category for the event is disabled, the conversion will not
+// happen.
+//
+//   class MyData : public kudu::debug::ConvertableToTraceFormat {
+//    public:
+//     MyData() {}
+//     virtual void AppendAsTraceFormat(std::string* out) const OVERRIDE {
+//       out->append("{\"foo\":1}");
+//     }
+//    private:
+//     virtual ~MyData() {}
+//     DISALLOW_COPY_AND_ASSIGN(MyData);
+//   };
+//
+//   TRACE_EVENT1("foo", "bar", "data",
+//                scoped_refptr<ConvertableToTraceFormat>(new MyData()));
+//
+// The trace framework will take ownership if the passed pointer and it will
+// be free'd when the trace buffer is flushed.
+//
+// Note, we only do the conversion when the buffer is flushed, so the provided
+// data object should not be modified after it's passed to the trace framework.
+//
+//
+// Thread Safety:
+// A thread safe singleton and mutex are used for thread safety. Category
+// enabled flags are used to limit the performance impact when the system
+// is not enabled.
+//
+// TRACE_EVENT macros first cache a pointer to a category. The categories are
+// statically allocated and safe at all times, even after exit. Fetching a
+// category is protected by the TraceLog::lock_. Multiple threads initializing
+// the static variable is safe, as they will be serialized by the lock and
+// multiple calls will return the same pointer to the category.
+//
+// Then the category_group_enabled flag is checked. This is a unsigned char, and
+// not intended to be multithread safe. It optimizes access to AddTraceEvent
+// which is threadsafe internally via TraceLog::lock_. The enabled flag may
+// cause some threads to incorrectly call or skip calling AddTraceEvent near
+// the time of the system being enabled or disabled. This is acceptable as
+// we tolerate some data loss while the system is being enabled/disabled and
+// because AddTraceEvent is threadsafe internally and checks the enabled state
+// again under lock.
+//
+// Without the use of these static category pointers and enabled flags all
+// trace points would carry a significant performance cost of acquiring a lock
+// and resolving the category.
+
+#ifndef KUDU_UTIL_DEBUG_TRACE_EVENT_H_
+#define KUDU_UTIL_DEBUG_TRACE_EVENT_H_
+
+#include <string>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/util/debug/trace_event_impl.h"
+#include "kudu/util/debug/trace_event_memory.h"
+#include "kudu/util/thread.h"
+#include "kudu/gutil/walltime.h"
+
+// By default, const char* argument values are assumed to have long-lived scope
+// and will not be copied. Use this macro to force a const char* to be copied.
+#define TRACE_STR_COPY(str) \
+    trace_event_internal::TraceStringWithCopy(str)
+
+// This will mark the trace event as disabled by default. The user will need
+// to explicitly enable the event.
+#define TRACE_DISABLED_BY_DEFAULT(name) "disabled-by-default-" name
+
+// By default, uint64 ID argument values are not mangled with the Process ID in
+// TRACE_EVENT_ASYNC macros. Use this macro to force Process ID mangling.
+#define TRACE_ID_MANGLE(id) \
+    trace_event_internal::TraceID::ForceMangle(id)
+
+// By default, pointers are mangled with the Process ID in TRACE_EVENT_ASYNC
+// macros. Use this macro to prevent Process ID mangling.
+#define TRACE_ID_DONT_MANGLE(id) \
+    trace_event_internal::TraceID::DontMangle(id)
+
+// Records a pair of begin and end events called "name" for the current
+// scope, with 0, 1 or 2 associated arguments. If the category is not
+// enabled, then this does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_EVENT0(category_group, name) \
+    INTERNAL_TRACE_MEMORY(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD_SCOPED(category_group, name)
+#define TRACE_EVENT1(category_group, name, arg1_name, arg1_val) \
+    INTERNAL_TRACE_MEMORY(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD_SCOPED(category_group, name, arg1_name, arg1_val)
+#define TRACE_EVENT2( \
+    category_group, name, arg1_name, arg1_val, arg2_name, arg2_val) \
+  INTERNAL_TRACE_MEMORY(category_group, name) \
+  INTERNAL_TRACE_EVENT_ADD_SCOPED( \
+      category_group, name, arg1_name, arg1_val, arg2_name, arg2_val)
+
+// Records events like TRACE_EVENT2 but uses |memory_tag| for memory tracing.
+// Use this where |name| is too generic to accurately aggregate allocations.
+#define TRACE_EVENT_WITH_MEMORY_TAG2( \
+    category, name, memory_tag, arg1_name, arg1_val, arg2_name, arg2_val) \
+  INTERNAL_TRACE_MEMORY(category, memory_tag) \
+  INTERNAL_TRACE_EVENT_ADD_SCOPED( \
+      category, name, arg1_name, arg1_val, arg2_name, arg2_val)
+
+// UNSHIPPED_TRACE_EVENT* are like TRACE_EVENT* except that they are not
+// included in official builds.
+
+#if OFFICIAL_BUILD
+#undef TRACING_IS_OFFICIAL_BUILD
+#define TRACING_IS_OFFICIAL_BUILD 1
+#elif !defined(TRACING_IS_OFFICIAL_BUILD)
+#define TRACING_IS_OFFICIAL_BUILD 0
+#endif
+
+#if TRACING_IS_OFFICIAL_BUILD
+#define UNSHIPPED_TRACE_EVENT0(category_group, name) (void)0
+#define UNSHIPPED_TRACE_EVENT1(category_group, name, arg1_name, arg1_val) \
+    (void)0
+#define UNSHIPPED_TRACE_EVENT2(category_group, name, arg1_name, arg1_val, \
+                               arg2_name, arg2_val) (void)0
+#define UNSHIPPED_TRACE_EVENT_INSTANT0(category_group, name, scope) (void)0
+#define UNSHIPPED_TRACE_EVENT_INSTANT1(category_group, name, scope, \
+                                       arg1_name, arg1_val) (void)0
+#define UNSHIPPED_TRACE_EVENT_INSTANT2(category_group, name, scope, \
+                                       arg1_name, arg1_val, \
+                                       arg2_name, arg2_val) (void)0
+#else
+#define UNSHIPPED_TRACE_EVENT0(category_group, name) \
+    TRACE_EVENT0(category_group, name)
+#define UNSHIPPED_TRACE_EVENT1(category_group, name, arg1_name, arg1_val) \
+    TRACE_EVENT1(category_group, name, arg1_name, arg1_val)
+#define UNSHIPPED_TRACE_EVENT2(category_group, name, arg1_name, arg1_val, \
+                               arg2_name, arg2_val) \
+    TRACE_EVENT2(category_group, name, arg1_name, arg1_val, arg2_name, arg2_val)
+#define UNSHIPPED_TRACE_EVENT_INSTANT0(category_group, name, scope) \
+    TRACE_EVENT_INSTANT0(category_group, name, scope)
+#define UNSHIPPED_TRACE_EVENT_INSTANT1(category_group, name, scope, \
+                                       arg1_name, arg1_val) \
+    TRACE_EVENT_INSTANT1(category_group, name, scope, arg1_name, arg1_val)
+#define UNSHIPPED_TRACE_EVENT_INSTANT2(category_group, name, scope, \
+                                       arg1_name, arg1_val, \
+                                       arg2_name, arg2_val) \
+    TRACE_EVENT_INSTANT2(category_group, name, scope, arg1_name, arg1_val, \
+                         arg2_name, arg2_val)
+#endif
+
+// Records a single event called "name" immediately, with 0, 1 or 2
+// associated arguments. If the category is not enabled, then this
+// does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_EVENT_INSTANT0(category_group, name, scope) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_NONE | scope)
+#define TRACE_EVENT_INSTANT1(category_group, name, scope, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_NONE | scope, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_INSTANT2(category_group, name, scope, arg1_name, arg1_val, \
+                             arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_NONE | scope, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_INSTANT0(category_group, name, scope) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_COPY | scope)
+#define TRACE_EVENT_COPY_INSTANT1(category_group, name, scope, \
+                                  arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_COPY | scope, arg1_name, \
+        arg1_val)
+#define TRACE_EVENT_COPY_INSTANT2(category_group, name, scope, \
+                                  arg1_name, arg1_val, \
+                                  arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_INSTANT, \
+        category_group, name, TRACE_EVENT_FLAG_COPY | scope, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+
+// Sets the current sample state to the given category and name (both must be
+// constant strings). These states are intended for a sampling profiler.
+// Implementation note: we store category and name together because we don't
+// want the inconsistency/expense of storing two pointers.
+// |thread_bucket| is [0..2] and is used to statically isolate samples in one
+// thread from others.
+#define TRACE_EVENT_SET_SAMPLING_STATE_FOR_BUCKET( \
+    bucket_number, category, name)                 \
+        trace_event_internal::                     \
+        TraceEventSamplingStateScope<bucket_number>::Set(category "\0" name)
+
+// Returns a current sampling state of the given bucket.
+#define TRACE_EVENT_GET_SAMPLING_STATE_FOR_BUCKET(bucket_number) \
+    trace_event_internal::TraceEventSamplingStateScope<bucket_number>::Current()
+
+// Creates a scope of a sampling state of the given bucket.
+//
+// {  // The sampling state is set within this scope.
+//    TRACE_EVENT_SAMPLING_STATE_SCOPE_FOR_BUCKET(0, "category", "name");
+//    ...;
+// }
+#define TRACE_EVENT_SCOPED_SAMPLING_STATE_FOR_BUCKET(                   \
+    bucket_number, category, name)                                      \
+    trace_event_internal::TraceEventSamplingStateScope<bucket_number>   \
+        traceEventSamplingScope(category "\0" name);
+
+// Syntactic sugars for the sampling tracing in the main thread.
+#define TRACE_EVENT_SCOPED_SAMPLING_STATE(category, name) \
+    TRACE_EVENT_SCOPED_SAMPLING_STATE_FOR_BUCKET(0, category, name)
+#define TRACE_EVENT_GET_SAMPLING_STATE() \
+    TRACE_EVENT_GET_SAMPLING_STATE_FOR_BUCKET(0)
+#define TRACE_EVENT_SET_SAMPLING_STATE(category, name) \
+    TRACE_EVENT_SET_SAMPLING_STATE_FOR_BUCKET(0, category, name)
+
+
+// Records a single BEGIN event called "name" immediately, with 0, 1 or 2
+// associated arguments. If the category is not enabled, then this
+// does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_EVENT_BEGIN0(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_BEGIN1(category_group, name, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_BEGIN2(category_group, name, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val, \
+        arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_BEGIN0(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_BEGIN1(category_group, name, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_BEGIN2(category_group, name, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_BEGIN, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, arg1_name, arg1_val, \
+        arg2_name, arg2_val)
+
+// Similar to TRACE_EVENT_BEGINx but with a custom |at| timestamp provided.
+// - |id| is used to match the _BEGIN event with the _END event.
+//   Events are considered to match if their category_group, name and id values
+//   all match. |id| must either be a pointer or an integer value up to 64 bits.
+//   If it's a pointer, the bits will be xored with a hash of the process ID so
+//   that the same pointer on two different processes will not collide.
+#define TRACE_EVENT_BEGIN_WITH_ID_TID_AND_TIMESTAMP0(category_group, \
+        name, id, thread_id, timestamp) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID_TID_AND_TIMESTAMP( \
+        TRACE_EVENT_PHASE_ASYNC_BEGIN, category_group, name, id, thread_id, \
+        timestamp, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_COPY_BEGIN_WITH_ID_TID_AND_TIMESTAMP0( \
+        category_group, name, id, thread_id, timestamp) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID_TID_AND_TIMESTAMP( \
+        TRACE_EVENT_PHASE_ASYNC_BEGIN, category_group, name, id, thread_id, \
+        timestamp, TRACE_EVENT_FLAG_COPY)
+
+// Records a single END event for "name" immediately. If the category
+// is not enabled, then this does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_EVENT_END0(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_END1(category_group, name, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_END2(category_group, name, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val, \
+        arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_END0(category_group, name) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_END1(category_group, name, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_END2(category_group, name, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_END, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, arg1_name, arg1_val, \
+        arg2_name, arg2_val)
+
+// Similar to TRACE_EVENT_ENDx but with a custom |at| timestamp provided.
+// - |id| is used to match the _BEGIN event with the _END event.
+//   Events are considered to match if their category_group, name and id values
+//   all match. |id| must either be a pointer or an integer value up to 64 bits.
+//   If it's a pointer, the bits will be xored with a hash of the process ID so
+//   that the same pointer on two different processes will not collide.
+#define TRACE_EVENT_END_WITH_ID_TID_AND_TIMESTAMP0(category_group, \
+        name, id, thread_id, timestamp) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID_TID_AND_TIMESTAMP( \
+        TRACE_EVENT_PHASE_ASYNC_END, category_group, name, id, thread_id, \
+        timestamp, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_COPY_END_WITH_ID_TID_AND_TIMESTAMP0( \
+        category_group, name, id, thread_id, timestamp) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID_TID_AND_TIMESTAMP( \
+        TRACE_EVENT_PHASE_ASYNC_END, category_group, name, id, thread_id, \
+        timestamp, TRACE_EVENT_FLAG_COPY)
+
+// Records the value of a counter called "name" immediately. Value
+// must be representable as a 32 bit integer.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_COUNTER1(category_group, name, value) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, \
+        "value", static_cast<int>(value))
+#define TRACE_COPY_COUNTER1(category_group, name, value) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, \
+        "value", static_cast<int>(value))
+
+// Records the values of a multi-parted counter called "name" immediately.
+// The UI will treat value1 and value2 as parts of a whole, displaying their
+// values as a stacked-bar chart.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+#define TRACE_COUNTER2(category_group, name, value1_name, value1_val, \
+        value2_name, value2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, TRACE_EVENT_FLAG_NONE, \
+        value1_name, static_cast<int>(value1_val), \
+        value2_name, static_cast<int>(value2_val))
+#define TRACE_COPY_COUNTER2(category_group, name, value1_name, value1_val, \
+        value2_name, value2_val) \
+    INTERNAL_TRACE_EVENT_ADD(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, TRACE_EVENT_FLAG_COPY, \
+        value1_name, static_cast<int>(value1_val), \
+        value2_name, static_cast<int>(value2_val))
+
+// Records the value of a counter called "name" immediately. Value
+// must be representable as a 32 bit integer.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+// - |id| is used to disambiguate counters with the same name. It must either
+//   be a pointer or an integer value up to 64 bits. If it's a pointer, the bits
+//   will be xored with a hash of the process ID so that the same pointer on
+//   two different processes will not collide.
+#define TRACE_COUNTER_ID1(category_group, name, id, value) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        "value", static_cast<int>(value))
+#define TRACE_COPY_COUNTER_ID1(category_group, name, id, value) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        "value", static_cast<int>(value))
+
+// Records the values of a multi-parted counter called "name" immediately.
+// The UI will treat value1 and value2 as parts of a whole, displaying their
+// values as a stacked-bar chart.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+// - |id| is used to disambiguate counters with the same name. It must either
+//   be a pointer or an integer value up to 64 bits. If it's a pointer, the bits
+//   will be xored with a hash of the process ID so that the same pointer on
+//   two different processes will not collide.
+#define TRACE_COUNTER_ID2(category_group, name, id, value1_name, value1_val, \
+        value2_name, value2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        value1_name, static_cast<int>(value1_val), \
+        value2_name, static_cast<int>(value2_val))
+#define TRACE_COPY_COUNTER_ID2(category_group, name, id, value1_name, \
+        value1_val, value2_name, value2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_COUNTER, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        value1_name, static_cast<int>(value1_val), \
+        value2_name, static_cast<int>(value2_val))
+
+
+// Records a single ASYNC_BEGIN event called "name" immediately, with 0, 1 or 2
+// associated arguments. If the category is not enabled, then this
+// does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+// - |id| is used to match the ASYNC_BEGIN event with the ASYNC_END event. ASYNC
+//   events are considered to match if their category_group, name and id values
+//   all match. |id| must either be a pointer or an integer value up to 64 bits.
+//   If it's a pointer, the bits will be xored with a hash of the process ID so
+//   that the same pointer on two different processes will not collide.
+//
+// An asynchronous operation can consist of multiple phases. The first phase is
+// defined by the ASYNC_BEGIN calls. Additional phases can be defined using the
+// ASYNC_STEP_INTO or ASYNC_STEP_PAST macros. The ASYNC_STEP_INTO macro will
+// annotate the block following the call. The ASYNC_STEP_PAST macro will
+// annotate the block prior to the call. Note that any particular event must use
+// only STEP_INTO or STEP_PAST macros; they can not mix and match. When the
+// operation completes, call ASYNC_END.
+//
+// An ASYNC trace typically occurs on a single thread (if not, they will only be
+// drawn on the thread defined in the ASYNC_BEGIN event), but all events in that
+// operation must use the same |name| and |id|. Each step can have its own
+// args.
+#define TRACE_EVENT_ASYNC_BEGIN0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_ASYNC_BEGIN1(category_group, name, id, arg1_name, \
+        arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_ASYNC_BEGIN2(category_group, name, id, arg1_name, \
+        arg1_val, arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_ASYNC_BEGIN0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_ASYNC_BEGIN1(category_group, name, id, arg1_name, \
+        arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_ASYNC_BEGIN2(category_group, name, id, arg1_name, \
+        arg1_val, arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+
+// Records a single ASYNC_STEP_INTO event for |step| immediately. If the
+// category is not enabled, then this does nothing. The |name| and |id| must
+// match the ASYNC_BEGIN event above. The |step| param identifies this step
+// within the async event. This should be called at the beginning of the next
+// phase of an asynchronous operation. The ASYNC_BEGIN event must not have any
+// ASYNC_STEP_PAST events.
+#define TRACE_EVENT_ASYNC_STEP_INTO0(category_group, name, id, step) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_STEP_INTO, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step)
+#define TRACE_EVENT_ASYNC_STEP_INTO1(category_group, name, id, step, \
+                                     arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_STEP_INTO, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step, \
+        arg1_name, arg1_val)
+
+// Records a single ASYNC_STEP_PAST event for |step| immediately. If the
+// category is not enabled, then this does nothing. The |name| and |id| must
+// match the ASYNC_BEGIN event above. The |step| param identifies this step
+// within the async event. This should be called at the beginning of the next
+// phase of an asynchronous operation. The ASYNC_BEGIN event must not have any
+// ASYNC_STEP_INTO events.
+#define TRACE_EVENT_ASYNC_STEP_PAST0(category_group, name, id, step) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_STEP_PAST, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step)
+#define TRACE_EVENT_ASYNC_STEP_PAST1(category_group, name, id, step, \
+                                     arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_STEP_PAST, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step, \
+        arg1_name, arg1_val)
+
+// Records a single ASYNC_END event for "name" immediately. If the category
+// is not enabled, then this does nothing.
+#define TRACE_EVENT_ASYNC_END0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_ASYNC_END1(category_group, name, id, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_ASYNC_END2(category_group, name, id, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_ASYNC_END0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_ASYNC_END1(category_group, name, id, arg1_name, \
+        arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_ASYNC_END2(category_group, name, id, arg1_name, \
+        arg1_val, arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_ASYNC_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+
+
+// Records a single FLOW_BEGIN event called "name" immediately, with 0, 1 or 2
+// associated arguments. If the category is not enabled, then this
+// does nothing.
+// - category and name strings must have application lifetime (statics or
+//   literals). They may not include " chars.
+// - |id| is used to match the FLOW_BEGIN event with the FLOW_END event. FLOW
+//   events are considered to match if their category_group, name and id values
+//   all match. |id| must either be a pointer or an integer value up to 64 bits.
+//   If it's a pointer, the bits will be xored with a hash of the process ID so
+//   that the same pointer on two different processes will not collide.
+// FLOW events are different from ASYNC events in how they are drawn by the
+// tracing UI. A FLOW defines asynchronous data flow, such as posting a task
+// (FLOW_BEGIN) and later executing that task (FLOW_END). Expect FLOWs to be
+// drawn as lines or arrows from FLOW_BEGIN scopes to FLOW_END scopes. Similar
+// to ASYNC, a FLOW can consist of multiple phases. The first phase is defined
+// by the FLOW_BEGIN calls. Additional phases can be defined using the FLOW_STEP
+// macros. When the operation completes, call FLOW_END. An async operation can
+// span threads and processes, but all events in that operation must use the
+// same |name| and |id|. Each event can have its own args.
+#define TRACE_EVENT_FLOW_BEGIN0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_FLOW_BEGIN1(category_group, name, id, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_FLOW_BEGIN2(category_group, name, id, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_FLOW_BEGIN0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_FLOW_BEGIN1(category_group, name, id, arg1_name, \
+        arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_FLOW_BEGIN2(category_group, name, id, arg1_name, \
+        arg1_val, arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_BEGIN, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+
+// Records a single FLOW_STEP event for |step| immediately. If the category
+// is not enabled, then this does nothing. The |name| and |id| must match the
+// FLOW_BEGIN event above. The |step| param identifies this step within the
+// async event. This should be called at the beginning of the next phase of an
+// asynchronous operation.
+#define TRACE_EVENT_FLOW_STEP0(category_group, name, id, step) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_STEP, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step)
+#define TRACE_EVENT_FLOW_STEP1(category_group, name, id, step, \
+        arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_STEP, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, "step", step, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_FLOW_STEP0(category_group, name, id, step) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_STEP, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, "step", step)
+#define TRACE_EVENT_COPY_FLOW_STEP1(category_group, name, id, step, \
+        arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_STEP, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, "step", step, \
+        arg1_name, arg1_val)
+
+// Records a single FLOW_END event for "name" immediately. If the category
+// is not enabled, then this does nothing.
+#define TRACE_EVENT_FLOW_END0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE)
+#define TRACE_EVENT_FLOW_END1(category_group, name, id, arg1_name, arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, arg1_name, arg1_val)
+#define TRACE_EVENT_FLOW_END2(category_group, name, id, arg1_name, arg1_val, \
+        arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_NONE, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+#define TRACE_EVENT_COPY_FLOW_END0(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY)
+#define TRACE_EVENT_COPY_FLOW_END1(category_group, name, id, arg1_name, \
+        arg1_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val)
+#define TRACE_EVENT_COPY_FLOW_END2(category_group, name, id, arg1_name, \
+        arg1_val, arg2_name, arg2_val) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_FLOW_END, \
+        category_group, name, id, TRACE_EVENT_FLAG_COPY, \
+        arg1_name, arg1_val, arg2_name, arg2_val)
+
+// Macros to track the life time and value of arbitrary client objects.
+// See also TraceTrackableObject.
+#define TRACE_EVENT_OBJECT_CREATED_WITH_ID(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_CREATE_OBJECT, \
+        category_group, name, TRACE_ID_DONT_MANGLE(id), TRACE_EVENT_FLAG_NONE)
+
+#define TRACE_EVENT_OBJECT_SNAPSHOT_WITH_ID(category_group, name, id, snapshot) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_SNAPSHOT_OBJECT, \
+        category_group, name, TRACE_ID_DONT_MANGLE(id), TRACE_EVENT_FLAG_NONE,\
+        "snapshot", snapshot)
+
+#define TRACE_EVENT_OBJECT_DELETED_WITH_ID(category_group, name, id) \
+    INTERNAL_TRACE_EVENT_ADD_WITH_ID(TRACE_EVENT_PHASE_DELETE_OBJECT, \
+        category_group, name, TRACE_ID_DONT_MANGLE(id), TRACE_EVENT_FLAG_NONE)
+
+#define INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE() \
+    PREDICT_FALSE(*INTERNAL_TRACE_EVENT_UID(category_group_enabled) & \
+        (kudu::debug::TraceLog::ENABLED_FOR_RECORDING | \
+         kudu::debug::TraceLog::ENABLED_FOR_EVENT_CALLBACK))
+
+// Macro to efficiently determine if a given category group is enabled.
+#define TRACE_EVENT_CATEGORY_GROUP_ENABLED(category_group, ret) \
+    do { \
+      INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group); \
+      if (INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE()) { \
+        *ret = true; \
+      } else { \
+        *ret = false; \
+      } \
+    } while (0)
+
+// Macro to efficiently determine, through polling, if a new trace has begun.
+#define TRACE_EVENT_IS_NEW_TRACE(ret) \
+    do { \
+      static int INTERNAL_TRACE_EVENT_UID(lastRecordingNumber) = 0; \
+      int num_traces_recorded = TRACE_EVENT_API_GET_NUM_TRACES_RECORDED(); \
+      if (num_traces_recorded != -1 && \
+          num_traces_recorded != \
+          INTERNAL_TRACE_EVENT_UID(lastRecordingNumber)) { \
+        INTERNAL_TRACE_EVENT_UID(lastRecordingNumber) = \
+            num_traces_recorded; \
+        *ret = true; \
+      } else { \
+        *ret = false; \
+      } \
+    } while (0)
+
+////////////////////////////////////////////////////////////////////////////////
+// Implementation specific tracing API definitions.
+
+// Get a pointer to the enabled state of the given trace category. Only
+// long-lived literal strings should be given as the category group. The
+// returned pointer can be held permanently in a local static for example. If
+// the unsigned char is non-zero, tracing is enabled. If tracing is enabled,
+// TRACE_EVENT_API_ADD_TRACE_EVENT can be called. It's OK if tracing is disabled
+// between the load of the tracing state and the call to
+// TRACE_EVENT_API_ADD_TRACE_EVENT, because this flag only provides an early out
+// for best performance when tracing is disabled.
+// const unsigned char*
+//     TRACE_EVENT_API_GET_CATEGORY_GROUP_ENABLED(const char* category_group)
+#define TRACE_EVENT_API_GET_CATEGORY_GROUP_ENABLED \
+    kudu::debug::TraceLog::GetCategoryGroupEnabled
+
+// Get the number of times traces have been recorded. This is used to implement
+// the TRACE_EVENT_IS_NEW_TRACE facility.
+// unsigned int TRACE_EVENT_API_GET_NUM_TRACES_RECORDED()
+#define TRACE_EVENT_API_GET_NUM_TRACES_RECORDED \
+    kudu::debug::TraceLog::GetInstance()->GetNumTracesRecorded
+
+// Add a trace event to the platform tracing system.
+// kudu::debug::TraceEventHandle TRACE_EVENT_API_ADD_TRACE_EVENT(
+//                    char phase,
+//                    const unsigned char* category_group_enabled,
+//                    const char* name,
+//                    uint64_t id,
+//                    int num_args,
+//                    const char** arg_names,
+//                    const unsigned char* arg_types,
+//                    const uint64_t* arg_values,
+//                    unsigned char flags)
+#define TRACE_EVENT_API_ADD_TRACE_EVENT \
+    kudu::debug::TraceLog::GetInstance()->AddTraceEvent
+
+// Add a trace event to the platform tracing system.
+// kudu::debug::TraceEventHandle TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_TIMESTAMP(
+//                    char phase,
+//                    const unsigned char* category_group_enabled,
+//                    const char* name,
+//                    uint64_t id,
+//                    int thread_id,
+//                    const MicrosecondsInt64& timestamp,
+//                    int num_args,
+//                    const char** arg_names,
+//                    const unsigned char* arg_types,
+//                    const uint64_t* arg_values,
+//                    unsigned char flags)
+#define TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP \
+    kudu::debug::TraceLog::GetInstance()->AddTraceEventWithThreadIdAndTimestamp
+
+// Set the duration field of a COMPLETE trace event.
+// void TRACE_EVENT_API_UPDATE_TRACE_EVENT_DURATION(
+//     const unsigned char* category_group_enabled,
+//     const char* name,
+//     kudu::debug::TraceEventHandle id)
+#define TRACE_EVENT_API_UPDATE_TRACE_EVENT_DURATION \
+    kudu::debug::TraceLog::GetInstance()->UpdateTraceEventDuration
+
+// Defines atomic operations used internally by the tracing system.
+#define TRACE_EVENT_API_ATOMIC_WORD AtomicWord
+#define TRACE_EVENT_API_ATOMIC_LOAD(var) base::subtle::NoBarrier_Load(&(var))
+#define TRACE_EVENT_API_ATOMIC_STORE(var, value) \
+    base::subtle::NoBarrier_Store(&(var), (value))
+
+// Defines visibility for classes in trace_event.h
+#define TRACE_EVENT_API_CLASS_EXPORT BASE_EXPORT
+
+// The thread buckets for the sampling profiler.
+TRACE_EVENT_API_CLASS_EXPORT extern \
+    TRACE_EVENT_API_ATOMIC_WORD g_trace_state[3];
+
+#define TRACE_EVENT_API_THREAD_BUCKET(thread_bucket)                           \
+    g_trace_state[thread_bucket]
+
+////////////////////////////////////////////////////////////////////////////////
+
+// Implementation detail: trace event macros create temporary variables
+// to keep instrumentation overhead low. These macros give each temporary
+// variable a unique name based on the line number to prevent name collisions.
+#define INTERNAL_TRACE_EVENT_UID3(a,b) \
+    trace_event_unique_##a##b
+#define INTERNAL_TRACE_EVENT_UID2(a,b) \
+    INTERNAL_TRACE_EVENT_UID3(a,b)
+#define INTERNAL_TRACE_EVENT_UID(name_prefix) \
+    INTERNAL_TRACE_EVENT_UID2(name_prefix, __LINE__)
+
+// Implementation detail: internal macro to create static category.
+// No barriers are needed, because this code is designed to operate safely
+// even when the unsigned char* points to garbage data (which may be the case
+// on processors without cache coherency).
+#define INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO_CUSTOM_VARIABLES( \
+    category_group, atomic, category_group_enabled) \
+    category_group_enabled = \
+        reinterpret_cast<const unsigned char*>(TRACE_EVENT_API_ATOMIC_LOAD( \
+            atomic)); \
+    if (PREDICT_FALSE(!category_group_enabled)) { \
+      category_group_enabled = \
+          TRACE_EVENT_API_GET_CATEGORY_GROUP_ENABLED(category_group); \
+      TRACE_EVENT_API_ATOMIC_STORE(atomic, \
+          reinterpret_cast<TRACE_EVENT_API_ATOMIC_WORD>( \
+              category_group_enabled)); \
+    }
+
+#define INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group) \
+    static TRACE_EVENT_API_ATOMIC_WORD INTERNAL_TRACE_EVENT_UID(atomic) = 0; \
+    const unsigned char* INTERNAL_TRACE_EVENT_UID(category_group_enabled); \
+    INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO_CUSTOM_VARIABLES(category_group, \
+        INTERNAL_TRACE_EVENT_UID(atomic), \
+        INTERNAL_TRACE_EVENT_UID(category_group_enabled));
+
+// Implementation detail: internal macro to create static category and add
+// event if the category is enabled.
+#define INTERNAL_TRACE_EVENT_ADD(phase, category_group, name, flags, ...) \
+    do { \
+      INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group); \
+      if (INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE()) { \
+        trace_event_internal::AddTraceEvent( \
+            phase, INTERNAL_TRACE_EVENT_UID(category_group_enabled), name, \
+            trace_event_internal::kNoEventId, flags, ##__VA_ARGS__); \
+      } \
+    } while (0)
+
+// Implementation detail: internal macro to create static category and add begin
+// event if the category is enabled. Also adds the end event when the scope
+// ends.
+#define INTERNAL_TRACE_EVENT_ADD_SCOPED(category_group, name, ...) \
+    INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group); \
+    trace_event_internal::ScopedTracer INTERNAL_TRACE_EVENT_UID(tracer); \
+    if (INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE()) { \
+      kudu::debug::TraceEventHandle h = trace_event_internal::AddTraceEvent( \
+          TRACE_EVENT_PHASE_COMPLETE, \
+          INTERNAL_TRACE_EVENT_UID(category_group_enabled), \
+          name, trace_event_internal::kNoEventId, \
+          TRACE_EVENT_FLAG_NONE, ##__VA_ARGS__); \
+      INTERNAL_TRACE_EVENT_UID(tracer).Initialize( \
+          INTERNAL_TRACE_EVENT_UID(category_group_enabled), name, h); \
+    }
+
+// Implementation detail: internal macro to create static category and add
+// event if the category is enabled.
+#define INTERNAL_TRACE_EVENT_ADD_WITH_ID(phase, category_group, name, id, \
+                                         flags, ...) \
+    do { \
+      INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group); \
+      if (INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE()) { \
+        unsigned char trace_event_flags = flags | TRACE_EVENT_FLAG_HAS_ID; \
+        trace_event_internal::TraceID trace_event_trace_id( \
+            id, &trace_event_flags); \
+        trace_event_internal::AddTraceEvent( \
+            phase, INTERNAL_TRACE_EVENT_UID(category_group_enabled), \
+            name, trace_event_trace_id.data(), trace_event_flags, \
+            ##__VA_ARGS__); \
+      } \
+    } while (0)
+
+// Implementation detail: internal macro to create static category and add
+// event if the category is enabled.
+#define INTERNAL_TRACE_EVENT_ADD_WITH_ID_TID_AND_TIMESTAMP(phase, \
+        category_group, name, id, thread_id, timestamp, flags, ...) \
+    do { \
+      INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO(category_group); \
+      if (INTERNAL_TRACE_EVENT_CATEGORY_GROUP_ENABLED_FOR_RECORDING_MODE()) { \
+        unsigned char trace_event_flags = flags | TRACE_EVENT_FLAG_HAS_ID; \
+        trace_event_internal::TraceID trace_event_trace_id( \
+            id, &trace_event_flags); \
+        trace_event_internal::AddTraceEventWithThreadIdAndTimestamp( \
+            phase, INTERNAL_TRACE_EVENT_UID(category_group_enabled), \
+            name, trace_event_trace_id.data(), \
+            thread_id, timestamp, \
+            trace_event_flags, ##__VA_ARGS__); \
+      } \
+    } while (0)
+
+// Notes regarding the following definitions:
+// New values can be added and propagated to third party libraries, but existing
+// definitions must never be changed, because third party libraries may use old
+// definitions.
+
+// Phase indicates the nature of an event entry. E.g. part of a begin/end pair.
+#define TRACE_EVENT_PHASE_BEGIN    ('B')
+#define TRACE_EVENT_PHASE_END      ('E')
+#define TRACE_EVENT_PHASE_COMPLETE ('X')
+#define TRACE_EVENT_PHASE_INSTANT  ('i')
+#define TRACE_EVENT_PHASE_ASYNC_BEGIN ('S')
+#define TRACE_EVENT_PHASE_ASYNC_STEP_INTO  ('T')
+#define TRACE_EVENT_PHASE_ASYNC_STEP_PAST  ('p')
+#define TRACE_EVENT_PHASE_ASYNC_END   ('F')
+#define TRACE_EVENT_PHASE_FLOW_BEGIN ('s')
+#define TRACE_EVENT_PHASE_FLOW_STEP  ('t')
+#define TRACE_EVENT_PHASE_FLOW_END   ('f')
+#define TRACE_EVENT_PHASE_METADATA ('M')
+#define TRACE_EVENT_PHASE_COUNTER  ('C')
+#define TRACE_EVENT_PHASE_SAMPLE  ('P')
+#define TRACE_EVENT_PHASE_CREATE_OBJECT ('N')
+#define TRACE_EVENT_PHASE_SNAPSHOT_OBJECT ('O')
+#define TRACE_EVENT_PHASE_DELETE_OBJECT ('D')
+
+// Flags for changing the behavior of TRACE_EVENT_API_ADD_TRACE_EVENT.
+#define TRACE_EVENT_FLAG_NONE         (static_cast<unsigned char>(0))
+#define TRACE_EVENT_FLAG_COPY         (static_cast<unsigned char>(1 << 0))
+#define TRACE_EVENT_FLAG_HAS_ID       (static_cast<unsigned char>(1 << 1))
+#define TRACE_EVENT_FLAG_MANGLE_ID    (static_cast<unsigned char>(1 << 2))
+#define TRACE_EVENT_FLAG_SCOPE_OFFSET (static_cast<unsigned char>(1 << 3))
+
+#define TRACE_EVENT_FLAG_SCOPE_MASK   (static_cast<unsigned char>( \
+    TRACE_EVENT_FLAG_SCOPE_OFFSET | (TRACE_EVENT_FLAG_SCOPE_OFFSET << 1)))
+
+// Type values for identifying types in the TraceValue union.
+#define TRACE_VALUE_TYPE_BOOL         (static_cast<unsigned char>(1))
+#define TRACE_VALUE_TYPE_UINT         (static_cast<unsigned char>(2))
+#define TRACE_VALUE_TYPE_INT          (static_cast<unsigned char>(3))
+#define TRACE_VALUE_TYPE_DOUBLE       (static_cast<unsigned char>(4))
+#define TRACE_VALUE_TYPE_POINTER      (static_cast<unsigned char>(5))
+#define TRACE_VALUE_TYPE_STRING       (static_cast<unsigned char>(6))
+#define TRACE_VALUE_TYPE_COPY_STRING  (static_cast<unsigned char>(7))
+#define TRACE_VALUE_TYPE_CONVERTABLE  (static_cast<unsigned char>(8))
+
+// Enum reflecting the scope of an INSTANT event. Must fit within
+// TRACE_EVENT_FLAG_SCOPE_MASK.
+#define TRACE_EVENT_SCOPE_GLOBAL  (static_cast<unsigned char>(0 << 3))
+#define TRACE_EVENT_SCOPE_PROCESS (static_cast<unsigned char>(1 << 3))
+#define TRACE_EVENT_SCOPE_THREAD  (static_cast<unsigned char>(2 << 3))
+
+#define TRACE_EVENT_SCOPE_NAME_GLOBAL  ('g')
+#define TRACE_EVENT_SCOPE_NAME_PROCESS ('p')
+#define TRACE_EVENT_SCOPE_NAME_THREAD  ('t')
+
+namespace trace_event_internal {
+
+// Specify these values when the corresponding argument of AddTraceEvent is not
+// used.
+const int kZeroNumArgs = 0;
+const uint64_t kNoEventId = 0;
+
+// TraceID encapsulates an ID that can either be an integer or pointer. Pointers
+// are by default mangled with the Process ID so that they are unlikely to
+// collide when the same pointer is used on different processes.
+class TraceID {
+ public:
+  class DontMangle {
+   public:
+    explicit DontMangle(const void* id)
+        : data_(static_cast<uint64_t>(
+              reinterpret_cast<unsigned long>(id))) {}
+    explicit DontMangle(uint64_t id) : data_(id) {}
+    explicit DontMangle(unsigned int id) : data_(id) {}
+    explicit DontMangle(unsigned short id) : data_(id) {}
+    explicit DontMangle(unsigned char id) : data_(id) {}
+    explicit DontMangle(long long id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit DontMangle(long id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit DontMangle(int id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit DontMangle(short id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit DontMangle(signed char id)
+        : data_(static_cast<uint64_t>(id)) {}
+    uint64_t data() const { return data_; }
+   private:
+    uint64_t data_;
+  };
+
+  class ForceMangle {
+   public:
+    explicit ForceMangle(uint64_t id) : data_(id) {}
+    explicit ForceMangle(unsigned int id) : data_(id) {}
+    explicit ForceMangle(unsigned short id) : data_(id) {}
+    explicit ForceMangle(unsigned char id) : data_(id) {}
+    explicit ForceMangle(long long id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit ForceMangle(long id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit ForceMangle(int id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit ForceMangle(short id)
+        : data_(static_cast<uint64_t>(id)) {}
+    explicit ForceMangle(signed char id)
+        : data_(static_cast<uint64_t>(id)) {}
+    uint64_t data() const { return data_; }
+   private:
+    uint64_t data_;
+  };
+
+  TraceID(const void* id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(
+              reinterpret_cast<unsigned long>(id))) {
+    *flags |= TRACE_EVENT_FLAG_MANGLE_ID;
+  }
+  TraceID(ForceMangle id, unsigned char* flags) : data_(id.data()) {
+    *flags |= TRACE_EVENT_FLAG_MANGLE_ID;
+  }
+  TraceID(DontMangle id, unsigned char* flags) : data_(id.data()) {
+  }
+  TraceID(uint64_t id, unsigned char* flags)
+      : data_(id) { (void)flags; }
+  TraceID(unsigned int id, unsigned char* flags)
+      : data_(id) { (void)flags; }
+  TraceID(unsigned short id, unsigned char* flags)
+      : data_(id) { (void)flags; }
+  TraceID(unsigned char id, unsigned char* flags)
+      : data_(id) { (void)flags; }
+  TraceID(long long id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(id)) { (void)flags; }
+  TraceID(long id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(id)) { (void)flags; }
+  TraceID(int id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(id)) { (void)flags; }
+  TraceID(short id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(id)) { (void)flags; }
+  TraceID(signed char id, unsigned char* flags)
+      : data_(static_cast<uint64_t>(id)) { (void)flags; }
+
+  uint64_t data() const { return data_; }
+
+ private:
+  uint64_t data_;
+};
+
+// Simple union to store various types as uint64_t.
+union TraceValueUnion {
+  bool as_bool;
+  uint64_t as_uint;
+  long long as_int;
+  double as_double;
+  const void* as_pointer;
+  const char* as_string;
+};
+
+// Simple container for const char* that should be copied instead of retained.
+class TraceStringWithCopy {
+ public:
+  explicit TraceStringWithCopy(const char* str) : str_(str) {}
+  const char* str() const { return str_; }
+ private:
+  const char* str_;
+};
+
+// Define SetTraceValue for each allowed type. It stores the type and
+// value in the return arguments. This allows this API to avoid declaring any
+// structures so that it is portable to third_party libraries.
+#define INTERNAL_DECLARE_SET_TRACE_VALUE(actual_type, \
+                                         arg_expression, \
+                                         union_member, \
+                                         value_type_id) \
+    static inline void SetTraceValue( \
+        actual_type arg, \
+        unsigned char* type, \
+        uint64_t* value) { \
+      TraceValueUnion type_value; \
+      type_value.union_member = arg_expression; \
+      *type = value_type_id; \
+      *value = type_value.as_uint; \
+    }
+// Simpler form for int types that can be safely casted.
+#define INTERNAL_DECLARE_SET_TRACE_VALUE_INT(actual_type, \
+                                             value_type_id) \
+    static inline void SetTraceValue( \
+        actual_type arg, \
+        unsigned char* type, \
+        uint64_t* value) { \
+      *type = value_type_id; \
+      *value = static_cast<uint64_t>(arg); \
+    }
+
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(uint64_t, TRACE_VALUE_TYPE_UINT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(unsigned int, TRACE_VALUE_TYPE_UINT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(unsigned short, TRACE_VALUE_TYPE_UINT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(unsigned char, TRACE_VALUE_TYPE_UINT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(long long, TRACE_VALUE_TYPE_INT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(long, TRACE_VALUE_TYPE_INT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(int, TRACE_VALUE_TYPE_INT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(short, TRACE_VALUE_TYPE_INT)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(signed char, TRACE_VALUE_TYPE_INT)
+INTERNAL_DECLARE_SET_TRACE_VALUE(bool, arg, as_bool, TRACE_VALUE_TYPE_BOOL)
+INTERNAL_DECLARE_SET_TRACE_VALUE(double, arg, as_double,
+                                 TRACE_VALUE_TYPE_DOUBLE)
+INTERNAL_DECLARE_SET_TRACE_VALUE(const void*, arg, as_pointer,
+                                 TRACE_VALUE_TYPE_POINTER)
+INTERNAL_DECLARE_SET_TRACE_VALUE(const char*, arg, as_string,
+                                 TRACE_VALUE_TYPE_STRING)
+INTERNAL_DECLARE_SET_TRACE_VALUE(const TraceStringWithCopy&, arg.str(),
+                                 as_string, TRACE_VALUE_TYPE_COPY_STRING)
+#if defined(__APPLE__)
+INTERNAL_DECLARE_SET_TRACE_VALUE_INT(size_t, TRACE_VALUE_TYPE_UINT)
+#endif
+
+#undef INTERNAL_DECLARE_SET_TRACE_VALUE
+#undef INTERNAL_DECLARE_SET_TRACE_VALUE_INT
+
+// std::string version of SetTraceValue so that trace arguments can be strings.
+static inline void SetTraceValue(const std::string& arg,
+                                 unsigned char* type,
+                                 uint64_t* value) {
+  TraceValueUnion type_value;
+  type_value.as_string = arg.c_str();
+  *type = TRACE_VALUE_TYPE_COPY_STRING;
+  *value = type_value.as_uint;
+}
+
+// These AddTraceEvent and AddTraceEventWithThreadIdAndTimestamp template
+// functions are defined here instead of in the macro, because the arg_values
+// could be temporary objects, such as std::string. In order to store
+// pointers to the internal c_str and pass through to the tracing API,
+// the arg_values must live throughout these procedures.
+
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const scoped_refptr<kudu::debug::ConvertableToTraceFormat>& arg1_val) {
+  const int num_args = 1;
+  unsigned char arg_types[1] = { TRACE_VALUE_TYPE_CONVERTABLE };
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, &arg1_name, arg_types, NULL, &arg1_val, flags);
+}
+
+template<class ARG1_TYPE>
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const ARG1_TYPE& arg1_val,
+    const char* arg2_name,
+    const scoped_refptr<kudu::debug::ConvertableToTraceFormat>& arg2_val) {
+  const int num_args = 2;
+  const char* arg_names[2] = { arg1_name, arg2_name };
+
+  unsigned char arg_types[2];
+  uint64_t arg_values[2];
+  SetTraceValue(arg1_val, &arg_types[0], &arg_values[0]);
+  arg_types[1] = TRACE_VALUE_TYPE_CONVERTABLE;
+
+  scoped_refptr<kudu::debug::ConvertableToTraceFormat> convertable_values[2];
+  convertable_values[1] = arg2_val;
+
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, arg_names, arg_types, arg_values, convertable_values, flags);
+}
+
+template<class ARG2_TYPE>
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const scoped_refptr<kudu::debug::ConvertableToTraceFormat>& arg1_val,
+    const char* arg2_name,
+    const ARG2_TYPE& arg2_val) {
+  const int num_args = 2;
+  const char* arg_names[2] = { arg1_name, arg2_name };
+
+  unsigned char arg_types[2];
+  uint64_t arg_values[2];
+  arg_types[0] = TRACE_VALUE_TYPE_CONVERTABLE;
+  arg_values[0] = 0;
+  SetTraceValue(arg2_val, &arg_types[1], &arg_values[1]);
+
+  scoped_refptr<kudu::debug::ConvertableToTraceFormat> convertable_values[2];
+  convertable_values[0] = arg1_val;
+
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, arg_names, arg_types, arg_values, convertable_values, flags);
+}
+
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const scoped_refptr<kudu::debug::ConvertableToTraceFormat>& arg1_val,
+    const char* arg2_name,
+    const scoped_refptr<kudu::debug::ConvertableToTraceFormat>& arg2_val) {
+  const int num_args = 2;
+  const char* arg_names[2] = { arg1_name, arg2_name };
+  unsigned char arg_types[2] =
+      { TRACE_VALUE_TYPE_CONVERTABLE, TRACE_VALUE_TYPE_CONVERTABLE };
+  scoped_refptr<kudu::debug::ConvertableToTraceFormat> convertable_values[2] =
+      { arg1_val, arg2_val };
+
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, arg_names, arg_types, NULL, convertable_values, flags);
+}
+
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags) {
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      kZeroNumArgs, NULL, NULL, NULL, NULL, flags);
+}
+
+static inline kudu::debug::TraceEventHandle AddTraceEvent(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    unsigned char flags) {
+  int thread_id = static_cast<int>(kudu::Thread::UniqueThreadId());
+  MicrosecondsInt64 now = GetMonoTimeMicros();
+  return AddTraceEventWithThreadIdAndTimestamp(phase, category_group_enabled,
+                                               name, id, thread_id, now, flags);
+}
+
+template<class ARG1_TYPE>
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const ARG1_TYPE& arg1_val) {
+  const int num_args = 1;
+  unsigned char arg_types[1];
+  uint64_t arg_values[1];
+  SetTraceValue(arg1_val, &arg_types[0], &arg_values[0]);
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, &arg1_name, arg_types, arg_values, NULL, flags);
+}
+
+template<class ARG1_TYPE>
+static inline kudu::debug::TraceEventHandle AddTraceEvent(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    unsigned char flags,
+    const char* arg1_name,
+    const ARG1_TYPE& arg1_val) {
+  int thread_id = static_cast<int>(kudu::Thread::UniqueThreadId());
+  MicrosecondsInt64 now = GetMonoTimeMicros();
+  return AddTraceEventWithThreadIdAndTimestamp(phase, category_group_enabled,
+                                               name, id, thread_id, now, flags,
+                                               arg1_name, arg1_val);
+}
+
+template<class ARG1_TYPE, class ARG2_TYPE>
+static inline kudu::debug::TraceEventHandle
+AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    unsigned char flags,
+    const char* arg1_name,
+    const ARG1_TYPE& arg1_val,
+    const char* arg2_name,
+    const ARG2_TYPE& arg2_val) {
+  const int num_args = 2;
+  const char* arg_names[2] = { arg1_name, arg2_name };
+  unsigned char arg_types[2];
+  uint64_t arg_values[2];
+  SetTraceValue(arg1_val, &arg_types[0], &arg_values[0]);
+  SetTraceValue(arg2_val, &arg_types[1], &arg_values[1]);
+  return TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+      phase, category_group_enabled, name, id, thread_id, timestamp,
+      num_args, arg_names, arg_types, arg_values, NULL, flags);
+}
+
+template<class ARG1_TYPE, class ARG2_TYPE>
+static inline kudu::debug::TraceEventHandle AddTraceEvent(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    unsigned char flags,
+    const char* arg1_name,
+    const ARG1_TYPE& arg1_val,
+    const char* arg2_name,
+    const ARG2_TYPE& arg2_val) {
+  int thread_id = static_cast<int>(kudu::Thread::UniqueThreadId());
+  MicrosecondsInt64 now = GetMonoTimeMicros();
+  return AddTraceEventWithThreadIdAndTimestamp(phase, category_group_enabled,
+                                               name, id, thread_id, now, flags,
+                                               arg1_name, arg1_val,
+                                               arg2_name, arg2_val);
+}
+
+// Used by TRACE_EVENTx macros. Do not use directly.
+class TRACE_EVENT_API_CLASS_EXPORT ScopedTracer {
+ public:
+  // Note: members of data_ intentionally left uninitialized. See Initialize.
+  ScopedTracer() : p_data_(NULL) {}
+
+  ~ScopedTracer() {
+    if (p_data_ && *data_.category_group_enabled)
+      TRACE_EVENT_API_UPDATE_TRACE_EVENT_DURATION(
+          data_.category_group_enabled, data_.name, data_.event_handle);
+  }
+
+  void Initialize(const unsigned char* category_group_enabled,
+                  const char* name,
+                  kudu::debug::TraceEventHandle event_handle) {
+    data_.category_group_enabled = category_group_enabled;
+    data_.name = name;
+    data_.event_handle = event_handle;
+    p_data_ = &data_;
+  }
+
+ private:
+  // This Data struct workaround is to avoid initializing all the members
+  // in Data during construction of this object, since this object is always
+  // constructed, even when tracing is disabled. If the members of Data were
+  // members of this class instead, compiler warnings occur about potential
+  // uninitialized accesses.
+  struct Data {
+    const unsigned char* category_group_enabled;
+    const char* name;
+    kudu::debug::TraceEventHandle event_handle;
+  };
+  Data* p_data_;
+  Data data_;
+};
+
+// Used by TRACE_EVENT_BINARY_EFFICIENTx macro. Do not use directly.
+class TRACE_EVENT_API_CLASS_EXPORT ScopedTraceBinaryEfficient {
+ public:
+  ScopedTraceBinaryEfficient(const char* category_group, const char* name);
+  ~ScopedTraceBinaryEfficient();
+
+ private:
+  const unsigned char* category_group_enabled_;
+  const char* name_;
+  kudu::debug::TraceEventHandle event_handle_;
+};
+
+// This macro generates less code then TRACE_EVENT0 but is also
+// slower to execute when tracing is off. It should generally only be
+// used with code that is seldom executed or conditionally executed
+// when debugging.
+// For now the category_group must be "gpu".
+#define TRACE_EVENT_BINARY_EFFICIENT0(category_group, name) \
+    trace_event_internal::ScopedTraceBinaryEfficient \
+        INTERNAL_TRACE_EVENT_UID(scoped_trace)(category_group, name);
+
+// TraceEventSamplingStateScope records the current sampling state
+// and sets a new sampling state. When the scope exists, it restores
+// the sampling state having recorded.
+template<size_t BucketNumber>
+class TraceEventSamplingStateScope {
+ public:
+  TraceEventSamplingStateScope(const char* category_and_name) {
+    previous_state_ = TraceEventSamplingStateScope<BucketNumber>::Current();
+    TraceEventSamplingStateScope<BucketNumber>::Set(category_and_name);
+  }
+
+  ~TraceEventSamplingStateScope() {
+    TraceEventSamplingStateScope<BucketNumber>::Set(previous_state_);
+  }
+
+  static inline const char* Current() {
+    return reinterpret_cast<const char*>(TRACE_EVENT_API_ATOMIC_LOAD(
+      g_trace_state[BucketNumber]));
+  }
+
+  static inline void Set(const char* category_and_name) {
+    TRACE_EVENT_API_ATOMIC_STORE(
+      g_trace_state[BucketNumber],
+      reinterpret_cast<TRACE_EVENT_API_ATOMIC_WORD>(
+        const_cast<char*>(category_and_name)));
+  }
+
+ private:
+  const char* previous_state_;
+};
+
+}  // namespace trace_event_internal
+
+namespace kudu {
+namespace debug {
+
+template<typename IDType> class TraceScopedTrackableObject {
+ public:
+  TraceScopedTrackableObject(const char* category_group, const char* name,
+      IDType id)
+    : category_group_(category_group),
+      name_(name),
+      id_(id) {
+    TRACE_EVENT_OBJECT_CREATED_WITH_ID(category_group_, name_, id_);
+  }
+
+  template <typename ArgType> void snapshot(ArgType snapshot) {
+    TRACE_EVENT_OBJECT_SNAPSHOT_WITH_ID(category_group_, name_, id_, snapshot);
+  }
+
+  ~TraceScopedTrackableObject() {
+    TRACE_EVENT_OBJECT_DELETED_WITH_ID(category_group_, name_, id_);
+  }
+
+ private:
+  const char* category_group_;
+  const char* name_;
+  IDType id_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceScopedTrackableObject);
+};
+
+} // namespace debug
+} // namespace kudu
+
+#endif /* KUDU_UTIL_DEBUG_TRACE_EVENT_H_ */


[04/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/thread.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/thread.cc b/be/src/kudu/util/thread.cc
new file mode 100644
index 0000000..4abc7c1
--- /dev/null
+++ b/be/src/kudu/util/thread.cc
@@ -0,0 +1,628 @@
+// 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.
+//
+// Copied from Impala and adapted to Kudu.
+
+#include "kudu/util/thread.h"
+
+#if defined(__linux__)
+#include <sys/prctl.h>
+#endif // defined(__linux__)
+#include <sys/resource.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cerrno>
+#include <cstring>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <boost/bind.hpp>
+#include <boost/smart_ptr/shared_ptr.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/mathlimits.h"
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/kernel_stack_watchdog.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/os-util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/url-coding.h"
+#include "kudu/util/web_callback_registry.h"
+
+using boost::bind;
+using boost::mem_fn;
+using std::endl;
+using std::map;
+using std::ostringstream;
+using std::shared_ptr;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+METRIC_DEFINE_gauge_uint64(server, threads_started,
+                           "Threads Started",
+                           kudu::MetricUnit::kThreads,
+                           "Total number of threads started on this server",
+                           kudu::EXPOSE_AS_COUNTER);
+
+METRIC_DEFINE_gauge_uint64(server, threads_running,
+                           "Threads Running",
+                           kudu::MetricUnit::kThreads,
+                           "Current number of running threads");
+
+METRIC_DEFINE_gauge_uint64(server, cpu_utime,
+                           "User CPU Time",
+                           kudu::MetricUnit::kMilliseconds,
+                           "Total user CPU time of the process",
+                           kudu::EXPOSE_AS_COUNTER);
+
+METRIC_DEFINE_gauge_uint64(server, cpu_stime,
+                           "System CPU Time",
+                           kudu::MetricUnit::kMilliseconds,
+                           "Total system CPU time of the process",
+                           kudu::EXPOSE_AS_COUNTER);
+
+METRIC_DEFINE_gauge_uint64(server, voluntary_context_switches,
+                           "Voluntary Context Switches",
+                           kudu::MetricUnit::kContextSwitches,
+                           "Total voluntary context switches",
+                           kudu::EXPOSE_AS_COUNTER);
+
+METRIC_DEFINE_gauge_uint64(server, involuntary_context_switches,
+                           "Involuntary Context Switches",
+                           kudu::MetricUnit::kContextSwitches,
+                           "Total involuntary context switches",
+                           kudu::EXPOSE_AS_COUNTER);
+
+DEFINE_int32(thread_inject_start_latency_ms, 0,
+             "Number of ms to sleep when starting a new thread. (For tests).");
+TAG_FLAG(thread_inject_start_latency_ms, hidden);
+TAG_FLAG(thread_inject_start_latency_ms, unsafe);
+
+namespace kudu {
+
+static uint64_t GetCpuUTime() {
+  rusage ru;
+  CHECK_ERR(getrusage(RUSAGE_SELF, &ru));
+  return ru.ru_utime.tv_sec * 1000UL + ru.ru_utime.tv_usec / 1000UL;
+}
+
+static uint64_t GetCpuSTime() {
+  rusage ru;
+  CHECK_ERR(getrusage(RUSAGE_SELF, &ru));
+  return ru.ru_stime.tv_sec * 1000UL + ru.ru_stime.tv_usec / 1000UL;
+}
+
+static uint64_t GetVoluntaryContextSwitches() {
+  rusage ru;
+  CHECK_ERR(getrusage(RUSAGE_SELF, &ru));
+  return ru.ru_nvcsw;;
+}
+
+static uint64_t GetInVoluntaryContextSwitches() {
+  rusage ru;
+  CHECK_ERR(getrusage(RUSAGE_SELF, &ru));
+  return ru.ru_nivcsw;
+}
+
+class ThreadMgr;
+
+__thread Thread* Thread::tls_ = NULL;
+
+// Singleton instance of ThreadMgr. Only visible in this file, used only by Thread.
+// The Thread class adds a reference to thread_manager while it is supervising a thread so
+// that a race between the end of the process's main thread (and therefore the destruction
+// of thread_manager) and the end of a thread that tries to remove itself from the
+// manager after the destruction can be avoided.
+static shared_ptr<ThreadMgr> thread_manager;
+
+// Controls the single (lazy) initialization of thread_manager.
+static GoogleOnceType once = GOOGLE_ONCE_INIT;
+
+// A singleton class that tracks all live threads, and groups them together for easy
+// auditing. Used only by Thread.
+class ThreadMgr {
+ public:
+  ThreadMgr()
+      : threads_started_metric_(0),
+        threads_running_metric_(0) {
+  }
+
+  ~ThreadMgr() {
+    MutexLock l(lock_);
+    thread_categories_.clear();
+  }
+
+  static void SetThreadName(const std::string& name, int64_t tid);
+
+  Status StartInstrumentation(const scoped_refptr<MetricEntity>& metrics, WebCallbackRegistry* web);
+
+  // Registers a thread to the supplied category. The key is a pthread_t,
+  // not the system TID, since pthread_t is less prone to being recycled.
+  void AddThread(const pthread_t& pthread_id, const string& name, const string& category,
+      int64_t tid);
+
+  // Removes a thread from the supplied category. If the thread has
+  // already been removed, this is a no-op.
+  void RemoveThread(const pthread_t& pthread_id, const string& category);
+
+ private:
+  // Container class for any details we want to capture about a thread
+  // TODO: Add start-time.
+  // TODO: Track fragment ID.
+  class ThreadDescriptor {
+   public:
+    ThreadDescriptor() { }
+    ThreadDescriptor(string category, string name, int64_t thread_id)
+        : name_(std::move(name)),
+          category_(std::move(category)),
+          thread_id_(thread_id) {}
+
+    const string& name() const { return name_; }
+    const string& category() const { return category_; }
+    int64_t thread_id() const { return thread_id_; }
+
+   private:
+    string name_;
+    string category_;
+    int64_t thread_id_;
+  };
+
+  // A ThreadCategory is a set of threads that are logically related.
+  // TODO: unordered_map is incompatible with pthread_t, but would be more
+  // efficient here.
+  typedef map<const pthread_t, ThreadDescriptor> ThreadCategory;
+
+  // All thread categorys, keyed on the category name.
+  typedef map<string, ThreadCategory> ThreadCategoryMap;
+
+  // Protects thread_categories_ and thread metrics.
+  Mutex lock_;
+
+  // All thread categorys that ever contained a thread, even if empty
+  ThreadCategoryMap thread_categories_;
+
+  // Counters to track all-time total number of threads, and the
+  // current number of running threads.
+  uint64_t threads_started_metric_;
+  uint64_t threads_running_metric_;
+
+  // Metric callbacks.
+  uint64_t ReadThreadsStarted();
+  uint64_t ReadThreadsRunning();
+
+  // Webpage callback; prints all threads by category.
+  void ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
+                         WebCallbackRegistry::PrerenderedWebResponse* resp);
+  void PrintThreadCategoryRows(const ThreadCategory& category, ostringstream* output);
+};
+
+void ThreadMgr::SetThreadName(const string& name, int64_t tid) {
+  // On linux we can get the thread names to show up in the debugger by setting
+  // the process name for the LWP.  We don't want to do this for the main
+  // thread because that would rename the process, causing tools like killall
+  // to stop working.
+  if (tid == getpid()) {
+    return;
+  }
+
+#if defined(__linux__)
+  // http://0pointer.de/blog/projects/name-your-threads.html
+  // Set the name for the LWP (which gets truncated to 15 characters).
+  // Note that glibc also has a 'pthread_setname_np' api, but it may not be
+  // available everywhere and it's only benefit over using prctl directly is
+  // that it can set the name of threads other than the current thread.
+  int err = prctl(PR_SET_NAME, name.c_str());
+#else
+  int err = pthread_setname_np(name.c_str());
+#endif // defined(__linux__)
+  // We expect EPERM failures in sandboxed processes, just ignore those.
+  if (err < 0 && errno != EPERM) {
+    PLOG(ERROR) << "SetThreadName";
+  }
+}
+
+Status ThreadMgr::StartInstrumentation(const scoped_refptr<MetricEntity>& metrics,
+                                       WebCallbackRegistry* web) {
+  MutexLock l(lock_);
+
+  // Use function gauges here so that we can register a unique copy of these metrics in
+  // multiple tservers, even though the ThreadMgr is itself a singleton.
+  metrics->NeverRetire(
+      METRIC_threads_started.InstantiateFunctionGauge(metrics,
+        Bind(&ThreadMgr::ReadThreadsStarted, Unretained(this))));
+  metrics->NeverRetire(
+      METRIC_threads_running.InstantiateFunctionGauge(metrics,
+        Bind(&ThreadMgr::ReadThreadsRunning, Unretained(this))));
+  metrics->NeverRetire(
+      METRIC_cpu_utime.InstantiateFunctionGauge(metrics,
+        Bind(&GetCpuUTime)));
+  metrics->NeverRetire(
+      METRIC_cpu_stime.InstantiateFunctionGauge(metrics,
+        Bind(&GetCpuSTime)));
+  metrics->NeverRetire(
+      METRIC_voluntary_context_switches.InstantiateFunctionGauge(metrics,
+        Bind(&GetVoluntaryContextSwitches)));
+  metrics->NeverRetire(
+      METRIC_involuntary_context_switches.InstantiateFunctionGauge(metrics,
+        Bind(&GetInVoluntaryContextSwitches)));
+
+  if (web) {
+    WebCallbackRegistry::PrerenderedPathHandlerCallback thread_callback =
+        bind<void>(mem_fn(&ThreadMgr::ThreadPathHandler), this, _1, _2);
+    DCHECK_NOTNULL(web)->RegisterPrerenderedPathHandler("/threadz", "Threads", thread_callback,
+                                                        true /* is_styled*/,
+                                                        true /* is_on_nav_bar */);
+  }
+  return Status::OK();
+}
+
+uint64_t ThreadMgr::ReadThreadsStarted() {
+  MutexLock l(lock_);
+  return threads_started_metric_;
+}
+
+uint64_t ThreadMgr::ReadThreadsRunning() {
+  MutexLock l(lock_);
+  return threads_running_metric_;
+}
+
+void ThreadMgr::AddThread(const pthread_t& pthread_id, const string& name,
+    const string& category, int64_t tid) {
+  // These annotations cause TSAN to ignore the synchronization on lock_
+  // without causing the subsequent mutations to be treated as data races
+  // in and of themselves (that's what IGNORE_READS_AND_WRITES does).
+  //
+  // Why do we need them here and in SuperviseThread()? TSAN operates by
+  // observing synchronization events and using them to establish "happens
+  // before" relationships between threads. Where these relationships are
+  // not built, shared state access constitutes a data race. The
+  // synchronization events here, in RemoveThread(), and in
+  // SuperviseThread() may cause TSAN to establish a "happens before"
+  // relationship between thread functors, ignoring potential data races.
+  // The annotations prevent this from happening.
+  ANNOTATE_IGNORE_SYNC_BEGIN();
+  ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN();
+  {
+    MutexLock l(lock_);
+    thread_categories_[category][pthread_id] = ThreadDescriptor(category, name, tid);
+    threads_running_metric_++;
+    threads_started_metric_++;
+  }
+  ANNOTATE_IGNORE_SYNC_END();
+  ANNOTATE_IGNORE_READS_AND_WRITES_END();
+}
+
+void ThreadMgr::RemoveThread(const pthread_t& pthread_id, const string& category) {
+  ANNOTATE_IGNORE_SYNC_BEGIN();
+  ANNOTATE_IGNORE_READS_AND_WRITES_BEGIN();
+  {
+    MutexLock l(lock_);
+    auto category_it = thread_categories_.find(category);
+    DCHECK(category_it != thread_categories_.end());
+    category_it->second.erase(pthread_id);
+    threads_running_metric_--;
+  }
+  ANNOTATE_IGNORE_SYNC_END();
+  ANNOTATE_IGNORE_READS_AND_WRITES_END();
+}
+
+void ThreadMgr::PrintThreadCategoryRows(const ThreadCategory& category,
+    ostringstream* output) {
+  for (const ThreadCategory::value_type& thread : category) {
+    ThreadStats stats;
+    Status status = GetThreadStats(thread.second.thread_id(), &stats);
+    if (!status.ok()) {
+      KLOG_EVERY_N(INFO, 100) << "Could not get per-thread statistics: "
+                              << status.ToString();
+    }
+    (*output) << "<tr><td>" << thread.second.name() << "</td><td>"
+              << (static_cast<double>(stats.user_ns) / 1e9) << "</td><td>"
+              << (static_cast<double>(stats.kernel_ns) / 1e9) << "</td><td>"
+              << (static_cast<double>(stats.iowait_ns) / 1e9) << "</td></tr>";
+  }
+}
+
+void ThreadMgr::ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
+                                  WebCallbackRegistry::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
+  MutexLock l(lock_);
+  vector<const ThreadCategory*> categories_to_print;
+  auto category_name = req.parsed_args.find("group");
+  if (category_name != req.parsed_args.end()) {
+    string group = EscapeForHtmlToString(category_name->second);
+    (*output) << "<h2>Thread Group: " << group << "</h2>" << endl;
+    if (group != "all") {
+      ThreadCategoryMap::const_iterator category = thread_categories_.find(group);
+      if (category == thread_categories_.end()) {
+        (*output) << "Thread group '" << group << "' not found" << endl;
+        return;
+      }
+      categories_to_print.push_back(&category->second);
+      (*output) << "<h3>" << category->first << " : " << category->second.size()
+                << "</h3>";
+    } else {
+      for (const ThreadCategoryMap::value_type& category : thread_categories_) {
+        categories_to_print.push_back(&category.second);
+      }
+      (*output) << "<h3>All Threads : </h3>";
+    }
+
+    (*output) << "<table class='table table-hover table-border'>";
+    (*output) << "<thead><tr><th>Thread name</th><th>Cumulative User CPU(s)</th>"
+              << "<th>Cumulative Kernel CPU(s)</th>"
+              << "<th>Cumulative IO-wait(s)</th></tr></thead>";
+    (*output) << "<tbody>\n";
+
+    for (const ThreadCategory* category : categories_to_print) {
+      PrintThreadCategoryRows(*category, output);
+    }
+    (*output) << "</tbody></table>";
+  } else {
+    (*output) << "<h2>Thread Groups</h2>";
+    (*output) << "<h4>" << threads_running_metric_ << " thread(s) running";
+    (*output) << "<a href='/threadz?group=all'><h3>All Threads</h3>";
+
+    for (const ThreadCategoryMap::value_type& category : thread_categories_) {
+      string category_arg;
+      UrlEncode(category.first, &category_arg);
+      (*output) << "<a href='/threadz?group=" << category_arg << "'><h3>"
+                << category.first << " : " << category.second.size() << "</h3></a>";
+    }
+  }
+}
+
+static void InitThreading() {
+  thread_manager.reset(new ThreadMgr());
+}
+
+Status StartThreadInstrumentation(const scoped_refptr<MetricEntity>& server_metrics,
+                                  WebCallbackRegistry* web) {
+  GoogleOnceInit(&once, &InitThreading);
+  return thread_manager->StartInstrumentation(server_metrics, web);
+}
+
+ThreadJoiner::ThreadJoiner(Thread* thr)
+  : thread_(CHECK_NOTNULL(thr)),
+    warn_after_ms_(kDefaultWarnAfterMs),
+    warn_every_ms_(kDefaultWarnEveryMs),
+    give_up_after_ms_(kDefaultGiveUpAfterMs) {
+}
+
+ThreadJoiner& ThreadJoiner::warn_after_ms(int ms) {
+  warn_after_ms_ = ms;
+  return *this;
+}
+
+ThreadJoiner& ThreadJoiner::warn_every_ms(int ms) {
+  warn_every_ms_ = ms;
+  return *this;
+}
+
+ThreadJoiner& ThreadJoiner::give_up_after_ms(int ms) {
+  give_up_after_ms_ = ms;
+  return *this;
+}
+
+Status ThreadJoiner::Join() {
+  if (Thread::current_thread() &&
+      Thread::current_thread()->tid() == thread_->tid()) {
+    return Status::InvalidArgument("Can't join on own thread", thread_->name_);
+  }
+
+  // Early exit: double join is a no-op.
+  if (!thread_->joinable_) {
+    return Status::OK();
+  }
+
+  int waited_ms = 0;
+  bool keep_trying = true;
+  while (keep_trying) {
+    if (waited_ms >= warn_after_ms_) {
+      LOG(WARNING) << Substitute("Waited for $0ms trying to join with $1 (tid $2)",
+                                 waited_ms, thread_->name_, thread_->tid_);
+    }
+
+    int remaining_before_giveup = MathLimits<int>::kMax;
+    if (give_up_after_ms_ != -1) {
+      remaining_before_giveup = give_up_after_ms_ - waited_ms;
+    }
+
+    int remaining_before_next_warn = warn_every_ms_;
+    if (waited_ms < warn_after_ms_) {
+      remaining_before_next_warn = warn_after_ms_ - waited_ms;
+    }
+
+    if (remaining_before_giveup < remaining_before_next_warn) {
+      keep_trying = false;
+    }
+
+    int wait_for = std::min(remaining_before_giveup, remaining_before_next_warn);
+
+    if (thread_->done_.WaitFor(MonoDelta::FromMilliseconds(wait_for))) {
+      // Unconditionally join before returning, to guarantee that any TLS
+      // has been destroyed (pthread_key_create() destructors only run
+      // after a pthread's user method has returned).
+      int ret = pthread_join(thread_->thread_, NULL);
+      CHECK_EQ(ret, 0);
+      thread_->joinable_ = false;
+      return Status::OK();
+    }
+    waited_ms += wait_for;
+  }
+  return Status::Aborted(strings::Substitute("Timed out after $0ms joining on $1",
+                                             waited_ms, thread_->name_));
+}
+
+Thread::~Thread() {
+  if (joinable_) {
+    int ret = pthread_detach(thread_);
+    CHECK_EQ(ret, 0);
+  }
+}
+
+std::string Thread::ToString() const {
+  return Substitute("Thread $0 (name: \"$1\", category: \"$2\")", tid(), name_, category_);
+}
+
+int64_t Thread::WaitForTid() const {
+  const string log_prefix = Substitute("$0 ($1) ", name_, category_);
+  SCOPED_LOG_SLOW_EXECUTION_PREFIX(WARNING, 500 /* ms */, log_prefix,
+                                   "waiting for new thread to publish its TID");
+  int loop_count = 0;
+  while (true) {
+    int64_t t = Acquire_Load(&tid_);
+    if (t != PARENT_WAITING_TID) return t;
+    boost::detail::yield(loop_count++);
+  }
+}
+
+
+Status Thread::StartThread(const std::string& category, const std::string& name,
+                           const ThreadFunctor& functor, uint64_t flags,
+                           scoped_refptr<Thread> *holder) {
+  TRACE_COUNTER_INCREMENT("threads_started", 1);
+  TRACE_COUNTER_SCOPE_LATENCY_US("thread_start_us");
+  GoogleOnceInit(&once, &InitThreading);
+
+  const string log_prefix = Substitute("$0 ($1) ", name, category);
+  SCOPED_LOG_SLOW_EXECUTION_PREFIX(WARNING, 500 /* ms */, log_prefix, "starting thread");
+
+  // Temporary reference for the duration of this function.
+  scoped_refptr<Thread> t(new Thread(category, name, functor));
+
+  // Optional, and only set if the thread was successfully created.
+  //
+  // We have to set this before we even start the thread because it's
+  // allowed for the thread functor to access 'holder'.
+  if (holder) {
+    *holder = t;
+  }
+
+  t->tid_ = PARENT_WAITING_TID;
+
+  // Add a reference count to the thread since SuperviseThread() needs to
+  // access the thread object, and we have no guarantee that our caller
+  // won't drop the reference as soon as we return. This is dereferenced
+  // in FinishThread().
+  t->AddRef();
+
+  auto cleanup = MakeScopedCleanup([&]() {
+      // If we failed to create the thread, we need to undo all of our prep work.
+      t->tid_ = INVALID_TID;
+      t->Release();
+    });
+
+  if (PREDICT_FALSE(FLAGS_thread_inject_start_latency_ms > 0)) {
+    LOG(INFO) << "Injecting " << FLAGS_thread_inject_start_latency_ms << "ms sleep on thread start";
+    SleepFor(MonoDelta::FromMilliseconds(FLAGS_thread_inject_start_latency_ms));
+  }
+
+  {
+    SCOPED_LOG_SLOW_EXECUTION_PREFIX(WARNING, 500 /* ms */, log_prefix, "creating pthread");
+    SCOPED_WATCH_STACK((flags & NO_STACK_WATCHDOG) ? 0 : 250);
+    int ret = pthread_create(&t->thread_, NULL, &Thread::SuperviseThread, t.get());
+    if (ret) {
+      return Status::RuntimeError("Could not create thread", strerror(ret), ret);
+    }
+  }
+
+  // The thread has been created and is now joinable.
+  //
+  // Why set this in the parent and not the child? Because only the parent
+  // (or someone communicating with the parent) can join, so joinable must
+  // be set before the parent returns.
+  t->joinable_ = true;
+  cleanup.cancel();
+
+  VLOG(2) << "Started thread " << t->tid()<< " - " << category << ":" << name;
+  return Status::OK();
+}
+
+void* Thread::SuperviseThread(void* arg) {
+  Thread* t = static_cast<Thread*>(arg);
+  int64_t system_tid = Thread::CurrentThreadId();
+  PCHECK(system_tid != -1);
+
+  // Take an additional reference to the thread manager, which we'll need below.
+  ANNOTATE_IGNORE_SYNC_BEGIN();
+  shared_ptr<ThreadMgr> thread_mgr_ref = thread_manager;
+  ANNOTATE_IGNORE_SYNC_END();
+
+  // Set up the TLS.
+  //
+  // We could store a scoped_refptr in the TLS itself, but as its
+  // lifecycle is poorly defined, we'll use a bare pointer. We
+  // already incremented the reference count in StartThread.
+  Thread::tls_ = t;
+
+  // Publish our tid to 'tid_', which unblocks any callers waiting in
+  // WaitForTid().
+  Release_Store(&t->tid_, system_tid);
+
+  string name = strings::Substitute("$0-$1", t->name(), system_tid);
+  thread_manager->SetThreadName(name, t->tid_);
+  thread_manager->AddThread(pthread_self(), name, t->category(), t->tid_);
+
+  // FinishThread() is guaranteed to run (even if functor_ throws an
+  // exception) because pthread_cleanup_push() creates a scoped object
+  // whose destructor invokes the provided callback.
+  pthread_cleanup_push(&Thread::FinishThread, t);
+  t->functor_();
+  pthread_cleanup_pop(true);
+
+  return NULL;
+}
+
+void Thread::FinishThread(void* arg) {
+  Thread* t = static_cast<Thread*>(arg);
+
+  // We're here either because of the explicit pthread_cleanup_pop() in
+  // SuperviseThread() or through pthread_exit(). In either case,
+  // thread_manager is guaranteed to be live because thread_mgr_ref in
+  // SuperviseThread() is still live.
+  thread_manager->RemoveThread(pthread_self(), t->category());
+
+  // Signal any Joiner that we're done.
+  t->done_.CountDown();
+
+  VLOG(2) << "Ended thread " << t->tid_ << " - " << t->category() << ":" << t->name();
+  t->Release();
+  // NOTE: the above 'Release' call could be the last reference to 'this',
+  // so 'this' could be destructed at this point. Do not add any code
+  // following here!
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/thread.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/thread.h b/be/src/kudu/util/thread.h
new file mode 100644
index 0000000..dd035f8
--- /dev/null
+++ b/be/src/kudu/util/thread.h
@@ -0,0 +1,373 @@
+// 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.
+//
+// Copied from Impala and adapted to Kudu.
+
+#ifndef KUDU_UTIL_THREAD_H
+#define KUDU_UTIL_THREAD_H
+
+#include <pthread.h>
+#if defined(__linux__)
+#include <syscall.h>
+#else
+#include <sys/syscall.h>
+#endif
+#include <unistd.h>
+
+#include <cstdint>
+#include <string>
+#include <utility>
+
+#include <boost/bind.hpp>     // IWYU pragma: keep
+#include <boost/function.hpp> // IWYU pragma: keep
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class MetricEntity;
+class Thread;
+class WebCallbackRegistry;
+
+// Utility to join on a thread, printing warning messages if it
+// takes too long. For example:
+//
+//   ThreadJoiner(&my_thread, "processing thread")
+//     .warn_after_ms(1000)
+//     .warn_every_ms(5000)
+//     .Join();
+//
+// TODO: would be nice to offer a way to use ptrace() or signals to
+// dump the stack trace of the thread we're trying to join on if it
+// gets stuck. But, after looking for 20 minutes or so, it seems
+// pretty complicated to get right.
+class ThreadJoiner {
+ public:
+  explicit ThreadJoiner(Thread* thread);
+
+  // Start emitting warnings after this many milliseconds.
+  //
+  // Default: 1000 ms.
+  ThreadJoiner& warn_after_ms(int ms);
+
+  // After the warnings after started, emit another warning at the
+  // given interval.
+  //
+  // Default: 1000 ms.
+  ThreadJoiner& warn_every_ms(int ms);
+
+  // If the thread has not stopped after this number of milliseconds, give up
+  // joining on it and return Status::Aborted.
+  //
+  // -1 (the default) means to wait forever trying to join.
+  ThreadJoiner& give_up_after_ms(int ms);
+
+  // Join the thread, subject to the above parameters. If the thread joining
+  // fails for any reason, returns RuntimeError. If it times out, returns
+  // Aborted.
+  Status Join();
+
+ private:
+  enum {
+    kDefaultWarnAfterMs = 1000,
+    kDefaultWarnEveryMs = 1000,
+    kDefaultGiveUpAfterMs = -1 // forever
+  };
+
+  Thread* thread_;
+
+  int warn_after_ms_;
+  int warn_every_ms_;
+  int give_up_after_ms_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadJoiner);
+};
+
+// Thin wrapper around pthread that can register itself with the singleton ThreadMgr
+// (a private class implemented in thread.cc entirely, which tracks all live threads so
+// that they may be monitored via the debug webpages). This class has a limited subset of
+// boost::thread's API. Construction is almost the same, but clients must supply a
+// category and a name for each thread so that they can be identified in the debug web
+// UI. Otherwise, Join() is the only supported method from boost::thread.
+//
+// Each Thread object knows its operating system thread ID (TID), which can be used to
+// attach debuggers to specific threads, to retrieve resource-usage statistics from the
+// operating system, and to assign threads to resource control groups.
+//
+// Threads are shared objects, but in a degenerate way. They may only have
+// up to two referents: the caller that created the thread (parent), and
+// the thread itself (child). Moreover, the only two methods to mutate state
+// (Join() and the destructor) are constrained: the child may not Join() on
+// itself, and the destructor is only run when there's one referent left.
+// These constraints allow us to access thread internals without any locks.
+class Thread : public RefCountedThreadSafe<Thread> {
+ public:
+
+  // Flags passed to Thread::CreateWithFlags().
+  enum CreateFlags {
+    NO_FLAGS = 0,
+
+    // Disable the use of KernelStackWatchdog to detect and log slow
+    // thread creations. This is necessary when starting the kernel stack
+    // watchdog thread itself to avoid reentrancy.
+    NO_STACK_WATCHDOG = 1 << 0
+  };
+
+  // This constructor pattern mimics that in boost::thread. There is
+  // one constructor for each number of arguments that the thread
+  // function accepts. To extend the set of acceptable signatures, add
+  // another constructor with <class F, class A1.... class An>.
+  //
+  // In general:
+  //  - category: string identifying the thread category to which this thread belongs,
+  //    used for organising threads together on the debug UI.
+  //  - name: name of this thread. Will be appended with "-<thread-id>" to ensure
+  //    uniqueness.
+  //  - F - a method type that supports operator(), and the instance passed to the
+  //    constructor is executed immediately in a separate thread.
+  //  - A1...An - argument types whose instances are passed to f(...)
+  //  - holder - optional shared pointer to hold a reference to the created thread.
+  template <class F>
+  static Status CreateWithFlags(const std::string& category, const std::string& name,
+                                const F& f, uint64_t flags,
+                                scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, f, flags, holder);
+
+  }
+  template <class F>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, f, NO_FLAGS, holder);
+  }
+
+  template <class F, class A1>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1), NO_FLAGS, holder);
+  }
+
+  template <class F, class A1, class A2>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, const A2& a2, scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1, a2), NO_FLAGS, holder);
+  }
+
+  template <class F, class A1, class A2, class A3>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, const A2& a2, const A3& a3, scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1, a2, a3), NO_FLAGS, holder);
+  }
+
+  template <class F, class A1, class A2, class A3, class A4>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, const A2& a2, const A3& a3, const A4& a4,
+                       scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1, a2, a3, a4), NO_FLAGS, holder);
+  }
+
+  template <class F, class A1, class A2, class A3, class A4, class A5>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, const A2& a2, const A3& a3, const A4& a4, const A5& a5,
+                       scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1, a2, a3, a4, a5), NO_FLAGS, holder);
+  }
+
+  template <class F, class A1, class A2, class A3, class A4, class A5, class A6>
+  static Status Create(const std::string& category, const std::string& name, const F& f,
+                       const A1& a1, const A2& a2, const A3& a3, const A4& a4, const A5& a5,
+                       const A6& a6, scoped_refptr<Thread>* holder) {
+    return StartThread(category, name, boost::bind(f, a1, a2, a3, a4, a5, a6), NO_FLAGS, holder);
+  }
+
+  // Emulates boost::thread and detaches.
+  ~Thread();
+
+  // Blocks until this thread finishes execution. Once this method returns, the thread
+  // will be unregistered with the ThreadMgr and will not appear in the debug UI.
+  void Join() { ThreadJoiner(this).Join(); }
+
+  // The thread ID assigned to this thread by the operating system. If the thread
+  // has not yet started running, returns INVALID_TID.
+  //
+  // NOTE: this may block for a short amount of time if the thread has just been
+  // started.
+  int64_t tid() const {
+    int64_t t = base::subtle::Acquire_Load(&tid_);
+    if (t != PARENT_WAITING_TID) {
+      return tid_;
+    }
+    return WaitForTid();
+  }
+
+  // Returns the thread's pthread ID.
+  pthread_t pthread_id() const { return thread_; }
+
+  const std::string& name() const { return name_; }
+  const std::string& category() const { return category_; }
+
+  // Return a string representation of the thread identifying information.
+  std::string ToString() const;
+
+  // The current thread of execution, or NULL if the current thread isn't a kudu::Thread.
+  // This call is signal-safe.
+  static Thread* current_thread() { return tls_; }
+
+  // Returns a unique, stable identifier for this thread. Note that this is a static
+  // method and thus can be used on any thread, including the main thread of the
+  // process.
+  //
+  // In general, this should be used when a value is required that is unique to
+  // a thread and must work on any thread including the main process thread.
+  //
+  // NOTE: this is _not_ the TID, but rather a unique value assigned by the
+  // thread implementation. So, this value should not be presented to the user
+  // in log messages, etc.
+  static int64_t UniqueThreadId() {
+#if defined(__linux__)
+    // This cast is a little bit ugly, but it is significantly faster than
+    // calling syscall(SYS_gettid). In particular, this speeds up some code
+    // paths in the tracing implementation.
+    return static_cast<int64_t>(pthread_self());
+#elif defined(__APPLE__)
+    uint64_t tid;
+    CHECK_EQ(0, pthread_threadid_np(NULL, &tid));
+    return tid;
+#else
+#error Unsupported platform
+#endif
+  }
+
+  // Returns the system thread ID (tid on Linux) for the current thread. Note
+  // that this is a static method and thus can be used from any thread,
+  // including the main thread of the process. This is in contrast to
+  // Thread::tid(), which only works on kudu::Threads.
+  //
+  // Thread::tid() will return the same value, but the value is cached in the
+  // Thread object, so will be faster to call.
+  //
+  // Thread::UniqueThreadId() (or Thread::tid()) should be preferred for
+  // performance sensistive code, however it is only guaranteed to return a
+  // unique and stable thread ID, not necessarily the system thread ID.
+  static int64_t CurrentThreadId() {
+#if defined(__linux__)
+    return syscall(SYS_gettid);
+#else
+    return UniqueThreadId();
+#endif
+  }
+
+ private:
+  friend class ThreadJoiner;
+
+  // See 'tid_' docs.
+  enum {
+    INVALID_TID = -1,
+    PARENT_WAITING_TID = -2,
+  };
+
+  // Function object that wraps the user-supplied function to run in a separate thread.
+  typedef boost::function<void ()> ThreadFunctor;
+
+  Thread(std::string category, std::string name, ThreadFunctor functor)
+      : thread_(0),
+        category_(std::move(category)),
+        name_(std::move(name)),
+        tid_(INVALID_TID),
+        functor_(std::move(functor)),
+        done_(1),
+        joinable_(false) {}
+
+  // Library-specific thread ID.
+  pthread_t thread_;
+
+  // Name and category for this thread.
+  const std::string category_;
+  const std::string name_;
+
+  // OS-specific thread ID. Once the constructor finishes StartThread(),
+  // guaranteed to be set either to a non-negative integer, or to INVALID_TID.
+  //
+  // The tid_ member goes through the following states:
+  // 1. INVALID_TID: the thread has not been started, or has already exited.
+  // 2. PARENT_WAITING_TID: the parent has started the thread, but the
+  //    thread has not yet begun running. Therefore the TID is not yet known
+  //    but it will be set once the thread starts.
+  // 3. <positive value>: the thread is running.
+  int64_t tid_;
+
+  // User function to be executed by this thread.
+  const ThreadFunctor functor_;
+
+  // Joiners wait on this latch to be notified if the thread is done.
+  //
+  // Note that Joiners must additionally pthread_join(), otherwise certain
+  // resources that callers expect to be destroyed (like TLS) may still be
+  // alive when a Joiner finishes.
+  CountDownLatch done_;
+
+  bool joinable_;
+
+  // Thread local pointer to the current thread of execution. Will be NULL if the current
+  // thread is not a Thread.
+  static __thread Thread* tls_;
+
+  // Wait for the running thread to publish its tid.
+  int64_t WaitForTid() const;
+
+  // Starts the thread running SuperviseThread(), and returns once that thread has
+  // initialised and its TID has been read. Waits for notification from the started
+  // thread that initialisation is complete before returning. On success, stores a
+  // reference to the thread in holder.
+  static Status StartThread(const std::string& category, const std::string& name,
+                            const ThreadFunctor& functor, uint64_t flags,
+                            scoped_refptr<Thread>* holder);
+
+  // Wrapper for the user-supplied function. Invoked from the new thread,
+  // with the Thread as its only argument. Executes functor_, but before
+  // doing so registers with the global ThreadMgr and reads the thread's
+  // system ID. After functor_ terminates, unregisters with the ThreadMgr.
+  // Always returns NULL.
+  //
+  // SuperviseThread() notifies StartThread() when thread initialisation is
+  // completed via the tid_, which is set to the new thread's system ID.
+  // By that point in time SuperviseThread() has also taken a reference to
+  // the Thread object, allowing it to safely refer to it even after the
+  // caller drops its reference.
+  //
+  // Additionally, StartThread() notifies SuperviseThread() when the actual
+  // Thread object has been assigned (SuperviseThread() is spinning during
+  // this time). Without this, the new thread may reference the actual
+  // Thread object before it has been assigned by StartThread(). See
+  // KUDU-11 for more details.
+  static void* SuperviseThread(void* arg);
+
+  // Invoked when the user-supplied function finishes or in the case of an
+  // abrupt exit (i.e. pthread_exit()). Cleans up after SuperviseThread().
+  static void FinishThread(void* arg);
+};
+
+// Registers /threadz with the debug webserver, and creates thread-tracking metrics under
+// the given entity. If 'web' is NULL, does not register the path handler.
+Status StartThreadInstrumentation(const scoped_refptr<MetricEntity>& server_metrics,
+                                  WebCallbackRegistry* web);
+} // namespace kudu
+
+#endif /* KUDU_UTIL_THREAD_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/thread_restrictions.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/thread_restrictions.cc b/be/src/kudu/util/thread_restrictions.cc
new file mode 100644
index 0000000..f956fd9
--- /dev/null
+++ b/be/src/kudu/util/thread_restrictions.cc
@@ -0,0 +1,87 @@
+// 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 <ostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadlocal.h"
+#include "kudu/util/thread_restrictions.h"
+
+#ifdef ENABLE_THREAD_RESTRICTIONS
+
+namespace kudu {
+
+namespace {
+
+struct LocalThreadRestrictions {
+  LocalThreadRestrictions()
+    : io_allowed(true),
+      wait_allowed(true),
+      singleton_allowed(true) {
+  }
+
+  bool io_allowed;
+  bool wait_allowed;
+  bool singleton_allowed;
+};
+
+LocalThreadRestrictions* LoadTLS() {
+  // Disable leak check. LSAN sometimes gets false positives on thread locals.
+  // See: https://github.com/google/sanitizers/issues/757
+  debug::ScopedLeakCheckDisabler d;
+  BLOCK_STATIC_THREAD_LOCAL(LocalThreadRestrictions, local_thread_restrictions);
+  return local_thread_restrictions;
+}
+
+} // anonymous namespace
+
+bool ThreadRestrictions::SetIOAllowed(bool allowed) {
+  bool previous_allowed = LoadTLS()->io_allowed;
+  LoadTLS()->io_allowed = allowed;
+  return previous_allowed;
+}
+
+void ThreadRestrictions::AssertIOAllowed() {
+  CHECK(LoadTLS()->io_allowed)
+    << "Function marked as IO-only was called from a thread that "
+    << "disallows IO!  If this thread really should be allowed to "
+    << "make IO calls, adjust the call to "
+    << "kudu::ThreadRestrictions::SetIOAllowed() in this thread's "
+    << "startup. "
+    << (Thread::current_thread() ? Thread::current_thread()->ToString() : "(not a kudu::Thread)");
+}
+
+bool ThreadRestrictions::SetWaitAllowed(bool allowed) {
+  bool previous_allowed = LoadTLS()->wait_allowed;
+  LoadTLS()->wait_allowed = allowed;
+  return previous_allowed;
+}
+
+void ThreadRestrictions::AssertWaitAllowed() {
+  CHECK(LoadTLS()->wait_allowed)
+    << "Waiting is not allowed to be used on this thread to prevent "
+    << "server-wide latency aberrations and deadlocks. "
+    << (Thread::current_thread() ? Thread::current_thread()->ToString() : "(not a kudu::Thread)");
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/thread_restrictions.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/thread_restrictions.h b/be/src/kudu/util/thread_restrictions.h
new file mode 100644
index 0000000..23f0cd5
--- /dev/null
+++ b/be/src/kudu/util/thread_restrictions.h
@@ -0,0 +1,121 @@
+// 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.
+//
+// Some portions: Copyright (c) 2012, The Chromium Authors.
+#ifndef KUDU_UTIL_THREAD_RESTRICTIONS_H
+#define KUDU_UTIL_THREAD_RESTRICTIONS_H
+
+#include "kudu/gutil/macros.h"
+
+#ifndef NDEBUG
+#define ENABLE_THREAD_RESTRICTIONS 1
+#endif
+
+namespace kudu {
+
+// Certain behavior is disallowed on certain threads.  ThreadRestrictions helps
+// enforce these rules.  Examples of such rules:
+//
+// * Do not do blocking IO
+// * Do not wait on synchronization variables or sleep
+//
+// Here's more about how the protection works:
+//
+// 1) If a thread should not be allowed to make IO calls, mark it:
+//      ThreadRestrictions::SetIOAllowed(false);
+//    By default, threads *are* allowed to make IO calls.
+//    In particular, threads like RPC reactors should never do blocking IO
+//    because it may stall other unrelated requests.
+//
+// 2) If a function makes a call that will go out to disk, check whether the
+//    current thread is allowed:
+//      ThreadRestrictions::AssertIOAllowed();
+//
+//
+// Style tip: where should you put AssertIOAllowed checks?  It's best
+// if you put them as close to the disk access as possible, at the
+// lowest level.  This rule is simple to follow and helps catch all
+// callers.  For example, if your function GoDoSomeBlockingDiskCall()
+// only calls other functions in Kudu and doesn't access the underlying
+// disk, you should go add the AssertIOAllowed checks in the helper functions.
+class ThreadRestrictions {
+ public:
+  // Constructing a ScopedAllowIO temporarily allows IO for the current
+  // thread.  Doing this is almost certainly always incorrect, but sometimes
+  // it makes more sense to allow an exception and file a bug in the backlog
+  // to improve it later.
+  class ScopedAllowIO {
+   public:
+    ScopedAllowIO() { previous_value_ = SetIOAllowed(true); }
+    ~ScopedAllowIO() { SetIOAllowed(previous_value_); }
+   private:
+    // Whether IO is allowed when the ScopedAllowIO was constructed.
+    bool previous_value_;
+
+    DISALLOW_COPY_AND_ASSIGN(ScopedAllowIO);
+  };
+
+  // Constructing a ScopedAllowWait temporarily allows waiting on the current
+  // thread.  Doing this is almost always incorrect: consider carefully whether
+  // you should instead be deferring work to a different thread.
+  class ScopedAllowWait {
+   public:
+    ScopedAllowWait() { previous_value_ = SetWaitAllowed(true); }
+    ~ScopedAllowWait() { SetWaitAllowed(previous_value_); }
+   private:
+    // Whether singleton use is allowed when the ScopedAllowWait was
+    // constructed.
+    bool previous_value_;
+
+    DISALLOW_COPY_AND_ASSIGN(ScopedAllowWait);
+  };
+
+
+#if ENABLE_THREAD_RESTRICTIONS
+  // Set whether the current thread to make IO calls.
+  // Threads start out in the *allowed* state.
+  // Returns the previous value.
+  static bool SetIOAllowed(bool allowed);
+
+  // Check whether the current thread is allowed to make IO calls,
+  // and FATALs if not.  See the block comment above the class for
+  // a discussion of where to add these checks.
+  static void AssertIOAllowed();
+
+  // Set whether the current thread may wait/block.  Returns the previous
+  // value.
+  static bool SetWaitAllowed(bool allowed);
+
+  // Check whether the current thread is allowed to wait/block.
+  // FATALs if not.
+  static void AssertWaitAllowed();
+#else
+  // Inline the empty definitions of these functions so that they can be
+  // compiled out.
+  static bool SetIOAllowed(bool allowed) { return true; }
+  static void AssertIOAllowed() {}
+  static bool SetWaitAllowed(bool allowed) { return true; }
+  static void AssertWaitAllowed() {}
+#endif
+
+ private:
+  DISALLOW_IMPLICIT_CONSTRUCTORS(ThreadRestrictions);
+};
+
+} // namespace kudu
+
+#endif /* KUDU_UTIL_THREAD_RESTRICTIONS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadlocal.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadlocal.cc b/be/src/kudu/util/threadlocal.cc
new file mode 100644
index 0000000..444ed15
--- /dev/null
+++ b/be/src/kudu/util/threadlocal.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/util/threadlocal.h"
+
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include <pthread.h>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/once.h"
+#include "kudu/util/errno.h"
+
+namespace kudu {
+namespace threadlocal {
+namespace internal {
+
+// One key used by the entire process to attach destructors on thread exit.
+static pthread_key_t destructors_key;
+
+// The above key must only be initialized once per process.
+static GoogleOnceType once = GOOGLE_ONCE_INIT;
+
+namespace {
+
+// List of destructors for all thread locals instantiated on a given thread.
+struct PerThreadDestructorList {
+  void (*destructor)(void*);
+  void* arg;
+  PerThreadDestructorList* next;
+};
+
+} // anonymous namespace
+
+// Call all the destructors associated with all THREAD_LOCAL instances in this
+// thread.
+static void InvokeDestructors(void* t) {
+  PerThreadDestructorList* d = reinterpret_cast<PerThreadDestructorList*>(t);
+  while (d != nullptr) {
+    d->destructor(d->arg);
+    PerThreadDestructorList* next = d->next;
+    delete d;
+    d = next;
+  }
+}
+
+// This key must be initialized only once.
+static void CreateKey() {
+  int ret = pthread_key_create(&destructors_key, &InvokeDestructors);
+  // Linux supports up to 1024 keys, we will use only one for all thread locals.
+  CHECK_EQ(0, ret) << "pthread_key_create() failed, cannot add destructor to thread: "
+      << "error " << ret << ": " << ErrnoToString(ret);
+}
+
+// Adds a destructor to the list.
+void AddDestructor(void (*destructor)(void*), void* arg) {
+  GoogleOnceInit(&once, &CreateKey);
+
+  // Returns NULL if nothing is set yet.
+  std::unique_ptr<PerThreadDestructorList> p(new PerThreadDestructorList());
+  p->destructor = destructor;
+  p->arg = arg;
+  p->next = reinterpret_cast<PerThreadDestructorList*>(pthread_getspecific(destructors_key));
+  int ret = pthread_setspecific(destructors_key, p.release());
+  // The only time this check should fail is if we are out of memory, or if
+  // somehow key creation failed, which should be caught by the above CHECK.
+  CHECK_EQ(0, ret) << "pthread_setspecific() failed, cannot update destructor list: "
+      << "error " << ret << ": " << ErrnoToString(ret);
+}
+
+} // namespace internal
+} // namespace threadlocal
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadlocal.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadlocal.h b/be/src/kudu/util/threadlocal.h
new file mode 100644
index 0000000..ebe1910
--- /dev/null
+++ b/be/src/kudu/util/threadlocal.h
@@ -0,0 +1,128 @@
+// 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.
+#ifndef KUDU_UTIL_THREADLOCAL_H_
+#define KUDU_UTIL_THREADLOCAL_H_
+
+// Block-scoped static thread local implementation.
+//
+// Usage is similar to a C++11 thread_local. The BLOCK_STATIC_THREAD_LOCAL macro
+// defines a thread-local pointer to the specified type, which is lazily
+// instantiated by any thread entering the block for the first time. The
+// constructor for the type T is invoked at macro execution time, as expected,
+// and its destructor is invoked when the corresponding thread's Runnable
+// returns, or when the thread exits.
+//
+// Inspired by Poco <http://pocoproject.org/docs/Poco.ThreadLocal.html>,
+// Andrew Tomazos <http://stackoverflow.com/questions/12049684/>, and
+// the C++11 thread_local API.
+//
+// Example usage:
+//
+// // Invokes a 3-arg constructor on SomeClass:
+// BLOCK_STATIC_THREAD_LOCAL(SomeClass, instance, arg1, arg2, arg3);
+// instance->DoSomething();
+//
+#define BLOCK_STATIC_THREAD_LOCAL(T, t, ...)                                    \
+static __thread T* t;                                                           \
+do {                                                                            \
+  if (PREDICT_FALSE(t == NULL)) {                                               \
+    t = new T(__VA_ARGS__);                                                     \
+    threadlocal::internal::AddDestructor(threadlocal::internal::Destroy<T>, t); \
+  }                                                                             \
+} while (false)
+
+// Class-scoped static thread local implementation.
+//
+// Very similar in implementation to the above block-scoped version, but
+// requires a bit more syntax and vigilance to use properly.
+//
+// DECLARE_STATIC_THREAD_LOCAL(Type, instance_var_) must be placed in the
+// class header, as usual for variable declarations.
+//
+// Because these variables are static, they must also be defined in the impl
+// file with DEFINE_STATIC_THREAD_LOCAL(Type, Classname, instance_var_),
+// which is very much like defining any static member, i.e. int Foo::member_.
+//
+// Finally, each thread must initialize the instance before using it by calling
+// INIT_STATIC_THREAD_LOCAL(Type, instance_var_, ...). This is a cheap
+// call, and may be invoked at the top of any method which may reference a
+// thread-local variable.
+//
+// Due to all of these requirements, you should probably declare TLS members
+// as private.
+//
+// Example usage:
+//
+// // foo.h
+// #include "kudu/utils/file.h"
+// class Foo {
+//  public:
+//   void DoSomething(std::string s);
+//  private:
+//   DECLARE_STATIC_THREAD_LOCAL(utils::File, file_);
+// };
+//
+// // foo.cc
+// #include "kudu/foo.h"
+// DEFINE_STATIC_THREAD_LOCAL(utils::File, Foo, file_);
+// void Foo::WriteToFile(std::string s) {
+//   // Call constructor if necessary.
+//   INIT_STATIC_THREAD_LOCAL(utils::File, file_, "/tmp/file_location.txt");
+//   file_->Write(s);
+// }
+
+// Goes in the class declaration (usually in a header file).
+// dtor must be destructed _after_ t, so it gets defined first.
+// Uses a mangled variable name for dtor since it must also be a member of the
+// class.
+#define DECLARE_STATIC_THREAD_LOCAL(T, t)                                                     \
+static __thread T* t
+
+// You must also define the instance in the .cc file.
+#define DEFINE_STATIC_THREAD_LOCAL(T, Class, t)                                               \
+__thread T* Class::t
+
+// Must be invoked at least once by each thread that will access t.
+#define INIT_STATIC_THREAD_LOCAL(T, t, ...)                                       \
+do {                                                                              \
+  if (PREDICT_FALSE(t == NULL)) {                                                 \
+    t = new T(__VA_ARGS__);                                                       \
+    threadlocal::internal::AddDestructor(threadlocal::internal::Destroy<T>, t);   \
+  }                                                                               \
+} while (false)
+
+// Internal implementation below.
+
+namespace kudu {
+namespace threadlocal {
+namespace internal {
+
+// Add a destructor to the list.
+void AddDestructor(void (*destructor)(void*), void* arg);
+
+// Destroy the passed object of type T.
+template<class T>
+static void Destroy(void* t) {
+  // With tcmalloc, this should be pretty cheap (same thread as new).
+  delete reinterpret_cast<T*>(t);
+}
+
+} // namespace internal
+} // namespace threadlocal
+} // namespace kudu
+
+#endif // KUDU_UTIL_THREADLOCAL_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/threadlocal_cache.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/threadlocal_cache.h b/be/src/kudu/util/threadlocal_cache.h
new file mode 100644
index 0000000..e9ab3c2
--- /dev/null
+++ b/be/src/kudu/util/threadlocal_cache.h
@@ -0,0 +1,110 @@
+// 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.
+
+#pragma once
+
+#include "kudu/util/threadlocal.h"
+
+#include <boost/optional/optional.hpp>
+#include <array>
+#include <memory>
+#include <utility>
+
+namespace kudu {
+
+// A small thread-local cache for arbitrary objects.
+//
+// This can be used as a contention-free "lookaside" type cache for frequently-accessed
+// objects to avoid having to go to a less-efficient centralized cache.
+//
+// 'Key' must be copyable, and comparable using operator==().
+// 'T' has no particular requirements.
+template<class Key, class T>
+class ThreadLocalCache {
+ public:
+  // The number of entries in the cache.
+  // NOTE: this should always be a power of two for good performance, so that the
+  // compiler can optimize the modulo operations into bit-mask operations.
+  static constexpr int kItemCapacity = 4;
+
+  // Look up a key in the cache. Returns either the existing entry with this key,
+  // or nullptr if no entry matched.
+  T* Lookup(const Key& key) {
+    // Our cache is so small that a linear search is likely to be more efficient than
+    // any kind of actual hashing. We always start the search at wherever we most
+    // recently found a hit.
+    for (int i = 0; i < kItemCapacity; i++) {
+      int idx = (last_hit_ + i) % kItemCapacity;
+      auto& p = cache_[idx];
+      if (p.first == key) {
+        last_hit_ = idx;
+        return p.second.get_ptr();
+      }
+    }
+    return nullptr;
+  }
+
+  // Insert a new entry into the cache. If the cache is full (as it usually is in the
+  // steady state), this replaces one of the existing entries. The 'args' are forwarded
+  // to T's constructor.
+  //
+  // NOTE: entries returned by a previous call to Lookup() may possibly be invalidated
+  // by this function.
+  template<typename ... Args>
+  T* EmplaceNew(const Key& key, Args&&... args) {
+    auto& p = cache_[next_slot_++ % kItemCapacity];
+    p.second.emplace(std::forward<Args>(args)...);
+    p.first = key;
+    return p.second.get_ptr();
+  }
+
+  // Get the the cache instance for this thread, creating it if it has not yet been
+  // created.
+  //
+  // The instance is automatically deleted and any cached items destructed when the
+  // thread exits.
+  static ThreadLocalCache* GetInstance() {
+    INIT_STATIC_THREAD_LOCAL(ThreadLocalCache, tl_instance_);
+    return tl_instance_;
+  }
+
+ private:
+  using EntryPair = std::pair<Key, boost::optional<T>>;
+  std::array<EntryPair, kItemCapacity> cache_;
+
+  // The next slot that we will write into. We always modulo this by the capacity
+  // before use.
+  uint8_t next_slot_ = 0;
+  // The slot where we last got a cache hit, so we can start our search at the same
+  // spot, optimizing for the case of repeated lookups of the same hot element.
+  uint8_t last_hit_ = 0;
+
+  static_assert(kItemCapacity <= 1 << (sizeof(next_slot_) * 8),
+                "next_slot_ must be large enough for capacity");
+  static_assert(kItemCapacity <= 1 << (sizeof(last_hit_) * 8),
+                "last_hit_ must be large enough for capacity");
+
+  DECLARE_STATIC_THREAD_LOCAL(ThreadLocalCache, tl_instance_);
+};
+
+// Define the thread-local storage for the ThreadLocalCache template.
+// We can't use DEFINE_STATIC_THREAD_LOCAL here because the commas in the
+// template arguments confuse the C preprocessor.
+template<class K, class T>
+__thread ThreadLocalCache<K,T>* ThreadLocalCache<K,T>::tl_instance_;
+
+} // namespace kudu


[51/51] [abbrv] impala git commit: IMPALA-7288: Fix Codegen Crash in FinalizeModule()

Posted by ta...@apache.org.
IMPALA-7288: Fix Codegen Crash in FinalizeModule()

Currently codegen crashed during FinalizeModule() where it tries to
clean up half-baked handcrafted functions. This happens only for the
cases where the code generating the handcrafted IR calls
eraseFromParent() on failure which also deletes the memory held by the
function pointer and therefore causes a crash during clean up in
FinalizeModule().

Testing:
Added regression tests that verify that failure code paths in
the previously offending methods don't crash Impala.

Change-Id: I2f0b527909a9fb3090996bb7510e4d58350c21b0
Reviewed-on: http://gerrit.cloudera.org:8080/10933
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/540611e8
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/540611e8
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/540611e8

Branch: refs/heads/master
Commit: 540611e863fe99b3d3ae35f8b94a745a68b9eba2
Parents: e2aafae
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Thu Jul 12 14:43:07 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Fri Jul 13 04:07:19 2018 +0000

----------------------------------------------------------------------
 be/src/exec/hash-table.cc        |  2 --
 be/src/exec/hdfs-avro-scanner.cc |  1 -
 be/src/exec/hdfs-scanner.cc      |  1 -
 tests/query_test/test_codegen.py | 17 +++++++++++++++++
 4 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/540611e8/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index 38e0d26..47a9a72 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -763,7 +763,6 @@ Status HashTableCtx::CodegenEvalRow(
     llvm::Function* expr_fn;
     Status status = exprs[i]->GetCodegendComputeFn(codegen, &expr_fn);
     if (!status.ok()) {
-      (*fn)->eraseFromParent(); // deletes function
       *fn = NULL;
       return Status(Substitute(
           "Problem with HashTableCtx::CodegenEvalRow(): $0", status.GetDetail()));
@@ -1113,7 +1112,6 @@ Status HashTableCtx::CodegenEquals(
     llvm::Function* expr_fn;
     Status status = build_exprs_[i]->GetCodegendComputeFn(codegen, &expr_fn);
     if (!status.ok()) {
-      (*fn)->eraseFromParent(); // deletes function
       *fn = NULL;
       return Status(
           Substitute("Problem with HashTableCtx::CodegenEquals: $0", status.GetDetail()));

http://git-wip-us.apache.org/repos/asf/impala/blob/540611e8/be/src/exec/hdfs-avro-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.cc b/be/src/exec/hdfs-avro-scanner.cc
index 9a92763..51d4ed6 100644
--- a/be/src/exec/hdfs-avro-scanner.cc
+++ b/be/src/exec/hdfs-avro-scanner.cc
@@ -841,7 +841,6 @@ Status HdfsAvroScanner::CodegenMaterializeTuple(const HdfsScanNodeBase* node,
         bail_out_block, this_val, pool_val, tuple_val, data_val, data_end_val);
     if (!status.ok()) {
       VLOG_QUERY << status.GetDetail();
-      helper_fn->eraseFromParent();
       return status;
     }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/540611e8/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index da44e9a..8947138 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -491,7 +491,6 @@ Status HdfsScanner::CodegenWriteCompleteTuple(const HdfsScanNodeBase* node,
         stringstream ss;
         ss << "Failed to codegen conjunct: " << status.GetDetail();
         state->LogError(ErrorMsg(TErrorCode::GENERAL, ss.str()));
-        fn->eraseFromParent();
         return status;
       }
       if (node->materialized_slots().size() + conjunct_idx

http://git-wip-us.apache.org/repos/asf/impala/blob/540611e8/tests/query_test/test_codegen.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_codegen.py b/tests/query_test/test_codegen.py
index a4c02f5..5bc0cc6 100644
--- a/tests/query_test/test_codegen.py
+++ b/tests/query_test/test_codegen.py
@@ -56,3 +56,20 @@ class TestCodegen(ImpalaTestSuite):
   def test_datastream_sender_codegen(self, vector):
     """Test the KrpcDataStreamSender's codegen logic"""
     self.run_test_case('QueryTest/datastream-sender-codegen', vector)
+
+  def test_codegen_failure_for_char_type(self, vector):
+    """IMPALA-7288: Regression tests for the codegen failure path when working with a
+    CHAR column type"""
+    # Test failure path in HashTableCtx::CodegenEquals().
+    result = self.execute_query("select 1 from functional.chars_tiny t1, "
+                                "functional.chars_tiny t2 "
+                                "where t1.cs = cast(t2.cs as string)");
+    assert "Codegen Disabled: Problem with HashTableCtx::CodegenEquals: ScalarFnCall" \
+           " Codegen not supported for CHAR" in str(result.runtime_profile)
+
+    # Test failure path in HashTableCtx::CodegenEvalRow().
+    result = self.execute_query("select 1 from functional.chars_tiny t1, "
+                                "functional.chars_tiny t2 where t1.cs = "
+                                "FROM_TIMESTAMP(cast(t2.cs as string), 'yyyyMMdd')");
+    assert "Codegen Disabled: Problem with HashTableCtx::CodegenEvalRow(): ScalarFnCall" \
+           " Codegen not supported for CHAR" in str(result.runtime_profile)


[14/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/memory.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/memory.h b/be/src/kudu/util/memory/memory.h
new file mode 100644
index 0000000..315631b
--- /dev/null
+++ b/be/src/kudu/util/memory/memory.h
@@ -0,0 +1,970 @@
+// Copyright 2010 Google Inc.  All Rights Reserved
+//
+// 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.
+//
+//
+// Classes for memory management, used by materializations
+// (arenas, segments, and STL collections parametrized via arena allocators)
+// so that memory usage can be controlled at the application level.
+//
+// Materializations can be parametrized by specifying an instance of a
+// BufferAllocator. The allocator implements
+// memory management policy (e.g. setting allocation limits). Allocators may
+// be shared between multiple materializations; e.g. you can designate a
+// single allocator per a single user request, thus setting bounds on memory
+// usage on a per-request basis.
+
+#ifndef KUDU_UTIL_MEMORY_MEMORY_H_
+#define KUDU_UTIL_MEMORY_MEMORY_H_
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <ostream>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/util/boost_mutex_utils.h"
+#include "kudu/util/memory/overwrite.h"
+#include "kudu/util/mutex.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/singleton.h"
+
+namespace kudu {
+
+class BufferAllocator;
+class MemTracker;
+
+// Wrapper for a block of data allocated by a BufferAllocator. Owns the block.
+// (To release the block, destroy the buffer - it will then return it via the
+// same allocator that has been used to create it).
+class Buffer {
+ public:
+  ~Buffer();
+
+  void* data() const { return data_; }   // The data buffer.
+  size_t size() const { return size_; }  // In bytes.
+
+ private:
+  friend class BufferAllocator;
+
+  Buffer(void* data, size_t size, BufferAllocator* allocator)
+      : data_(CHECK_NOTNULL(data)),
+        size_(size),
+        allocator_(allocator) {
+#ifndef NDEBUG
+    OverwriteWithPattern(reinterpret_cast<char*>(data_), size_,
+                         "NEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEW"
+                         "NEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEW"
+                         "NEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEWNEW");
+#endif
+  }
+
+  // Called by a successful realloc.
+  void Update(void* new_data, size_t new_size) {
+#ifndef NDEBUG
+    if (new_size > size_) {
+      OverwriteWithPattern(reinterpret_cast<char*>(new_data) + size_,
+                           new_size - size_, "NEW");
+    }
+#endif
+    data_ = new_data;
+    size_ = new_size;
+  }
+
+  void* data_;
+  size_t size_;
+  BufferAllocator* const allocator_;
+  DISALLOW_COPY_AND_ASSIGN(Buffer);
+};
+
+// Allocators allow applications to control memory usage. They are
+// used by materializations to allocate blocks of memory arenas.
+// BufferAllocator is an abstract class that defines a common contract of
+// all implementations of allocators. Specific allocators provide specific
+// features, e.g. enforced resource limits, thread safety, etc.
+class BufferAllocator {
+ public:
+  virtual ~BufferAllocator() {}
+
+  // Called by the user when a new block of memory is needed. The 'requested'
+  // parameter specifies how much memory (in bytes) the user would like to get.
+  // The 'minimal' parameter specifies how much he is willing to settle for.
+  // The allocator returns a buffer sized in the range [minimal, requested],
+  // or NULL if the request can't be satisfied. When the buffer is destroyed,
+  // its destructor calls the FreeInternal() method on its allocator.
+  // CAVEAT: The allocator must outlive all buffers returned by it.
+  //
+  // Corner cases:
+  // 1. If requested == 0, the allocator will always return a non-NULL Buffer
+  //    with a non-NULL data pointer and zero capacity.
+  // 2. If minimal == 0, the allocator will always return a non-NULL Buffer
+  //    with a non-NULL data pointer, possibly with zero capacity.
+  Buffer* BestEffortAllocate(size_t requested, size_t minimal) {
+    DCHECK_LE(minimal, requested);
+    Buffer* result = AllocateInternal(requested, minimal, this);
+    LogAllocation(requested, minimal, result);
+    return result;
+  }
+
+  // Called by the user when a new block of memory is needed. Equivalent to
+  // BestEffortAllocate(requested, requested).
+  Buffer* Allocate(size_t requested) {
+    return BestEffortAllocate(requested, requested);
+  }
+
+  // Called by the user when a previously allocated block needs to be resized.
+  // Mimics semantics of <cstdlib> realloc. The 'requested' and 'minimal'
+  // represent the desired final buffer size, with semantics as in the Allocate.
+  // If the 'buffer' parameter is NULL, the call is equivalent to
+  // Allocate(requested, minimal). Otherwise, a reallocation of the buffer's
+  // data is attempted. On success, the original 'buffer' parameter is returned,
+  // but the buffer itself might have updated size and data. On failure,
+  // returns NULL, and leaves the input buffer unmodified.
+  // Reallocation might happen in-place, preserving the original data
+  // pointer, but it is not guaranteed - e.g. this function might degenerate to
+  // Allocate-Copy-Free. Either way, the content of the data buffer, up to the
+  // minimum of the new and old size, is preserved.
+  //
+  // Corner cases:
+  // 1. If requested == 0, the allocator will always return a non-NULL Buffer
+  //    with a non-NULL data pointer and zero capacity.
+  // 2. If minimal == 0, the allocator will always return a non-NULL Buffer
+  //    with a non-NULL data pointer, possibly with zero capacity.
+  Buffer* BestEffortReallocate(size_t requested,
+                               size_t minimal,
+                               Buffer* buffer) {
+    DCHECK_LE(minimal, requested);
+    Buffer* result;
+    if (buffer == NULL) {
+      result = AllocateInternal(requested, minimal, this);
+      LogAllocation(requested, minimal, result);
+      return result;
+    } else {
+      result =  ReallocateInternal(requested, minimal, buffer, this) ?
+          buffer : NULL;
+      LogAllocation(requested, minimal, buffer);
+      return result;
+    }
+  }
+
+  // Called by the user when a previously allocated block needs to be resized.
+  // Equivalent to BestEffortReallocate(requested, requested, buffer).
+  Buffer* Reallocate(size_t requested, Buffer* buffer) {
+    return BestEffortReallocate(requested, requested, buffer);
+  }
+
+  // Returns the amount of memory (in bytes) still available for this allocator.
+  // For unbounded allocators (like raw HeapBufferAllocator) this is the highest
+  // size_t value possible.
+  // TODO(user): consider making pure virtual.
+  virtual size_t Available() const { return std::numeric_limits<size_t>::max(); }
+
+ protected:
+  friend class Buffer;
+
+  BufferAllocator() {}
+
+  // Expose the constructor to subclasses of BufferAllocator.
+  Buffer* CreateBuffer(void* data,
+                       size_t size,
+                       BufferAllocator* allocator) {
+    return new Buffer(data, size, allocator);
+  }
+
+  // Expose Buffer::Update to subclasses of BufferAllocator.
+  void UpdateBuffer(void* new_data, size_t new_size, Buffer* buffer) {
+    buffer->Update(new_data, new_size);
+  }
+
+  // Called by chained buffer allocators.
+  Buffer* DelegateAllocate(BufferAllocator* delegate,
+                           size_t requested,
+                           size_t minimal,
+                           BufferAllocator* originator) {
+    return delegate->AllocateInternal(requested, minimal, originator);
+  }
+
+  // Called by chained buffer allocators.
+  bool DelegateReallocate(BufferAllocator* delegate,
+                          size_t requested,
+                          size_t minimal,
+                          Buffer* buffer,
+                          BufferAllocator* originator) {
+    return delegate->ReallocateInternal(requested, minimal, buffer, originator);
+  }
+
+  // Called by chained buffer allocators.
+  void DelegateFree(BufferAllocator* delegate, Buffer* buffer) {
+    delegate->FreeInternal(buffer);
+  }
+
+ private:
+  // Implemented by concrete subclasses.
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) = 0;
+
+  // Implemented by concrete subclasses. Returns false on failure.
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) = 0;
+
+  // Implemented by concrete subclasses.
+  virtual void FreeInternal(Buffer* buffer) = 0;
+
+  // Logs a warning message if the allocation failed or if it returned less than
+  // the required number of bytes.
+  void LogAllocation(size_t required, size_t minimal, Buffer* buffer);
+
+  DISALLOW_COPY_AND_ASSIGN(BufferAllocator);
+};
+
+// Allocates buffers on the heap, with no memory limits. Uses standard C
+// allocation functions (malloc, realloc, free).
+class HeapBufferAllocator : public BufferAllocator {
+ public:
+  virtual ~HeapBufferAllocator() {}
+
+  // Returns a singleton instance of the heap allocator.
+  static HeapBufferAllocator* Get() {
+    return Singleton<HeapBufferAllocator>::get();
+  }
+
+  virtual size_t Available() const OVERRIDE {
+    return std::numeric_limits<size_t>::max();
+  }
+
+ private:
+  // Allocates memory that is aligned to 16 way.
+  // Use if you want to boost SIMD operations on the memory area.
+  const bool aligned_mode_;
+
+  friend class Singleton<HeapBufferAllocator>;
+
+  // Always allocates 'requested'-sized buffer, or returns NULL on OOM.
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE;
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE;
+
+  void* Malloc(size_t size);
+  void* Realloc(void* previousData, size_t previousSize, size_t newSize);
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE;
+
+  HeapBufferAllocator();
+  explicit HeapBufferAllocator(bool aligned_mode)
+      : aligned_mode_(aligned_mode) {}
+
+  DISALLOW_COPY_AND_ASSIGN(HeapBufferAllocator);
+};
+
+// Wrapper around the delegate allocator, that clears all newly allocated
+// (and reallocated) memory.
+class ClearingBufferAllocator : public BufferAllocator {
+ public:
+  // Does not take ownership of the delegate.
+  explicit ClearingBufferAllocator(BufferAllocator* delegate)
+      : delegate_(delegate) {}
+
+  virtual size_t Available() const OVERRIDE {
+    return delegate_->Available();
+  }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE;
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE;
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE;
+
+  BufferAllocator* delegate_;
+  DISALLOW_COPY_AND_ASSIGN(ClearingBufferAllocator);
+};
+
+// Abstract policy for modifying allocation requests - e.g. enforcing quotas.
+class Mediator {
+ public:
+  Mediator() {}
+  virtual ~Mediator() {}
+
+  // Called by an allocator when a allocation request is processed.
+  // Must return a value in the range [minimal, requested], or zero. Returning
+  // zero (if minimal is non-zero) indicates denial to allocate. Returning
+  // non-zero indicates that the request should be capped at that value.
+  virtual size_t Allocate(size_t requested, size_t minimal) = 0;
+
+  // Called by an allocator when the specified amount (in bytes) is released.
+  virtual void Free(size_t amount) = 0;
+
+  // TODO(user): consider making pure virtual.
+  virtual size_t Available() const { return std::numeric_limits<size_t>::max(); }
+};
+
+// Optionally thread-safe skeletal implementation of a 'quota' abstraction,
+// providing methods to allocate resources against the quota, and return them.
+template<bool thread_safe>
+class Quota : public Mediator {
+ public:
+  explicit Quota(bool enforced) : usage_(0), enforced_(enforced) {}
+  virtual ~Quota() {}
+
+  // Returns a value in range [minimal, requested] if not exceeding remaining
+  // quota or if the quota is not enforced (soft quota), and adjusts the usage
+  // value accordingly.  Otherwise, returns zero. The semantics of 'remaining
+  // quota' are defined by subclasses (that must supply GetQuotaInternal()
+  // method).
+  virtual size_t Allocate(size_t requested, size_t minimal) OVERRIDE;
+
+  virtual void Free(size_t amount) OVERRIDE;
+
+  // Returns memory still available in the quota. For unenforced Quota objects,
+  // you are still able to perform _minimal_ allocations when the available
+  // quota is 0 (or less than "minimal" param).
+  virtual size_t Available() const OVERRIDE {
+    lock_guard_maybe<Mutex> lock(Quota<thread_safe>::mutex());
+    const size_t quota = GetQuotaInternal();
+    return (usage_ >= quota) ? 0 : (quota - usage_);
+  }
+
+  // Returns the current quota value.
+  size_t GetQuota() const;
+
+  // Returns the current usage value, defined as a sum of all the values
+  // granted by calls to Allocate, less these released via calls to Free.
+  size_t GetUsage() const;
+
+  bool enforced() const {
+    return enforced_;
+  }
+
+ protected:
+  // Overridden by specific implementations, to define semantics of
+  // the quota, i.e. the total amount of resources that the mediator will
+  // allocate. Called directly from GetQuota that optionally provides
+  // thread safety. An 'Allocate' request will succeed if
+  // GetUsage() + minimal <= GetQuota() or if the quota is not enforced (soft
+  // quota).
+  virtual size_t GetQuotaInternal() const = 0;
+
+  Mutex* mutex() const { return thread_safe ? &mutex_ : NULL; }
+
+ private:
+  mutable Mutex mutex_;
+  size_t usage_;
+  bool enforced_;
+  DISALLOW_COPY_AND_ASSIGN(Quota);
+};
+
+// Optionally thread-safe static quota implementation (where quota is explicitly
+// set to a concrete numeric value).
+template<bool thread_safe>
+class StaticQuota : public Quota<thread_safe> {
+ public:
+  explicit StaticQuota(size_t quota)
+      : Quota<thread_safe>(true) {
+    SetQuota(quota);
+  }
+  StaticQuota(size_t quota, bool enforced)
+      : Quota<thread_safe>(enforced) {
+    SetQuota(quota);
+  }
+  virtual ~StaticQuota() {}
+
+  // Sets quota to the new value.
+  void SetQuota(const size_t quota);
+
+ protected:
+  virtual size_t GetQuotaInternal() const { return quota_; }
+
+ private:
+  size_t quota_;
+  DISALLOW_COPY_AND_ASSIGN(StaticQuota);
+};
+
+// Places resource limits on another allocator, using the specified Mediator
+// (e.g. quota) implementation.
+//
+// If the mediator and the delegate allocator are thread-safe, this allocator
+// is also thread-safe, to the extent that it will not introduce any
+// state inconsistencies. However, without additional synchronization,
+// allocation requests are not atomic end-to-end. This way, it is deadlock-
+// resilient (even if you have cyclic relationships between allocators) and
+// allows better concurrency. But, it may cause over-conservative
+// allocations under memory contention, if you have multiple levels of
+// mediating allocators. For example, if two requests that can't both be
+// satisfied are submitted concurrently, it may happen that one of them succeeds
+// but gets smaller buffer allocated than it would if the requests were strictly
+// ordered. This is usually not a problem, however, as you don't really want to
+// operate so close to memory limits that some of your allocations can't be
+// satisfied. If you do have a simple, cascading graph of allocators though,
+// and want to force requests be atomic end-to-end, put a
+// ThreadSafeBufferAllocator at the entry point.
+class MediatingBufferAllocator : public BufferAllocator {
+ public:
+  // Does not take ownership of the delegate, nor the mediator, allowing
+  // both to be reused.
+  MediatingBufferAllocator(BufferAllocator* const delegate,
+                           Mediator* const mediator)
+      : delegate_(delegate),
+        mediator_(mediator) {}
+
+  virtual ~MediatingBufferAllocator() {}
+
+  virtual size_t Available() const OVERRIDE {
+    return std::min(delegate_->Available(), mediator_->Available());
+  }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE;
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE;
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE;
+
+  BufferAllocator* delegate_;
+  Mediator* const mediator_;
+};
+
+// Convenience non-thread-safe static memory bounds enforcer.
+// Combines MediatingBufferAllocator with a StaticQuota.
+class MemoryLimit : public BufferAllocator {
+ public:
+  // Creates a limiter based on the default, heap allocator. Quota is infinite.
+  // (Can be set using SetQuota).
+  MemoryLimit()
+      : quota_(std::numeric_limits<size_t>::max()),
+        allocator_(HeapBufferAllocator::Get(), &quota_) {}
+
+  // Creates a limiter based on the default, heap allocator.
+  explicit MemoryLimit(size_t quota)
+      : quota_(quota),
+        allocator_(HeapBufferAllocator::Get(), &quota_) {}
+
+  // Creates a limiter relaying to the specified delegate allocator.
+  MemoryLimit(size_t quota, BufferAllocator* const delegate)
+      : quota_(quota),
+        allocator_(delegate, &quota_) {}
+
+  // Creates a (possibly non-enforcing) limiter relaying to the specified
+  // delegate allocator.
+  MemoryLimit(size_t quota, bool enforced, BufferAllocator* const delegate)
+      : quota_(quota, enforced),
+        allocator_(delegate, &quota_) {}
+
+  virtual ~MemoryLimit() {}
+
+  virtual size_t Available() const OVERRIDE {
+    return allocator_.Available();
+  }
+
+  size_t GetQuota() const { return quota_.GetQuota(); }
+  size_t GetUsage() const { return quota_.GetUsage(); }
+  void SetQuota(const size_t quota) { quota_.SetQuota(quota); }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE {
+    return DelegateAllocate(&allocator_, requested, minimal, originator);
+  }
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE {
+    return DelegateReallocate(&allocator_, requested, minimal, buffer,
+                              originator);
+  }
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE {
+    DelegateFree(&allocator_, buffer);
+  }
+
+  StaticQuota<false> quota_;
+  MediatingBufferAllocator allocator_;
+};
+
+// An allocator that allows to bypass the (potential) soft quota below for a
+// given amount of memory usage. The goal is to make the allocation methods and
+// Available() work as if the allocator below had at least bypassed_amount of
+// soft quota. Of course this class doesn't allow to exceed the hard quota.
+class SoftQuotaBypassingBufferAllocator : public BufferAllocator {
+ public:
+  SoftQuotaBypassingBufferAllocator(BufferAllocator* allocator,
+                                    size_t bypassed_amount)
+      : allocator_(std::numeric_limits<size_t>::max(), allocator),
+        bypassed_amount_(bypassed_amount) {}
+
+  virtual size_t Available() const OVERRIDE {
+    const size_t usage = allocator_.GetUsage();
+    size_t available = allocator_.Available();
+    if (bypassed_amount_ > usage) {
+      available = std::max(bypassed_amount_ - usage, available);
+    }
+    return available;
+  }
+
+ private:
+  // Calculates how much to increase the minimal parameter to allocate more
+  // aggressively in the underlying allocator. This is to avoid getting only
+  // very small allocations when we exceed the soft quota below. The request
+  // with increased minimal size is more likely to fail because of exceeding
+  // hard quota, so we also fall back to the original minimal size.
+  size_t AdjustMinimal(size_t requested, size_t minimal) const {
+    return std::min(requested, std::max(minimal, Available()));
+  }
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE {
+    // Try increasing the "minimal" parameter to allocate more aggresively
+    // within the bypassed amount of soft quota.
+    Buffer* result = DelegateAllocate(&allocator_,
+                                      requested,
+                                      AdjustMinimal(requested, minimal),
+                                      originator);
+    if (result != NULL) {
+      return result;
+    } else {
+      return DelegateAllocate(&allocator_,
+                              requested,
+                              minimal,
+                              originator);
+    }
+  }
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE {
+    if (DelegateReallocate(&allocator_,
+                           requested,
+                           AdjustMinimal(requested, minimal),
+                           buffer,
+                           originator)) {
+      return true;
+    } else {
+      return DelegateReallocate(&allocator_,
+                                requested,
+                                minimal,
+                                buffer,
+                                originator);
+    }
+  }
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE {
+    DelegateFree(&allocator_, buffer);
+  }
+
+  // Using MemoryLimit with "infinite" limit to get GetUsage().
+  MemoryLimit allocator_;
+  size_t bypassed_amount_;
+};
+
+// An interface for a MemoryStatisticsCollector - an object which collects
+// information about the memory usage of the allocator. The collector will
+// gather statistics about memory usage based on information received from the
+// allocator.
+class MemoryStatisticsCollectorInterface {
+ public:
+  MemoryStatisticsCollectorInterface() {}
+
+  virtual ~MemoryStatisticsCollectorInterface() {}
+
+  // Informs the collector that the allocator granted bytes memory. Note that in
+  // the case of reallocation bytes should be the increase in total memory
+  // usage, not the total size of the buffer after reallocation.
+  virtual void AllocatedMemoryBytes(size_t bytes) = 0;
+
+  // Informs the collector that the allocator received a request for at least
+  // bytes memory, and rejected it (meaning that it granted nothing).
+  virtual void RefusedMemoryBytes(size_t bytes) = 0;
+
+  // Informs the collector that bytes memory have been released to the
+  // allocator.
+  virtual void FreedMemoryBytes(size_t bytes) = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(MemoryStatisticsCollectorInterface);
+};
+
+class MemoryStatisticsCollectingBufferAllocator : public BufferAllocator {
+ public:
+  // Does not take ownership of the delegate.
+  // Takes ownership of memory_stats_collector.
+  MemoryStatisticsCollectingBufferAllocator(
+      BufferAllocator* const delegate,
+      MemoryStatisticsCollectorInterface* const memory_stats_collector)
+      : delegate_(delegate),
+        memory_stats_collector_(memory_stats_collector) {}
+
+  virtual ~MemoryStatisticsCollectingBufferAllocator() {}
+
+  virtual size_t Available() const OVERRIDE {
+    return delegate_->Available();
+  }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE;
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE;
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE;
+
+  BufferAllocator* delegate_;
+  gscoped_ptr<MemoryStatisticsCollectorInterface>
+      memory_stats_collector_;
+};
+
+// BufferAllocator which uses MemTracker to keep track of and optionally
+// (if a limit is set on the MemTracker) regulate memory consumption.
+class MemoryTrackingBufferAllocator : public BufferAllocator {
+ public:
+  // Does not take ownership of the delegate. The delegate must remain
+  // valid for the lifetime of this allocator. Increments reference
+  // count for 'mem_tracker'.
+  // If 'mem_tracker' has a limit and 'enforce_limit' is true, then
+  // the classes calling this buffer allocator (whether directly, or
+  // through an Arena) must be able to handle the case when allocation
+  // fails. If 'enforce_limit' is false (this is the default), then
+  // allocation will always succeed.
+  MemoryTrackingBufferAllocator(BufferAllocator* const delegate,
+                                std::shared_ptr<MemTracker> mem_tracker,
+                                bool enforce_limit = false)
+      : delegate_(delegate),
+        mem_tracker_(std::move(mem_tracker)),
+        enforce_limit_(enforce_limit) {}
+
+  virtual ~MemoryTrackingBufferAllocator() {}
+
+  // If enforce limit is false, this always returns maximum possible value
+  // for int64_t (std::numeric_limits<int64_t>::max()). Otherwise, this
+  // is equivalent to calling mem_tracker_->SpareCapacity();
+  virtual size_t Available() const OVERRIDE;
+
+ private:
+
+  // If enforce_limit_ is true, this is equivalent to calling
+  // mem_tracker_->TryConsume(bytes). If enforce_limit_ is false and
+  // mem_tracker_->TryConsume(bytes) is false, we call
+  // mem_tracker_->Consume(bytes) and always return true.
+  bool TryConsume(int64_t bytes);
+
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE;
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE;
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE;
+
+  BufferAllocator* delegate_;
+  std::shared_ptr<MemTracker> mem_tracker_;
+  bool enforce_limit_;
+};
+
+// Synchronizes access to AllocateInternal and FreeInternal, and exposes the
+// mutex for use by subclasses. Allocation requests performed through this
+// allocator are atomic end-to-end. Template parameter DelegateAllocatorType
+// allows to specify a subclass of BufferAllocator for the delegate, to allow
+// subclasses of ThreadSafeBufferAllocator to access additional methods provided
+// by the allocator subclass. If this is not needed, it can be set to
+// BufferAllocator.
+template <class DelegateAllocatorType>
+class ThreadSafeBufferAllocator : public BufferAllocator {
+ public:
+  // Does not take ownership of the delegate.
+  explicit ThreadSafeBufferAllocator(DelegateAllocatorType* delegate)
+      : delegate_(delegate) {}
+  virtual ~ThreadSafeBufferAllocator() {}
+
+  virtual size_t Available() const OVERRIDE {
+    lock_guard_maybe<Mutex> lock(mutex());
+    return delegate()->Available();
+  }
+
+ protected:
+  Mutex* mutex() const { return &mutex_; }
+  // Expose the delegate allocator, with the precise type of the allocator
+  // specified by the template parameter. The delegate() methods themselves
+  // don't give any thread-safety guarantees. Protect all uses taking the Mutex
+  // exposed by the mutex() method.
+  DelegateAllocatorType* delegate() { return delegate_; }
+  const DelegateAllocatorType* delegate() const { return delegate_; }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE {
+    lock_guard_maybe<Mutex> lock(mutex());
+    return DelegateAllocate(delegate(), requested, minimal, originator);
+  }
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE {
+    lock_guard_maybe<Mutex> lock(mutex());
+    return DelegateReallocate(delegate(), requested, minimal, buffer,
+                              originator);
+  }
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE {
+    lock_guard_maybe<Mutex> lock(mutex());
+    DelegateFree(delegate(), buffer);
+  }
+
+  DelegateAllocatorType* delegate_;
+  mutable Mutex mutex_;
+  DISALLOW_COPY_AND_ASSIGN(ThreadSafeBufferAllocator);
+};
+
+// A version of ThreadSafeBufferAllocator that owns the supplied delegate
+// allocator.
+template <class DelegateAllocatorType>
+class OwningThreadSafeBufferAllocator
+    : public ThreadSafeBufferAllocator<DelegateAllocatorType> {
+ public:
+  explicit OwningThreadSafeBufferAllocator(DelegateAllocatorType* delegate)
+      : ThreadSafeBufferAllocator<DelegateAllocatorType>(delegate),
+        delegate_owned_(delegate) {}
+  virtual ~OwningThreadSafeBufferAllocator() {}
+
+ private:
+  gscoped_ptr<DelegateAllocatorType> delegate_owned_;
+};
+
+class ThreadSafeMemoryLimit
+    : public OwningThreadSafeBufferAllocator<MemoryLimit> {
+ public:
+  ThreadSafeMemoryLimit(size_t quota, bool enforced,
+                        BufferAllocator* const delegate)
+      : OwningThreadSafeBufferAllocator<MemoryLimit>(
+            new MemoryLimit(quota, enforced, delegate)) {}
+  virtual ~ThreadSafeMemoryLimit() {}
+
+  size_t GetQuota() const {
+    lock_guard_maybe<Mutex> lock(mutex());
+    return delegate()->GetQuota();
+  }
+  size_t GetUsage() const {
+    lock_guard_maybe<Mutex> lock(mutex());
+    return delegate()->GetUsage();
+  }
+  void SetQuota(const size_t quota) {
+    lock_guard_maybe<Mutex> lock(mutex());
+    delegate()->SetQuota(quota);
+  }
+};
+
+// A BufferAllocator that can be given ownership of many objects of given type.
+// These objects will then be deleted when the buffer allocator is destroyed.
+// The objects added last are deleted first (LIFO).
+template <typename OwnedType>
+class OwningBufferAllocator : public BufferAllocator {
+ public:
+  // Doesn't take ownership of delegate.
+  explicit OwningBufferAllocator(BufferAllocator* const delegate)
+      : delegate_(delegate) {}
+
+  virtual ~OwningBufferAllocator() {
+    // Delete elements starting from the end.
+    while (!owned_.empty()) {
+      OwnedType* p = owned_.back();
+      owned_.pop_back();
+      delete p;
+    }
+  }
+
+  // Add to the collection of objects owned by this allocator. The object added
+  // last is deleted first.
+  OwningBufferAllocator* Add(OwnedType* p) {
+    owned_.push_back(p);
+    return this;
+  }
+
+  virtual size_t Available() const OVERRIDE {
+    return delegate_->Available();
+  }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE {
+    return DelegateAllocate(delegate_, requested, minimal, originator);
+  }
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t minimal,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE {
+    return DelegateReallocate(delegate_, requested, minimal, buffer,
+                              originator);
+  }
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE {
+    DelegateFree(delegate_, buffer);
+  }
+
+  // Not using PointerVector here because we want to guarantee certain order of
+  // deleting elements (starting from the ones added last).
+  std::vector<OwnedType*> owned_;
+  BufferAllocator* delegate_;
+};
+
+// Buffer allocator that tries to guarantee the exact and consistent amount
+// of memory. Uses hard MemoryLimit to enforce the upper bound but also
+// guarantees consistent allocations by ignoring minimal requested amounts and
+// always returning the full amount of memory requested if available.
+// Allocations will fail if the memory requested would exceed the quota or if
+// the underlying allocator fails to provide the memory.
+class GuaranteeMemory : public BufferAllocator {
+ public:
+  // Doesn't take ownership of 'delegate'.
+  GuaranteeMemory(size_t memory_quota,
+                  BufferAllocator* delegate)
+      : limit_(memory_quota, true, delegate),
+        memory_guarantee_(memory_quota) {}
+
+  virtual size_t Available() const OVERRIDE {
+    return memory_guarantee_ - limit_.GetUsage();
+  }
+
+ private:
+  virtual Buffer* AllocateInternal(size_t requested,
+                                   size_t minimal,
+                                   BufferAllocator* originator) OVERRIDE {
+    if (requested > Available()) {
+      return NULL;
+    } else {
+      return DelegateAllocate(&limit_, requested, requested, originator);
+    }
+  }
+
+  virtual bool ReallocateInternal(size_t requested,
+                                  size_t /* minimal */,
+                                  Buffer* buffer,
+                                  BufferAllocator* originator) OVERRIDE {
+    int64_t additional_memory = requested - (buffer != NULL ? buffer->size() : 0);
+    return additional_memory <= static_cast<int64_t>(Available())
+        && DelegateReallocate(&limit_, requested, requested,
+                              buffer, originator);
+  }
+
+  virtual void FreeInternal(Buffer* buffer) OVERRIDE {
+    DelegateFree(&limit_, buffer);
+  }
+
+  MemoryLimit limit_;
+  size_t memory_guarantee_;
+  DISALLOW_COPY_AND_ASSIGN(GuaranteeMemory);
+};
+
+// Implementation of inline and template methods
+
+template<bool thread_safe>
+size_t Quota<thread_safe>::Allocate(const size_t requested,
+                                    const size_t minimal) {
+  lock_guard_maybe<Mutex> lock(mutex());
+  DCHECK_LE(minimal, requested)
+      << "\"minimal\" shouldn't be bigger than \"requested\"";
+  const size_t quota = GetQuotaInternal();
+  size_t allocation;
+  if (usage_ > quota || minimal > quota - usage_) {
+    // OOQ (Out of quota).
+    if (!enforced() && minimal <= std::numeric_limits<size_t>::max() - usage_) {
+      // The quota is unenforced and the value of "minimal" won't cause an
+      // overflow. Perform a minimal allocation.
+      allocation = minimal;
+    } else {
+      allocation = 0;
+    }
+    LOG(WARNING) << "Out of quota. Requested: " << requested
+                 << " bytes, or at least minimal: " << minimal
+                 << ". Current quota value is: " << quota
+                 << " while current usage is: " << usage_
+                 << ". The quota is " << (enforced() ? "" : "not ")
+                 << "enforced. "
+                 << ((allocation == 0) ? "Did not allocate any memory."
+                 : "Allocated the minimal value requested.");
+  } else {
+    allocation = std::min(requested, quota - usage_);
+  }
+  usage_ += allocation;
+  return allocation;
+}
+
+template<bool thread_safe>
+void Quota<thread_safe>::Free(size_t amount) {
+  lock_guard_maybe<Mutex> lock(mutex());
+  usage_ -= amount;
+  // threads allocate/free memory concurrently via the same Quota object that is
+  // not protected with a mutex (thread_safe == false).
+  if (usage_ > (std::numeric_limits<size_t>::max() - (1 << 28))) {
+    LOG(ERROR) << "Suspiciously big usage_ value: " << usage_
+               << " (could be a result size_t wrapping around below 0, "
+               << "for example as a result of race condition).";
+  }
+}
+
+template<bool thread_safe>
+size_t Quota<thread_safe>::GetQuota() const {
+  lock_guard_maybe<Mutex> lock(mutex());
+  return GetQuotaInternal();
+}
+
+template<bool thread_safe>
+size_t Quota<thread_safe>::GetUsage() const {
+  lock_guard_maybe<Mutex> lock(mutex());
+  return usage_;
+}
+
+template<bool thread_safe>
+void StaticQuota<thread_safe>::SetQuota(const size_t quota) {
+  lock_guard_maybe<Mutex> lock(Quota<thread_safe>::mutex());
+  quota_ = quota;
+}
+
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_MEMORY_MEMORY_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/overwrite.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/overwrite.cc b/be/src/kudu/util/memory/overwrite.cc
new file mode 100644
index 0000000..cca9227
--- /dev/null
+++ b/be/src/kudu/util/memory/overwrite.cc
@@ -0,0 +1,42 @@
+// 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 "kudu/util/memory/overwrite.h"
+
+#include "kudu/gutil/strings/stringpiece.h"
+
+#include <string.h>
+#include <glog/logging.h>
+namespace kudu {
+
+void OverwriteWithPattern(char* p, size_t len, StringPiece pattern) {
+  size_t pat_len = pattern.size();
+  CHECK_LT(0, pat_len);
+  size_t rem = len;
+  const char *pat_ptr = pattern.data();
+
+  for (; rem >= pat_len; rem -= pat_len) {
+    memcpy(p, pat_ptr, pat_len);
+    p += pat_len;
+  }
+
+  for (; rem > 0; rem--) {
+    *p++ = *pat_ptr++;
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/overwrite.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/overwrite.h b/be/src/kudu/util/memory/overwrite.h
new file mode 100644
index 0000000..37c6512
--- /dev/null
+++ b/be/src/kudu/util/memory/overwrite.h
@@ -0,0 +1,33 @@
+// 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.
+#ifndef KUDU_MEMORY_OVERWRITE_H
+#define KUDU_MEMORY_OVERWRITE_H
+
+#include <cstddef>
+
+#include "kudu/gutil/strings/stringpiece.h"
+
+namespace kudu {
+
+// Overwrite 'p' with enough repetitions of 'pattern' to fill 'len'
+// bytes. This is optimized at -O3 even in debug builds, so is
+// reasonably efficient to use.
+void OverwriteWithPattern(char* p, size_t len, StringPiece pattern);
+
+} // namespace kudu
+#endif /* KUDU_MEMORY_OVERWRITE_H */
+

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/metrics-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/metrics-test.cc b/be/src/kudu/util/metrics-test.cc
new file mode 100644
index 0000000..f8776aa
--- /dev/null
+++ b/be/src/kudu/util/metrics-test.cc
@@ -0,0 +1,388 @@
+// 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 <cstdint>
+#include <ostream>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::unordered_set;
+using std::vector;
+
+DECLARE_int32(metrics_retirement_age_ms);
+
+namespace kudu {
+
+METRIC_DEFINE_entity(test_entity);
+
+class MetricsTest : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    entity_ = METRIC_ENTITY_test_entity.Instantiate(&registry_, "my-test");
+  }
+
+ protected:
+  MetricRegistry registry_;
+  scoped_refptr<MetricEntity> entity_;
+};
+
+METRIC_DEFINE_counter(test_entity, test_counter, "My Test Counter", MetricUnit::kRequests,
+                      "Description of test counter");
+
+TEST_F(MetricsTest, SimpleCounterTest) {
+  scoped_refptr<Counter> requests =
+    new Counter(&METRIC_test_counter);
+  ASSERT_EQ("Description of test counter", requests->prototype()->description());
+  ASSERT_EQ(0, requests->value());
+  requests->Increment();
+  ASSERT_EQ(1, requests->value());
+  requests->IncrementBy(2);
+  ASSERT_EQ(3, requests->value());
+}
+
+METRIC_DEFINE_gauge_uint64(test_entity, test_gauge, "Test uint64 Gauge",
+                           MetricUnit::kBytes, "Description of Test Gauge");
+
+TEST_F(MetricsTest, SimpleAtomicGaugeTest) {
+  scoped_refptr<AtomicGauge<uint64_t> > mem_usage =
+    METRIC_test_gauge.Instantiate(entity_, 0);
+  ASSERT_EQ(METRIC_test_gauge.description(), mem_usage->prototype()->description());
+  ASSERT_EQ(0, mem_usage->value());
+  mem_usage->IncrementBy(7);
+  ASSERT_EQ(7, mem_usage->value());
+  mem_usage->set_value(5);
+  ASSERT_EQ(5, mem_usage->value());
+}
+
+METRIC_DEFINE_gauge_int64(test_entity, test_func_gauge, "Test Function Gauge",
+                          MetricUnit::kBytes, "Test Gauge 2");
+
+static int64_t MyFunction(int* metric_val) {
+  return (*metric_val)++;
+}
+
+TEST_F(MetricsTest, SimpleFunctionGaugeTest) {
+  int metric_val = 1000;
+  scoped_refptr<FunctionGauge<int64_t> > gauge =
+    METRIC_test_func_gauge.InstantiateFunctionGauge(
+      entity_, Bind(&MyFunction, Unretained(&metric_val)));
+
+  ASSERT_EQ(1000, gauge->value());
+  ASSERT_EQ(1001, gauge->value());
+
+  gauge->DetachToCurrentValue();
+  // After detaching, it should continue to return the same constant value.
+  ASSERT_EQ(1002, gauge->value());
+  ASSERT_EQ(1002, gauge->value());
+
+  // Test resetting to a constant.
+  gauge->DetachToConstant(2);
+  ASSERT_EQ(2, gauge->value());
+}
+
+TEST_F(MetricsTest, AutoDetachToLastValue) {
+  int metric_val = 1000;
+  scoped_refptr<FunctionGauge<int64_t> > gauge =
+    METRIC_test_func_gauge.InstantiateFunctionGauge(
+        entity_, Bind(&MyFunction, Unretained(&metric_val)));
+
+  ASSERT_EQ(1000, gauge->value());
+  ASSERT_EQ(1001, gauge->value());
+  {
+    FunctionGaugeDetacher detacher;
+    gauge->AutoDetachToLastValue(&detacher);
+    ASSERT_EQ(1002, gauge->value());
+    ASSERT_EQ(1003, gauge->value());
+  }
+
+  ASSERT_EQ(1004, gauge->value());
+  ASSERT_EQ(1004, gauge->value());
+}
+
+TEST_F(MetricsTest, AutoDetachToConstant) {
+  int metric_val = 1000;
+  scoped_refptr<FunctionGauge<int64_t> > gauge =
+    METRIC_test_func_gauge.InstantiateFunctionGauge(
+        entity_, Bind(&MyFunction, Unretained(&metric_val)));
+
+  ASSERT_EQ(1000, gauge->value());
+  ASSERT_EQ(1001, gauge->value());
+  {
+    FunctionGaugeDetacher detacher;
+    gauge->AutoDetach(&detacher, 12345);
+    ASSERT_EQ(1002, gauge->value());
+    ASSERT_EQ(1003, gauge->value());
+  }
+
+  ASSERT_EQ(12345, gauge->value());
+}
+
+METRIC_DEFINE_gauge_uint64(test_entity, counter_as_gauge, "Gauge exposed as Counter",
+                           MetricUnit::kBytes, "Gauge exposed as Counter",
+                           EXPOSE_AS_COUNTER);
+TEST_F(MetricsTest, TEstExposeGaugeAsCounter) {
+  ASSERT_EQ(MetricType::kCounter, METRIC_counter_as_gauge.type());
+}
+
+METRIC_DEFINE_histogram(test_entity, test_hist, "Test Histogram",
+                        MetricUnit::kMilliseconds, "foo", 1000000, 3);
+
+TEST_F(MetricsTest, SimpleHistogramTest) {
+  scoped_refptr<Histogram> hist = METRIC_test_hist.Instantiate(entity_);
+  hist->Increment(2);
+  hist->IncrementBy(4, 1);
+  ASSERT_EQ(2, hist->histogram_->MinValue());
+  ASSERT_EQ(3, hist->histogram_->MeanValue());
+  ASSERT_EQ(4, hist->histogram_->MaxValue());
+  ASSERT_EQ(2, hist->histogram_->TotalCount());
+  ASSERT_EQ(6, hist->histogram_->TotalSum());
+  // TODO: Test coverage needs to be improved a lot.
+}
+
+TEST_F(MetricsTest, JsonPrintTest) {
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+  test_counter->Increment();
+  entity_->SetAttribute("test_attr", "attr_val");
+
+  // Generate the JSON.
+  std::ostringstream out;
+  JsonWriter writer(&out, JsonWriter::PRETTY);
+  ASSERT_OK(entity_->WriteAsJson(&writer, { "*" }, MetricJsonOptions()));
+
+  // Now parse it back out.
+  JsonReader reader(out.str());
+  ASSERT_OK(reader.Init());
+
+  vector<const rapidjson::Value*> metrics;
+  ASSERT_OK(reader.ExtractObjectArray(reader.root(), "metrics", &metrics));
+  ASSERT_EQ(1, metrics.size());
+  string metric_name;
+  ASSERT_OK(reader.ExtractString(metrics[0], "name", &metric_name));
+  ASSERT_EQ("test_counter", metric_name);
+  int64_t metric_value;
+  ASSERT_OK(reader.ExtractInt64(metrics[0], "value", &metric_value));
+  ASSERT_EQ(1L, metric_value);
+
+  const rapidjson::Value* attributes;
+  ASSERT_OK(reader.ExtractObject(reader.root(), "attributes", &attributes));
+  string attr_value;
+  ASSERT_OK(reader.ExtractString(attributes, "test_attr", &attr_value));
+  ASSERT_EQ("attr_val", attr_value);
+
+  // Verify that metric filtering matches on substrings.
+  out.str("");
+  ASSERT_OK(entity_->WriteAsJson(&writer, { "test count" }, MetricJsonOptions()));
+  ASSERT_STR_CONTAINS(METRIC_test_counter.name(), out.str());
+
+  // Verify that, if we filter for a metric that isn't in this entity, we get no result.
+  out.str("");
+  ASSERT_OK(entity_->WriteAsJson(&writer, { "not_a_matching_metric" }, MetricJsonOptions()));
+  ASSERT_EQ("", out.str());
+
+  // Verify that filtering is case-insensitive.
+  out.str("");
+  ASSERT_OK(entity_->WriteAsJson(&writer, { "mY teST coUNteR" }, MetricJsonOptions()));
+  ASSERT_STR_CONTAINS(METRIC_test_counter.name(), out.str());
+}
+
+// Test that metrics are retired when they are no longer referenced.
+TEST_F(MetricsTest, RetirementTest) {
+  FLAGS_metrics_retirement_age_ms = 100;
+
+  const string kMetricName = "foo";
+  scoped_refptr<Counter> counter = METRIC_test_counter.Instantiate(entity_);
+  ASSERT_EQ(1, entity_->UnsafeMetricsMapForTests().size());
+
+  // Since we hold a reference to the counter, it should not get retired.
+  entity_->RetireOldMetrics();
+  ASSERT_EQ(1, entity_->UnsafeMetricsMapForTests().size());
+
+  // When we de-ref it, it should not get immediately retired, either, because
+  // we keep retirable metrics around for some amount of time. We try retiring
+  // a number of times to hit all the cases.
+  counter = nullptr;
+  for (int i = 0; i < 3; i++) {
+    entity_->RetireOldMetrics();
+    ASSERT_EQ(1, entity_->UnsafeMetricsMapForTests().size());
+  }
+
+  // If we wait for longer than the retirement time, and call retire again, we'll
+  // actually retire it.
+  SleepFor(MonoDelta::FromMilliseconds(FLAGS_metrics_retirement_age_ms * 1.5));
+  entity_->RetireOldMetrics();
+  ASSERT_EQ(0, entity_->UnsafeMetricsMapForTests().size());
+}
+
+TEST_F(MetricsTest, TestRetiringEntities) {
+  ASSERT_EQ(1, registry_.num_entities());
+
+  // Drop the reference to our entity.
+  entity_.reset();
+
+  // Retire metrics. Since there is nothing inside our entity, it should
+  // retire immediately (no need to loop).
+  registry_.RetireOldMetrics();
+
+  ASSERT_EQ(0, registry_.num_entities());
+}
+
+// Test that we can mark a metric to never be retired.
+TEST_F(MetricsTest, NeverRetireTest) {
+  entity_->NeverRetire(METRIC_test_hist.Instantiate(entity_));
+  FLAGS_metrics_retirement_age_ms = 0;
+
+  for (int i = 0; i < 3; i++) {
+    entity_->RetireOldMetrics();
+    ASSERT_EQ(1, entity_->UnsafeMetricsMapForTests().size());
+  }
+}
+
+TEST_F(MetricsTest, TestInstantiatingTwice) {
+  // Test that re-instantiating the same entity ID returns the same object.
+  scoped_refptr<MetricEntity> new_entity = METRIC_ENTITY_test_entity.Instantiate(
+      &registry_, entity_->id());
+  ASSERT_EQ(new_entity.get(), entity_.get());
+}
+
+TEST_F(MetricsTest, TestInstantiatingDifferentEntities) {
+  scoped_refptr<MetricEntity> new_entity = METRIC_ENTITY_test_entity.Instantiate(
+      &registry_, "some other ID");
+  ASSERT_NE(new_entity.get(), entity_.get());
+}
+
+TEST_F(MetricsTest, TestDumpJsonPrototypes) {
+  // Dump the prototype info.
+  std::ostringstream out;
+  JsonWriter w(&out, JsonWriter::PRETTY);
+  MetricPrototypeRegistry::get()->WriteAsJson(&w);
+  string json = out.str();
+
+  // Quick sanity check for one of our metrics defined in this file.
+  const char* expected =
+    "        {\n"
+    "            \"name\": \"test_func_gauge\",\n"
+    "            \"label\": \"Test Function Gauge\",\n"
+    "            \"type\": \"gauge\",\n"
+    "            \"unit\": \"bytes\",\n"
+    "            \"description\": \"Test Gauge 2\",\n"
+    "            \"entity_type\": \"test_entity\"\n"
+    "        }";
+  ASSERT_STR_CONTAINS(json, expected);
+
+  // Parse it.
+  rapidjson::Document d;
+  d.Parse<0>(json.c_str());
+
+  // Ensure that we got a reasonable number of metrics.
+  int num_metrics = d["metrics"].Size();
+  int num_entities = d["entities"].Size();
+  LOG(INFO) << "Parsed " << num_metrics << " metrics and " << num_entities << " entities";
+  ASSERT_GT(num_metrics, 5);
+  ASSERT_EQ(num_entities, 2);
+
+  // Spot-check that some metrics were properly registered and that the JSON was properly
+  // formed.
+  unordered_set<string> seen_metrics;
+  for (int i = 0; i < d["metrics"].Size(); i++) {
+    InsertOrDie(&seen_metrics, d["metrics"][i]["name"].GetString());
+  }
+  ASSERT_TRUE(ContainsKey(seen_metrics, "threads_started"));
+  ASSERT_TRUE(ContainsKey(seen_metrics, "test_hist"));
+}
+
+TEST_F(MetricsTest, TestDumpOnlyChanged) {
+  auto GetJson = [&](int64_t since_epoch) {
+    MetricJsonOptions opts;
+    opts.only_modified_in_or_after_epoch = since_epoch;
+    std::ostringstream out;
+    JsonWriter writer(&out, JsonWriter::COMPACT);
+    CHECK_OK(entity_->WriteAsJson(&writer, { "*" }, opts));
+    return out.str();
+  };
+
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+
+  int64_t epoch_when_modified = Metric::current_epoch();
+  test_counter->Increment();
+
+  // If we pass a "since dirty" epoch from before we incremented it, we should
+  // see the metric.
+  for (int i = 0; i < 2; i++) {
+    ASSERT_STR_CONTAINS(GetJson(epoch_when_modified), "{\"name\":\"test_counter\",\"value\":1}");
+    Metric::IncrementEpoch();
+  }
+
+  // If we pass a current epoch, we should see that the metric was not modified.
+  int64_t new_epoch = Metric::current_epoch();
+  ASSERT_STR_NOT_CONTAINS(GetJson(new_epoch), "test_counter");
+  // ... until we modify it again.
+  test_counter->Increment();
+  ASSERT_STR_CONTAINS(GetJson(new_epoch), "{\"name\":\"test_counter\",\"value\":2}");
+}
+
+
+// Test that 'include_untouched_metrics=false' prevents dumping counters and histograms
+// which have never been incremented.
+TEST_F(MetricsTest, TestDontDumpUntouched) {
+  // Instantiate a bunch of metrics.
+  int metric_val = 1000;
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+  scoped_refptr<Histogram> hist = METRIC_test_hist.Instantiate(entity_);
+  scoped_refptr<FunctionGauge<int64_t> > function_gauge =
+    METRIC_test_func_gauge.InstantiateFunctionGauge(
+        entity_, Bind(&MyFunction, Unretained(&metric_val)));
+  scoped_refptr<AtomicGauge<uint64_t> > atomic_gauge =
+    METRIC_test_gauge.Instantiate(entity_, 0);
+
+  MetricJsonOptions opts;
+  opts.include_untouched_metrics = false;
+  std::ostringstream out;
+  JsonWriter writer(&out, JsonWriter::COMPACT);
+  CHECK_OK(entity_->WriteAsJson(&writer, { "*" }, opts));
+  // Untouched counters and histograms should not be included.
+  ASSERT_STR_NOT_CONTAINS(out.str(), "test_counter");
+  ASSERT_STR_NOT_CONTAINS(out.str(), "test_hist");
+  // Untouched gauges need to be included, because we don't actually
+  // track whether they have been touched.
+  ASSERT_STR_CONTAINS(out.str(), "test_func_gauge");
+  ASSERT_STR_CONTAINS(out.str(), "test_gauge");
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/metrics.cc b/be/src/kudu/util/metrics.cc
new file mode 100644
index 0000000..dc30360
--- /dev/null
+++ b/be/src/kudu/util/metrics.cc
@@ -0,0 +1,746 @@
+// 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 "kudu/util/metrics.h"
+
+#include <iostream>
+#include <map>
+#include <utility>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/singleton.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/histogram.pb.h"
+#include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
+
+DEFINE_int32(metrics_retirement_age_ms, 120 * 1000,
+             "The minimum number of milliseconds a metric will be kept for after it is "
+             "no longer active. (Advanced option)");
+TAG_FLAG(metrics_retirement_age_ms, runtime);
+TAG_FLAG(metrics_retirement_age_ms, advanced);
+
+// Process/server-wide metrics should go into the 'server' entity.
+// More complex applications will define other entities.
+METRIC_DEFINE_entity(server);
+
+namespace kudu {
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+//
+// MetricUnit
+//
+
+const char* MetricUnit::Name(Type unit) {
+  switch (unit) {
+    case kCacheHits:
+      return "hits";
+    case kCacheQueries:
+      return "queries";
+    case kBytes:
+      return "bytes";
+    case kRequests:
+      return "requests";
+    case kEntries:
+      return "entries";
+    case kRows:
+      return "rows";
+    case kCells:
+      return "cells";
+    case kConnections:
+      return "connections";
+    case kOperations:
+      return "operations";
+    case kProbes:
+      return "probes";
+    case kNanoseconds:
+      return "nanoseconds";
+    case kMicroseconds:
+      return "microseconds";
+    case kMilliseconds:
+      return "milliseconds";
+    case kSeconds:
+      return "seconds";
+    case kThreads:
+      return "threads";
+    case kTransactions:
+      return "transactions";
+    case kUnits:
+      return "units";
+    case kScanners:
+      return "scanners";
+    case kMaintenanceOperations:
+      return "operations";
+    case kBlocks:
+      return "blocks";
+    case kHoles:
+      return "holes";
+    case kLogBlockContainers:
+      return "log block containers";
+    case kTasks:
+      return "tasks";
+    case kMessages:
+      return "messages";
+    case kContextSwitches:
+      return "context switches";
+    case kDataDirectories:
+      return "data directories";
+    case kState:
+      return "state";
+    case kSessions:
+      return "sessions";
+    case kTablets:
+      return "tablets";
+    default:
+      DCHECK(false) << "Unknown unit with type = " << unit;
+      return "UNKNOWN UNIT";
+  }
+}
+
+//
+// MetricType
+//
+
+const char* const MetricType::kGaugeType = "gauge";
+const char* const MetricType::kCounterType = "counter";
+const char* const MetricType::kHistogramType = "histogram";
+const char* MetricType::Name(MetricType::Type type) {
+  switch (type) {
+    case kGauge:
+      return kGaugeType;
+    case kCounter:
+      return kCounterType;
+    case kHistogram:
+      return kHistogramType;
+    default:
+      return "UNKNOWN TYPE";
+  }
+}
+
+//
+// MetricEntityPrototype
+//
+
+MetricEntityPrototype::MetricEntityPrototype(const char* name)
+  : name_(name) {
+  MetricPrototypeRegistry::get()->AddEntity(this);
+}
+
+MetricEntityPrototype::~MetricEntityPrototype() {
+}
+
+scoped_refptr<MetricEntity> MetricEntityPrototype::Instantiate(
+    MetricRegistry* registry,
+    const std::string& id,
+    const MetricEntity::AttributeMap& initial_attrs) const {
+  return registry->FindOrCreateEntity(this, id, initial_attrs);
+}
+
+
+//
+// MetricEntity
+//
+
+MetricEntity::MetricEntity(const MetricEntityPrototype* prototype,
+                           std::string id, AttributeMap attributes)
+    : prototype_(prototype),
+      id_(std::move(id)),
+      attributes_(std::move(attributes)),
+      published_(true) {}
+
+MetricEntity::~MetricEntity() {
+}
+
+void MetricEntity::CheckInstantiation(const MetricPrototype* proto) const {
+  CHECK_STREQ(prototype_->name(), proto->entity_type())
+    << "Metric " << proto->name() << " may not be instantiated entity of type "
+    << prototype_->name() << " (expected: " << proto->entity_type() << ")";
+}
+
+scoped_refptr<Metric> MetricEntity::FindOrNull(const MetricPrototype& prototype) const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return FindPtrOrNull(metric_map_, &prototype);
+}
+
+namespace {
+
+bool MatchMetricInList(const string& metric_name,
+                       const vector<string>& match_params) {
+  string metric_name_uc;
+  ToUpperCase(metric_name, &metric_name_uc);
+
+  for (const string& param : match_params) {
+    // Handle wildcard.
+    if (param == "*") return true;
+    // The parameter is a case-insensitive substring match of the metric name.
+    string param_uc;
+    ToUpperCase(param, &param_uc);
+    if (metric_name_uc.find(param_uc) != std::string::npos) {
+      return true;
+    }
+  }
+  return false;
+}
+
+} // anonymous namespace
+
+
+Status MetricEntity::WriteAsJson(JsonWriter* writer,
+                                 const vector<string>& requested_metrics,
+                                 const MetricJsonOptions& opts) const {
+  bool select_all = MatchMetricInList(id(), requested_metrics);
+
+  // We want the keys to be in alphabetical order when printing, so we use an ordered map here.
+  typedef std::map<const char*, scoped_refptr<Metric> > OrderedMetricMap;
+  OrderedMetricMap metrics;
+  AttributeMap attrs;
+  {
+    // Snapshot the metrics in this registry (not guaranteed to be a consistent snapshot)
+    std::lock_guard<simple_spinlock> l(lock_);
+    attrs = attributes_;
+    for (const MetricMap::value_type& val : metric_map_) {
+      const MetricPrototype* prototype = val.first;
+      const scoped_refptr<Metric>& metric = val.second;
+
+      if (select_all || MatchMetricInList(prototype->name(), requested_metrics)) {
+        InsertOrDie(&metrics, prototype->name(), metric);
+      }
+    }
+  }
+
+  // If we had a filter, and we didn't either match this entity or any metrics inside
+  // it, don't print the entity at all.
+  if (!requested_metrics.empty() && !select_all && metrics.empty()) {
+    return Status::OK();
+  }
+
+  writer->StartObject();
+
+  writer->String("type");
+  writer->String(prototype_->name());
+
+  writer->String("id");
+  writer->String(id_);
+
+  if (opts.include_entity_attributes) {
+    writer->String("attributes");
+    writer->StartObject();
+    for (const AttributeMap::value_type& val : attrs) {
+      writer->String(val.first);
+      writer->String(val.second);
+    }
+    writer->EndObject();
+  }
+
+  writer->String("metrics");
+  writer->StartArray();
+  for (OrderedMetricMap::value_type& val : metrics) {
+    const auto& m = val.second;
+    if (m->ModifiedInOrAfterEpoch(opts.only_modified_in_or_after_epoch)) {
+      if (!opts.include_untouched_metrics && m->IsUntouched()) {
+        continue;
+      }
+      WARN_NOT_OK(m->WriteAsJson(writer, opts),
+                  strings::Substitute("Failed to write $0 as JSON", val.first));
+    }
+  }
+  writer->EndArray();
+
+  writer->EndObject();
+
+  return Status::OK();
+}
+
+void MetricEntity::RetireOldMetrics() {
+  MonoTime now(MonoTime::Now());
+
+  std::lock_guard<simple_spinlock> l(lock_);
+  for (auto it = metric_map_.begin(); it != metric_map_.end();) {
+    const scoped_refptr<Metric>& metric = it->second;
+
+    if (PREDICT_TRUE(!metric->HasOneRef() && published_)) {
+      // The metric is still in use. Note that, in the case of "NeverRetire()", the metric
+      // will have a ref-count of 2 because it is reffed by the 'never_retire_metrics_'
+      // collection.
+
+      // Ensure that it is not marked for later retirement (this could happen in the case
+      // that a metric is un-reffed and then re-reffed later by looking it up from the
+      // registry).
+      metric->retire_time_ = MonoTime();
+      ++it;
+      continue;
+    }
+
+    if (!metric->retire_time_.Initialized()) {
+      VLOG(3) << "Metric " << it->first << " has become un-referenced or unpublished. "
+              << "Will retire after the retention interval";
+      // This is the first time we've seen this metric as retirable.
+      metric->retire_time_ =
+          now + MonoDelta::FromMilliseconds(FLAGS_metrics_retirement_age_ms);
+      ++it;
+      continue;
+    }
+
+    // If we've already seen this metric in a previous scan, check if it's
+    // time to retire it yet.
+    if (now < metric->retire_time_) {
+      VLOG(3) << "Metric " << it->first << " is un-referenced, but still within "
+              << "the retention interval";
+      ++it;
+      continue;
+    }
+
+
+    VLOG(2) << "Retiring metric " << it->first;
+    metric_map_.erase(it++);
+  }
+}
+
+void MetricEntity::NeverRetire(const scoped_refptr<Metric>& metric) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  never_retire_metrics_.push_back(metric);
+}
+
+void MetricEntity::SetAttributes(const AttributeMap& attrs) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  attributes_ = attrs;
+}
+
+void MetricEntity::SetAttribute(const string& key, const string& val) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  attributes_[key] = val;
+}
+
+//
+// MetricRegistry
+//
+
+MetricRegistry::MetricRegistry() {
+}
+
+MetricRegistry::~MetricRegistry() {
+}
+
+Status MetricRegistry::WriteAsJson(JsonWriter* writer,
+                                   const vector<string>& requested_metrics,
+                                   const MetricJsonOptions& opts) const {
+  EntityMap entities;
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    entities = entities_;
+  }
+
+  writer->StartArray();
+  for (const auto& e : entities) {
+    WARN_NOT_OK(e.second->WriteAsJson(writer, requested_metrics, opts),
+                Substitute("Failed to write entity $0 as JSON", e.second->id()));
+  }
+  writer->EndArray();
+
+  // Rather than having a thread poll metrics periodically to retire old ones,
+  // we'll just retire them here. The only downside is that, if no one is polling
+  // metrics, we may end up leaving them around indefinitely; however, metrics are
+  // small, and one might consider it a feature: if monitoring stops polling for
+  // metrics, we should keep them around until the next poll.
+  entities.clear(); // necessary to deref metrics we just dumped before doing retirement scan.
+  const_cast<MetricRegistry*>(this)->RetireOldMetrics();
+  return Status::OK();
+}
+
+void MetricRegistry::RetireOldMetrics() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  for (auto it = entities_.begin(); it != entities_.end();) {
+    it->second->RetireOldMetrics();
+
+    if (it->second->num_metrics() == 0 &&
+        (it->second->HasOneRef() || !it->second->published())) {
+      // This entity has no metrics and either has no more external references or has
+      // been marked as unpublished, so we can remove it.
+      // Unlike retiring the metrics themselves, we don't wait for any timeout
+      // to retire them -- we assume that that timed retention has been satisfied
+      // by holding onto the metrics inside the entity.
+      entities_.erase(it++);
+    } else {
+      ++it;
+    }
+  }
+}
+
+//
+// MetricPrototypeRegistry
+//
+MetricPrototypeRegistry* MetricPrototypeRegistry::get() {
+  return Singleton<MetricPrototypeRegistry>::get();
+}
+
+void MetricPrototypeRegistry::AddMetric(const MetricPrototype* prototype) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  metrics_.push_back(prototype);
+}
+
+void MetricPrototypeRegistry::AddEntity(const MetricEntityPrototype* prototype) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  entities_.push_back(prototype);
+}
+
+void MetricPrototypeRegistry::WriteAsJson(JsonWriter* writer) const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  MetricJsonOptions opts;
+  opts.include_schema_info = true;
+  writer->StartObject();
+
+  // Dump metric prototypes.
+  writer->String("metrics");
+  writer->StartArray();
+  for (const MetricPrototype* p : metrics_) {
+    writer->StartObject();
+    p->WriteFields(writer, opts);
+    writer->String("entity_type");
+    writer->String(p->entity_type());
+    writer->EndObject();
+  }
+  writer->EndArray();
+
+  // Dump entity prototypes.
+  writer->String("entities");
+  writer->StartArray();
+  for (const MetricEntityPrototype* p : entities_) {
+    writer->StartObject();
+    writer->String("name");
+    writer->String(p->name());
+    writer->EndObject();
+  }
+  writer->EndArray();
+
+  writer->EndObject();
+}
+
+void MetricPrototypeRegistry::WriteAsJson() const {
+  std::ostringstream s;
+  JsonWriter w(&s, JsonWriter::PRETTY);
+  WriteAsJson(&w);
+  std::cout << s.str() << std::endl;
+}
+
+//
+// MetricPrototype
+//
+MetricPrototype::MetricPrototype(CtorArgs args) : args_(args) {
+  MetricPrototypeRegistry::get()->AddMetric(this);
+}
+
+void MetricPrototype::WriteFields(JsonWriter* writer,
+                                  const MetricJsonOptions& opts) const {
+  writer->String("name");
+  writer->String(name());
+
+  if (opts.include_schema_info) {
+    writer->String("label");
+    writer->String(label());
+
+    writer->String("type");
+    writer->String(MetricType::Name(type()));
+
+    writer->String("unit");
+    writer->String(MetricUnit::Name(unit()));
+
+    writer->String("description");
+    writer->String(description());
+  }
+}
+
+//
+// FunctionGaugeDetacher
+//
+
+FunctionGaugeDetacher::FunctionGaugeDetacher() {
+}
+
+FunctionGaugeDetacher::~FunctionGaugeDetacher() {
+  for (const Closure& c : callbacks_) {
+    c.Run();
+  }
+}
+
+scoped_refptr<MetricEntity> MetricRegistry::FindOrCreateEntity(
+    const MetricEntityPrototype* prototype,
+    const std::string& id,
+    const MetricEntity::AttributeMap& initial_attributes) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  scoped_refptr<MetricEntity> e = FindPtrOrNull(entities_, id);
+  if (!e) {
+    e = new MetricEntity(prototype, id, initial_attributes);
+    InsertOrDie(&entities_, id, e);
+  } else if (!e->published()) {
+    e = new MetricEntity(prototype, id, initial_attributes);
+    entities_[id] = e;
+  } else {
+    e->SetAttributes(initial_attributes);
+  }
+  return e;
+}
+
+//
+// Metric
+//
+
+std::atomic<int64_t> Metric::g_epoch_;
+
+Metric::Metric(const MetricPrototype* prototype)
+    : prototype_(prototype),
+      m_epoch_(current_epoch()) {
+}
+
+Metric::~Metric() {
+}
+
+void Metric::IncrementEpoch() {
+  g_epoch_++;
+}
+
+void Metric::UpdateModificationEpochSlowPath() {
+  int64_t new_epoch, old_epoch;
+  // CAS loop to ensure that we never transition a metric's epoch backwards
+  // even if multiple threads race to update it.
+  do {
+    old_epoch = m_epoch_;
+    new_epoch = g_epoch_;
+  } while (old_epoch < new_epoch &&
+           !m_epoch_.compare_exchange_weak(old_epoch, new_epoch));
+}
+
+//
+// Gauge
+//
+
+Status Gauge::WriteAsJson(JsonWriter* writer,
+                          const MetricJsonOptions& opts) const {
+  writer->StartObject();
+
+  prototype_->WriteFields(writer, opts);
+
+  writer->String("value");
+  WriteValue(writer);
+
+  writer->EndObject();
+  return Status::OK();
+}
+
+//
+// StringGauge
+//
+
+StringGauge::StringGauge(const GaugePrototype<string>* proto,
+                         string initial_value)
+    : Gauge(proto), value_(std::move(initial_value)) {}
+
+std::string StringGauge::value() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return value_;
+}
+
+void StringGauge::set_value(const std::string& value) {
+  UpdateModificationEpoch();
+  std::lock_guard<simple_spinlock> l(lock_);
+  value_ = value;
+}
+
+void StringGauge::WriteValue(JsonWriter* writer) const {
+  writer->String(value());
+}
+
+//
+// Counter
+//
+// This implementation is optimized by using a striped counter. See LongAdder for details.
+
+scoped_refptr<Counter> CounterPrototype::Instantiate(const scoped_refptr<MetricEntity>& entity) {
+  return entity->FindOrCreateCounter(this);
+}
+
+Counter::Counter(const CounterPrototype* proto) : Metric(proto) {
+}
+
+int64_t Counter::value() const {
+  return value_.Value();
+}
+
+void Counter::Increment() {
+  IncrementBy(1);
+}
+
+void Counter::IncrementBy(int64_t amount) {
+  UpdateModificationEpoch();
+  value_.IncrementBy(amount);
+}
+
+Status Counter::WriteAsJson(JsonWriter* writer,
+                            const MetricJsonOptions& opts) const {
+  writer->StartObject();
+
+  prototype_->WriteFields(writer, opts);
+
+  writer->String("value");
+  writer->Int64(value());
+
+  writer->EndObject();
+  return Status::OK();
+}
+
+/////////////////////////////////////////////////
+// HistogramPrototype
+/////////////////////////////////////////////////
+
+HistogramPrototype::HistogramPrototype(const MetricPrototype::CtorArgs& args,
+                                       uint64_t max_trackable_value, int num_sig_digits)
+  : MetricPrototype(args),
+    max_trackable_value_(max_trackable_value),
+    num_sig_digits_(num_sig_digits) {
+  // Better to crash at definition time that at instantiation time.
+  CHECK(HdrHistogram::IsValidHighestTrackableValue(max_trackable_value))
+      << Substitute("Invalid max trackable value on histogram $0: $1",
+                    args.name_, max_trackable_value);
+  CHECK(HdrHistogram::IsValidNumSignificantDigits(num_sig_digits))
+      << Substitute("Invalid number of significant digits on histogram $0: $1",
+                    args.name_, num_sig_digits);
+}
+
+scoped_refptr<Histogram> HistogramPrototype::Instantiate(
+    const scoped_refptr<MetricEntity>& entity) {
+  return entity->FindOrCreateHistogram(this);
+}
+
+/////////////////////////////////////////////////
+// Histogram
+/////////////////////////////////////////////////
+
+Histogram::Histogram(const HistogramPrototype* proto)
+  : Metric(proto),
+    histogram_(new HdrHistogram(proto->max_trackable_value(), proto->num_sig_digits())) {
+}
+
+void Histogram::Increment(int64_t value) {
+  UpdateModificationEpoch();
+  histogram_->Increment(value);
+}
+
+void Histogram::IncrementBy(int64_t value, int64_t amount) {
+  UpdateModificationEpoch();
+  histogram_->IncrementBy(value, amount);
+}
+
+Status Histogram::WriteAsJson(JsonWriter* writer,
+                              const MetricJsonOptions& opts) const {
+
+  HistogramSnapshotPB snapshot;
+  RETURN_NOT_OK(GetHistogramSnapshotPB(&snapshot, opts));
+  writer->Protobuf(snapshot);
+  return Status::OK();
+}
+
+Status Histogram::GetHistogramSnapshotPB(HistogramSnapshotPB* snapshot_pb,
+                                         const MetricJsonOptions& opts) const {
+  snapshot_pb->set_name(prototype_->name());
+  if (opts.include_schema_info) {
+    snapshot_pb->set_type(MetricType::Name(prototype_->type()));
+    snapshot_pb->set_label(prototype_->label());
+    snapshot_pb->set_unit(MetricUnit::Name(prototype_->unit()));
+    snapshot_pb->set_description(prototype_->description());
+    snapshot_pb->set_max_trackable_value(histogram_->highest_trackable_value());
+    snapshot_pb->set_num_significant_digits(histogram_->num_significant_digits());
+  }
+  // Fast-path for a reasonably common case of an empty histogram. This occurs
+  // when a histogram is tracking some information about a feature not in
+  // use, for example.
+  if (histogram_->TotalCount() == 0) {
+    snapshot_pb->set_total_count(0);
+    snapshot_pb->set_total_sum(0);
+    snapshot_pb->set_min(0);
+    snapshot_pb->set_mean(0);
+    snapshot_pb->set_percentile_75(0);
+    snapshot_pb->set_percentile_95(0);
+    snapshot_pb->set_percentile_99(0);
+    snapshot_pb->set_percentile_99_9(0);
+    snapshot_pb->set_percentile_99_99(0);
+    snapshot_pb->set_max(0);
+  } else {
+    HdrHistogram snapshot(*histogram_);
+    snapshot_pb->set_total_count(snapshot.TotalCount());
+    snapshot_pb->set_total_sum(snapshot.TotalSum());
+    snapshot_pb->set_min(snapshot.MinValue());
+    snapshot_pb->set_mean(snapshot.MeanValue());
+    snapshot_pb->set_percentile_75(snapshot.ValueAtPercentile(75));
+    snapshot_pb->set_percentile_95(snapshot.ValueAtPercentile(95));
+    snapshot_pb->set_percentile_99(snapshot.ValueAtPercentile(99));
+    snapshot_pb->set_percentile_99_9(snapshot.ValueAtPercentile(99.9));
+    snapshot_pb->set_percentile_99_99(snapshot.ValueAtPercentile(99.99));
+    snapshot_pb->set_max(snapshot.MaxValue());
+
+    if (opts.include_raw_histograms) {
+      RecordedValuesIterator iter(&snapshot);
+      while (iter.HasNext()) {
+        HistogramIterationValue value;
+        RETURN_NOT_OK(iter.Next(&value));
+        snapshot_pb->add_values(value.value_iterated_to);
+        snapshot_pb->add_counts(value.count_at_value_iterated_to);
+      }
+    }
+  }
+  return Status::OK();
+}
+
+uint64_t Histogram::CountInBucketForValueForTests(uint64_t value) const {
+  return histogram_->CountInBucketForValue(value);
+}
+
+uint64_t Histogram::TotalCount() const {
+  return histogram_->TotalCount();
+}
+
+uint64_t Histogram::MinValueForTests() const {
+  return histogram_->MinValue();
+}
+
+uint64_t Histogram::MaxValueForTests() const {
+  return histogram_->MaxValue();
+}
+double Histogram::MeanValueForTests() const {
+  return histogram_->MeanValue();
+}
+
+ScopedLatencyMetric::ScopedLatencyMetric(Histogram* latency_hist)
+  : latency_hist_(latency_hist) {
+  if (latency_hist_) {
+    time_started_ = MonoTime::Now();
+  }
+}
+
+ScopedLatencyMetric::~ScopedLatencyMetric() {
+  if (latency_hist_ != nullptr) {
+    MonoTime time_now = MonoTime::Now();
+    latency_hist_->Increment((time_now - time_started_).ToMicroseconds());
+  }
+}
+
+} // namespace kudu


[41/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/reactor.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/reactor.cc b/be/src/kudu/rpc/reactor.cc
new file mode 100644
index 0000000..c1832ef
--- /dev/null
+++ b/be/src/kudu/rpc/reactor.cc
@@ -0,0 +1,918 @@
+// 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 "kudu/rpc/reactor.h"
+
+#include <cerrno>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/intrusive/list.hpp>
+#include <boost/optional.hpp>
+#include <ev++.h>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/client_negotiation.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/debug/sanitizer_scopes.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/thread_restrictions.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/trace.h"
+
+// When compiling on Mac OS X, use 'kqueue' instead of the default, 'select', for the event loop.
+// Otherwise we run into problems because 'select' can't handle connections when more than 1024
+// file descriptors are open by the process.
+#if defined(__APPLE__)
+static const int kDefaultLibEvFlags = ev::KQUEUE;
+#else
+static const int kDefaultLibEvFlags = ev::AUTO;
+#endif
+
+using std::string;
+using std::shared_ptr;
+using std::unique_ptr;
+using strings::Substitute;
+
+DEFINE_bool(rpc_reopen_outbound_connections, false,
+            "Open a new connection to the server for every RPC call. "
+            "If not enabled, an already existing connection to a "
+            "server is reused upon making another call to the same server. "
+            "When this flag is enabled, an already existing _idle_ connection "
+            "to the server is closed upon making another RPC call which would "
+            "reuse the connection otherwise. "
+            "Used by tests only.");
+TAG_FLAG(rpc_reopen_outbound_connections, unsafe);
+TAG_FLAG(rpc_reopen_outbound_connections, runtime);
+
+METRIC_DEFINE_histogram(server, reactor_load_percent,
+                        "Reactor Thread Load Percentage",
+                        kudu::MetricUnit::kUnits,
+                        "The percentage of time that the reactor is busy "
+                        "(not blocked awaiting network activity). If this metric "
+                        "shows significant samples nears 100%, increasing the "
+                        "number of reactors may be beneficial.", 100, 2);
+
+METRIC_DEFINE_histogram(server, reactor_active_latency_us,
+                        "Reactor Thread Active Latency",
+                        kudu::MetricUnit::kMicroseconds,
+                        "Histogram of the wall clock time for reactor thread wake-ups. "
+                        "The reactor thread is responsible for all network I/O and "
+                        "therefore outliers in this latency histogram directly contribute "
+                        "to the latency of both inbound and outbound RPCs.",
+                        1000000, 2);
+
+namespace kudu {
+namespace rpc {
+
+namespace {
+Status ShutdownError(bool aborted) {
+  const char* msg = "reactor is shutting down";
+  return aborted ?
+      Status::Aborted(msg, "", ESHUTDOWN) :
+      Status::ServiceUnavailable(msg, "", ESHUTDOWN);
+}
+
+// Callback for libev fatal errors (eg running out of file descriptors).
+// Unfortunately libev doesn't plumb these back through to the caller, but
+// instead just expects the callback to abort.
+//
+// This implementation is slightly preferable to the built-in one since
+// it uses a FATAL log message instead of printing to stderr, which might
+// not end up anywhere useful in a daemonized context.
+void LibevSysErr(const char* msg) throw() {
+  PLOG(FATAL) << "LibEV fatal error: " << msg;
+}
+
+void DoInitLibEv() {
+  ev::set_syserr_cb(LibevSysErr);
+}
+
+} // anonymous namespace
+
+ReactorThread::ReactorThread(Reactor *reactor, const MessengerBuilder& bld)
+  : loop_(kDefaultLibEvFlags),
+    cur_time_(MonoTime::Now()),
+    last_unused_tcp_scan_(cur_time_),
+    reactor_(reactor),
+    connection_keepalive_time_(bld.connection_keepalive_time_),
+    coarse_timer_granularity_(bld.coarse_timer_granularity_),
+    total_client_conns_cnt_(0),
+    total_server_conns_cnt_(0) {
+
+  if (bld.metric_entity_) {
+    invoke_us_histogram_ =
+        METRIC_reactor_active_latency_us.Instantiate(bld.metric_entity_);
+    load_percent_histogram_ =
+        METRIC_reactor_load_percent.Instantiate(bld.metric_entity_);
+  }
+}
+
+Status ReactorThread::Init() {
+  DCHECK(thread_.get() == nullptr) << "Already started";
+  DVLOG(6) << "Called ReactorThread::Init()";
+  // Register to get async notifications in our epoll loop.
+  async_.set(loop_);
+  async_.set<ReactorThread, &ReactorThread::AsyncHandler>(this); // NOLINT(*)
+  async_.start();
+
+  // Register the timer watcher.
+  // The timer is used for closing old TCP connections and applying
+  // backpressure.
+  timer_.set(loop_);
+  timer_.set<ReactorThread, &ReactorThread::TimerHandler>(this); // NOLINT(*)
+  timer_.start(coarse_timer_granularity_.ToSeconds(),
+               coarse_timer_granularity_.ToSeconds());
+
+  // Register our callbacks. ev++ doesn't provide handy wrappers for these.
+  ev_set_userdata(loop_, this);
+  ev_set_loop_release_cb(loop_, &ReactorThread::AboutToPollCb, &ReactorThread::PollCompleteCb);
+  ev_set_invoke_pending_cb(loop_, &ReactorThread::InvokePendingCb);
+
+  // Create Reactor thread.
+  return kudu::Thread::Create("reactor", "rpc reactor", &ReactorThread::RunThread, this, &thread_);
+}
+
+void ReactorThread::InvokePendingCb(struct ev_loop* loop) {
+  // Calculate the number of cycles spent calling our callbacks.
+  // This is called quite frequently so we use CycleClock rather than MonoTime
+  // since it's a bit faster.
+  int64_t start = CycleClock::Now();
+  ev_invoke_pending(loop);
+  int64_t dur_cycles = CycleClock::Now() - start;
+
+  // Contribute this to our histogram.
+  ReactorThread* thr = static_cast<ReactorThread*>(ev_userdata(loop));
+  if (thr->invoke_us_histogram_) {
+    thr->invoke_us_histogram_->Increment(dur_cycles * 1000000 / base::CyclesPerSecond());
+  }
+}
+
+void ReactorThread::AboutToPollCb(struct ev_loop* loop) noexcept {
+  // Store the current time in a member variable to be picked up below
+  // in PollCompleteCb.
+  ReactorThread* thr = static_cast<ReactorThread*>(ev_userdata(loop));
+  thr->cycle_clock_before_poll_ = CycleClock::Now();
+}
+
+void ReactorThread::PollCompleteCb(struct ev_loop* loop) noexcept {
+  // First things first, capture the time, so that this is as accurate as possible
+  int64_t cycle_clock_after_poll = CycleClock::Now();
+
+  // Record it in our accounting.
+  ReactorThread* thr = static_cast<ReactorThread*>(ev_userdata(loop));
+  DCHECK_NE(thr->cycle_clock_before_poll_, -1)
+      << "PollCompleteCb called without corresponding AboutToPollCb";
+
+  int64_t poll_cycles = cycle_clock_after_poll - thr->cycle_clock_before_poll_;
+  thr->cycle_clock_before_poll_ = -1;
+  thr->total_poll_cycles_ += poll_cycles;
+}
+
+void ReactorThread::Shutdown(Messenger::ShutdownMode mode) {
+  CHECK(reactor_->closing()) << "Should be called after setting closing_ flag";
+
+  VLOG(1) << name() << ": shutting down Reactor thread.";
+  WakeThread();
+
+  if (mode == Messenger::ShutdownMode::SYNC) {
+    // Join() will return a bad status if asked to join on the currently
+    // running thread.
+    CHECK_OK(ThreadJoiner(thread_.get()).Join());
+  }
+}
+
+void ReactorThread::ShutdownInternal() {
+  DCHECK(IsCurrentThread());
+
+  // Tear down any outbound TCP connections.
+  Status service_unavailable = ShutdownError(false);
+  VLOG(1) << name() << ": tearing down outbound TCP connections...";
+  for (const auto& elem : client_conns_) {
+    const auto& conn = elem.second;
+    VLOG(1) << name() << ": shutting down " << conn->ToString();
+    conn->Shutdown(service_unavailable);
+  }
+  client_conns_.clear();
+
+  // Tear down any inbound TCP connections.
+  VLOG(1) << name() << ": tearing down inbound TCP connections...";
+  for (const auto& conn : server_conns_) {
+    VLOG(1) << name() << ": shutting down " << conn->ToString();
+    conn->Shutdown(service_unavailable);
+  }
+  server_conns_.clear();
+
+  // Abort any scheduled tasks.
+  //
+  // These won't be found in the ReactorThread's list of pending tasks
+  // because they've been "run" (that is, they've been scheduled).
+  Status aborted = ShutdownError(true); // aborted
+  while (!scheduled_tasks_.empty()) {
+    DelayedTask* t = &scheduled_tasks_.front();
+    scheduled_tasks_.pop_front();
+    t->Abort(aborted); // should also free the task.
+  }
+
+  // Remove the OpenSSL thread state.
+  //
+  // As of OpenSSL 1.1, this [1] is a no-op and can be ignored.
+  //
+  // 1. https://www.openssl.org/docs/man1.1.0/crypto/ERR_remove_thread_state.html
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  ERR_remove_thread_state(nullptr);
+#endif
+}
+
+ReactorTask::ReactorTask() {
+}
+ReactorTask::~ReactorTask() {
+}
+
+Status ReactorThread::GetMetrics(ReactorMetrics* metrics) {
+  DCHECK(IsCurrentThread());
+  metrics->num_client_connections_ = client_conns_.size();
+  metrics->num_server_connections_ = server_conns_.size();
+  metrics->total_client_connections_ = total_client_conns_cnt_;
+  metrics->total_server_connections_ = total_server_conns_cnt_;
+  return Status::OK();
+}
+
+Status ReactorThread::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                                      DumpRunningRpcsResponsePB* resp) {
+  DCHECK(IsCurrentThread());
+  for (const scoped_refptr<Connection>& conn : server_conns_) {
+    RETURN_NOT_OK(conn->DumpPB(req, resp->add_inbound_connections()));
+  }
+  for (const conn_multimap_t::value_type& entry : client_conns_) {
+    Connection* conn = entry.second.get();
+    RETURN_NOT_OK(conn->DumpPB(req, resp->add_outbound_connections()));
+  }
+  return Status::OK();
+}
+
+void ReactorThread::WakeThread() {
+  // libev uses some lock-free synchronization, but doesn't have TSAN annotations.
+  // See http://lists.schmorp.de/pipermail/libev/2013q2/002178.html or KUDU-366
+  // for examples.
+  debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan;
+  async_.send();
+}
+
+// Handle async events.  These events are sent to the reactor by other
+// threads that want to bring something to our attention, like the fact that
+// we're shutting down, or the fact that there is a new outbound Transfer
+// ready to send.
+void ReactorThread::AsyncHandler(ev::async& /*watcher*/, int /*revents*/) {
+  DCHECK(IsCurrentThread());
+
+  if (PREDICT_FALSE(reactor_->closing())) {
+    ShutdownInternal();
+    loop_.break_loop(); // break the epoll loop and terminate the thread
+    return;
+  }
+
+  boost::intrusive::list<ReactorTask> tasks;
+  reactor_->DrainTaskQueue(&tasks);
+
+  while (!tasks.empty()) {
+    ReactorTask& task = tasks.front();
+    tasks.pop_front();
+    task.Run(this);
+  }
+}
+
+void ReactorThread::RegisterConnection(scoped_refptr<Connection> conn) {
+  DCHECK(IsCurrentThread());
+
+  Status s = StartConnectionNegotiation(conn);
+  if (PREDICT_FALSE(!s.ok())) {
+    LOG(ERROR) << "Server connection negotiation failed: " << s.ToString();
+    DestroyConnection(conn.get(), s);
+    return;
+  }
+  ++total_server_conns_cnt_;
+  server_conns_.emplace_back(std::move(conn));
+}
+
+void ReactorThread::AssignOutboundCall(shared_ptr<OutboundCall> call) {
+  DCHECK(IsCurrentThread());
+
+  // Skip if the outbound has been cancelled already.
+  if (PREDICT_FALSE(call->IsCancelled())) {
+    return;
+  }
+
+  scoped_refptr<Connection> conn;
+  Status s = FindOrStartConnection(call->conn_id(),
+                                   call->controller()->credentials_policy(),
+                                   &conn);
+  if (PREDICT_FALSE(!s.ok())) {
+    call->SetFailed(std::move(s), OutboundCall::Phase::CONNECTION_NEGOTIATION);
+    return;
+  }
+
+  conn->QueueOutboundCall(std::move(call));
+}
+
+void ReactorThread::CancelOutboundCall(const shared_ptr<OutboundCall>& call) {
+  DCHECK(IsCurrentThread());
+
+  // If the callback has been invoked already, the cancellation is a no-op.
+  // The controller may be gone already if the callback has been invoked.
+  if (call->IsFinished()) {
+    return;
+  }
+
+  scoped_refptr<Connection> conn;
+  if (FindConnection(call->conn_id(),
+                     call->controller()->credentials_policy(),
+                     &conn)) {
+    conn->CancelOutboundCall(call);
+  }
+  call->Cancel();
+}
+
+//
+// Handles timer events.  The periodic timer:
+//
+// 1. updates Reactor::cur_time_
+// 2. every tcp_conn_timeo_ seconds, close down connections older than
+//    tcp_conn_timeo_ seconds.
+//
+void ReactorThread::TimerHandler(ev::timer& /*watcher*/, int revents) {
+  DCHECK(IsCurrentThread());
+  if (EV_ERROR & revents) {
+    LOG(WARNING) << "Reactor " << name() << " got an error in "
+      "the timer handler.";
+    return;
+  }
+  cur_time_ = MonoTime::Now();
+
+  // Compute load percentage.
+  int64_t now_cycles = CycleClock::Now();
+  if (last_load_measurement_.time_cycles != -1) {
+    int64_t cycles_delta = (now_cycles - last_load_measurement_.time_cycles);
+    int64_t poll_cycles_delta = total_poll_cycles_ - last_load_measurement_.poll_cycles;
+    double poll_fraction = static_cast<double>(poll_cycles_delta) / cycles_delta;
+    double active_fraction = 1 - poll_fraction;
+    if (load_percent_histogram_) {
+      load_percent_histogram_->Increment(static_cast<int>(active_fraction * 100));
+    }
+  }
+  last_load_measurement_.time_cycles = now_cycles;
+  last_load_measurement_.poll_cycles = total_poll_cycles_;
+
+  ScanIdleConnections();
+}
+
+void ReactorThread::RegisterTimeout(ev::timer *watcher) {
+  watcher->set(loop_);
+}
+
+void ReactorThread::ScanIdleConnections() {
+  DCHECK(IsCurrentThread());
+  // Enforce TCP connection timeouts: server-side connections.
+  const auto server_conns_end = server_conns_.end();
+  uint64_t timed_out = 0;
+  // Scan for idle server connections if it's enabled.
+  if (connection_keepalive_time_ >= MonoDelta::FromMilliseconds(0)) {
+    for (auto it = server_conns_.begin(); it != server_conns_end; ) {
+      Connection* conn = it->get();
+      if (!conn->Idle()) {
+        VLOG(10) << "Connection " << conn->ToString() << " not idle";
+        ++it;
+        continue;
+      }
+
+      const MonoDelta connection_delta(cur_time_ - conn->last_activity_time());
+      if (connection_delta <= connection_keepalive_time_) {
+        ++it;
+        continue;
+      }
+
+      conn->Shutdown(Status::NetworkError(
+          Substitute("connection timed out after $0", connection_keepalive_time_.ToString())));
+      VLOG(1) << "Timing out connection " << conn->ToString() << " - it has been idle for "
+              << connection_delta.ToString();
+      ++timed_out;
+      it = server_conns_.erase(it);
+    }
+  }
+  // Take care of idle client-side connections marked for shutdown.
+  uint64_t shutdown = 0;
+  for (auto it = client_conns_.begin(); it != client_conns_.end();) {
+    Connection* conn = it->second.get();
+    if (conn->scheduled_for_shutdown() && conn->Idle()) {
+      conn->Shutdown(Status::NetworkError(
+          "connection has been marked for shutdown"));
+      it = client_conns_.erase(it);
+      ++shutdown;
+    } else {
+      ++it;
+    }
+  }
+  // TODO(aserbin): clients may want to set their keepalive timeout for idle
+  //                but not scheduled for shutdown connections.
+
+  VLOG_IF(1, timed_out > 0) << name() << ": timed out " << timed_out << " TCP connections.";
+  VLOG_IF(1, shutdown > 0) << name() << ": shutdown " << shutdown << " TCP connections.";
+}
+
+const std::string& ReactorThread::name() const {
+  return reactor_->name();
+}
+
+MonoTime ReactorThread::cur_time() const {
+  return cur_time_;
+}
+
+Reactor *ReactorThread::reactor() {
+  return reactor_;
+}
+
+bool ReactorThread::IsCurrentThread() const {
+  return thread_.get() == kudu::Thread::current_thread();
+}
+
+void ReactorThread::RunThread() {
+  ThreadRestrictions::SetWaitAllowed(false);
+  ThreadRestrictions::SetIOAllowed(false);
+  DVLOG(6) << "Calling ReactorThread::RunThread()...";
+  loop_.run(0);
+  VLOG(1) << name() << " thread exiting.";
+
+  // No longer need the messenger. This causes the messenger to
+  // get deleted when all the reactors exit.
+  reactor_->messenger_.reset();
+}
+
+bool ReactorThread::FindConnection(const ConnectionId& conn_id,
+                                   CredentialsPolicy cred_policy,
+                                   scoped_refptr<Connection>* conn) {
+  DCHECK(IsCurrentThread());
+  const auto range = client_conns_.equal_range(conn_id);
+  scoped_refptr<Connection> found_conn;
+  for (auto it = range.first; it != range.second;) {
+    const auto& c = it->second.get();
+    // * Do not use connections scheduled for shutdown to place new calls.
+    //
+    // * Do not use a connection with a non-compliant credentials policy.
+    //   Instead, open a new one, while marking the former as scheduled for
+    //   shutdown. This process converges: any connection that satisfies the
+    //   PRIMARY_CREDENTIALS policy automatically satisfies the ANY_CREDENTIALS
+    //   policy as well. The idea is to keep only one usable connection
+    //   identified by the specified 'conn_id'.
+    //
+    // * If the test-only 'one-connection-per-RPC' mode is enabled, connections
+    //   are re-established at every RPC call.
+    if (c->scheduled_for_shutdown() ||
+        !c->SatisfiesCredentialsPolicy(cred_policy) ||
+        PREDICT_FALSE(FLAGS_rpc_reopen_outbound_connections)) {
+      if (c->Idle()) {
+        // Shutdown idle connections to the target destination. Non-idle ones
+        // will be taken care of later by the idle connection scanner.
+        DCHECK_EQ(Connection::CLIENT, c->direction());
+        c->Shutdown(Status::NetworkError("connection is closed due to non-reuse policy"));
+        it = client_conns_.erase(it);
+        continue;
+      }
+      c->set_scheduled_for_shutdown();
+    } else {
+      DCHECK(!found_conn);
+      found_conn = c;
+      // Appropriate connection is found; continue further to take care of the
+      // rest of connections to mark them for shutdown if they are not
+      // satisfying the policy.
+    }
+    ++it;
+  }
+  if (found_conn) {
+    // Found matching not-to-be-shutdown connection: return it as the result.
+    conn->swap(found_conn);
+    return true;
+  }
+  return false;
+}
+
+Status ReactorThread::FindOrStartConnection(const ConnectionId& conn_id,
+                                            CredentialsPolicy cred_policy,
+                                            scoped_refptr<Connection>* conn) {
+  DCHECK(IsCurrentThread());
+  if (FindConnection(conn_id, cred_policy, conn)) {
+    return Status::OK();
+  }
+
+  // No connection to this remote. Need to create one.
+  VLOG(2) << name() << " FindOrStartConnection: creating "
+          << "new connection for " << conn_id.remote().ToString();
+
+  // Create a new socket and start connecting to the remote.
+  Socket sock;
+  RETURN_NOT_OK(CreateClientSocket(&sock));
+  RETURN_NOT_OK(StartConnect(&sock, conn_id.remote()));
+
+  unique_ptr<Socket> new_socket(new Socket(sock.Release()));
+
+  // Register the new connection in our map.
+  *conn = new Connection(
+      this, conn_id.remote(), std::move(new_socket), Connection::CLIENT, cred_policy);
+  (*conn)->set_outbound_connection_id(conn_id);
+
+  // Kick off blocking client connection negotiation.
+  Status s = StartConnectionNegotiation(*conn);
+  if (s.IsIllegalState()) {
+    // Return a nicer error message to the user indicating -- if we just
+    // forward the status we'd get something generic like "ThreadPool is closing".
+    return Status::ServiceUnavailable("Client RPC Messenger shutting down");
+  }
+  // Propagate any other errors as-is.
+  RETURN_NOT_OK_PREPEND(s, "Unable to start connection negotiation thread");
+
+  // Insert into the client connection map to avoid duplicate connection requests.
+  client_conns_.emplace(conn_id, *conn);
+  ++total_client_conns_cnt_;
+
+  return Status::OK();
+}
+
+Status ReactorThread::StartConnectionNegotiation(const scoped_refptr<Connection>& conn) {
+  DCHECK(IsCurrentThread());
+
+  // Set a limit on how long the server will negotiate with a new client.
+  MonoTime deadline = MonoTime::Now() +
+      MonoDelta::FromMilliseconds(reactor()->messenger()->rpc_negotiation_timeout_ms());
+
+  scoped_refptr<Trace> trace(new Trace());
+  ADOPT_TRACE(trace.get());
+  TRACE("Submitting negotiation task for $0", conn->ToString());
+  auto authentication = reactor()->messenger()->authentication();
+  auto encryption = reactor()->messenger()->encryption();
+  ThreadPool* negotiation_pool =
+      reactor()->messenger()->negotiation_pool(conn->direction());
+  RETURN_NOT_OK(negotiation_pool->SubmitClosure(
+        Bind(&Negotiation::RunNegotiation, conn, authentication, encryption, deadline)));
+  return Status::OK();
+}
+
+void ReactorThread::CompleteConnectionNegotiation(
+    const scoped_refptr<Connection>& conn,
+    const Status& status,
+    unique_ptr<ErrorStatusPB> rpc_error) {
+  DCHECK(IsCurrentThread());
+  if (PREDICT_FALSE(!status.ok())) {
+    DestroyConnection(conn.get(), status, std::move(rpc_error));
+    return;
+  }
+
+  // Switch the socket back to non-blocking mode after negotiation.
+  Status s = conn->SetNonBlocking(true);
+  if (PREDICT_FALSE(!s.ok())) {
+    LOG(DFATAL) << "Unable to set connection to non-blocking mode: " << s.ToString();
+    DestroyConnection(conn.get(), s, std::move(rpc_error));
+    return;
+  }
+
+  conn->MarkNegotiationComplete();
+  conn->EpollRegister(loop_);
+}
+
+Status ReactorThread::CreateClientSocket(Socket *sock) {
+  Status ret = sock->Init(Socket::FLAG_NONBLOCKING);
+  if (ret.ok()) {
+    ret = sock->SetNoDelay(true);
+  }
+  LOG_IF(WARNING, !ret.ok())
+      << "failed to create an outbound connection because a new socket could not be created: "
+      << ret.ToString();
+  return ret;
+}
+
+Status ReactorThread::StartConnect(Socket *sock, const Sockaddr& remote) {
+  const Status ret = sock->Connect(remote);
+  if (ret.ok()) {
+    VLOG(3) << "StartConnect: connect finished immediately for " << remote.ToString();
+    return Status::OK();
+  }
+
+  int posix_code = ret.posix_code();
+  if (Socket::IsTemporarySocketError(posix_code) || posix_code == EINPROGRESS) {
+    VLOG(3) << "StartConnect: connect in progress for " << remote.ToString();
+    return Status::OK();
+  }
+
+  LOG(WARNING) << "Failed to create an outbound connection to " << remote.ToString()
+               << " because connect() failed: " << ret.ToString();
+  return ret;
+}
+
+void ReactorThread::DestroyConnection(Connection *conn,
+                                      const Status& conn_status,
+                                      unique_ptr<ErrorStatusPB> rpc_error) {
+  DCHECK(IsCurrentThread());
+
+  conn->Shutdown(conn_status, std::move(rpc_error));
+
+  // Unlink connection from lists.
+  if (conn->direction() == Connection::CLIENT) {
+    const auto range = client_conns_.equal_range(conn->outbound_connection_id());
+    CHECK(range.first != range.second) << "Couldn't find connection " << conn->ToString();
+    // The client_conns_ container is a multi-map.
+    for (auto it = range.first; it != range.second;) {
+      if (it->second.get() == conn) {
+        it = client_conns_.erase(it);
+        break;
+      }
+      ++it;
+    }
+  } else if (conn->direction() == Connection::SERVER) {
+    auto it = server_conns_.begin();
+    while (it != server_conns_.end()) {
+      if ((*it).get() == conn) {
+        server_conns_.erase(it);
+        break;
+      }
+      ++it;
+    }
+  }
+}
+
+DelayedTask::DelayedTask(boost::function<void(const Status&)> func,
+                         MonoDelta when)
+    : func_(std::move(func)),
+      when_(when),
+      thread_(nullptr) {
+}
+
+void DelayedTask::Run(ReactorThread* thread) {
+  DCHECK(thread_ == nullptr) << "Task has already been scheduled";
+  DCHECK(thread->IsCurrentThread());
+  DCHECK(!is_linked()) << "Should not be linked on pending_tasks_ anymore";
+
+  // Schedule the task to run later.
+  thread_ = thread;
+  timer_.set(thread->loop_);
+  timer_.set<DelayedTask, &DelayedTask::TimerHandler>(this); // NOLINT(*)
+  timer_.start(when_.ToSeconds(), // after
+               0);                // repeat
+  thread_->scheduled_tasks_.push_back(*this);
+}
+
+void DelayedTask::Abort(const Status& abort_status) {
+  func_(abort_status);
+  delete this;
+}
+
+void DelayedTask::TimerHandler(ev::timer& /*watcher*/, int revents) {
+  DCHECK(is_linked()) << "should be linked on scheduled_tasks_";
+  // We will free this task's memory.
+  thread_->scheduled_tasks_.erase(thread_->scheduled_tasks_.iterator_to(*this));
+
+  if (EV_ERROR & revents) {
+    string msg = "Delayed task got an error in its timer handler";
+    LOG(WARNING) << msg;
+    Abort(Status::Aborted(msg)); // Will delete 'this'.
+  } else {
+    func_(Status::OK());
+    delete this;
+  }
+}
+
+Reactor::Reactor(shared_ptr<Messenger> messenger,
+                 int index, const MessengerBuilder& bld)
+    : messenger_(std::move(messenger)),
+      name_(StringPrintf("%s_R%03d", messenger_->name().c_str(), index)),
+      closing_(false),
+      thread_(this, bld) {
+  static std::once_flag libev_once;
+  std::call_once(libev_once, DoInitLibEv);
+}
+
+Status Reactor::Init() {
+  DVLOG(6) << "Called Reactor::Init()";
+  return thread_.Init();
+}
+
+void Reactor::Shutdown(Messenger::ShutdownMode mode) {
+  {
+    std::lock_guard<LockType> l(lock_);
+    if (closing_) {
+      return;
+    }
+    closing_ = true;
+  }
+
+  thread_.Shutdown(mode);
+
+  // Abort all pending tasks. No new tasks can get scheduled after this
+  // because ScheduleReactorTask() tests the closing_ flag set above.
+  Status aborted = ShutdownError(true);
+  while (!pending_tasks_.empty()) {
+    ReactorTask& task = pending_tasks_.front();
+    pending_tasks_.pop_front();
+    task.Abort(aborted);
+  }
+}
+
+Reactor::~Reactor() {
+  Shutdown(Messenger::ShutdownMode::ASYNC);
+}
+
+const std::string& Reactor::name() const {
+  return name_;
+}
+
+bool Reactor::closing() const {
+  std::lock_guard<LockType> l(lock_);
+  return closing_;
+}
+
+// Task to call an arbitrary function within the reactor thread.
+class RunFunctionTask : public ReactorTask {
+ public:
+  explicit RunFunctionTask(boost::function<Status()> f)
+      : function_(std::move(f)), latch_(1) {}
+
+  void Run(ReactorThread* /*reactor*/) override {
+    status_ = function_();
+    latch_.CountDown();
+  }
+  void Abort(const Status& status) override {
+    status_ = status;
+    latch_.CountDown();
+  }
+
+  // Wait until the function has completed, and return the Status
+  // returned by the function.
+  Status Wait() {
+    latch_.Wait();
+    return status_;
+  }
+
+ private:
+  boost::function<Status()> function_;
+  Status status_;
+  CountDownLatch latch_;
+};
+
+Status Reactor::GetMetrics(ReactorMetrics *metrics) {
+  return RunOnReactorThread(boost::bind(&ReactorThread::GetMetrics, &thread_, metrics));
+}
+
+Status Reactor::RunOnReactorThread(const boost::function<Status()>& f) {
+  RunFunctionTask task(f);
+  ScheduleReactorTask(&task);
+  return task.Wait();
+}
+
+Status Reactor::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                                DumpRunningRpcsResponsePB* resp) {
+  return RunOnReactorThread(boost::bind(&ReactorThread::DumpRunningRpcs, &thread_,
+                                        boost::ref(req), resp));
+}
+
+class RegisterConnectionTask : public ReactorTask {
+ public:
+  explicit RegisterConnectionTask(scoped_refptr<Connection> conn)
+      : conn_(std::move(conn)) {
+  }
+
+  void Run(ReactorThread* reactor) override {
+    reactor->RegisterConnection(std::move(conn_));
+    delete this;
+  }
+
+  void Abort(const Status& /*status*/) override {
+    // We don't need to Shutdown the connection since it was never registered.
+    // This is only used for inbound connections, and inbound connections will
+    // never have any calls added to them until they've been registered.
+    delete this;
+  }
+
+ private:
+  scoped_refptr<Connection> conn_;
+};
+
+void Reactor::RegisterInboundSocket(Socket *socket, const Sockaddr& remote) {
+  VLOG(3) << name_ << ": new inbound connection to " << remote.ToString();
+  unique_ptr<Socket> new_socket(new Socket(socket->Release()));
+  auto task = new RegisterConnectionTask(
+      new Connection(&thread_, remote, std::move(new_socket), Connection::SERVER));
+  ScheduleReactorTask(task);
+}
+
+// Task which runs in the reactor thread to assign an outbound call
+// to a connection.
+class AssignOutboundCallTask : public ReactorTask {
+ public:
+  explicit AssignOutboundCallTask(shared_ptr<OutboundCall> call)
+      : call_(std::move(call)) {}
+
+  void Run(ReactorThread* reactor) override {
+    reactor->AssignOutboundCall(std::move(call_));
+    delete this;
+  }
+
+  void Abort(const Status& status) override {
+    // It doesn't matter what is the actual phase of the OutboundCall: just set
+    // it to Phase::REMOTE_CALL to finalize the state of the call.
+    call_->SetFailed(status, OutboundCall::Phase::REMOTE_CALL);
+    delete this;
+  }
+
+ private:
+  shared_ptr<OutboundCall> call_;
+};
+
+void Reactor::QueueOutboundCall(const shared_ptr<OutboundCall>& call) {
+  DVLOG(3) << name_ << ": queueing outbound call "
+           << call->ToString() << " to remote " << call->conn_id().remote().ToString();
+  // Test cancellation when 'call_' is in 'READY' state.
+  if (PREDICT_FALSE(call->ShouldInjectCancellation())) {
+    QueueCancellation(call);
+  }
+  ScheduleReactorTask(new AssignOutboundCallTask(call));
+}
+
+class CancellationTask : public ReactorTask {
+ public:
+  explicit CancellationTask(shared_ptr<OutboundCall> call)
+      : call_(std::move(call)) {}
+
+  void Run(ReactorThread* reactor) override {
+    reactor->CancelOutboundCall(call_);
+    delete this;
+  }
+
+  void Abort(const Status& /*status*/) override {
+    delete this;
+  }
+
+ private:
+  shared_ptr<OutboundCall> call_;
+};
+
+void Reactor::QueueCancellation(const shared_ptr<OutboundCall>& call) {
+  ScheduleReactorTask(new CancellationTask(call));
+}
+
+void Reactor::ScheduleReactorTask(ReactorTask *task) {
+  {
+    std::unique_lock<LockType> l(lock_);
+    if (closing_) {
+      // We guarantee the reactor lock is not taken when calling Abort().
+      l.unlock();
+      task->Abort(ShutdownError(false));
+      return;
+    }
+    pending_tasks_.push_back(*task);
+  }
+  thread_.WakeThread();
+}
+
+bool Reactor::DrainTaskQueue(boost::intrusive::list<ReactorTask> *tasks) { // NOLINT(*)
+  std::lock_guard<LockType> l(lock_);
+  if (closing_) {
+    return false;
+  }
+  tasks->swap(pending_tasks_);
+  return true;
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/reactor.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/reactor.h b/be/src/kudu/rpc/reactor.h
new file mode 100644
index 0000000..8884f54
--- /dev/null
+++ b/be/src/kudu/rpc/reactor.h
@@ -0,0 +1,427 @@
+// 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.
+#ifndef KUDU_RPC_REACTOR_H
+#define KUDU_RPC_REACTOR_H
+
+#include <cstdint>
+#include <list>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include <boost/function.hpp> // IWYU pragma: keep
+#include <boost/intrusive/list.hpp>
+#include <boost/intrusive/list_hook.hpp>
+#include <ev++.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+namespace kudu {
+
+class Sockaddr;
+class Socket;
+
+namespace rpc {
+
+typedef std::list<scoped_refptr<Connection>> conn_list_t;
+
+class DumpRunningRpcsRequestPB;
+class DumpRunningRpcsResponsePB;
+class OutboundCall;
+class Reactor;
+class ReactorThread;
+enum class CredentialsPolicy;
+
+// Simple metrics information from within a reactor.
+// TODO(todd): switch these over to use util/metrics.h style metrics.
+struct ReactorMetrics {
+  // Number of client RPC connections currently connected.
+  int32_t num_client_connections_;
+  // Number of server RPC connections currently connected.
+  int32_t num_server_connections_;
+
+  // Total number of client RPC connections opened during Reactor's lifetime.
+  uint64_t total_client_connections_;
+  // Total number of server RPC connections opened during Reactor's lifetime.
+  uint64_t total_server_connections_;
+};
+
+// A task which can be enqueued to run on the reactor thread.
+class ReactorTask : public boost::intrusive::list_base_hook<> {
+ public:
+  ReactorTask();
+
+  // Run the task. 'reactor' is guaranteed to be the current thread.
+  virtual void Run(ReactorThread *reactor) = 0;
+
+  // Abort the task, in the case that the reactor shut down before the
+  // task could be processed. This may or may not run on the reactor thread
+  // itself.
+  //
+  // The Reactor guarantees that the Reactor lock is free when this
+  // method is called.
+  virtual void Abort(const Status &abort_status) {}
+
+  virtual ~ReactorTask();
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(ReactorTask);
+};
+
+// A ReactorTask that is scheduled to run at some point in the future.
+//
+// Semantically it works like RunFunctionTask with a few key differences:
+// 1. The user function is called during Abort. Put another way, the
+//    user function is _always_ invoked, even during reactor shutdown.
+// 2. To differentiate between Abort and non-Abort, the user function
+//    receives a Status as its first argument.
+class DelayedTask : public ReactorTask {
+ public:
+  DelayedTask(boost::function<void(const Status &)> func, MonoDelta when);
+
+  // Schedules the task for running later but doesn't actually run it yet.
+  void Run(ReactorThread* thread) override;
+
+  // Behaves like ReactorTask::Abort.
+  void Abort(const Status& abort_status) override;
+
+ private:
+  // libev callback for when the registered timer fires.
+  void TimerHandler(ev::timer& watcher, int revents);
+
+  // User function to invoke when timer fires or when task is aborted.
+  const boost::function<void(const Status&)> func_;
+
+  // Delay to apply to this task.
+  const MonoDelta when_;
+
+  // Link back to registering reactor thread.
+  ReactorThread* thread_;
+
+  // libev timer. Set when Run() is invoked.
+  ev::timer timer_;
+};
+
+// A ReactorThread is a libev event handler thread which manages I/O
+// on a list of sockets.
+//
+// All methods in this class are _only_ called from the reactor thread itself
+// except where otherwise specified. New methods should DCHECK(IsCurrentThread())
+// to ensure this.
+class ReactorThread {
+ public:
+  friend class Connection;
+
+  // Client-side connection map. Multiple connections could be open to a remote
+  // server if multiple credential policies are used for individual RPCs.
+  typedef std::unordered_multimap<ConnectionId, scoped_refptr<Connection>,
+                                  ConnectionIdHash, ConnectionIdEqual>
+      conn_multimap_t;
+
+  ReactorThread(Reactor *reactor, const MessengerBuilder &bld);
+
+  // This may be called from another thread.
+  Status Init();
+
+  // Add any connections on this reactor thread into the given status dump.
+  Status DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                         DumpRunningRpcsResponsePB* resp);
+
+  // Shuts down a reactor thread, optionally waiting for it to exit.
+  // Reactor::Shutdown() must have been called already.
+  //
+  // If mode == SYNC, may not be called from the reactor thread itself.
+  void Shutdown(Messenger::ShutdownMode mode);
+
+  // This method is thread-safe.
+  void WakeThread();
+
+  // libev callback for handling async notifications in our epoll thread.
+  void AsyncHandler(ev::async &watcher, int revents);
+
+  // libev callback for handling timer events in our epoll thread.
+  void TimerHandler(ev::timer &watcher, int revents);
+
+  // Register an epoll timer watcher with our event loop.
+  // Does not set a timeout or start it.
+  void RegisterTimeout(ev::timer *watcher);
+
+  // This may be called from another thread.
+  const std::string &name() const;
+
+  MonoTime cur_time() const;
+
+  // This may be called from another thread.
+  Reactor *reactor();
+
+  // Return true if this reactor thread is the thread currently
+  // running. Should be used in DCHECK assertions.
+  bool IsCurrentThread() const;
+
+  // Begin the process of connection negotiation.
+  // Must be called from the reactor thread.
+  Status StartConnectionNegotiation(const scoped_refptr<Connection>& conn);
+
+  // Transition back from negotiating to processing requests.
+  // Must be called from the reactor thread.
+  void CompleteConnectionNegotiation(const scoped_refptr<Connection>& conn,
+                                     const Status& status,
+                                     std::unique_ptr<ErrorStatusPB> rpc_error);
+
+  // Collect metrics.
+  // Must be called from the reactor thread.
+  Status GetMetrics(ReactorMetrics *metrics);
+
+ private:
+  friend class AssignOutboundCallTask;
+  friend class CancellationTask;
+  friend class RegisterConnectionTask;
+  friend class DelayedTask;
+
+  // Run the main event loop of the reactor.
+  void RunThread();
+
+  // When libev has noticed that it needs to wake up an application watcher,
+  // it calls this callback. The callback simply calls back into libev's
+  // ev_invoke_pending() to trigger all the watcher callbacks, but
+  // wraps it with latency measurements.
+  static void InvokePendingCb(struct ev_loop* loop);
+
+  // Similarly, libev calls these functions before/after invoking epoll_wait().
+  // We use these to measure the amount of time spent waiting.
+  //
+  // NOTE: 'noexcept' is required to avoid compilation errors due to libev's
+  // use of the same exception specification.
+  static void AboutToPollCb(struct ev_loop* loop) noexcept;
+  static void PollCompleteCb(struct ev_loop* loop) noexcept;
+
+  // Find a connection to the given remote and returns it in 'conn'.
+  // Returns true if a connection is found. Returns false otherwise.
+  bool FindConnection(const ConnectionId& conn_id,
+                      CredentialsPolicy cred_policy,
+                      scoped_refptr<Connection>* conn);
+
+  // Find or create a new connection to the given remote.
+  // If such a connection already exists, returns that, otherwise creates a new one.
+  // May return a bad Status if the connect() call fails.
+  // The resulting connection object is managed internally by the reactor thread.
+  Status FindOrStartConnection(const ConnectionId& conn_id,
+                               CredentialsPolicy cred_policy,
+                               scoped_refptr<Connection>* conn);
+
+  // Shut down the given connection, removing it from the connection tracking
+  // structures of this reactor.
+  //
+  // The connection is not explicitly deleted -- shared_ptr reference counting
+  // may hold on to the object after this, but callers should assume that it
+  // _may_ be deleted by this call.
+  void DestroyConnection(Connection *conn, const Status &conn_status,
+                         std::unique_ptr<ErrorStatusPB> rpc_error = {});
+
+  // Scan any open connections for idle ones that have been idle longer than
+  // connection_keepalive_time_. If connection_keepalive_time_ < 0, the scan
+  // is skipped.
+  void ScanIdleConnections();
+
+  // Create a new client socket (non-blocking, NODELAY)
+  static Status CreateClientSocket(Socket *sock);
+
+  // Initiate a new connection on the given socket.
+  static Status StartConnect(Socket *sock, const Sockaddr &remote);
+
+  // Assign a new outbound call to the appropriate connection object.
+  // If this fails, the call is marked failed and completed.
+  void AssignOutboundCall(std::shared_ptr<OutboundCall> call);
+
+  // Cancel the outbound call. May update corresponding connection
+  // object to remove call from the CallAwaitingResponse object.
+  // Also mark the call as slated for cancellation so the callback
+  // may be invoked early if the RPC hasn't yet been sent or if it's
+  // waiting for a response from the remote.
+  void CancelOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // Register a new connection.
+  void RegisterConnection(scoped_refptr<Connection> conn);
+
+  // Actually perform shutdown of the thread, tearing down any connections,
+  // etc. This is called from within the thread.
+  void ShutdownInternal();
+
+  scoped_refptr<kudu::Thread> thread_;
+
+  // our epoll object (or kqueue, etc).
+  ev::dynamic_loop loop_;
+
+  // Used by other threads to notify the reactor thread
+  ev::async async_;
+
+  // Handles the periodic timer.
+  ev::timer timer_;
+
+  // Scheduled (but not yet run) delayed tasks.
+  //
+  // Each task owns its own memory and must be freed by its TaskRun and
+  // Abort members, provided it was allocated on the heap.
+  boost::intrusive::list<DelayedTask> scheduled_tasks_;
+
+  // The current monotonic time.  Updated every coarse_timer_granularity_secs_.
+  MonoTime cur_time_;
+
+  // last time we did TCP timeouts.
+  MonoTime last_unused_tcp_scan_;
+
+  // Map of sockaddrs to Connection objects for outbound (client) connections.
+  conn_multimap_t client_conns_;
+
+  // List of current connections coming into the server.
+  conn_list_t server_conns_;
+
+  Reactor *reactor_;
+
+  // If a connection has been idle for this much time, it is torn down.
+  const MonoDelta connection_keepalive_time_;
+
+  // Scan for idle connections on this granularity.
+  const MonoDelta coarse_timer_granularity_;
+
+  // Metrics.
+  scoped_refptr<Histogram> invoke_us_histogram_;
+  scoped_refptr<Histogram> load_percent_histogram_;
+
+  // Total number of client connections opened during Reactor's lifetime.
+  uint64_t total_client_conns_cnt_;
+
+  // Total number of server connections opened during Reactor's lifetime.
+  uint64_t total_server_conns_cnt_;
+
+  // Set prior to calling epoll and then reset back to -1 after each invocation
+  // completes. Used for accounting total_poll_cycles_.
+  int64_t cycle_clock_before_poll_ = -1;
+
+  // The total number of cycles spent in epoll_wait() since this thread
+  // started.
+  int64_t total_poll_cycles_ = 0;
+
+  // Accounting for determining load average in each cycle of TimerHandler.
+  struct {
+    // The cycle-time at which the load average was last calculated.
+    int64_t time_cycles = -1;
+    // The value of total_poll_cycles_ at the last-recorded time.
+    int64_t poll_cycles = -1;
+  } last_load_measurement_;
+};
+
+// A Reactor manages a ReactorThread
+class Reactor {
+ public:
+  Reactor(std::shared_ptr<Messenger> messenger,
+          int index,
+          const MessengerBuilder &bld);
+  Status Init();
+
+  // Shuts down the reactor and its corresponding thread, optionally waiting
+  // until the thread has exited.
+  void Shutdown(Messenger::ShutdownMode mode);
+
+  ~Reactor();
+
+  const std::string &name() const;
+
+  // Collect metrics about the reactor.
+  Status GetMetrics(ReactorMetrics *metrics);
+
+  // Add any connections on this reactor thread into the given status dump.
+  Status DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                         DumpRunningRpcsResponsePB* resp);
+
+  // Queue a new incoming connection. Takes ownership of the underlying fd from
+  // 'socket', but not the Socket object itself.
+  // If the reactor is already shut down, takes care of closing the socket.
+  void RegisterInboundSocket(Socket *socket, const Sockaddr &remote);
+
+  // Queue a new call to be sent. If the reactor is already shut down, marks
+  // the call as failed.
+  void QueueOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // Queue a new reactor task to cancel an outbound call.
+  void QueueCancellation(const std::shared_ptr<OutboundCall> &call);
+
+  // Schedule the given task's Run() method to be called on the
+  // reactor thread.
+  // If the reactor shuts down before it is run, the Abort method will be
+  // called.
+  // Does _not_ take ownership of 'task' -- the task should take care of
+  // deleting itself after running if it is allocated on the heap.
+  void ScheduleReactorTask(ReactorTask *task);
+
+  Status RunOnReactorThread(const boost::function<Status()>& f);
+
+  // If the Reactor is closing, returns false.
+  // Otherwise, drains the pending_tasks_ queue into the provided list.
+  bool DrainTaskQueue(boost::intrusive::list<ReactorTask> *tasks);
+
+  Messenger *messenger() const {
+    return messenger_.get();
+  }
+
+  // Indicates whether the reactor is shutting down.
+  //
+  // This method is thread-safe.
+  bool closing() const;
+
+  // Is this reactor's thread the current thread?
+  bool IsCurrentThread() const {
+    return thread_.IsCurrentThread();
+  }
+
+ private:
+  friend class ReactorThread;
+  typedef simple_spinlock LockType;
+  mutable LockType lock_;
+
+  // parent messenger
+  std::shared_ptr<Messenger> messenger_;
+
+  const std::string name_;
+
+  // Whether the reactor is shutting down.
+  // Guarded by lock_.
+  bool closing_;
+
+  // Tasks to be run within the reactor thread.
+  // Guarded by lock_.
+  boost::intrusive::list<ReactorTask> pending_tasks_; // NOLINT(build/include_what_you_use)
+
+  ReactorThread thread_;
+
+  DISALLOW_COPY_AND_ASSIGN(Reactor);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/remote_method.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/remote_method.cc b/be/src/kudu/rpc/remote_method.cc
new file mode 100644
index 0000000..70b0d02
--- /dev/null
+++ b/be/src/kudu/rpc/remote_method.cc
@@ -0,0 +1,53 @@
+// 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 <ostream>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+
+namespace kudu {
+namespace rpc {
+
+using strings::Substitute;
+
+RemoteMethod::RemoteMethod(std::string service_name,
+                           std::string method_name)
+    : service_name_(std::move(service_name)),
+      method_name_(std::move(method_name)) {}
+
+void RemoteMethod::FromPB(const RemoteMethodPB& pb) {
+  DCHECK(pb.IsInitialized()) << "PB is uninitialized: " << pb.InitializationErrorString();
+  service_name_ = pb.service_name();
+  method_name_ = pb.method_name();
+}
+
+void RemoteMethod::ToPB(RemoteMethodPB* pb) const {
+  pb->set_service_name(service_name_);
+  pb->set_method_name(method_name_);
+}
+
+std::string RemoteMethod::ToString() const {
+  return Substitute("$0.$1", service_name_, method_name_);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/remote_method.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/remote_method.h b/be/src/kudu/rpc/remote_method.h
new file mode 100644
index 0000000..a0a35bb
--- /dev/null
+++ b/be/src/kudu/rpc/remote_method.h
@@ -0,0 +1,51 @@
+// 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.
+#ifndef KUDU_RPC_REMOTE_METHOD_H_
+#define KUDU_RPC_REMOTE_METHOD_H_
+
+#include <string>
+
+namespace kudu {
+namespace rpc {
+
+class RemoteMethodPB;
+
+// Simple class that acts as a container for a fully qualified remote RPC name
+// and converts to/from RemoteMethodPB.
+// This class is also copyable and assignable for convenience reasons.
+class RemoteMethod {
+ public:
+  RemoteMethod() {}
+  RemoteMethod(std::string service_name, std::string method_name);
+  std::string service_name() const { return service_name_; }
+  std::string method_name() const { return method_name_; }
+
+  // Encode/decode to/from 'pb'.
+  void FromPB(const RemoteMethodPB& pb);
+  void ToPB(RemoteMethodPB* pb) const;
+
+  std::string ToString() const;
+
+ private:
+  std::string service_name_;
+  std::string method_name_;
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif // KUDU_RPC_REMOTE_METHOD_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/remote_user.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/remote_user.cc b/be/src/kudu/rpc/remote_user.cc
new file mode 100644
index 0000000..047f10a
--- /dev/null
+++ b/be/src/kudu/rpc/remote_user.cc
@@ -0,0 +1,40 @@
+// 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 "kudu/rpc/remote_user.h"
+
+#include <string>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+string RemoteUser::ToString() const {
+  string ret;
+  strings::SubstituteAndAppend(&ret, "{username='$0'", username_);
+  if (principal_) {
+    strings::SubstituteAndAppend(&ret, ", principal='$0'", *principal_);
+  }
+  ret.append("}");
+  return ret;
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/remote_user.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/remote_user.h b/be/src/kudu/rpc/remote_user.h
new file mode 100644
index 0000000..77ef294
--- /dev/null
+++ b/be/src/kudu/rpc/remote_user.h
@@ -0,0 +1,99 @@
+// 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.
+#pragma once
+
+#include <string>
+#include <utility>
+
+#include <boost/optional/optional.hpp>
+
+namespace kudu {
+namespace rpc {
+
+// Server-side view of the remote authenticated user.
+//
+// This class may be read by multiple threads concurrently after
+// its initialization during RPC negotiation.
+class RemoteUser {
+ public:
+  // The method by which the remote user authenticated.
+  enum Method {
+    // No authentication (authentication was not required by the server
+    // and the user provided a username but it was not validated in any way)
+    UNAUTHENTICATED,
+    // Kerberos-authenticated.
+    KERBEROS,
+    // Authenticated by a Kudu authentication token.
+    AUTHN_TOKEN,
+    // Authenticated by a client certificate.
+    CLIENT_CERT
+  };
+
+  Method authenticated_by() const {
+    return authenticated_by_;
+  }
+
+  const std::string& username() const { return username_; }
+
+  boost::optional<std::string> principal() const {
+    return principal_;
+  }
+
+  void SetAuthenticatedByKerberos(std::string username,
+                                  std::string principal) {
+    authenticated_by_ = KERBEROS;
+    username_ = std::move(username);
+    principal_ = std::move(principal);
+  }
+
+  void SetUnauthenticated(std::string username) {
+    authenticated_by_ = UNAUTHENTICATED;
+    username_ = std::move(username);
+    principal_ = boost::none;
+  }
+
+  void SetAuthenticatedByClientCert(std::string username,
+                                    boost::optional<std::string> principal) {
+    authenticated_by_ = CLIENT_CERT;
+    username_ = std::move(username);
+    principal_ = std::move(principal);
+  }
+
+  void SetAuthenticatedByToken(std::string username) {
+    authenticated_by_ = AUTHN_TOKEN;
+    username_ = std::move(username);
+    principal_ = boost::none;
+  }
+
+  // Returns a string representation of the object.
+  std::string ToString() const;
+
+ private:
+  // The real username of the remote user. In the case of a Kerberos
+  // principal, this has already been mapped to a local username.
+  // TODO(todd): actually do the above mapping.
+  std::string username_;
+
+  // The full principal of the remote user. This is only set in the
+  // case of a strong-authenticated user.
+  boost::optional<std::string> principal_;
+
+  Method authenticated_by_ = UNAUTHENTICATED;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/request_tracker-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/request_tracker-test.cc b/be/src/kudu/rpc/request_tracker-test.cc
new file mode 100644
index 0000000..f1e4d55
--- /dev/null
+++ b/be/src/kudu/rpc/request_tracker-test.cc
@@ -0,0 +1,86 @@
+// 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 <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/request_tracker.h"
+#include "kudu/util/test_macros.h"
+
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+TEST(RequestTrackerTest, TestSequenceNumberGeneration) {
+  const int MAX = 10;
+
+  scoped_refptr<RequestTracker> tracker_(new RequestTracker("test_client"));
+
+  // A new tracker should have no incomplete RPCs
+  RequestTracker::SequenceNumber seq_no = tracker_->FirstIncomplete();
+  ASSERT_EQ(seq_no, RequestTracker::kNoSeqNo);
+
+  vector<RequestTracker::SequenceNumber> generated_seq_nos;
+
+  // Generate MAX in flight RPCs, making sure they are correctly returned.
+  for (int i = 0; i < MAX; i++) {
+    ASSERT_OK(tracker_->NewSeqNo(&seq_no));
+    generated_seq_nos.push_back(seq_no);
+  }
+
+  // Now we should get a first incomplete.
+  ASSERT_EQ(generated_seq_nos[0], tracker_->FirstIncomplete());
+
+  // Marking 'first_incomplete' as done, should advance the first incomplete.
+  tracker_->RpcCompleted(tracker_->FirstIncomplete());
+
+  ASSERT_EQ(generated_seq_nos[1], tracker_->FirstIncomplete());
+
+  // Marking a 'middle' rpc, should not advance 'first_incomplete'.
+  tracker_->RpcCompleted(generated_seq_nos[5]);
+  ASSERT_EQ(generated_seq_nos[1], tracker_->FirstIncomplete());
+
+  // Marking half the rpc as complete should advance FirstIncomplete.
+  // Note that this also tests that RequestTracker::RpcCompleted() is idempotent, i.e. that
+  // marking the same sequence number as complete twice is a no-op.
+  for (int i = 0; i < MAX / 2; i++) {
+    tracker_->RpcCompleted(generated_seq_nos[i]);
+  }
+
+  ASSERT_EQ(generated_seq_nos[6], tracker_->FirstIncomplete());
+
+  for (int i = MAX / 2; i <= MAX; i++) {
+    ASSERT_OK(tracker_->NewSeqNo(&seq_no));
+    generated_seq_nos.push_back(seq_no);
+  }
+
+  // Marking them all as completed should cause RequestTracker::FirstIncomplete() to return
+  // Status::NotFound() again.
+  for (auto seq_no : generated_seq_nos) {
+    tracker_->RpcCompleted(seq_no);
+  }
+
+  ASSERT_EQ(tracker_->FirstIncomplete(), RequestTracker::kNoSeqNo);
+}
+
+} // namespace rpc
+} // namespace kudu
+

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/request_tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/request_tracker.cc b/be/src/kudu/rpc/request_tracker.cc
new file mode 100644
index 0000000..07806f8
--- /dev/null
+++ b/be/src/kudu/rpc/request_tracker.cc
@@ -0,0 +1,55 @@
+// 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 "kudu/rpc/request_tracker.h"
+
+#include <mutex>
+#include <string>
+#include <utility>
+
+#include "kudu/gutil/map-util.h"
+
+namespace kudu {
+namespace rpc {
+
+const RequestTracker::SequenceNumber RequestTracker::kNoSeqNo = -1;
+
+RequestTracker::RequestTracker(std::string client_id)
+    : client_id_(std::move(client_id)),
+      next_(0) {}
+
+Status RequestTracker::NewSeqNo(SequenceNumber* seq_no) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  *seq_no = next_;
+  InsertOrDie(&incomplete_rpcs_, *seq_no);
+  next_++;
+  return Status::OK();
+}
+
+RequestTracker::SequenceNumber RequestTracker::FirstIncomplete() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  if (incomplete_rpcs_.empty()) return kNoSeqNo;
+  return *incomplete_rpcs_.begin();
+}
+
+void RequestTracker::RpcCompleted(const SequenceNumber& seq_no) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  incomplete_rpcs_.erase(seq_no);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/request_tracker.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/request_tracker.h b/be/src/kudu/rpc/request_tracker.h
new file mode 100644
index 0000000..5cc3995
--- /dev/null
+++ b/be/src/kudu/rpc/request_tracker.h
@@ -0,0 +1,87 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <set>
+#include <string>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace rpc {
+
+// RequestTracker implementation, inspired by:
+// "Implementing Linearizability at Large Scale and Low Latency" by Colin Lee et al.
+//
+// This generates sequence numbers for retriable RPCs and tracks the ongoing ones.
+// The main point of this is to enable exactly-once semantics, i.e. making sure that
+// an RPC is only executed once, by uniquely identifying each RPC that is sent to
+// the server.
+//
+// Note that the sequence numbers here are differet from RPC 'call ids'. A call id
+// uniquely identifies a call _to a server_. All calls have a call id that is
+// assigned incrementally. Sequence numbers, on the other hand, uniquely identify
+// the RPC operation itself. That is, if an RPC is retried on another server it will
+// have a different call id, but the same sequence number.
+//
+// By keeping track of the RPCs that are in-flight and which ones are completed
+// we can determine the first incomplete RPC. When this information is sent
+// to the server it can use it to garbage collect RPC results that it might be
+// saving for future retries, since it now knows there won't be any.
+//
+// This class is thread safe.
+class RequestTracker : public RefCountedThreadSafe<RequestTracker> {
+ public:
+  typedef int64_t SequenceNumber;
+  static const RequestTracker::SequenceNumber kNoSeqNo;
+  explicit RequestTracker(std::string client_id);
+
+  // Creates a new, unique, sequence number.
+  // Sequence numbers are assigned in increasing integer order.
+  // Returns Status::OK() and sets 'seq_no' if it was able to generate a sequence number
+  // or returns Status::ServiceUnavailable() if too many RPCs are in-flight, in which case
+  // the caller should try again later.
+  Status NewSeqNo(SequenceNumber* seq_no);
+
+  // Returns the sequence number of the first incomplete RPC.
+  // If there is no incomplete RPC returns kNoSeqNo.
+  SequenceNumber FirstIncomplete();
+
+  // Marks the rpc with 'seq_no' as completed.
+  void RpcCompleted(const SequenceNumber& seq_no);
+
+  // Returns the client id for this request tracker.
+  const std::string& client_id() { return client_id_; }
+ private:
+  // The client id for this request tracker.
+  const std::string client_id_;
+
+  // Lock that protects all non-const fields.
+  simple_spinlock lock_;
+
+  // The next sequence number.
+  SequenceNumber next_;
+
+  // The (ordered) set of incomplete RPCs.
+  std::set<SequenceNumber> incomplete_rpcs_;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/response_callback.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/response_callback.h b/be/src/kudu/rpc/response_callback.h
new file mode 100644
index 0000000..8c4fc03
--- /dev/null
+++ b/be/src/kudu/rpc/response_callback.h
@@ -0,0 +1,31 @@
+// 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.
+
+#ifndef KUDU_RPC_RESPONSE_CALLBACK_H
+#define KUDU_RPC_RESPONSE_CALLBACK_H
+
+#include <boost/function.hpp>
+
+namespace kudu {
+namespace rpc {
+
+typedef boost::function<void()> ResponseCallback;
+
+}
+}
+
+#endif


[24/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_impl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_impl.h b/be/src/kudu/util/debug/trace_event_impl.h
new file mode 100644
index 0000000..2650e8b
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_impl.h
@@ -0,0 +1,726 @@
+// Copyright (c) 2012 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#ifndef KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_
+#define KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_
+
+#include <cstddef>
+#include <cstdint>
+#include <iosfwd>
+#include <stack>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/mutex.h"
+
+// Older style trace macros with explicit id and extra data
+// Only these macros result in publishing data to ETW as currently implemented.
+#define TRACE_EVENT_BEGIN_ETW(name, id, extra) \
+    base::debug::TraceLog::AddTraceEventEtw( \
+        TRACE_EVENT_PHASE_BEGIN, \
+        name, reinterpret_cast<const void*>(id), extra)
+
+#define TRACE_EVENT_END_ETW(name, id, extra) \
+    base::debug::TraceLog::AddTraceEventEtw( \
+        TRACE_EVENT_PHASE_END, \
+        name, reinterpret_cast<const void*>(id), extra)
+
+#define TRACE_EVENT_INSTANT_ETW(name, id, extra) \
+    base::debug::TraceLog::AddTraceEventEtw( \
+        TRACE_EVENT_PHASE_INSTANT, \
+        name, reinterpret_cast<const void*>(id), extra)
+
+template <typename Type>
+class Singleton;
+
+#if defined(COMPILER_GCC)
+namespace BASE_HASH_NAMESPACE {
+template <>
+struct hash<kudu::Thread*> {
+  std::size_t operator()(kudu::Thread* value) const {
+    return reinterpret_cast<std::size_t>(value);
+  }
+};
+}  // BASE_HASH_NAMESPACE
+#endif
+
+namespace kudu {
+
+class RefCountedString;
+class Thread;
+
+namespace debug {
+
+// For any argument of type TRACE_VALUE_TYPE_CONVERTABLE the provided
+// class must implement this interface.
+class ConvertableToTraceFormat : public kudu::RefCountedThreadSafe<ConvertableToTraceFormat> {
+ public:
+  // Append the class info to the provided |out| string. The appended
+  // data must be a valid JSON object. Strings must be properly quoted, and
+  // escaped. There is no processing applied to the content after it is
+  // appended.
+  virtual void AppendAsTraceFormat(std::string* out) const = 0;
+
+ protected:
+  virtual ~ConvertableToTraceFormat() {}
+
+ private:
+  friend class kudu::RefCountedThreadSafe<ConvertableToTraceFormat>;
+};
+
+struct TraceEventHandle {
+  uint32_t chunk_seq;
+  uint16_t chunk_index;
+  uint16_t event_index;
+};
+
+const int kTraceMaxNumArgs = 2;
+
+class BASE_EXPORT TraceEvent {
+ public:
+  union TraceValue {
+    bool as_bool;
+    uint64_t as_uint;
+    long long as_int;
+    double as_double;
+    const void* as_pointer;
+    const char* as_string;
+  };
+
+  TraceEvent();
+  ~TraceEvent();
+
+  // We don't need to copy TraceEvent except when TraceEventBuffer is cloned.
+  // Use explicit copy method to avoid accidentally misuse of copy.
+  void CopyFrom(const TraceEvent& other);
+
+  void Initialize(
+      int thread_id,
+      MicrosecondsInt64 timestamp,
+      MicrosecondsInt64 thread_timestamp,
+      char phase,
+      const unsigned char* category_group_enabled,
+      const char* name,
+      uint64_t id,
+      int num_args,
+      const char** arg_names,
+      const unsigned char* arg_types,
+      const uint64_t* arg_values,
+      const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+      unsigned char flags);
+
+  void Reset();
+
+  void UpdateDuration(const MicrosecondsInt64& now, const MicrosecondsInt64& thread_now);
+
+  // Serialize event data to JSON
+  void AppendAsJSON(std::string* out) const;
+  void AppendPrettyPrinted(std::ostringstream* out) const;
+
+  static void AppendValueAsJSON(unsigned char type,
+                                TraceValue value,
+                                std::string* out);
+
+  MicrosecondsInt64 timestamp() const { return timestamp_; }
+  MicrosecondsInt64 thread_timestamp() const { return thread_timestamp_; }
+  char phase() const { return phase_; }
+  int thread_id() const { return thread_id_; }
+  MicrosecondsInt64 duration() const { return duration_; }
+  MicrosecondsInt64 thread_duration() const { return thread_duration_; }
+  uint64_t id() const { return id_; }
+  unsigned char flags() const { return flags_; }
+
+  // Exposed for unittesting:
+
+  const kudu::RefCountedString* parameter_copy_storage() const {
+    return parameter_copy_storage_.get();
+  }
+
+  const unsigned char* category_group_enabled() const {
+    return category_group_enabled_;
+  }
+
+  const char* name() const { return name_; }
+
+#if defined(OS_ANDROID)
+  void SendToATrace();
+#endif
+
+ private:
+  // Note: these are ordered by size (largest first) for optimal packing.
+  MicrosecondsInt64 timestamp_;
+  MicrosecondsInt64 thread_timestamp_;
+  MicrosecondsInt64 duration_;
+  MicrosecondsInt64 thread_duration_;
+  // id_ can be used to store phase-specific data.
+  uint64_t id_;
+  TraceValue arg_values_[kTraceMaxNumArgs];
+  const char* arg_names_[kTraceMaxNumArgs];
+  scoped_refptr<ConvertableToTraceFormat> convertable_values_[kTraceMaxNumArgs];
+  const unsigned char* category_group_enabled_;
+  const char* name_;
+  scoped_refptr<kudu::RefCountedString> parameter_copy_storage_;
+  int thread_id_;
+  char phase_;
+  unsigned char flags_;
+  unsigned char arg_types_[kTraceMaxNumArgs];
+
+  DISALLOW_COPY_AND_ASSIGN(TraceEvent);
+};
+
+// TraceBufferChunk is the basic unit of TraceBuffer.
+class BASE_EXPORT TraceBufferChunk {
+ public:
+  explicit TraceBufferChunk(uint32_t seq)
+      : next_free_(0),
+        seq_(seq) {
+  }
+
+  void Reset(uint32_t new_seq);
+  TraceEvent* AddTraceEvent(size_t* event_index);
+  bool IsFull() const { return next_free_ == kTraceBufferChunkSize; }
+
+  uint32_t seq() const { return seq_; }
+  size_t capacity() const { return kTraceBufferChunkSize; }
+  size_t size() const { return next_free_; }
+
+  TraceEvent* GetEventAt(size_t index) {
+    DCHECK(index < size());
+    return &chunk_[index];
+  }
+  const TraceEvent* GetEventAt(size_t index) const {
+    DCHECK(index < size());
+    return &chunk_[index];
+  }
+
+  gscoped_ptr<TraceBufferChunk> Clone() const;
+
+  static const size_t kTraceBufferChunkSize = 64;
+
+ private:
+  size_t next_free_;
+  TraceEvent chunk_[kTraceBufferChunkSize];
+  uint32_t seq_;
+};
+
+// TraceBuffer holds the events as they are collected.
+class BASE_EXPORT TraceBuffer {
+ public:
+  virtual ~TraceBuffer() {}
+
+  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t *index) = 0;
+  virtual void ReturnChunk(size_t index,
+                           gscoped_ptr<TraceBufferChunk> chunk) = 0;
+
+  virtual bool IsFull() const = 0;
+  virtual size_t Size() const = 0;
+  virtual size_t Capacity() const = 0;
+  virtual TraceEvent* GetEventByHandle(TraceEventHandle handle) = 0;
+
+  // For iteration. Each TraceBuffer can only be iterated once.
+  virtual const TraceBufferChunk* NextChunk() = 0;
+
+  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const = 0;
+};
+
+// TraceResultBuffer collects and converts trace fragments returned by TraceLog
+// to JSON output.
+class TraceResultBuffer {
+ public:
+  static std::string FlushTraceLogToString();
+  static std::string FlushTraceLogToStringButLeaveBufferIntact();
+
+ private:
+  TraceResultBuffer();
+  ~TraceResultBuffer();
+
+  static std::string DoFlush(bool leave_intact);
+
+  // Callback for TraceLog::Flush
+  void Collect(const scoped_refptr<RefCountedString>& s,
+               bool has_more_events);
+
+  bool first_;
+  std::string json_;
+};
+
+class BASE_EXPORT CategoryFilter {
+ public:
+  typedef std::vector<std::string> StringList;
+
+  // The default category filter, used when none is provided.
+  // Allows all categories through, except if they end in the suffix 'Debug' or
+  // 'Test'.
+  static const char* kDefaultCategoryFilterString;
+
+  // |filter_string| is a comma-delimited list of category wildcards.
+  // A category can have an optional '-' prefix to make it an excluded category.
+  // All the same rules apply above, so for example, having both included and
+  // excluded categories in the same list would not be supported.
+  //
+  // Example: CategoryFilter"test_MyTest*");
+  // Example: CategoryFilter("test_MyTest*,test_OtherStuff");
+  // Example: CategoryFilter("-excluded_category1,-excluded_category2");
+  // Example: CategoryFilter("-*,webkit"); would disable everything but webkit.
+  // Example: CategoryFilter("-webkit"); would enable everything but webkit.
+  //
+  // Category filters can also be used to configure synthetic delays.
+  //
+  // Example: CategoryFilter("DELAY(gpu.PresentingFrame;16)"); would make swap
+  //          buffers always take at least 16 ms.
+  // Example: CategoryFilter("DELAY(gpu.PresentingFrame;16;oneshot)"); would
+  //          make swap buffers take at least 16 ms the first time it is
+  //          called.
+  // Example: CategoryFilter("DELAY(gpu.PresentingFrame;16;alternating)");
+  //          would make swap buffers take at least 16 ms every other time it
+  //          is called.
+  explicit CategoryFilter(const std::string& filter_string);
+
+  CategoryFilter(const CategoryFilter& cf);
+
+  ~CategoryFilter();
+
+  CategoryFilter& operator=(const CategoryFilter& rhs);
+
+  // Writes the string representation of the CategoryFilter. This is a comma
+  // separated string, similar in nature to the one used to determine
+  // enabled/disabled category patterns, except here there is an arbitrary
+  // order, included categories go first, then excluded categories. Excluded
+  // categories are distinguished from included categories by the prefix '-'.
+  std::string ToString() const;
+
+  // Determines whether category group would be enabled or
+  // disabled by this category filter.
+  bool IsCategoryGroupEnabled(const char* category_group) const;
+
+  // Return a list of the synthetic delays specified in this category filter.
+  const StringList& GetSyntheticDelayValues() const;
+
+  // Merges nested_filter with the current CategoryFilter
+  void Merge(const CategoryFilter& nested_filter);
+
+  // Clears both included/excluded pattern lists. This would be equivalent to
+  // creating a CategoryFilter with an empty string, through the constructor.
+  // i.e: CategoryFilter("").
+  //
+  // When using an empty filter, all categories are considered included as we
+  // are not excluding anything.
+  void Clear();
+
+ private:
+  FRIEND_TEST(TraceEventTestFixture, CategoryFilter);
+
+  static bool IsEmptyOrContainsLeadingOrTrailingWhitespace(
+      const std::string& str);
+
+  void Initialize(const std::string& filter_string);
+  void WriteString(const StringList& values,
+                   std::string* out,
+                   bool included) const;
+  void WriteString(const StringList& delays, std::string* out) const;
+  bool HasIncludedPatterns() const;
+
+  bool DoesCategoryGroupContainCategory(const char* category_group,
+                                        const char* category) const;
+
+  StringList included_;
+  StringList disabled_;
+  StringList excluded_;
+  StringList delays_;
+};
+
+class TraceSamplingThread;
+
+class BASE_EXPORT TraceLog {
+ public:
+  enum Mode {
+    DISABLED = 0,
+    RECORDING_MODE,
+    MONITORING_MODE,
+  };
+
+  // Options determines how the trace buffer stores data.
+  enum Options {
+    // Record until the trace buffer is full.
+    RECORD_UNTIL_FULL = 1 << 0,
+
+    // Record until the user ends the trace. The trace buffer is a fixed size
+    // and we use it as a ring buffer during recording.
+    RECORD_CONTINUOUSLY = 1 << 1,
+
+    // Enable the sampling profiler in the recording mode.
+    ENABLE_SAMPLING = 1 << 2,
+
+    // Echo to console. Events are discarded.
+    ECHO_TO_CONSOLE = 1 << 3,
+  };
+
+  // The pointer returned from GetCategoryGroupEnabledInternal() points to a
+  // value with zero or more of the following bits. Used in this class only.
+  // The TRACE_EVENT macros should only use the value as a bool.
+  // These values must be in sync with macro values in TraceEvent.h in Blink.
+  enum CategoryGroupEnabledFlags {
+    // Category group enabled for the recording mode.
+    ENABLED_FOR_RECORDING = 1 << 0,
+    // Category group enabled for the monitoring mode.
+    ENABLED_FOR_MONITORING = 1 << 1,
+    // Category group enabled by SetEventCallbackEnabled().
+    ENABLED_FOR_EVENT_CALLBACK = 1 << 2,
+  };
+
+  static TraceLog* GetInstance();
+
+  // Get set of known category groups. This can change as new code paths are
+  // reached. The known category groups are inserted into |category_groups|.
+  void GetKnownCategoryGroups(std::vector<std::string>* category_groups);
+
+  // Retrieves a copy (for thread-safety) of the current CategoryFilter.
+  CategoryFilter GetCurrentCategoryFilter();
+
+  Options trace_options() const {
+    return static_cast<Options>(base::subtle::NoBarrier_Load(&trace_options_));
+  }
+
+  // Enables normal tracing (recording trace events in the trace buffer).
+  // See CategoryFilter comments for details on how to control what categories
+  // will be traced. If tracing has already been enabled, |category_filter| will
+  // be merged into the current category filter.
+  void SetEnabled(const CategoryFilter& category_filter,
+                  Mode mode, Options options);
+
+  // Disables normal tracing for all categories.
+  void SetDisabled();
+
+  bool IsEnabled() { return mode_ != DISABLED; }
+
+  // The number of times we have begun recording traces. If tracing is off,
+  // returns -1. If tracing is on, then it returns the number of times we have
+  // recorded a trace. By watching for this number to increment, you can
+  // passively discover when a new trace has begun. This is then used to
+  // implement the TRACE_EVENT_IS_NEW_TRACE() primitive.
+  int GetNumTracesRecorded();
+
+#if defined(OS_ANDROID)
+  void StartATrace();
+  void StopATrace();
+  void AddClockSyncMetadataEvent();
+#endif
+
+  // Enabled state listeners give a callback when tracing is enabled or
+  // disabled. This can be used to tie into other library's tracing systems
+  // on-demand.
+  class EnabledStateObserver {
+   public:
+    // Called just after the tracing system becomes enabled, outside of the
+    // |lock_|. TraceLog::IsEnabled() is true at this point.
+    virtual void OnTraceLogEnabled() = 0;
+
+    // Called just after the tracing system disables, outside of the |lock_|.
+    // TraceLog::IsEnabled() is false at this point.
+    virtual void OnTraceLogDisabled() = 0;
+  };
+  void AddEnabledStateObserver(EnabledStateObserver* listener);
+  void RemoveEnabledStateObserver(EnabledStateObserver* listener);
+  bool HasEnabledStateObserver(EnabledStateObserver* listener) const;
+
+  float GetBufferPercentFull() const;
+  bool BufferIsFull() const;
+
+  // Not using kudu::Callback because of its limited by 7 parameters.
+  // Also, using primitive type allows directly passing callback from WebCore.
+  // WARNING: It is possible for the previously set callback to be called
+  // after a call to SetEventCallbackEnabled() that replaces or a call to
+  // SetEventCallbackDisabled() that disables the callback.
+  // This callback may be invoked on any thread.
+  // For TRACE_EVENT_PHASE_COMPLETE events, the client will still receive pairs
+  // of TRACE_EVENT_PHASE_BEGIN and TRACE_EVENT_PHASE_END events to keep the
+  // interface simple.
+  typedef void (*EventCallback)(MicrosecondsInt64 timestamp,
+                                char phase,
+                                const unsigned char* category_group_enabled,
+                                const char* name,
+                                uint64_t id,
+                                int num_args,
+                                const char* const arg_names[],
+                                const unsigned char arg_types[],
+                                const uint64_t arg_values[],
+                                unsigned char flags);
+
+  // Enable tracing for EventCallback.
+  void SetEventCallbackEnabled(const CategoryFilter& category_filter,
+                               EventCallback cb);
+  void SetEventCallbackDisabled();
+
+  // Flush all collected events to the given output callback. The callback will
+  // be called one or more times synchronously from
+  // the current thread with IPC-bite-size chunks. The string format is
+  // undefined. Use TraceResultBuffer to convert one or more trace strings to
+  // JSON. The callback can be null if the caller doesn't want any data.
+  // Due to the implementation of thread-local buffers, flush can't be
+  // done when tracing is enabled. If called when tracing is enabled, the
+  // callback will be called directly with (empty_string, false) to indicate
+  // the end of this unsuccessful flush.
+  typedef kudu::Callback<void(const scoped_refptr<kudu::RefCountedString>&,
+                              bool has_more_events)> OutputCallback;
+  void Flush(const OutputCallback& cb);
+  void FlushButLeaveBufferIntact(const OutputCallback& flush_output_callback);
+
+  // Called by TRACE_EVENT* macros, don't call this directly.
+  // The name parameter is a category group for example:
+  // TRACE_EVENT0("renderer,webkit", "WebViewImpl::HandleInputEvent")
+  static const unsigned char* GetCategoryGroupEnabled(const char* name);
+  static const char* GetCategoryGroupName(
+      const unsigned char* category_group_enabled);
+
+  // Called by TRACE_EVENT* macros, don't call this directly.
+  // If |copy| is set, |name|, |arg_name1| and |arg_name2| will be deep copied
+  // into the event; see "Memory scoping note" and TRACE_EVENT_COPY_XXX above.
+  TraceEventHandle AddTraceEvent(
+      char phase,
+      const unsigned char* category_group_enabled,
+      const char* name,
+      uint64_t id,
+      int num_args,
+      const char** arg_names,
+      const unsigned char* arg_types,
+      const uint64_t* arg_values,
+      const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+      unsigned char flags);
+  TraceEventHandle AddTraceEventWithThreadIdAndTimestamp(
+      char phase,
+      const unsigned char* category_group_enabled,
+      const char* name,
+      uint64_t id,
+      int thread_id,
+      const MicrosecondsInt64& timestamp,
+      int num_args,
+      const char** arg_names,
+      const unsigned char* arg_types,
+      const uint64_t* arg_values,
+      const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+      unsigned char flags);
+  static void AddTraceEventEtw(char phase,
+                               const char* category_group,
+                               const void* id,
+                               const char* extra);
+  static void AddTraceEventEtw(char phase,
+                               const char* category_group,
+                               const void* id,
+                               const std::string& extra);
+
+  void UpdateTraceEventDuration(const unsigned char* category_group_enabled,
+                                const char* name,
+                                TraceEventHandle handle);
+
+  // For every matching event, the callback will be called.
+  typedef kudu::Callback<void()> WatchEventCallback;
+  void SetWatchEvent(const std::string& category_name,
+                     const std::string& event_name,
+                     const WatchEventCallback& callback);
+  // Cancel the watch event. If tracing is enabled, this may race with the
+  // watch event notification firing.
+  void CancelWatchEvent();
+
+  int process_id() const { return process_id_; }
+
+  // Allow tests to inspect TraceEvents.
+  size_t GetEventsSize() const { return logged_events_->Size(); }
+  TraceEvent* GetEventByHandle(TraceEventHandle handle);
+
+  void SetProcessID(int process_id);
+
+  // Process sort indices, if set, override the order of a process will appear
+  // relative to other processes in the trace viewer. Processes are sorted first
+  // on their sort index, ascending, then by their name, and then tid.
+  void SetProcessSortIndex(int sort_index);
+
+  // Sets the name of the process.
+  void SetProcessName(const std::string& process_name);
+
+  // Processes can have labels in addition to their names. Use labels, for
+  // instance, to list out the web page titles that a process is handling.
+  void UpdateProcessLabel(int label_id, const std::string& current_label);
+  void RemoveProcessLabel(int label_id);
+
+  // Thread sort indices, if set, override the order of a thread will appear
+  // within its process in the trace viewer. Threads are sorted first on their
+  // sort index, ascending, then by their name, and then tid.
+  void SetThreadSortIndex(int64_t tid , int sort_index);
+
+  // Allow setting an offset between the current MicrosecondsInt64 time and the time
+  // that should be reported.
+  void SetTimeOffset(MicrosecondsInt64 offset);
+
+  size_t GetObserverCountForTest() const;
+
+
+ private:
+  FRIEND_TEST(TraceEventTestFixture,
+                           TraceBufferRingBufferGetReturnChunk);
+  FRIEND_TEST(TraceEventTestFixture,
+                           TraceBufferRingBufferHalfIteration);
+  FRIEND_TEST(TraceEventTestFixture,
+                           TraceBufferRingBufferFullIteration);
+
+  // This allows constructor and destructor to be private and usable only
+  // by the Singleton class.
+  friend class Singleton<TraceLog>;
+
+  // Enable/disable each category group based on the current mode_,
+  // category_filter_, event_callback_ and event_callback_category_filter_.
+  // Enable the category group in the enabled mode if category_filter_ matches
+  // the category group, or event_callback_ is not null and
+  // event_callback_category_filter_ matches the category group.
+  void UpdateCategoryGroupEnabledFlags();
+  void UpdateCategoryGroupEnabledFlag(int category_index);
+
+  // Configure synthetic delays based on the values set in the current
+  // category filter.
+  void UpdateSyntheticDelaysFromCategoryFilter();
+
+  struct PerThreadInfo;
+  class OptionalAutoLock;
+  class ThreadLocalEventBuffer;
+
+  TraceLog();
+  ~TraceLog();
+  const unsigned char* GetCategoryGroupEnabledInternal(const char* name);
+  void AddMetadataEventsWhileLocked();
+
+  TraceBuffer* trace_buffer() const { return logged_events_.get(); }
+  TraceBuffer* CreateTraceBuffer();
+
+  std::string EventToConsoleMessage(unsigned char phase,
+                                    const MicrosecondsInt64& timestamp,
+                                    TraceEvent* trace_event);
+
+  TraceEvent* AddEventToThreadSharedChunkWhileLocked(TraceEventHandle* handle,
+                                                     bool check_buffer_is_full);
+  void CheckIfBufferIsFullWhileLocked();
+  void SetDisabledWhileLocked();
+
+  TraceEvent* GetEventByHandleInternal(TraceEventHandle handle,
+                                       OptionalAutoLock* lock);
+
+  void ConvertTraceEventsToTraceFormat(gscoped_ptr<TraceBuffer> logged_events,
+                                       const OutputCallback& flush_output_callback);
+  void FinishFlush(int generation,
+                   const OutputCallback& flush_output_callback);
+
+  // Called when a thread which has registered trace events is about to exit.
+  void ThreadExiting();
+
+  // The static callback registered as a thread destructor.
+  static void ThreadExitingCB(void* arg);
+
+  int generation() const {
+    return static_cast<int>(base::subtle::NoBarrier_Load(&generation_));
+  }
+  bool CheckGeneration(int generation) const {
+    return generation == this->generation();
+  }
+  void UseNextTraceBuffer();
+
+  MicrosecondsInt64 OffsetNow() const {
+    return OffsetTimestamp(GetMonoTimeMicros());
+  }
+  MicrosecondsInt64 OffsetTimestamp(const MicrosecondsInt64& timestamp) const {
+    return timestamp - time_offset_;
+  }
+
+  // Create a new PerThreadInfo object for the current thread,
+  // and register it in the active_threads_ list.
+  PerThreadInfo* SetupThreadLocalBuffer();
+
+  // This lock protects TraceLog member accesses (except for members protected
+  // by thread_info_lock_) from arbitrary threads.
+  mutable base::SpinLock lock_;
+  // This lock protects accesses to thread_names_, thread_event_start_times_
+  // and thread_colors_.
+  base::SpinLock thread_info_lock_;
+  int locked_line_;
+  Mode mode_;
+  int num_traces_recorded_;
+  gscoped_ptr<TraceBuffer> logged_events_;
+  AtomicWord /* EventCallback */ event_callback_;
+  bool dispatching_to_observer_list_;
+  std::vector<EnabledStateObserver*> enabled_state_observer_list_;
+
+  std::string process_name_;
+  std::unordered_map<int, std::string> process_labels_;
+  int process_sort_index_;
+  std::unordered_map<int, int> thread_sort_indices_;
+  std::unordered_map<int, std::string> thread_names_;
+
+  // The following two maps are used only when ECHO_TO_CONSOLE.
+  std::unordered_map<int, std::stack<MicrosecondsInt64> > thread_event_start_times_;
+  std::unordered_map<std::string, int> thread_colors_;
+
+  // XORed with TraceID to make it unlikely to collide with other processes.
+  uint64_t process_id_hash_;
+
+  int process_id_;
+
+  MicrosecondsInt64 time_offset_;
+
+  // Allow tests to wake up when certain events occur.
+  WatchEventCallback watch_event_callback_;
+  AtomicWord /* const unsigned char* */ watch_category_;
+  std::string watch_event_name_;
+
+  AtomicWord /* Options */ trace_options_;
+
+  // Sampling thread handles.
+  gscoped_ptr<TraceSamplingThread> sampling_thread_;
+  scoped_refptr<kudu::Thread> sampling_thread_handle_;
+
+  CategoryFilter category_filter_;
+  CategoryFilter event_callback_category_filter_;
+
+  struct PerThreadInfo {
+    ThreadLocalEventBuffer* event_buffer_;
+    base::subtle::Atomic32 is_in_trace_event_;
+
+    // Atomically take the event_buffer_ member, setting it to NULL.
+    // Returns the old value of the member.
+    ThreadLocalEventBuffer* AtomicTakeBuffer();
+  };
+  static __thread PerThreadInfo* thread_local_info_;
+
+  Mutex active_threads_lock_;
+  // Map of PID -> PerThreadInfo
+  // Protected by active_threads_lock_.
+  typedef std::unordered_map<int64_t, PerThreadInfo*> ActiveThreadMap;
+  ActiveThreadMap active_threads_;
+
+  // For events which can't be added into the thread local buffer, e.g. events
+  // from threads without a message loop.
+  gscoped_ptr<TraceBufferChunk> thread_shared_chunk_;
+  size_t thread_shared_chunk_index_;
+
+  // The generation is incremented whenever tracing is enabled, and incremented
+  // again when the buffers are flushed. This ensures that trace events logged
+  // for a previous tracing session do not get accidentally flushed in the
+  // next tracing session.
+  AtomicWord generation_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceLog);
+};
+
+}  // namespace debug
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_impl_constants.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_impl_constants.cc b/be/src/kudu/util/debug/trace_event_impl_constants.cc
new file mode 100644
index 0000000..bf45ed7
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_impl_constants.cc
@@ -0,0 +1,14 @@
+// Copyright (c) 2013 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "kudu/util/debug/trace_event_impl.h"
+
+namespace kudu {
+namespace debug {
+
+// Enable everything but debug and test categories by default.
+const char* CategoryFilter::kDefaultCategoryFilterString = "-*Debug,-*Test";
+
+}  // namespace debug
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_memory.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_memory.h b/be/src/kudu/util/debug/trace_event_memory.h
new file mode 100644
index 0000000..6d9cf8d
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_memory.h
@@ -0,0 +1,28 @@
+// 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.
+#ifndef KUDU_DEBUG_TRACE_EVENT_MEMORY_H
+#define KUDU_DEBUG_TRACE_EVENT_MEMORY_H
+
+// Stub for this part of chromium tracing we haven't yet
+// imported.
+// The Chromium code relies on a locally patch tcmalloc.
+// See 5bc71bae28ea03689dbf50fe6baa15b574319091 in the Chromium
+// repository.
+
+#define INTERNAL_TRACE_MEMORY(category_group, name)
+
+#endif /* KUDU_DEBUG_TRACE_EVENT_MEMORY_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_synthetic_delay.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_synthetic_delay.cc b/be/src/kudu/util/debug/trace_event_synthetic_delay.cc
new file mode 100644
index 0000000..947ab88
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_synthetic_delay.cc
@@ -0,0 +1,238 @@
+// Copyright 2014 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "kudu/util/debug/trace_event_synthetic_delay.h"
+
+#include <cstring>
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/singleton.h"
+
+namespace {
+const int kMaxSyntheticDelays = 32;
+}  // namespace
+
+namespace kudu {
+namespace debug {
+
+TraceEventSyntheticDelayClock::TraceEventSyntheticDelayClock() {}
+TraceEventSyntheticDelayClock::~TraceEventSyntheticDelayClock() {}
+
+class TraceEventSyntheticDelayRegistry : public TraceEventSyntheticDelayClock {
+ public:
+  static TraceEventSyntheticDelayRegistry* GetInstance();
+
+  TraceEventSyntheticDelay* GetOrCreateDelay(const char* name);
+  void ResetAllDelays();
+
+  // TraceEventSyntheticDelayClock implementation.
+  virtual MonoTime Now() OVERRIDE;
+
+ private:
+  TraceEventSyntheticDelayRegistry();
+
+  friend class Singleton<TraceEventSyntheticDelayRegistry>;
+
+  Mutex lock_;
+  TraceEventSyntheticDelay delays_[kMaxSyntheticDelays];
+  TraceEventSyntheticDelay dummy_delay_;
+  base::subtle::Atomic32 delay_count_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceEventSyntheticDelayRegistry);
+};
+
+TraceEventSyntheticDelay::TraceEventSyntheticDelay()
+    : mode_(STATIC), begin_count_(0), trigger_count_(0), clock_(nullptr) {}
+
+TraceEventSyntheticDelay::~TraceEventSyntheticDelay() {}
+
+TraceEventSyntheticDelay* TraceEventSyntheticDelay::Lookup(
+    const std::string& name) {
+  return TraceEventSyntheticDelayRegistry::GetInstance()->GetOrCreateDelay(
+      name.c_str());
+}
+
+void TraceEventSyntheticDelay::Initialize(
+    const std::string& name,
+    TraceEventSyntheticDelayClock* clock) {
+  name_ = name;
+  clock_ = clock;
+}
+
+void TraceEventSyntheticDelay::SetTargetDuration(const MonoDelta& target_duration) {
+  MutexLock lock(lock_);
+  target_duration_ = target_duration;
+  trigger_count_ = 0;
+  begin_count_ = 0;
+}
+
+void TraceEventSyntheticDelay::SetMode(Mode mode) {
+  MutexLock lock(lock_);
+  mode_ = mode;
+}
+
+void TraceEventSyntheticDelay::SetClock(TraceEventSyntheticDelayClock* clock) {
+  MutexLock lock(lock_);
+  clock_ = clock;
+}
+
+void TraceEventSyntheticDelay::Begin() {
+  // Note that we check for a non-zero target duration without locking to keep
+  // things quick for the common case when delays are disabled. Since the delay
+  // calculation is done with a lock held, it will always be correct. The only
+  // downside of this is that we may fail to apply some delays when the target
+  // duration changes.
+  ANNOTATE_BENIGN_RACE(&target_duration_, "Synthetic delay duration");
+  if (!target_duration_.Initialized())
+    return;
+
+  MonoTime start_time = clock_->Now();
+  {
+    MutexLock lock(lock_);
+    if (++begin_count_ != 1)
+      return;
+    end_time_ = CalculateEndTimeLocked(start_time);
+  }
+}
+
+void TraceEventSyntheticDelay::BeginParallel(MonoTime* out_end_time) {
+  // See note in Begin().
+  ANNOTATE_BENIGN_RACE(&target_duration_, "Synthetic delay duration");
+  if (!target_duration_.Initialized()) {
+    *out_end_time = MonoTime();
+    return;
+  }
+
+  MonoTime start_time = clock_->Now();
+  {
+    MutexLock lock(lock_);
+    *out_end_time = CalculateEndTimeLocked(start_time);
+  }
+}
+
+void TraceEventSyntheticDelay::End() {
+  // See note in Begin().
+  ANNOTATE_BENIGN_RACE(&target_duration_, "Synthetic delay duration");
+  if (!target_duration_.Initialized())
+    return;
+
+  MonoTime end_time;
+  {
+    MutexLock lock(lock_);
+    if (!begin_count_ || --begin_count_ != 0)
+      return;
+    end_time = end_time_;
+  }
+  if (end_time.Initialized())
+    ApplyDelay(end_time);
+}
+
+void TraceEventSyntheticDelay::EndParallel(const MonoTime& end_time) {
+  if (end_time.Initialized())
+    ApplyDelay(end_time);
+}
+
+MonoTime TraceEventSyntheticDelay::CalculateEndTimeLocked(
+    const MonoTime& start_time) {
+  if (mode_ == ONE_SHOT && trigger_count_++)
+    return MonoTime();
+  else if (mode_ == ALTERNATING && trigger_count_++ % 2)
+    return MonoTime();
+  return start_time + target_duration_;
+}
+
+void TraceEventSyntheticDelay::ApplyDelay(const MonoTime& end_time) {
+  TRACE_EVENT0("synthetic_delay", name_.c_str());
+  while (clock_->Now() < end_time) {
+    // Busy loop.
+  }
+}
+
+TraceEventSyntheticDelayRegistry*
+TraceEventSyntheticDelayRegistry::GetInstance() {
+  return Singleton<TraceEventSyntheticDelayRegistry>::get();
+}
+
+TraceEventSyntheticDelayRegistry::TraceEventSyntheticDelayRegistry()
+    : delay_count_(0) {}
+
+TraceEventSyntheticDelay* TraceEventSyntheticDelayRegistry::GetOrCreateDelay(
+    const char* name) {
+  // Try to find an existing delay first without locking to make the common case
+  // fast.
+  int delay_count = base::subtle::Acquire_Load(&delay_count_);
+  for (int i = 0; i < delay_count; ++i) {
+    if (!strcmp(name, delays_[i].name_.c_str()))
+      return &delays_[i];
+  }
+
+  MutexLock lock(lock_);
+  delay_count = base::subtle::Acquire_Load(&delay_count_);
+  for (int i = 0; i < delay_count; ++i) {
+    if (!strcmp(name, delays_[i].name_.c_str()))
+      return &delays_[i];
+  }
+
+  DCHECK(delay_count < kMaxSyntheticDelays)
+      << "must increase kMaxSyntheticDelays";
+  if (delay_count >= kMaxSyntheticDelays)
+    return &dummy_delay_;
+
+  delays_[delay_count].Initialize(std::string(name), this);
+  base::subtle::Release_Store(&delay_count_, delay_count + 1);
+  return &delays_[delay_count];
+}
+
+MonoTime TraceEventSyntheticDelayRegistry::Now() {
+  return MonoTime::Now();
+}
+
+void TraceEventSyntheticDelayRegistry::ResetAllDelays() {
+  MutexLock lock(lock_);
+  int delay_count = base::subtle::Acquire_Load(&delay_count_);
+  for (int i = 0; i < delay_count; ++i) {
+    delays_[i].SetTargetDuration(MonoDelta());
+    delays_[i].SetClock(this);
+  }
+}
+
+void ResetTraceEventSyntheticDelays() {
+  TraceEventSyntheticDelayRegistry::GetInstance()->ResetAllDelays();
+}
+
+}  // namespace debug
+}  // namespace kudu
+
+namespace trace_event_internal {
+
+ScopedSyntheticDelay::ScopedSyntheticDelay(const char* name,
+                                           AtomicWord* impl_ptr)
+    : delay_impl_(GetOrCreateDelay(name, impl_ptr)) {
+  delay_impl_->BeginParallel(&end_time_);
+}
+
+ScopedSyntheticDelay::~ScopedSyntheticDelay() {
+  delay_impl_->EndParallel(end_time_);
+}
+
+kudu::debug::TraceEventSyntheticDelay* GetOrCreateDelay(
+    const char* name,
+    AtomicWord* impl_ptr) {
+  kudu::debug::TraceEventSyntheticDelay* delay_impl =
+      reinterpret_cast<kudu::debug::TraceEventSyntheticDelay*>(
+          base::subtle::Acquire_Load(impl_ptr));
+  if (!delay_impl) {
+    delay_impl = kudu::debug::TraceEventSyntheticDelayRegistry::GetInstance()
+                     ->GetOrCreateDelay(name);
+    base::subtle::Release_Store(
+        impl_ptr, reinterpret_cast<AtomicWord>(delay_impl));
+  }
+  return delay_impl;
+}
+
+}  // namespace trace_event_internal

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_synthetic_delay.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_synthetic_delay.h b/be/src/kudu/util/debug/trace_event_synthetic_delay.h
new file mode 100644
index 0000000..5866814
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_synthetic_delay.h
@@ -0,0 +1,166 @@
+// Copyright 2014 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+// The synthetic delay framework makes it possible to dynamically inject
+// arbitrary delays into into different parts of the codebase. This can be used,
+// for instance, for testing various task scheduling algorithms.
+//
+// The delays are specified in terms of a target duration for a given block of
+// code. If the code executes faster than the duration, the thread is made to
+// sleep until the deadline is met.
+//
+// Code can be instrumented for delays with two sets of macros. First, for
+// delays that should apply within a scope, use the following macro:
+//
+//   TRACE_EVENT_SYNTHETIC_DELAY("cc.LayerTreeHost.DrawAndSwap");
+//
+// For delaying operations that span multiple scopes, use:
+//
+//   TRACE_EVENT_SYNTHETIC_DELAY_BEGIN("cc.Scheduler.BeginMainFrame");
+//   ...
+//   TRACE_EVENT_SYNTHETIC_DELAY_END("cc.Scheduler.BeginMainFrame");
+//
+// Here BEGIN establishes the start time for the delay and END executes the
+// delay based on the remaining time. If BEGIN is called multiple times in a
+// row, END should be called a corresponding number of times. Only the last
+// call to END will have an effect.
+//
+// Note that a single delay may begin on one thread and end on another. This
+// implies that a single delay cannot not be applied in several threads at once.
+
+#ifndef KUDU_UTIL_DEBUG_TRACE_EVENT_SYNTHETIC_DELAY_H_
+#define KUDU_UTIL_DEBUG_TRACE_EVENT_SYNTHETIC_DELAY_H_
+
+#include <string>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+
+// Apply a named delay in the current scope.
+#define TRACE_EVENT_SYNTHETIC_DELAY(name)                                     \
+  static AtomicWord INTERNAL_TRACE_EVENT_UID(impl_ptr) = 0;                   \
+  trace_event_internal::ScopedSyntheticDelay INTERNAL_TRACE_EVENT_UID(delay)( \
+      name, &INTERNAL_TRACE_EVENT_UID(impl_ptr));
+
+// Begin a named delay, establishing its timing start point. May be called
+// multiple times as long as the calls to TRACE_EVENT_SYNTHETIC_DELAY_END are
+// balanced. Only the first call records the timing start point.
+#define TRACE_EVENT_SYNTHETIC_DELAY_BEGIN(name)                          \
+  do {                                                                   \
+    static AtomicWord impl_ptr = 0;                                      \
+    trace_event_internal::GetOrCreateDelay(name, &impl_ptr)->Begin();    \
+  } while (false)
+
+// End a named delay. The delay is applied only if this call matches the
+// first corresponding call to TRACE_EVENT_SYNTHETIC_DELAY_BEGIN with the
+// same delay.
+#define TRACE_EVENT_SYNTHETIC_DELAY_END(name)                         \
+  do {                                                                \
+    static AtomicWord impl_ptr = 0;                                   \
+    trace_event_internal::GetOrCreateDelay(name, &impl_ptr)->End();   \
+  } while (false)
+
+namespace kudu {
+namespace debug {
+
+// Time source for computing delay durations. Used for testing.
+class TRACE_EVENT_API_CLASS_EXPORT TraceEventSyntheticDelayClock {
+ public:
+  TraceEventSyntheticDelayClock();
+  virtual ~TraceEventSyntheticDelayClock();
+  virtual MonoTime Now() = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(TraceEventSyntheticDelayClock);
+};
+
+// Single delay point instance.
+class TRACE_EVENT_API_CLASS_EXPORT TraceEventSyntheticDelay {
+ public:
+  enum Mode {
+    STATIC,      // Apply the configured delay every time.
+    ONE_SHOT,    // Apply the configured delay just once.
+    ALTERNATING  // Apply the configured delay every other time.
+  };
+
+  // Returns an existing named delay instance or creates a new one with |name|.
+  static TraceEventSyntheticDelay* Lookup(const std::string& name);
+
+  void SetTargetDuration(const MonoDelta& target_duration);
+  void SetMode(Mode mode);
+  void SetClock(TraceEventSyntheticDelayClock* clock);
+
+  // Begin the delay, establishing its timing start point. May be called
+  // multiple times as long as the calls to End() are balanced. Only the first
+  // call records the timing start point.
+  void Begin();
+
+  // End the delay. The delay is applied only if this call matches the first
+  // corresponding call to Begin() with the same delay.
+  void End();
+
+  // Begin a parallel instance of the delay. Several parallel instances may be
+  // active simultaneously and will complete independently. The computed end
+  // time for the delay is stored in |out_end_time|, which should later be
+  // passed to EndParallel().
+  void BeginParallel(MonoTime* out_end_time);
+
+  // End a previously started parallel delay. |end_time| is the delay end point
+  // computed by BeginParallel().
+  void EndParallel(const MonoTime& end_time);
+
+ private:
+  TraceEventSyntheticDelay();
+  ~TraceEventSyntheticDelay();
+  friend class TraceEventSyntheticDelayRegistry;
+
+  void Initialize(const std::string& name,
+                  TraceEventSyntheticDelayClock* clock);
+  MonoTime CalculateEndTimeLocked(const MonoTime& start_time);
+  void ApplyDelay(const MonoTime& end_time);
+
+  Mutex lock_;
+  Mode mode_;
+  std::string name_;
+  int begin_count_;
+  int trigger_count_;
+  MonoTime end_time_;
+  MonoDelta target_duration_;
+  TraceEventSyntheticDelayClock* clock_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceEventSyntheticDelay);
+};
+
+// Set the target durations of all registered synthetic delay points to zero.
+TRACE_EVENT_API_CLASS_EXPORT void ResetTraceEventSyntheticDelays();
+
+}  // namespace debug
+}  // namespace kudu
+
+namespace trace_event_internal {
+
+// Helper class for scoped delays. Do not use directly.
+class TRACE_EVENT_API_CLASS_EXPORT ScopedSyntheticDelay {
+ public:
+  explicit ScopedSyntheticDelay(const char* name,
+                                AtomicWord* impl_ptr);
+  ~ScopedSyntheticDelay();
+
+ private:
+  kudu::debug::TraceEventSyntheticDelay* delay_impl_;
+  kudu::MonoTime end_time_;
+
+  DISALLOW_COPY_AND_ASSIGN(ScopedSyntheticDelay);
+};
+
+// Helper for registering delays. Do not use directly.
+TRACE_EVENT_API_CLASS_EXPORT kudu::debug::TraceEventSyntheticDelay*
+    GetOrCreateDelay(const char* name, AtomicWord* impl_ptr);
+
+}  // namespace trace_event_internal
+
+#endif /* KUDU_UTIL_DEBUG_TRACE_EVENT_SYNTHETIC_DELAY_H_ */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_logging.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_logging.h b/be/src/kudu/util/debug/trace_logging.h
new file mode 100644
index 0000000..1a3b39e
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_logging.h
@@ -0,0 +1,132 @@
+// 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 header defines the following macros:
+//
+// VLOG_AND_TRACE(category, vlevel)
+//
+//   Write a log message to VLOG(vlevel) as well as the current
+//   trace event buffer as an "INSTANT" trace event type. If the
+//   given vlog level is not enabled, this will still result in a
+//   trace buffer entry.
+//
+//   The provided 'category' should be a trace event category, which
+//   allows the users to filter which trace events to enable.
+//   For example:
+//
+//    VLOG_AND_TRACE("my_subsystem", 1) << "This always shows up in trace buffers "
+//        << "but only shows up in the log if VLOG(1) level logging is enabled.";
+//
+//   Most VLOG(1) level log messages are reasonable to use this macro.
+//   Note that there is slightly more overhead to this macro as opposed
+//   to just using VLOG(1).
+//
+//   Note that, like VLOG(n), this macro avoids evaluating its arguments unless
+//   either trace recording or VLOG(n) is enabled. In the case that both are enabled,
+//   the arguments are only evaluated once.
+//
+//
+// LOG_AND_TRACE(category, severity)
+//
+//   Same as the above, but always logs at the given severity level in addition
+//   to writing to the trace buffer.
+
+#ifndef KUDU_DEBUG_TRACE_LOGGING_H
+#define KUDU_DEBUG_TRACE_LOGGING_H
+
+#include <glog/logging.h>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/debug/trace_event.h"
+
+// The inner workings of these macros are a bit arcane:
+// - We make use of the fact that a block can be embedded within a ternary expression.
+//   This allows us to determine whether the trace event is enabled before we decide
+//   to evaluate the arguments.
+// - We have to use google::LogMessageVoidify so that we can put 'void(0)' on one side
+//   of the ternary expression and the log stream on the other. This technique is
+//   cribbed from glog/logging.h.
+#define VLOG_AND_TRACE_INTERNAL(category, vlevel) \
+  kudu::debug::TraceGLog(__FILE__, __LINE__, category, google::GLOG_INFO, \
+                         /* send_to_log= */VLOG_IS_ON(vlevel)).stream()
+
+#define VLOG_AND_TRACE(category, vlevel)                              \
+  !( {                                                                \
+      bool enabled;                                                   \
+      TRACE_EVENT_CATEGORY_GROUP_ENABLED(category, &enabled);         \
+      enabled || VLOG_IS_ON(vlevel);                                  \
+    } ) ? static_cast<void>(0) :                                      \
+          google::LogMessageVoidify() & VLOG_AND_TRACE_INTERNAL(category, vlevel) // NOLINT(*)
+
+
+#define LOG_AND_TRACE(category, severity) \
+  kudu::debug::TraceGLog(__FILE__, __LINE__, category, \
+                        google::GLOG_ ## severity, /* send_to_log= */true).stream()
+
+namespace kudu {
+namespace debug {
+
+class TraceGLog {
+ public:
+  TraceGLog(const char* file, int line, const char* category,
+           google::LogSeverity severity, bool send_to_log)
+    : sink_(category),
+      google_msg_(file, line, severity, &sink_, send_to_log) {
+  }
+
+  std::ostream& stream() {
+    return google_msg_.stream();
+  }
+
+ private:
+  class TraceLogSink : public google::LogSink {
+   public:
+    explicit TraceLogSink(const char* category) : category_(category) {}
+    void send(google::LogSeverity severity, const char* full_filename,
+              const char* base_filename, int line,
+              const struct ::tm* tm_time, const char* message,
+              size_t message_len) override {
+      // Rather than calling TRACE_EVENT_INSTANT here, we have to do it from
+      // the destructor. This is because glog holds its internal mutex while
+      // calling send(). So, if we try to use TRACE_EVENT here, and --trace_to_console
+      // is enabled, then we'd end up calling back into glog when its lock is already
+      // held. glog isn't re-entrant, so that causes a crash.
+      //
+      // By just storing the string here, and then emitting the trace in the dtor,
+      // we defer the tracing until the google::LogMessage has destructed and the
+      // glog lock is available again.
+      str_ = ToString(severity, base_filename, line,
+                      tm_time, message, message_len);
+    }
+    virtual ~TraceLogSink() {
+      TRACE_EVENT_INSTANT1(category_, "vlog", TRACE_EVENT_SCOPE_THREAD,
+                           "msg", str_);
+    }
+
+   private:
+    const char* const category_;
+    std::string str_;
+  };
+
+  TraceLogSink sink_;
+  google::LogMessage google_msg_;
+};
+
+} // namespace debug
+} // namespace kudu
+#endif /* KUDU_DEBUG_TRACE_LOGGING_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/unwind_safeness.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/unwind_safeness.cc b/be/src/kudu/util/debug/unwind_safeness.cc
new file mode 100644
index 0000000..c8e0adf
--- /dev/null
+++ b/be/src/kudu/util/debug/unwind_safeness.cc
@@ -0,0 +1,164 @@
+// 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.
+
+// Override various libdl functions which can race with libunwind.
+// The overridden versions set a threadlocal variable and our
+// stack-tracing code checks the threadlocal before calling into
+// libunwind.
+//
+// Based on public domain code by Aliaksey Kandratsenka at
+// https://github.com/alk/unwind_safeness_helper
+
+#include "kudu/util/debug/unwind_safeness.h"
+
+#include <dlfcn.h>
+#include <stddef.h>
+
+#include <ostream>
+
+#include <glog/logging.h>
+
+#define CALL_ORIG(func_name, ...) \
+  ((decltype(&func_name))g_orig_ ## func_name)(__VA_ARGS__)
+
+// Don't hook dl_iterate_phdr in TSAN builds since TSAN already instruments this
+// function and blocks signals while calling it. And skip it for macOS; it
+// doesn't exist there.
+#if !defined(THREAD_SANITIZER) && !defined(__APPLE__)
+#define HOOK_DL_ITERATE_PHDR 1
+#endif
+
+typedef int (*dl_iterate_phdr_cbtype)(struct dl_phdr_info *, size_t, void *);
+
+namespace {
+
+// Whether InitializeIfNecessary() has been called.
+bool g_initted;
+
+// The original versions of our wrapped functions.
+void* g_orig_dlopen;
+void* g_orig_dlclose;
+#ifdef HOOK_DL_ITERATE_PHDR
+void* g_orig_dl_iterate_phdr;
+#endif
+
+// The depth of calls into libdl.
+__thread int g_unsafeness_depth;
+
+// Scoped helper to track the recursion depth of calls into libdl
+struct ScopedBumpDepth {
+  ScopedBumpDepth() {
+    g_unsafeness_depth++;
+  }
+  ~ScopedBumpDepth() {
+    g_unsafeness_depth--;
+  }
+};
+
+void *dlsym_or_die(const char *sym) {
+  dlerror();
+  void* ret = dlsym(RTLD_NEXT, sym);
+  char* error = dlerror();
+  CHECK(!error) << "failed to find symbol " << sym << ": " << error;
+  return ret;
+}
+
+// Initialize the global variables which store the original references. This is
+// set up as a constructor so that we're guaranteed to call this before main()
+// while we are still single-threaded.
+//
+// NOTE: We _also_ call explicitly this from each of the wrappers, because
+// there are some cases where the constructors of dynamic libraries may call
+// dlopen, and there is no guarantee that our own constructor runs before
+// the constructor of other libraries.
+//
+// A couple examples of the above:
+//
+// 1) In ASAN builds, the sanitizer runtime ends up calling dl_iterate_phdr from its
+//    initialization.
+// 2) OpenSSL in FIPS mode calls dlopen() within its constructor.
+__attribute__((constructor))
+void InitIfNecessary() {
+  // Dynamic library initialization is always single-threaded, so there's no
+  // need for any synchronization here.
+  if (g_initted) return;
+
+  g_orig_dlopen = dlsym_or_die("dlopen");
+  g_orig_dlclose = dlsym_or_die("dlclose");
+#ifdef HOOK_DL_ITERATE_PHDR
+  // Failing to hook dl_iterate_phdr is non-fatal.
+  //
+  // In toolchains where the linker is passed --as-needed by default, a
+  // dynamically linked binary that doesn't directly reference any kudu_util
+  // symbols will omit a DT_NEEDED entry for kudu_util. Such a binary will
+  // no doubt emit a DT_NEEDED entry for libc, which means libc will wind up
+  // _before_ kudu_util in dlsym's search order. The net effect: the dlsym()
+  // call below will fail.
+  //
+  // All Ubuntu releases since Natty[1] behave in this way, except that many
+  // of them are also vulnerable to a glibc bug[2] that'll cause such a
+  // failure to go unreported by dlerror(). In newer releases, the failure
+  // is reported and dlsym_or_die() crashes the process.
+  //
+  // Given that the subset of affected binaries is small, and given that
+  // dynamic linkage isn't used in production anyway, we'll just treat the
+  // hook attempt as a best effort. Affected binaries that actually attempt
+  // to invoke dl_iterate_phdr will dereference a null pointer and crash, so
+  // if this is ever becomes a problem, we'll know right away.a
+  //
+  // 1. https://wiki.ubuntu.com/NattyNarwhal/ToolchainTransition
+  // 2. https://sourceware.org/bugzilla/show_bug.cgi?id=19509
+  g_orig_dl_iterate_phdr = dlsym(RTLD_NEXT, "dl_iterate_phdr");
+#endif
+  g_initted = true;
+}
+
+} // anonymous namespace
+
+namespace kudu {
+namespace debug {
+
+bool SafeToUnwindStack() {
+  return g_unsafeness_depth == 0;
+}
+
+} // namespace debug
+} // namespace kudu
+
+extern "C" {
+
+void *dlopen(const char *filename, int flag) { // NOLINT
+  InitIfNecessary();
+  ScopedBumpDepth d;
+  return CALL_ORIG(dlopen, filename, flag);
+}
+
+int dlclose(void *handle) { // NOLINT
+  InitIfNecessary();
+  ScopedBumpDepth d;
+  return CALL_ORIG(dlclose, handle);
+}
+
+#ifdef HOOK_DL_ITERATE_PHDR
+int dl_iterate_phdr(dl_iterate_phdr_cbtype callback, void *data) { // NOLINT
+  InitIfNecessary();
+  ScopedBumpDepth d;
+  return CALL_ORIG(dl_iterate_phdr, callback, data);
+}
+#endif
+
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/unwind_safeness.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/unwind_safeness.h b/be/src/kudu/util/debug/unwind_safeness.h
new file mode 100644
index 0000000..4aab6f9
--- /dev/null
+++ b/be/src/kudu/util/debug/unwind_safeness.h
@@ -0,0 +1,29 @@
+// 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.
+#pragma once
+
+namespace kudu {
+namespace debug {
+
+// Return true if it is currently safe to unwind the call stack.
+//
+// It's almost always safe unless we are in a signal handler context
+// inside a call into libdl.
+bool SafeToUnwindStack();
+
+} // namespace debug
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug_ref_counted.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug_ref_counted.h b/be/src/kudu/util/debug_ref_counted.h
new file mode 100644
index 0000000..7c2deca
--- /dev/null
+++ b/be/src/kudu/util/debug_ref_counted.h
@@ -0,0 +1,56 @@
+// 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.
+
+#ifndef KUDU_UTIL_DEBUG_REF_COUNTED_H_
+#define KUDU_UTIL_DEBUG_REF_COUNTED_H_
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/debug-util.h"
+
+namespace kudu {
+
+// For use in debugging. Change a ref-counted class to inherit from this,
+// instead of RefCountedThreadSafe, and fill your logs with stack traces.
+template <class T, typename Traits = DefaultRefCountedThreadSafeTraits<T> >
+class DebugRefCountedThreadSafe : public RefCountedThreadSafe<T, Traits> {
+ public:
+  DebugRefCountedThreadSafe() {}
+
+  void AddRef() const {
+    RefCountedThreadSafe<T, Traits>::AddRef();
+    LOG(INFO) << "Incremented ref on " << this << ":\n" << GetStackTrace();
+  }
+
+  void Release() const {
+    LOG(INFO) << "Decrementing ref on " << this << ":\n" << GetStackTrace();
+    RefCountedThreadSafe<T, Traits>::Release();
+  }
+
+ protected:
+  ~DebugRefCountedThreadSafe() {}
+
+ private:
+  friend struct DefaultRefCountedThreadSafeTraits<T>;
+
+  DISALLOW_COPY_AND_ASSIGN(DebugRefCountedThreadSafe);
+};
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_DEBUG_REF_COUNTED_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/decimal_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/decimal_util-test.cc b/be/src/kudu/util/decimal_util-test.cc
new file mode 100644
index 0000000..d7bfc35
--- /dev/null
+++ b/be/src/kudu/util/decimal_util-test.cc
@@ -0,0 +1,81 @@
+// 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 <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/decimal_util.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(TestDecimalUtil, TestMaxUnscaledDecimal) {
+  ASSERT_EQ(9, MaxUnscaledDecimal(1));
+  ASSERT_EQ(99999, MaxUnscaledDecimal(5));
+  ASSERT_EQ(kMaxUnscaledDecimal32, MaxUnscaledDecimal(kMaxDecimal32Precision));
+  ASSERT_EQ(kMaxUnscaledDecimal64, MaxUnscaledDecimal(kMaxDecimal64Precision));
+  ASSERT_EQ(kMaxUnscaledDecimal128, MaxUnscaledDecimal(kMaxDecimal128Precision));
+}
+
+TEST(TestDecimalUtil, TestMinUnscaledDecimal) {
+  ASSERT_EQ(-9, MinUnscaledDecimal(1));
+  ASSERT_EQ(-99999, MinUnscaledDecimal(5));
+  ASSERT_EQ(kMinUnscaledDecimal32, MinUnscaledDecimal(kMaxDecimal32Precision));
+  ASSERT_EQ(kMinUnscaledDecimal64, MinUnscaledDecimal(kMaxDecimal64Precision));
+  ASSERT_EQ(kMinUnscaledDecimal128, MinUnscaledDecimal(kMaxDecimal128Precision));
+}
+
+TEST(TestDecimalUtil, TestToString) {
+  ASSERT_EQ("999999999",
+            DecimalToString(kMaxUnscaledDecimal32, kDefaultDecimalScale));
+  ASSERT_EQ("0.999999999",
+            DecimalToString(kMaxUnscaledDecimal32, kMaxDecimal32Precision));
+  ASSERT_EQ("-999999999",
+            DecimalToString(kMinUnscaledDecimal32, kDefaultDecimalScale));
+  ASSERT_EQ("-0.999999999",
+            DecimalToString(kMinUnscaledDecimal32, kMaxDecimal32Precision));
+
+  ASSERT_EQ("999999999999999999",
+            DecimalToString(kMaxUnscaledDecimal64, kDefaultDecimalScale));
+  ASSERT_EQ("0.999999999999999999",
+            DecimalToString(kMaxUnscaledDecimal64, kMaxDecimal64Precision));
+  ASSERT_EQ("-999999999999999999",
+            DecimalToString(kMinUnscaledDecimal64, kDefaultDecimalScale));
+  ASSERT_EQ("-0.999999999999999999",
+            DecimalToString(kMinUnscaledDecimal64, kMaxDecimal64Precision));
+
+  ASSERT_EQ("99999999999999999999999999999999999999",
+            DecimalToString(kMaxUnscaledDecimal128, kDefaultDecimalScale));
+  ASSERT_EQ("0.99999999999999999999999999999999999999",
+            DecimalToString(kMaxUnscaledDecimal128, kMaxDecimal128Precision));
+  ASSERT_EQ("-99999999999999999999999999999999999999",
+            DecimalToString(kMinUnscaledDecimal128, kDefaultDecimalScale));
+  ASSERT_EQ("-0.99999999999999999999999999999999999999",
+            DecimalToString(kMinUnscaledDecimal128, kMaxDecimal128Precision));
+
+  ASSERT_EQ("0", DecimalToString(0, 0));
+  ASSERT_EQ("12345", DecimalToString(12345, 0));
+  ASSERT_EQ("-12345", DecimalToString(-12345, 0));
+  ASSERT_EQ("123.45", DecimalToString(12345, 2));
+  ASSERT_EQ("-123.45", DecimalToString(-12345, 2));
+  ASSERT_EQ("0.00012345", DecimalToString(12345, 8));
+  ASSERT_EQ("-0.00012345", DecimalToString(-12345, 8));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/decimal_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/decimal_util.cc b/be/src/kudu/util/decimal_util.cc
new file mode 100644
index 0000000..0e04494
--- /dev/null
+++ b/be/src/kudu/util/decimal_util.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/util/decimal_util.h"
+
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+using std::string;
+
+// Workaround for an ASAN build issue documented here:
+// https://bugs.llvm.org/show_bug.cgi?id=16404
+ATTRIBUTE_NO_SANITIZE_UNDEFINED
+int128_t MaxUnscaledDecimal(int8_t precision) {
+  DCHECK_GE(precision, kMinDecimalPrecision);
+  DCHECK_LE(precision, kMaxDecimalPrecision);
+  int128_t result = 1;
+  for (; precision > 0; precision--) {
+    result = result * 10;
+  }
+  return result - 1;
+}
+
+int128_t MinUnscaledDecimal(int8_t precision) {
+  return -MaxUnscaledDecimal(precision);
+}
+
+// Workaround for an ASAN build issue documented here:
+// https://bugs.llvm.org/show_bug.cgi?id=16404
+ATTRIBUTE_NO_SANITIZE_UNDEFINED
+string DecimalToString(int128_t d, int8_t scale) {
+  // 38 digits, 1 extra leading zero, decimal point,
+  // and sign are good for 128-bit or smaller decimals.
+  char local[41];
+  char *p = local + sizeof(local);
+  int128_t n = d < 0? -d : d;
+  int position = 0;
+  while (n) {
+    // Print the decimal in the scale position.
+    // No decimal is output when scale is 0.
+    if (scale != 0 && position == scale) {
+      *--p = '.';
+    }
+    // Unroll the next digits.
+    *--p = '0' + n % 10;
+    n /= 10;
+    position++;
+  }
+  // True if the value is between 1 and -1.
+  bool fractional = position <= scale;
+  // Pad with zeros until the scale
+  while (position < scale) {
+    *--p = '0';
+    position++;
+  }
+  // Add leading "0.".
+  if (fractional) {
+    if (d != 0) {
+      *--p = '.';
+    }
+    *--p = '0';
+  }
+  // Add sign for negative values.
+  if (d < 0) {
+    *--p = '-';
+  }
+  return string(p, local + sizeof(local));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/decimal_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/decimal_util.h b/be/src/kudu/util/decimal_util.h
new file mode 100644
index 0000000..a465412
--- /dev/null
+++ b/be/src/kudu/util/decimal_util.h
@@ -0,0 +1,69 @@
+// 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.
+
+#pragma once
+
+#include <stdint.h>
+#include <string>
+
+#include "kudu/util/int128.h"
+
+namespace kudu {
+  // Maximum precision and absolute value of a Decimal that can be stored
+  // in 4 bytes.
+  static const int8_t kMaxDecimal32Precision = 9;
+  static const int32_t kMaxUnscaledDecimal32 = 999999999; // 9 9's
+  static const int32_t kMinUnscaledDecimal32 = -kMaxUnscaledDecimal32; // 9 9's
+
+  // Maximum precision and absolute value of a valid Decimal can be
+  // stored in 8 bytes.
+  static const int8_t kMaxDecimal64Precision = 18;
+  static const int64_t kMaxUnscaledDecimal64 = 999999999999999999; // 18 9's
+  static const int64_t kMinUnscaledDecimal64 = -kMaxUnscaledDecimal64; // 18 9's
+
+  // Maximum precision and absolute value of a valid Decimal can be
+  // stored in 16 bytes.
+  static const int8_t kMaxDecimal128Precision = 38;
+  // Hacky calculation because int128 literals are not supported.
+  static const int128_t kMaxUnscaledDecimal128 =
+      (((static_cast<int128_t>(999999999999999999) * 1000000000000000000) +
+          999999999999999999) * 100) + 99; // 38 9's
+  static const int128_t kMinUnscaledDecimal128 = -kMaxUnscaledDecimal128;
+
+  // Minimum and maximum precision for any Decimal.
+  static const int8_t kMinDecimalPrecision = 1;
+  static const int8_t kMaxDecimalPrecision = kMaxDecimal128Precision;
+  // Maximum absolute value for any Decimal.
+  static const int128_t kMaxUnscaledDecimal = kMaxUnscaledDecimal128;
+  static const int128_t kMinUnscaledDecimal = kMinUnscaledDecimal128;
+
+  // Minimum scale for any Decimal.
+  static const int8_t kMinDecimalScale = 0;
+  static const int8_t kDefaultDecimalScale = 0;
+  // The maximum scale is the Decimal's precision.
+
+  // Returns the maximum unscaled decimal value that can be stored
+  // based on the precision.
+  int128_t MaxUnscaledDecimal(int8_t precision);
+
+  // Returns the maximum unscaled decimal value that can be stored
+  // based on the precision.
+  int128_t MinUnscaledDecimal(int8_t precision);
+
+  std::string DecimalToString(int128_t value, int8_t scale);
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/easy_json-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/easy_json-test.cc b/be/src/kudu/util/easy_json-test.cc
new file mode 100644
index 0000000..7074512
--- /dev/null
+++ b/be/src/kudu/util/easy_json-test.cc
@@ -0,0 +1,106 @@
+// 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 <string>
+
+#include <gtest/gtest.h>
+#include <rapidjson/document.h>
+#include <rapidjson/rapidjson.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/util/easy_json.h"
+#include "kudu/util/test_util.h"
+
+using rapidjson::SizeType;
+using rapidjson::Value;
+using std::string;
+
+namespace kudu {
+
+class EasyJsonTest: public KuduTest {};
+
+TEST_F(EasyJsonTest, TestNull) {
+  EasyJson ej;
+  ASSERT_TRUE(ej.value().IsNull());
+}
+
+TEST_F(EasyJsonTest, TestBasic) {
+  EasyJson ej;
+  ej.SetObject();
+  ej.Set("1", true);
+  ej.Set("2", kint32min);
+  ej.Set("4", kint64min);
+  ej.Set("6", 1.0);
+  ej.Set("7", "string");
+
+  Value& v = ej.value();
+
+  ASSERT_EQ(v["1"].GetBool(), true);
+  ASSERT_EQ(v["2"].GetInt(), kint32min);
+  ASSERT_EQ(v["4"].GetInt64(), kint64min);
+  ASSERT_EQ(v["6"].GetDouble(), 1.0);
+  ASSERT_EQ(string(v["7"].GetString()), "string");
+}
+
+TEST_F(EasyJsonTest, TestNested) {
+  EasyJson ej;
+  ej.SetObject();
+  ej.Get("nested").SetObject();
+  ej.Get("nested").Set("nested_attr", true);
+  ASSERT_EQ(ej.value()["nested"]["nested_attr"].GetBool(), true);
+
+  ej.Get("nested_array").SetArray();
+  ej.Get("nested_array").PushBack(1);
+  ej.Get("nested_array").PushBack(2);
+  ASSERT_EQ(ej.value()["nested_array"][SizeType(0)].GetInt(), 1);
+  ASSERT_EQ(ej.value()["nested_array"][SizeType(1)].GetInt(), 2);
+}
+
+TEST_F(EasyJsonTest, TestCompactSyntax) {
+  EasyJson ej;
+  ej["nested"]["nested_attr"] = true;
+  ASSERT_EQ(ej.value()["nested"]["nested_attr"].GetBool(), true);
+
+  for (int i = 0; i < 2; i++) {
+    ej["nested_array"][i] = i + 1;
+  }
+  ASSERT_EQ(ej.value()["nested_array"][SizeType(0)].GetInt(), 1);
+  ASSERT_EQ(ej.value()["nested_array"][SizeType(1)].GetInt(), 2);
+}
+
+TEST_F(EasyJsonTest, TestComplexInitializer) {
+  EasyJson ej;
+  ej = EasyJson::kObject;
+  ASSERT_TRUE(ej.value().IsObject());
+
+  EasyJson nested_arr = ej.Set("nested_arr", EasyJson::kArray);
+  ASSERT_TRUE(nested_arr.value().IsArray());
+
+  EasyJson nested_obj = nested_arr.PushBack(EasyJson::kObject);
+  ASSERT_TRUE(ej["nested_arr"][0].value().IsObject());
+}
+
+TEST_F(EasyJsonTest, TestAllocatorLifetime) {
+  EasyJson* root = new EasyJson;
+  EasyJson child = (*root)["child"];
+  delete root;
+
+  child["child_attr"] = 1;
+  ASSERT_EQ(child.value()["child_attr"].GetInt(), 1);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/easy_json.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/easy_json.cc b/be/src/kudu/util/easy_json.cc
new file mode 100644
index 0000000..9057b50
--- /dev/null
+++ b/be/src/kudu/util/easy_json.cc
@@ -0,0 +1,212 @@
+// 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 "kudu/util/easy_json.h"
+
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+#include <rapidjson/document.h>
+#include <rapidjson/rapidjson.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+using rapidjson::SizeType;
+using rapidjson::Value;
+using std::string;
+
+namespace kudu {
+
+EasyJson::EasyJson() : alloc_(new EasyJsonAllocator), value_(&alloc_->value()) {}
+
+EasyJson::EasyJson(EasyJson::ComplexTypeInitializer type)
+    : alloc_(new EasyJsonAllocator), value_(&alloc_->value()) {
+  if (type == kObject) {
+    value_->SetObject();
+  } else if (type == kArray) {
+    value_->SetArray();
+  }
+}
+
+EasyJson EasyJson::Get(const string& key) {
+  if (!value_->IsObject()) {
+    value_->SetObject();
+  }
+  if (!value_->HasMember(key.c_str())) {
+    Value key_val(key.c_str(), alloc_->allocator());
+    value_->AddMember(key_val, Value().SetNull(), alloc_->allocator());
+  }
+  return EasyJson(&(*value_)[key.c_str()], alloc_);
+}
+
+EasyJson EasyJson::Get(int index) {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  while (SizeType(index) >= value_->Size()) {
+    value_->PushBack(Value().SetNull(), alloc_->allocator());
+  }
+  return EasyJson(&(*value_)[index], alloc_);
+}
+
+EasyJson EasyJson::operator[](const string& key) {
+  return Get(key);
+}
+
+EasyJson EasyJson::operator[](int index) {
+  return Get(index);
+}
+
+EasyJson& EasyJson::operator=(const string& val) {
+  value_->SetString(val.c_str(), alloc_->allocator());
+  return *this;
+}
+template<typename T>
+EasyJson& EasyJson::operator=(T val) {
+  *value_ = val;
+  return *this;
+}
+template EasyJson& EasyJson::operator=<bool>(bool val);
+template EasyJson& EasyJson::operator=<int32_t>(int32_t val);
+template EasyJson& EasyJson::operator=<int64_t>(int64_t val);
+template EasyJson& EasyJson::operator=<uint32_t>(uint32_t val);
+template EasyJson& EasyJson::operator=<uint64_t>(uint64_t val);
+template EasyJson& EasyJson::operator=<double>(double val);
+template<> EasyJson& EasyJson::operator=<const char*>(const char* val) {
+  value_->SetString(val, alloc_->allocator());
+  return *this;
+}
+template<> EasyJson& EasyJson::operator=<EasyJson::ComplexTypeInitializer>(
+    EasyJson::ComplexTypeInitializer val) {
+  if (val == kObject) {
+    value_->SetObject();
+  } else if (val == kArray) {
+    value_->SetArray();
+  }
+  return (*this);
+}
+
+EasyJson& EasyJson::SetObject() {
+  if (!value_->IsObject()) {
+    value_->SetObject();
+  }
+  return *this;
+}
+
+EasyJson& EasyJson::SetArray() {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  return *this;
+}
+
+EasyJson EasyJson::Set(const string& key, const string& val) {
+  return (Get(key) = val);
+}
+template<typename T>
+EasyJson EasyJson::Set(const string& key, T val) {
+  return (Get(key) = val);
+}
+template EasyJson EasyJson::Set<bool>(const string& key, bool val);
+template EasyJson EasyJson::Set<int32_t>(const string& key, int32_t val);
+template EasyJson EasyJson::Set<int64_t>(const string& key, int64_t val);
+template EasyJson EasyJson::Set<uint32_t>(const string& key, uint32_t val);
+template EasyJson EasyJson::Set<uint64_t>(const string& key, uint64_t val);
+template EasyJson EasyJson::Set<double>(const string& key, double val);
+template EasyJson EasyJson::Set<const char*>(const string& key, const char* val);
+template EasyJson EasyJson::Set<EasyJson::ComplexTypeInitializer>(
+    const string& key,
+    EasyJson::ComplexTypeInitializer val);
+
+EasyJson EasyJson::Set(int index, const string& val) {
+  return (Get(index) = val);
+}
+template<typename T>
+EasyJson EasyJson::Set(int index, T val) {
+  return (Get(index) = val);
+}
+template EasyJson EasyJson::Set<bool>(int index, bool val);
+template EasyJson EasyJson::Set<int32_t>(int index, int32_t val);
+template EasyJson EasyJson::Set<int64_t>(int index, int64_t val);
+template EasyJson EasyJson::Set<uint32_t>(int index, uint32_t val);
+template EasyJson EasyJson::Set<uint64_t>(int index, uint64_t val);
+template EasyJson EasyJson::Set<double>(int index, double val);
+template EasyJson EasyJson::Set<const char*>(int index, const char* val);
+template EasyJson EasyJson::Set<EasyJson::ComplexTypeInitializer>(
+    int index,
+    EasyJson::ComplexTypeInitializer val);
+
+EasyJson EasyJson::PushBack(const string& val) {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  Value push_val(val.c_str(), alloc_->allocator());
+  value_->PushBack(push_val, alloc_->allocator());
+  return EasyJson(&(*value_)[value_->Size() - 1], alloc_);
+}
+template<typename T>
+EasyJson EasyJson::PushBack(T val) {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  value_->PushBack(val, alloc_->allocator());
+  return EasyJson(&(*value_)[value_->Size() - 1], alloc_);
+}
+template EasyJson EasyJson::PushBack<bool>(bool val);
+template EasyJson EasyJson::PushBack<int32_t>(int32_t val);
+template EasyJson EasyJson::PushBack<int64_t>(int64_t val);
+template EasyJson EasyJson::PushBack<uint32_t>(uint32_t val);
+template EasyJson EasyJson::PushBack<uint64_t>(uint64_t val);
+template EasyJson EasyJson::PushBack<double>(double val);
+template<> EasyJson EasyJson::PushBack<const char*>(const char* val) {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  Value push_val(val, alloc_->allocator());
+  value_->PushBack(push_val, alloc_->allocator());
+  return EasyJson(&(*value_)[value_->Size() - 1], alloc_);
+}
+template<> EasyJson EasyJson::PushBack<EasyJson::ComplexTypeInitializer>(
+    EasyJson::ComplexTypeInitializer val) {
+  if (!value_->IsArray()) {
+    value_->SetArray();
+  }
+  Value push_val;
+  if (val == kObject) {
+    push_val.SetObject();
+  } else if (val == kArray) {
+    push_val.SetArray();
+  } else {
+    LOG(FATAL) << "Unknown initializer type";
+  }
+  value_->PushBack(push_val, alloc_->allocator());
+  return EasyJson(&(*value_)[value_->Size() - 1], alloc_);
+}
+
+string EasyJson::ToString() const {
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  value_->Accept(writer);
+  return buffer.GetString();
+}
+
+EasyJson::EasyJson(Value* value, scoped_refptr<EasyJsonAllocator> alloc)
+    : alloc_(std::move(alloc)), value_(value) {}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/easy_json.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/easy_json.h b/be/src/kudu/util/easy_json.h
new file mode 100644
index 0000000..bd0365a
--- /dev/null
+++ b/be/src/kudu/util/easy_json.h
@@ -0,0 +1,190 @@
+// 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.
+#pragma once
+
+#include <string>
+
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/ref_counted.h"
+
+namespace kudu {
+
+// A wrapper around rapidjson Value objects, to simplify usage.
+// Intended solely for building json objects, not writing/parsing.
+//
+// Simplifies code like this:
+//
+//   rapidjson::Document d;
+//   rapidjson::Value v;
+//   v.SetObject();
+//   rapidjson::Value list;
+//   list.SetArray();
+//   v.AddMember("list", list, d.GetAllocator());
+//   v["list"].PushBack(rapidjson::Value().SetString("element"), d.GetAllocator());
+//
+// To this:
+//
+//   EasyJson ej;
+//   ej["list"][0] = "element";
+//
+// Client code should build objects as demonstrated above,
+// then call EasyJson::value() to obtain a reference to the
+// built rapidjson Value.
+class EasyJson {
+ public:
+  // Used for initializing EasyJson's with complex types.
+  // For example:
+  //
+  //  EasyJson array;
+  //  EasyJson nested = array.PushBack(EasyJson::kObject);
+  //  nested["attr"] = "val";
+  //  // array = [ { "attr": "val" } ]
+  enum ComplexTypeInitializer {
+    kObject,
+    kArray
+  };
+
+  EasyJson();
+  // Initializes the EasyJson object with the given type.
+  explicit EasyJson(ComplexTypeInitializer type);
+  ~EasyJson() = default;
+
+  // Returns the child EasyJson associated with key.
+  //
+  // Note: this method can mutate the EasyJson object
+  // as follows:
+  //
+  // If this EasyJson's underlying Value is not an object
+  // (i.e. !this->value().IsObject()), then its Value is
+  // coerced to an object, overwriting the old Value.
+  // If the given key does not exist, a Null-valued
+  // EasyJson associated with key is created.
+  EasyJson Get(const std::string& key);
+
+  // Returns the child EasyJson at index.
+  //
+  // Note: this method can mutate the EasyJson object
+  // as follows:
+  //
+  // If this EasyJson's underlying Value is not an array
+  // (i.e. !this->value().IsArray()), then its Value is
+  // coerced to an array, overwriting the old Value.
+  // If index >= this->value().Size(), then the underlying
+  // array's size is increased to index + 1 (new indices
+  // are filled with Null values).
+  EasyJson Get(int index);
+
+  // Same as Get(key).
+  EasyJson operator[](const std::string& key);
+  // Same as Get(index).
+  EasyJson operator[](int index);
+
+  // Sets the underlying Value equal to val.
+  // Returns a reference to the object itself.
+  //
+  // 'val' can be a bool, int32_t, int64_t, double,
+  // char*, string, or ComplexTypeInitializer.
+  EasyJson& operator=(const std::string& val);
+  template<typename T>
+  EasyJson& operator=(T val);
+
+  // Sets the underlying Value to an object.
+  // Returns a reference to the object itself.
+  //
+  // i.e. after calling SetObject(),
+  // value().IsObject() == true
+  EasyJson& SetObject();
+  // Sets the underlying Value to an array.
+  // Returns a reference to the object itself.
+  //
+  // i.e. after calling SetArray(),
+  // value().IsArray() == true
+  EasyJson& SetArray();
+
+  // Associates val with key.
+  // Returns the child object.
+  //
+  // If this EasyJson's underlying Value is not an object
+  // (i.e. !this->value().IsObject()), then its Value is
+  // coerced to an object, overwriting the old Value.
+  // If the given key does not exist, a new child entry
+  // is created with the given value.
+  EasyJson Set(const std::string& key, const std::string& val);
+  template<typename T>
+  EasyJson Set(const std::string& key, T val);
+
+  // Stores val at index.
+  // Returns the child object.
+  //
+  // If this EasyJson's underlying Value is not an array
+  // (i.e. !this->value().IsArray()), then its Value is
+  // coerced to an array, overwriting the old Value.
+  // If index >= this->value().Size(), then the underlying
+  // array's size is increased to index + 1 (new indices
+  // are filled with Null values).
+  EasyJson Set(int index, const std::string& val);
+  template<typename T>
+  EasyJson Set(int index, T val);
+
+  // Appends val to the underlying array.
+  // Returns a reference to the new child object.
+  //
+  // If this EasyJson's underlying Value is not an array
+  // (i.e. !this->value().IsArray()), then its Value is
+  // coerced to an array, overwriting the old Value.
+  EasyJson PushBack(const std::string& val);
+  template<typename T>
+  EasyJson PushBack(T val);
+
+  // Returns a reference to the underlying Value.
+  rapidjson::Value& value() const { return *value_; }
+
+  // Returns a string representation of the underlying json.
+  std::string ToString() const;
+
+ private:
+  // One instance of EasyJsonAllocator is shared among a root
+  // EasyJson object and all of its descendants. The allocator
+  // owns the underlying rapidjson Value, and a rapidjson
+  // allocator (via a rapidjson::Document).
+  class EasyJsonAllocator : public RefCounted<EasyJsonAllocator> {
+   public:
+    rapidjson::Value& value() { return value_; }
+    rapidjson::Document::AllocatorType& allocator() { return value_.GetAllocator(); }
+   private:
+    friend class RefCounted<EasyJsonAllocator>;
+    ~EasyJsonAllocator() = default;
+
+    // The underlying rapidjson::Value object (Document is
+    // a subclass of Value that has its own allocator).
+    rapidjson::Document value_;
+  };
+
+  // Used to instantiate descendant objects.
+  EasyJson(rapidjson::Value* value, scoped_refptr<EasyJsonAllocator> alloc);
+
+  // One allocator is shared among an EasyJson object and
+  // all of its descendants.
+  scoped_refptr<EasyJsonAllocator> alloc_;
+
+  // A pointer to the underlying Value in the object
+  // tree owned by alloc_.
+  rapidjson::Value* value_;
+};
+
+} // namespace kudu


[15/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memcmpable_varint.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memcmpable_varint.cc b/be/src/kudu/util/memcmpable_varint.cc
new file mode 100644
index 0000000..b30eff6
--- /dev/null
+++ b/be/src/kudu/util/memcmpable_varint.cc
@@ -0,0 +1,257 @@
+// 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 contains code derived from sqlite4, distributed in the public domain.
+//
+// A variable length integer is an encoding of 64-bit unsigned integers
+// into between 1 and 9 bytes.  The encoding is designed so that small
+// (and common) values take much less space that larger values.  Additional
+// properties:
+//
+//    *  The length of the varint can be determined after examining just
+//       the first byte of the encoding.
+//
+//    *  Varints compare in numerical order using memcmp().
+//
+//************************************************************************
+//
+// Treat each byte of the encoding as an unsigned integer between 0 and 255.
+// Let the bytes of the encoding be called A0, A1, A2, ..., A8.
+//
+// DECODE
+//
+// If A0 is between 0 and 240 inclusive, then the result is the value of A0.
+//
+// If A0 is between 241 and 248 inclusive, then the result is
+// 240+256*(A0-241)+A1.
+//
+// If A0 is 249 then the result is 2288+256*A1+A2.
+//
+// If A0 is 250 then the result is A1..A3 as a 3-byte big-ending integer.
+//
+// If A0 is 251 then the result is A1..A4 as a 4-byte big-ending integer.
+//
+// If A0 is 252 then the result is A1..A5 as a 5-byte big-ending integer.
+//
+// If A0 is 253 then the result is A1..A6 as a 6-byte big-ending integer.
+//
+// If A0 is 254 then the result is A1..A7 as a 7-byte big-ending integer.
+//
+// If A0 is 255 then the result is A1..A8 as a 8-byte big-ending integer.
+//
+// ENCODE
+//
+// Let the input value be V.
+//
+// If V<=240 then output a single by A0 equal to V.
+//
+// If V<=2287 then output A0 as (V-240)/256 + 241 and A1 as (V-240)%256.
+//
+// If V<=67823 then output A0 as 249, A1 as (V-2288)/256, and A2
+// as (V-2288)%256.
+//
+// If V<=16777215 then output A0 as 250 and A1 through A3 as a big-endian
+// 3-byte integer.
+//
+// If V<=4294967295 then output A0 as 251 and A1..A4 as a big-ending
+// 4-byte integer.
+//
+// If V<=1099511627775 then output A0 as 252 and A1..A5 as a big-ending
+// 5-byte integer.
+//
+// If V<=281474976710655 then output A0 as 253 and A1..A6 as a big-ending
+// 6-byte integer.
+//
+// If V<=72057594037927935 then output A0 as 254 and A1..A7 as a
+// big-ending 7-byte integer.
+//
+// Otherwise then output A0 as 255 and A1..A8 as a big-ending 8-byte integer.
+//
+// SUMMARY
+//
+//    Bytes    Max Value    Digits
+//    -------  ---------    ---------
+//      1      240           2.3
+//      2      2287          3.3
+//      3      67823         4.8
+//      4      2**24-1       7.2
+//      5      2**32-1       9.6
+//      6      2**40-1      12.0
+//      7      2**48-1      14.4
+//      8      2**56-1      16.8
+//      9      2**64-1      19.2
+
+#include <cstddef>
+
+#include <glog/logging.h>
+
+#include "kudu/util/faststring.h"
+#include "kudu/util/memcmpable_varint.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+////////////////////////////////////////////////////////////
+// Begin code ripped from sqlite4
+////////////////////////////////////////////////////////////
+
+// This function is borrowed from sqlite4/varint.c
+static void varintWrite32(uint8_t *z, uint32_t y) {
+  z[0] = (uint8_t)(y>>24);
+  z[1] = (uint8_t)(y>>16);
+  z[2] = (uint8_t)(y>>8);
+  z[3] = (uint8_t)(y);
+}
+
+
+// Write a varint into z[].  The buffer z[] must be at least 9 characters
+// long to accommodate the largest possible varint.  Return the number of
+// bytes of z[] used.
+//
+// This function is borrowed from sqlite4/varint.c
+static size_t sqlite4PutVarint64(uint8_t *z, uint64_t x) {
+  uint64_t w, y;
+  if (x <= 240) {
+    z[0] = (uint8_t)x;
+    return 1;
+  }
+  if (x <= 2287) {
+    y = (uint64_t)(x - 240);
+    z[0] = (uint8_t)(y/256 + 241);
+    z[1] = (uint8_t)(y%256);
+    return 2;
+  }
+  if (x <= 67823) {
+    y = (uint64_t)(x - 2288);
+    z[0] = 249;
+    z[1] = (uint8_t)(y/256);
+    z[2] = (uint8_t)(y%256);
+    return 3;
+  }
+  y = (uint64_t)x;
+  w = (uint64_t)(x>>32);
+  if (w == 0) {
+    if (y <= 16777215) {
+      z[0] = 250;
+      z[1] = (uint8_t)(y>>16);
+      z[2] = (uint8_t)(y>>8);
+      z[3] = (uint8_t)(y);
+      return 4;
+    }
+    z[0] = 251;
+    varintWrite32(z+1, y);
+    return 5;
+  }
+  if (w <= 255) {
+    z[0] = 252;
+    z[1] = (uint8_t)w;
+    varintWrite32(z+2, y);
+    return 6;
+  }
+  if (w <= 65535) {
+    z[0] = 253;
+    z[1] = (uint8_t)(w>>8);
+    z[2] = (uint8_t)w;
+    varintWrite32(z+3, y);
+    return 7;
+  }
+  if (w <= 16777215) {
+    z[0] = 254;
+    z[1] = (uint8_t)(w>>16);
+    z[2] = (uint8_t)(w>>8);
+    z[3] = (uint8_t)w;
+    varintWrite32(z+4, y);
+    return 8;
+  }
+  z[0] = 255;
+  varintWrite32(z+1, w);
+  varintWrite32(z+5, y);
+  return 9;
+}
+
+// Decode the varint in the first n bytes z[].  Write the integer value
+// into *pResult and return the number of bytes in the varint.
+//
+// If the decode fails because there are not enough bytes in z[] then
+// return 0;
+//
+// Borrowed from sqlite4 varint.c
+static int sqlite4GetVarint64(
+  const uint8_t *z,
+  int n,
+  uint64_t *p_result) {
+  unsigned int x;
+  if ( n < 1) return 0;
+  if (z[0] <= 240) {
+    *p_result = z[0];
+    return 1;
+  }
+  if (z[0] <= 248) {
+    if ( n < 2) return 0;
+    *p_result = (z[0]-241)*256 + z[1] + 240;
+    return 2;
+  }
+  if (n < z[0]-246 ) return 0;
+  if (z[0] == 249) {
+    *p_result = 2288 + 256*z[1] + z[2];
+    return 3;
+  }
+  if (z[0] == 250) {
+    *p_result = (z[1]<<16) + (z[2]<<8) + z[3];
+    return 4;
+  }
+  x = (z[1]<<24) + (z[2]<<16) + (z[3]<<8) + z[4];
+  if (z[0] == 251) {
+    *p_result = x;
+    return 5;
+  }
+  if (z[0] == 252) {
+    *p_result = (((uint64_t)x)<<8) + z[5];
+    return 6;
+  }
+  if (z[0] == 253) {
+    *p_result = (((uint64_t)x)<<16) + (z[5]<<8) + z[6];
+    return 7;
+  }
+  if (z[0] == 254) {
+    *p_result = (((uint64_t)x)<<24) + (z[5]<<16) + (z[6]<<8) + z[7];
+    return 8;
+  }
+  *p_result = (((uint64_t)x)<<32) +
+               (0xffffffff & ((z[5]<<24) + (z[6]<<16) + (z[7]<<8) + z[8]));
+  return 9;
+}
+
+////////////////////////////////////////////////////////////
+// End code ripped from sqlite4
+////////////////////////////////////////////////////////////
+
+void PutMemcmpableVarint64(faststring *dst, uint64_t value) {
+  uint8_t buf[9];
+  int used = sqlite4PutVarint64(buf, value);
+  DCHECK_LE(used, sizeof(buf));
+  dst->append(buf, used);
+}
+
+bool GetMemcmpableVarint64(Slice *input, uint64_t *value) {
+  size_t size = sqlite4GetVarint64(input->data(), input->size(), value);
+  input->remove_prefix(size);
+  return size > 0;
+}
+
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memcmpable_varint.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memcmpable_varint.h b/be/src/kudu/util/memcmpable_varint.h
new file mode 100644
index 0000000..955f89d
--- /dev/null
+++ b/be/src/kudu/util/memcmpable_varint.h
@@ -0,0 +1,45 @@
+// 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 is an alternate varint format, borrowed from sqlite4, that differs from the
+// varint in util/coding.h in that its serialized form can be compared with memcmp(),
+// yielding the same result as comparing the original integers.
+//
+// The serialized form also has the property that multiple such varints can be strung
+// together to form a composite key, which itself is memcmpable.
+//
+// See memcmpable_varint.cc for further description.
+
+#ifndef KUDU_UTIL_MEMCMPABLE_VARINT_H
+#define KUDU_UTIL_MEMCMPABLE_VARINT_H
+
+#include <cstdint>
+
+namespace kudu {
+
+class Slice;
+class faststring;
+
+void PutMemcmpableVarint64(faststring *dst, uint64_t value);
+
+// Standard Get... routines parse a value from the beginning of a Slice
+// and advance the slice past the parsed value.
+bool GetMemcmpableVarint64(Slice *input, uint64_t *value);
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/arena-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/arena-test.cc b/be/src/kudu/util/memory/arena-test.cc
new file mode 100644
index 0000000..695e305
--- /dev/null
+++ b/be/src/kudu/util/memory/arena-test.cc
@@ -0,0 +1,205 @@
+// 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 <cstdint>
+#include <cstring>
+#include <memory>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/util/memory/arena.h"
+#include "kudu/util/memory/memory.h"
+#include "kudu/util/mem_tracker.h"
+
+DEFINE_int32(num_threads, 16, "Number of threads to test");
+DEFINE_int32(allocs_per_thread, 10000, "Number of allocations each thread should do");
+DEFINE_int32(alloc_size, 4, "number of bytes in each allocation");
+
+namespace kudu {
+
+using std::shared_ptr;
+using std::string;
+using std::thread;
+using std::vector;
+
+template<class ArenaType>
+static void AllocateThread(ArenaType *arena, uint8_t thread_index) {
+  std::vector<void *> ptrs;
+  ptrs.reserve(FLAGS_allocs_per_thread);
+
+  char buf[FLAGS_alloc_size];
+  memset(buf, thread_index, FLAGS_alloc_size);
+
+  for (int i = 0; i < FLAGS_allocs_per_thread; i++) {
+    void *alloced = arena->AllocateBytes(FLAGS_alloc_size);
+    CHECK(alloced);
+    memcpy(alloced, buf, FLAGS_alloc_size);
+    ptrs.push_back(alloced);
+  }
+
+  for (void *p : ptrs) {
+    if (memcmp(buf, p, FLAGS_alloc_size) != 0) {
+      FAIL() << StringPrintf("overwritten pointer at %p", p);
+    }
+  }
+}
+
+// Non-templated function to forward to above -- simplifies thread creation
+static void AllocateThreadTSArena(ThreadSafeArena *arena, uint8_t thread_index) {
+  AllocateThread(arena, thread_index);
+}
+
+
+TEST(TestArena, TestSingleThreaded) {
+  Arena arena(128);
+  AllocateThread(&arena, 0);
+}
+
+
+
+TEST(TestArena, TestMultiThreaded) {
+  CHECK(FLAGS_num_threads < 256);
+
+  ThreadSafeArena arena(1024);
+
+  vector<thread> threads;
+  for (uint8_t i = 0; i < FLAGS_num_threads; i++) {
+    threads.emplace_back(AllocateThreadTSArena, &arena, (uint8_t)i);
+  }
+
+  for (thread& thr : threads) {
+    thr.join();
+  }
+}
+
+TEST(TestArena, TestAlignment) {
+  ThreadSafeArena arena(1024);
+  for (int i = 0; i < 1000; i++) {
+    int alignment = 1 << (1 % 5);
+
+    void *ret = arena.AllocateBytesAligned(5, alignment);
+    ASSERT_EQ(0, (uintptr_t)(ret) % alignment) <<
+      "failed to align on " << alignment << "b boundary: " <<
+      ret;
+  }
+}
+
+TEST(TestArena, TestObjectAlignment) {
+  struct MyStruct {
+    int64_t v;
+  };
+  Arena a(256);
+  // Allocate a junk byte to ensure that the next allocation isn't "accidentally" aligned.
+  a.AllocateBytes(1);
+  void* v = a.NewObject<MyStruct>();
+  ASSERT_EQ(reinterpret_cast<uintptr_t>(v) % alignof(MyStruct), 0);
+}
+
+
+// MemTrackers update their ancestors when consuming and releasing memory to compute
+// usage totals. However, the lifetimes of parent and child trackers can be different.
+// Validate that child trackers can still correctly update their parent stats even when
+// the parents go out of scope.
+TEST(TestArena, TestMemoryTrackerParentReferences) {
+  // Set up a parent and child MemTracker.
+  const string parent_id = "parent-id";
+  const string child_id = "child-id";
+  shared_ptr<MemTracker> child_tracker;
+  {
+    shared_ptr<MemTracker> parent_tracker = MemTracker::CreateTracker(1024, parent_id);
+    child_tracker = MemTracker::CreateTracker(-1, child_id, parent_tracker);
+    // Parent falls out of scope here. Should still be owned by the child.
+  }
+  shared_ptr<MemoryTrackingBufferAllocator> allocator(
+      new MemoryTrackingBufferAllocator(HeapBufferAllocator::Get(), child_tracker));
+  MemoryTrackingArena arena(256, allocator);
+
+  // Try some child operations.
+  ASSERT_EQ(256, child_tracker->consumption());
+  void *allocated = arena.AllocateBytes(256);
+  ASSERT_TRUE(allocated);
+  ASSERT_EQ(256, child_tracker->consumption());
+  allocated = arena.AllocateBytes(256);
+  ASSERT_TRUE(allocated);
+  ASSERT_EQ(768, child_tracker->consumption());
+}
+
+TEST(TestArena, TestMemoryTrackingDontEnforce) {
+  shared_ptr<MemTracker> mem_tracker = MemTracker::CreateTracker(1024, "arena-test-tracker");
+  shared_ptr<MemoryTrackingBufferAllocator> allocator(
+      new MemoryTrackingBufferAllocator(HeapBufferAllocator::Get(), mem_tracker));
+  MemoryTrackingArena arena(256, allocator);
+  ASSERT_EQ(256, mem_tracker->consumption());
+  void *allocated = arena.AllocateBytes(256);
+  ASSERT_TRUE(allocated);
+  ASSERT_EQ(256, mem_tracker->consumption());
+  allocated = arena.AllocateBytes(256);
+  ASSERT_TRUE(allocated);
+  ASSERT_EQ(768, mem_tracker->consumption());
+
+  // In DEBUG mode after Reset() the last component of an arena is
+  // cleared, but is then created again; in release mode, the last
+  // component is not cleared. In either case, after Reset()
+  // consumption() should equal the size of the last component which
+  // is 512 bytes.
+  arena.Reset();
+  ASSERT_EQ(512, mem_tracker->consumption());
+
+  // Allocate beyond allowed consumption. This should still go
+  // through, since enforce_limit is false.
+  allocated = arena.AllocateBytes(1024);
+  ASSERT_TRUE(allocated);
+
+  ASSERT_EQ(1536, mem_tracker->consumption());
+}
+
+TEST(TestArena, TestMemoryTrackingEnforced) {
+  shared_ptr<MemTracker> mem_tracker = MemTracker::CreateTracker(1024, "arena-test-tracker");
+  shared_ptr<MemoryTrackingBufferAllocator> allocator(
+      new MemoryTrackingBufferAllocator(HeapBufferAllocator::Get(), mem_tracker,
+                                        // enforce limit
+                                        true));
+  MemoryTrackingArena arena(256, allocator);
+  ASSERT_EQ(256, mem_tracker->consumption());
+  void *allocated = arena.AllocateBytes(256);
+  ASSERT_TRUE(allocated);
+  ASSERT_EQ(256, mem_tracker->consumption());
+  allocated = arena.AllocateBytes(1024);
+  ASSERT_FALSE(allocated);
+  ASSERT_EQ(256, mem_tracker->consumption());
+}
+
+TEST(TestArena, TestSTLAllocator) {
+  Arena a(256);
+  typedef vector<int, ArenaAllocator<int, false> > ArenaVector;
+  ArenaAllocator<int, false> alloc(&a);
+  ArenaVector v(alloc);
+  for (int i = 0; i < 10000; i++) {
+    v.push_back(i);
+  }
+  for (int i = 0; i < 10000; i++) {
+    ASSERT_EQ(i, v[i]);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/arena.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/arena.cc b/be/src/kudu/util/memory/arena.cc
new file mode 100644
index 0000000..b580dbc
--- /dev/null
+++ b/be/src/kudu/util/memory/arena.cc
@@ -0,0 +1,167 @@
+// Copyright 2010 Google Inc.  All Rights Reserved
+//
+// 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 "kudu/util/memory/arena.h"
+
+#include <algorithm>
+#include <memory>
+#include <mutex>
+
+using std::min;
+using std::unique_ptr;
+
+namespace kudu {
+
+template <bool THREADSAFE>
+const size_t ArenaBase<THREADSAFE>::kMinimumChunkSize = 16;
+
+// The max size of our allocations is set to this magic number
+// corresponding to 127 tcmalloc pages (each being 8KB). tcmalloc
+// internally keeps a free-list of spans up to this size. Larger
+// allocations have to go through a linear search through free
+// space, which can get quite slow in a fragmented heap.
+//
+// See the definition of kMaxPages in tcmalloc/src/common.h
+// as well as https://github.com/gperftools/gperftools/issues/535
+// for a description of the performance issue.
+constexpr int kMaxTcmallocFastAllocation = 8192 * 127;
+
+template <bool THREADSAFE>
+ArenaBase<THREADSAFE>::ArenaBase(BufferAllocator* buffer_allocator,
+                                 size_t initial_buffer_size)
+    : buffer_allocator_(buffer_allocator),
+      max_buffer_size_(kMaxTcmallocFastAllocation),
+      arena_footprint_(0) {
+  AddComponent(CHECK_NOTNULL(NewComponent(initial_buffer_size, 0)));
+}
+
+template <bool THREADSAFE>
+ArenaBase<THREADSAFE>::ArenaBase(size_t initial_buffer_size)
+    : ArenaBase<THREADSAFE>(HeapBufferAllocator::Get(),
+                            initial_buffer_size) {
+}
+
+template <bool THREADSAFE>
+void ArenaBase<THREADSAFE>::SetMaxBufferSize(size_t size) {
+  DCHECK_LE(size, kMaxTcmallocFastAllocation);
+  max_buffer_size_ = size;
+}
+
+template <bool THREADSAFE>
+void *ArenaBase<THREADSAFE>::AllocateBytesFallback(const size_t size, const size_t align) {
+  std::lock_guard<mutex_type> lock(component_lock_);
+
+  // It's possible another thread raced with us and already allocated
+  // a new component, in which case we should try the "fast path" again
+  Component* cur = AcquireLoadCurrent();
+  void * result = cur->AllocateBytesAligned(size, align);
+  if (PREDICT_FALSE(result != nullptr)) return result;
+
+  // Really need to allocate more space.
+  size_t next_component_size = min(2 * cur->size(), max_buffer_size_);
+  // But, allocate enough, even if the request is large. In this case,
+  // might violate the max_element_size bound.
+  if (next_component_size < size) {
+    next_component_size = size;
+  }
+  // If soft quota is exhausted we will only get the "minimal" amount of memory
+  // we ask for. In this case if we always use "size" as minimal, we may degrade
+  // to allocating a lot of tiny components, one for each string added to the
+  // arena. This would be very inefficient, so let's first try something between
+  // "size" and "next_component_size". If it fails due to hard quota being
+  // exhausted, we'll fall back to using "size" as minimal.
+  size_t minimal = (size + next_component_size) / 2;
+  CHECK_LE(size, minimal);
+  CHECK_LE(minimal, next_component_size);
+  // Now, just make sure we can actually get the memory.
+  Component* component = NewComponent(next_component_size, minimal);
+  if (component == nullptr) {
+    component = NewComponent(next_component_size, size);
+  }
+  if (!component) return nullptr;
+
+  // Now, must succeed. The component has at least 'size' bytes.
+  result = component->AllocateBytesAligned(size, align);
+  CHECK(result != nullptr);
+
+  // Now add it to the arena.
+  AddComponent(component);
+
+  return result;
+}
+
+template <bool THREADSAFE>
+typename ArenaBase<THREADSAFE>::Component* ArenaBase<THREADSAFE>::NewComponent(
+  size_t requested_size,
+  size_t minimum_size) {
+  Buffer* buffer = buffer_allocator_->BestEffortAllocate(requested_size,
+                                                         minimum_size);
+  if (buffer == nullptr) return nullptr;
+
+  CHECK_EQ(reinterpret_cast<uintptr_t>(buffer->data()) & (16 - 1), 0)
+    << "Components should be 16-byte aligned: " << buffer->data();
+
+  ASAN_POISON_MEMORY_REGION(buffer->data(), buffer->size());
+
+  return new Component(buffer);
+}
+
+// LOCKING: component_lock_ must be held by the current thread.
+template <bool THREADSAFE>
+void ArenaBase<THREADSAFE>::AddComponent(ArenaBase::Component *component) {
+  ReleaseStoreCurrent(component);
+  arena_.push_back(unique_ptr<Component>(component));
+  arena_footprint_ += component->size();
+}
+
+template <bool THREADSAFE>
+void ArenaBase<THREADSAFE>::Reset() {
+  std::lock_guard<mutex_type> lock(component_lock_);
+
+  if (PREDICT_FALSE(arena_.size() > 1)) {
+    unique_ptr<Component> last = std::move(arena_.back());
+    arena_.clear();
+    arena_.emplace_back(std::move(last));
+    ReleaseStoreCurrent(arena_[0].get());
+  }
+  arena_.back()->Reset();
+  arena_footprint_ = arena_.back()->size();
+
+#ifndef NDEBUG
+  // In debug mode release the last component too for (hopefully) better
+  // detection of memory-related bugs (invalid shallow copies, etc.).
+  size_t last_size = arena_.back()->size();
+  arena_.clear();
+  AddComponent(CHECK_NOTNULL(NewComponent(last_size, 0)));
+  arena_footprint_ = 0;
+#endif
+}
+
+template <bool THREADSAFE>
+size_t ArenaBase<THREADSAFE>::memory_footprint() const {
+  std::lock_guard<mutex_type> lock(component_lock_);
+  return arena_footprint_;
+}
+
+// Explicit instantiation.
+template class ArenaBase<true>;
+template class ArenaBase<false>;
+
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/arena.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/arena.h b/be/src/kudu/util/memory/arena.h
new file mode 100644
index 0000000..6d9843b
--- /dev/null
+++ b/be/src/kudu/util/memory/arena.h
@@ -0,0 +1,501 @@
+// Copyright 2010 Google Inc.  All Rights Reserved
+//
+// 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.
+//
+//
+// Memory arena for variable-length datatypes and STL collections.
+
+#ifndef KUDU_UTIL_MEMORY_ARENA_H_
+#define KUDU_UTIL_MEMORY_ARENA_H_
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <new>
+#include <ostream>
+#include <vector>
+
+#include <boost/signals2/dummy_mutex.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/alignment.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/memory/memory.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+template<bool THREADSAFE> struct ArenaTraits;
+
+template <> struct ArenaTraits<true> {
+  typedef Atomic32 offset_type;
+  typedef Mutex mutex_type;
+  typedef simple_spinlock spinlock_type;
+};
+
+template <> struct ArenaTraits<false> {
+  typedef uint32_t offset_type;
+  // For non-threadsafe, we don't need any real locking.
+  typedef boost::signals2::dummy_mutex mutex_type;
+  typedef boost::signals2::dummy_mutex spinlock_type;
+};
+
+// A helper class for storing variable-length blobs (e.g. strings). Once a blob
+// is added to the arena, its index stays fixed. No reallocation happens.
+// Instead, the arena keeps a list of buffers. When it needs to grow, it
+// allocates a new buffer. Each subsequent buffer is 2x larger, than its
+// predecessor, until the maximum specified buffer size is reached.
+// The buffers are furnished by a designated allocator.
+//
+// This class is thread-safe with the fast path lock-free.
+template <bool THREADSAFE>
+class ArenaBase {
+ public:
+  // Arenas are required to have a minimum size of at least this amount.
+  static const size_t kMinimumChunkSize;
+
+  // Creates a new arena, with a single buffer of size up-to initial_buffer_size
+  // and maximum capacity (i.e. total sizes of all buffers)
+  // possibly limited by the buffer allocator. The allocator might cap the
+  // initial allocation request arbitrarily (down to zero). As a consequence,
+  // arena construction never fails due to OOM.
+  //
+  // Calls to AllocateBytes() will then give out bytes from the working buffer
+  // until it is exhausted. Then, a subsequent working buffer will be allocated.
+  // The size of the next buffer is normally 2x the size of the previous buffer.
+  // It might be capped by the allocator, or by the max_buffer_size of the Arena,
+  // settable by SetMaxBufferSize below.
+  //
+  // The default maximum buffer size is ~1MB. See 'SetMaxBufferSize' for details
+  // on when you would want to configure this differently.
+  ArenaBase(BufferAllocator* buffer_allocator,
+            size_t initial_buffer_size);
+
+  // Creates an arena using a default (heap) allocator.
+  explicit ArenaBase(size_t initial_buffer_size);
+
+  // Set the maximum buffer size allocated for this arena.
+  // The maximum buffer size allowed is slightly less than ~1MB (8192 * 127 bytes).
+  //
+  // Consider the following pros/cons of large buffer sizes:
+  //
+  // Pros:
+  //   - Fewer heap allocations if the arena will hold a lot of data.
+  //     (hence better allocation performance out of the arena)
+  //   - Better page locality for objects allocated out of the same arena,
+  //     especially if huge pages are in use.
+  //   - Less internal fragmentation at the "end" of each buffer if the
+  //     size of allocations from the arena is close to the size of the
+  //     buffer. For example, with a 128KB max buffer size and 65KB
+  //     allocations, we will only be able to make one allocation from
+  //     each buffer and waste nearly 50% of memory.
+  // Cons:
+  //   - Larger heap allocations may be more difficult to fulfill if the
+  //     heap is fragmented.
+  //
+  // Overall, if you aren't sure, just leave it at the default.
+  //
+  // NOTE: this method is not thread-safe, even in the thread-safe variant.
+  // It is expected to call this only immediately after constructing the
+  // Arena instance, but before making any allocations.
+  void SetMaxBufferSize(size_t size);
+
+  // Adds content of the specified Slice to the arena, and returns a
+  // pointer to it. The pointer is guaranteed to remain valid during the
+  // lifetime of the arena. The Slice object itself is not copied. The
+  // size information is not stored.
+  // (Normal use case is that the caller already has an array of Slices,
+  // where it keeps these pointers together with size information).
+  // If this request would make the arena grow and the allocator denies that,
+  // returns NULL and leaves the arena unchanged.
+  uint8_t *AddSlice(const Slice& value);
+
+  // Same as above.
+  void * AddBytes(const void *data, size_t len);
+
+  // Handy wrapper for placement-new.
+  //
+  // This ensures that the returned object is properly aligned based on
+  // alignof(T).
+  template<class T, typename ... Args>
+  T *NewObject(Args&&... args);
+
+  // Relocate the given Slice into the arena, setting 'dst' and
+  // returning true if successful.
+  // It is legal for 'dst' to be a pointer to 'src'.
+  // See AddSlice above for detail on memory lifetime.
+  bool RelocateSlice(const Slice &src, Slice *dst);
+
+  // Similar to the above, but for StringPiece.
+  bool RelocateStringPiece(const StringPiece& src, StringPiece* sp);
+
+  // Reserves a blob of the specified size in the arena, and returns a pointer
+  // to it. The caller can then fill the allocated memory. The pointer is
+  // guaranteed to remain valid during the lifetime of the arena.
+  // If this request would make the arena grow and the allocator denies that,
+  // returns NULL and leaves the arena unchanged.
+  void* AllocateBytes(const size_t size) {
+    return AllocateBytesAligned(size, 1);
+  }
+
+  // Allocate bytes, ensuring a specified alignment.
+  // NOTE: alignment MUST be a power of two, or else this will break.
+  void* AllocateBytesAligned(const size_t size, const size_t alignment);
+
+  // Removes all data from the arena. (Invalidates all pointers returned by
+  // AddSlice and AllocateBytes). Does not cause memory allocation.
+  // May reduce memory footprint, as it discards all allocated buffers but
+  // the last one.
+  // Unless allocations exceed max_buffer_size, repetitive filling up and
+  // resetting normally lead to quickly settling memory footprint and ceasing
+  // buffer allocations, as the arena keeps reusing a single, large buffer.
+  void Reset();
+
+  // Returns the memory footprint of this arena, in bytes, defined as a sum of
+  // all buffer sizes. Always greater or equal to the total number of
+  // bytes allocated out of the arena.
+  size_t memory_footprint() const;
+
+ private:
+  typedef typename ArenaTraits<THREADSAFE>::mutex_type mutex_type;
+  // Encapsulates a single buffer in the arena.
+  class Component;
+
+  // Fallback for AllocateBytes non-fast-path
+  void* AllocateBytesFallback(const size_t size, const size_t align);
+
+  Component* NewComponent(size_t requested_size, size_t minimum_size);
+  void AddComponent(Component *component);
+
+  // Load the current component, with "Acquire" semantics (see atomicops.h)
+  // if the arena is meant to be thread-safe.
+  inline Component* AcquireLoadCurrent() {
+    if (THREADSAFE) {
+      return reinterpret_cast<Component*>(
+        base::subtle::Acquire_Load(reinterpret_cast<AtomicWord*>(&current_)));
+    } else {
+      return current_;
+    }
+  }
+
+  // Store the current component, with "Release" semantics (see atomicops.h)
+  // if the arena is meant to be thread-safe.
+  inline void ReleaseStoreCurrent(Component* c) {
+    if (THREADSAFE) {
+      base::subtle::Release_Store(reinterpret_cast<AtomicWord*>(&current_),
+                                  reinterpret_cast<AtomicWord>(c));
+    } else {
+      current_ = c;
+    }
+  }
+
+  BufferAllocator* const buffer_allocator_;
+  std::vector<std::unique_ptr<Component> > arena_;
+
+  // The current component to allocate from.
+  // Use AcquireLoadCurrent and ReleaseStoreCurrent to load/store.
+  Component* current_;
+  size_t max_buffer_size_;
+  size_t arena_footprint_;
+
+  // Lock covering 'slow path' allocation, when new components are
+  // allocated and added to the arena's list. Also covers any other
+  // mutation of the component data structure (eg Reset).
+  mutable mutex_type component_lock_;
+
+  DISALLOW_COPY_AND_ASSIGN(ArenaBase);
+};
+
+// STL-compliant allocator, for use with hash_maps and other structures
+// which share lifetime with an Arena. Enables memory control and improves
+// performance.
+template<class T, bool THREADSAFE> class ArenaAllocator {
+ public:
+  typedef T value_type;
+  typedef size_t size_type;
+  typedef ptrdiff_t difference_type;
+
+  typedef T* pointer;
+  typedef const T* const_pointer;
+  typedef T& reference;
+  typedef const T& const_reference;
+  pointer index(reference r) const  { return &r; }
+  const_pointer index(const_reference r) const  { return &r; }
+  size_type max_size() const  { return size_t(-1) / sizeof(T); }
+
+  explicit ArenaAllocator(ArenaBase<THREADSAFE>* arena) : arena_(arena) {
+    CHECK_NOTNULL(arena_);
+  }
+
+  ~ArenaAllocator() { }
+
+  pointer allocate(size_type n, std::allocator<void>::const_pointer /*hint*/ = 0) {
+    return reinterpret_cast<T*>(arena_->AllocateBytes(n * sizeof(T)));
+  }
+
+  void deallocate(pointer p, size_type n) {}
+
+  void construct(pointer p, const T& val) {
+    new(reinterpret_cast<void*>(p)) T(val);
+  }
+
+  void destroy(pointer p) { p->~T(); }
+
+  template<class U> struct rebind {
+    typedef ArenaAllocator<U, THREADSAFE> other;
+  };
+
+  template<class U, bool TS> ArenaAllocator(const ArenaAllocator<U, TS>& other)
+      : arena_(other.arena()) { }
+
+  template<class U, bool TS> bool operator==(const ArenaAllocator<U, TS>& other) const {
+    return arena_ == other.arena();
+  }
+
+  template<class U, bool TS> bool operator!=(const ArenaAllocator<U, TS>& other) const {
+    return arena_ != other.arena();
+  }
+
+  ArenaBase<THREADSAFE> *arena() const {
+    return arena_;
+  }
+
+ private:
+
+  ArenaBase<THREADSAFE>* arena_;
+};
+
+
+class Arena : public ArenaBase<false> {
+ public:
+  explicit Arena(size_t initial_buffer_size) :
+    ArenaBase<false>(initial_buffer_size)
+  {}
+};
+
+class ThreadSafeArena : public ArenaBase<true> {
+ public:
+  explicit ThreadSafeArena(size_t initial_buffer_size) :
+    ArenaBase<true>(initial_buffer_size)
+  {}
+};
+
+// Arena implementation that is integrated with MemTracker in order to
+// track heap-allocated space consumed by the arena.
+
+class MemoryTrackingArena : public ArenaBase<false> {
+ public:
+
+  MemoryTrackingArena(
+      size_t initial_buffer_size,
+      const std::shared_ptr<MemoryTrackingBufferAllocator>& tracking_allocator)
+      : ArenaBase<false>(tracking_allocator.get(), initial_buffer_size),
+        tracking_allocator_(tracking_allocator) {}
+
+  ~MemoryTrackingArena() {
+  }
+
+ private:
+
+  // This is required in order for the Arena to survive even after tablet is shut down,
+  // e.g., in the case of Scanners running scanners (see tablet_server-test.cc)
+  std::shared_ptr<MemoryTrackingBufferAllocator> tracking_allocator_;
+};
+
+class ThreadSafeMemoryTrackingArena : public ArenaBase<true> {
+ public:
+
+  ThreadSafeMemoryTrackingArena(
+      size_t initial_buffer_size,
+      const std::shared_ptr<MemoryTrackingBufferAllocator>& tracking_allocator)
+      : ArenaBase<true>(tracking_allocator.get(), initial_buffer_size),
+        tracking_allocator_(tracking_allocator) {}
+
+  ~ThreadSafeMemoryTrackingArena() {
+  }
+
+ private:
+
+  // See comment in MemoryTrackingArena above.
+  std::shared_ptr<MemoryTrackingBufferAllocator> tracking_allocator_;
+};
+
+// Implementation of inline and template methods
+
+template<bool THREADSAFE>
+class ArenaBase<THREADSAFE>::Component {
+ public:
+  explicit Component(Buffer* buffer)
+      : buffer_(buffer),
+        data_(static_cast<uint8_t*>(buffer->data())),
+        offset_(0),
+        size_(buffer->size()) {}
+
+  // Tries to reserve space in this component. Returns the pointer to the
+  // reserved space if successful; NULL on failure (if there's no more room).
+  uint8_t* AllocateBytes(const size_t size) {
+    return AllocateBytesAligned(size, 1);
+  }
+
+  uint8_t *AllocateBytesAligned(const size_t size, const size_t alignment);
+
+  size_t size() const { return size_; }
+  void Reset() {
+    ASAN_POISON_MEMORY_REGION(data_, size_);
+    offset_ = 0;
+  }
+
+ private:
+  // Mark the given range unpoisoned in ASAN.
+  // This is a no-op in a non-ASAN build.
+  void AsanUnpoison(const void* addr, size_t size);
+
+  gscoped_ptr<Buffer> buffer_;
+  uint8_t* const data_;
+  typename ArenaTraits<THREADSAFE>::offset_type offset_;
+  const size_t size_;
+
+#ifdef ADDRESS_SANITIZER
+  // Lock used around unpoisoning memory when ASAN is enabled.
+  // ASAN does not support concurrent unpoison calls that may overlap a particular
+  // memory word (8 bytes).
+  typedef typename ArenaTraits<THREADSAFE>::spinlock_type spinlock_type;
+  spinlock_type asan_lock_;
+#endif
+  DISALLOW_COPY_AND_ASSIGN(Component);
+};
+
+
+// Thread-safe implementation
+template <>
+inline uint8_t *ArenaBase<true>::Component::AllocateBytesAligned(
+  const size_t size, const size_t alignment) {
+  // Special case check the allowed alignments. Currently, we only ensure
+  // the allocated buffer components are 16-byte aligned, and the code path
+  // doesn't support larger alignment.
+  DCHECK(alignment == 1 || alignment == 2 || alignment == 4 ||
+         alignment == 8 || alignment == 16)
+    << "bad alignment: " << alignment;
+  retry:
+  Atomic32 offset = Acquire_Load(&offset_);
+
+  Atomic32 aligned = KUDU_ALIGN_UP(offset, alignment);
+  Atomic32 new_offset = aligned + size;
+
+  if (PREDICT_TRUE(new_offset <= size_)) {
+    bool success = Acquire_CompareAndSwap(&offset_, offset, new_offset) == offset;
+    if (PREDICT_TRUE(success)) {
+      AsanUnpoison(data_ + aligned, size);
+      return data_ + aligned;
+    } else {
+      // Raced with another allocator
+      goto retry;
+    }
+  } else {
+    return NULL;
+  }
+}
+
+// Non-Threadsafe implementation
+template <>
+inline uint8_t *ArenaBase<false>::Component::AllocateBytesAligned(
+  const size_t size, const size_t alignment) {
+  DCHECK(alignment == 1 || alignment == 2 || alignment == 4 ||
+         alignment == 8 || alignment == 16)
+    << "bad alignment: " << alignment;
+  size_t aligned = KUDU_ALIGN_UP(offset_, alignment);
+  uint8_t* destination = data_ + aligned;
+  size_t save_offset = offset_;
+  offset_ = aligned + size;
+  if (PREDICT_TRUE(offset_ <= size_)) {
+    AsanUnpoison(data_ + aligned, size);
+    return destination;
+  } else {
+    offset_ = save_offset;
+    return NULL;
+  }
+}
+
+template <bool THREADSAFE>
+inline void ArenaBase<THREADSAFE>::Component::AsanUnpoison(const void* addr, size_t size) {
+#ifdef ADDRESS_SANITIZER
+  std::lock_guard<spinlock_type> l(asan_lock_);
+  ASAN_UNPOISON_MEMORY_REGION(addr, size);
+#endif
+}
+
+// Fast-path allocation should get inlined, and fall-back
+// to non-inline function call for allocation failure
+template <bool THREADSAFE>
+inline void *ArenaBase<THREADSAFE>::AllocateBytesAligned(const size_t size, const size_t align) {
+  void* result = AcquireLoadCurrent()->AllocateBytesAligned(size, align);
+  if (PREDICT_TRUE(result != NULL)) return result;
+  return AllocateBytesFallback(size, align);
+}
+
+template <bool THREADSAFE>
+inline uint8_t* ArenaBase<THREADSAFE>::AddSlice(const Slice& value) {
+  return reinterpret_cast<uint8_t *>(AddBytes(value.data(), value.size()));
+}
+
+template <bool THREADSAFE>
+inline void *ArenaBase<THREADSAFE>::AddBytes(const void *data, size_t len) {
+  void* destination = AllocateBytes(len);
+  if (destination == NULL) return NULL;
+  memcpy(destination, data, len);
+  return destination;
+}
+
+template <bool THREADSAFE>
+inline bool ArenaBase<THREADSAFE>::RelocateSlice(const Slice &src, Slice *dst) {
+  void* destination = AllocateBytes(src.size());
+  if (destination == NULL) return false;
+  memcpy(destination, src.data(), src.size());
+  *dst = Slice(reinterpret_cast<uint8_t *>(destination), src.size());
+  return true;
+}
+
+
+template <bool THREADSAFE>
+inline bool ArenaBase<THREADSAFE>::RelocateStringPiece(const StringPiece& src, StringPiece* sp) {
+  Slice slice(src.data(), src.size());
+  if (!RelocateSlice(slice, &slice)) return false;
+  *sp = StringPiece(reinterpret_cast<const char*>(slice.data()), slice.size());
+  return true;
+}
+
+template<bool THREADSAFE>
+template<class T, class ... Args>
+inline T *ArenaBase<THREADSAFE>::NewObject(Args&&... args) {
+  void *mem = AllocateBytesAligned(sizeof(T), alignof(T));
+  if (mem == NULL) throw std::bad_alloc();
+  return new (mem) T(std::forward<Args>(args)...);
+}
+
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_MEMORY_ARENA_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/memory/memory.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/memory/memory.cc b/be/src/kudu/util/memory/memory.cc
new file mode 100644
index 0000000..b3964df
--- /dev/null
+++ b/be/src/kudu/util/memory/memory.cc
@@ -0,0 +1,339 @@
+// Copyright 2010 Google Inc.  All Rights Reserved
+//
+// 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 "kudu/util/memory/memory.h"
+
+#include <mm_malloc.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <cstring>
+
+#include <gflags/gflags.h>
+
+#include "kudu/util/alignment.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/memory/overwrite.h"
+#include "kudu/util/mem_tracker.h"
+
+using std::copy;
+using std::min;
+
+// TODO(onufry) - test whether the code still tests OK if we set this to true,
+// or remove this code and add a test that Google allocator does not change it's
+// contract - 16-aligned in -c opt and %16 == 8 in debug.
+DEFINE_bool(allocator_aligned_mode, false,
+            "Use 16-byte alignment instead of 8-byte, "
+            "unless explicitly specified otherwise - to boost SIMD");
+TAG_FLAG(allocator_aligned_mode, hidden);
+
+namespace kudu {
+
+namespace {
+static char dummy_buffer[0] = {};
+}
+
+Buffer::~Buffer() {
+#if !defined(NDEBUG) && !defined(ADDRESS_SANITIZER)
+  // "unrolling" the string "BAD" makes for a much more efficient
+  // OverwriteWithPattern call in debug mode, so we can keep this
+  // useful bit of code without tests going slower!
+  //
+  // In ASAN mode, we don't bother with this, because when we free the memory, ASAN will
+  // prevent us from accessing it anyway.
+  OverwriteWithPattern(reinterpret_cast<char*>(data_), size_,
+                       "BADBADBADBADBADBADBADBADBADBADBAD"
+                       "BADBADBADBADBADBADBADBADBADBADBAD"
+                       "BADBADBADBADBADBADBADBADBADBADBAD");
+#endif
+  if (allocator_ != nullptr) allocator_->FreeInternal(this);
+}
+
+void BufferAllocator::LogAllocation(size_t requested,
+                                    size_t minimal,
+                                    Buffer* buffer) {
+  if (buffer == nullptr) {
+    LOG(WARNING) << "Memory allocation failed. "
+                 << "Number of bytes requested: " << requested
+                 << ", minimal: " << minimal;
+    return;
+  }
+  if (buffer->size() < requested) {
+    LOG(WARNING) << "Memory allocation was shorter than requested. "
+                 << "Number of bytes requested to allocate: " << requested
+                 << ", minimal: " << minimal
+                 << ", and actually allocated: " << buffer->size();
+  }
+}
+
+HeapBufferAllocator::HeapBufferAllocator()
+  : aligned_mode_(FLAGS_allocator_aligned_mode) {
+}
+
+Buffer* HeapBufferAllocator::AllocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    BufferAllocator* const originator) {
+  DCHECK_LE(minimal, requested);
+  void* data;
+  size_t attempted = requested;
+  while (true) {
+    data = (attempted == 0) ? &dummy_buffer[0] : Malloc(attempted);
+    if (data != nullptr) {
+      return CreateBuffer(data, attempted, originator);
+    }
+    if (attempted == minimal) return nullptr;
+    attempted = minimal + (attempted - minimal - 1) / 2;
+  }
+}
+
+bool HeapBufferAllocator::ReallocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    Buffer* const buffer,
+    BufferAllocator* const originator) {
+  DCHECK_LE(minimal, requested);
+  void* data;
+  size_t attempted = requested;
+  while (true) {
+    if (attempted == 0) {
+      if (buffer->size() > 0) free(buffer->data());
+      data = &dummy_buffer[0];
+    } else {
+      if (buffer->size() > 0) {
+        data = Realloc(buffer->data(), buffer->size(), attempted);
+      } else {
+        data = Malloc(attempted);
+      }
+    }
+    if (data != nullptr) {
+      UpdateBuffer(data, attempted, buffer);
+      return true;
+    }
+    if (attempted == minimal) return false;
+    attempted = minimal + (attempted - minimal - 1) / 2;
+  }
+}
+
+void HeapBufferAllocator::FreeInternal(Buffer* buffer) {
+  if (buffer->size() > 0) free(buffer->data());
+}
+
+void* HeapBufferAllocator::Malloc(size_t size) {
+  if (aligned_mode_) {
+    void* data;
+    if (posix_memalign(&data, 16, KUDU_ALIGN_UP(size, 16))) {
+       return nullptr;
+    }
+    return data;
+  } else {
+    return malloc(size);
+  }
+}
+
+void* HeapBufferAllocator::Realloc(void* previous_data, size_t previous_size,
+                                   size_t new_size) {
+  if (aligned_mode_) {
+    void* data = Malloc(new_size);
+    if (data) {
+// NOTE(ptab): We should use realloc here to avoid memmory coping,
+// but it doesn't work on memory allocated by posix_memalign(...).
+// realloc reallocates the memory but doesn't preserve the content.
+// TODO(ptab): reiterate after some time to check if it is fixed (tcmalloc ?)
+      memcpy(data, previous_data, min(previous_size, new_size));
+      free(previous_data);
+      return data;
+    } else {
+      return nullptr;
+    }
+  } else {
+    return realloc(previous_data, new_size);
+  }
+}
+
+Buffer* ClearingBufferAllocator::AllocateInternal(size_t requested,
+                                                  size_t minimal,
+                                                  BufferAllocator* originator) {
+  Buffer* buffer = DelegateAllocate(delegate_, requested, minimal,
+                                    originator);
+  if (buffer != nullptr) memset(buffer->data(), 0, buffer->size());
+  return buffer;
+}
+
+bool ClearingBufferAllocator::ReallocateInternal(size_t requested,
+                                                 size_t minimal,
+                                                 Buffer* buffer,
+                                                 BufferAllocator* originator) {
+  size_t offset = (buffer != nullptr ? buffer->size() : 0);
+  bool success = DelegateReallocate(delegate_, requested, minimal, buffer,
+                                    originator);
+  if (success && buffer->size() > offset) {
+    memset(static_cast<char*>(buffer->data()) + offset, 0,
+           buffer->size() - offset);
+  }
+  return success;
+}
+
+void ClearingBufferAllocator::FreeInternal(Buffer* buffer) {
+  DelegateFree(delegate_, buffer);
+}
+
+Buffer* MediatingBufferAllocator::AllocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    BufferAllocator* const originator) {
+  // Allow the mediator to trim the request.
+  size_t granted;
+  if (requested > 0) {
+    granted = mediator_->Allocate(requested, minimal);
+    if (granted < minimal) return nullptr;
+  } else {
+    granted = 0;
+  }
+  Buffer* buffer = DelegateAllocate(delegate_, granted, minimal, originator);
+  if (buffer == nullptr) {
+    mediator_->Free(granted);
+  } else if (buffer->size() < granted) {
+    mediator_->Free(granted - buffer->size());
+  }
+  return buffer;
+}
+
+bool MediatingBufferAllocator::ReallocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    Buffer* const buffer,
+    BufferAllocator* const originator) {
+  // Allow the mediator to trim the request. Be conservative; assume that
+  // realloc may degenerate to malloc-memcpy-free.
+  size_t granted;
+  if (requested > 0) {
+    granted = mediator_->Allocate(requested, minimal);
+    if (granted < minimal) return false;
+  } else {
+    granted = 0;
+  }
+  size_t old_size = buffer->size();
+  if (DelegateReallocate(delegate_, granted, minimal, buffer, originator)) {
+    mediator_->Free(granted - buffer->size() + old_size);
+    return true;
+  } else {
+    mediator_->Free(granted);
+    return false;
+  }
+}
+
+void MediatingBufferAllocator::FreeInternal(Buffer* buffer) {
+  mediator_->Free(buffer->size());
+  DelegateFree(delegate_, buffer);
+}
+
+Buffer* MemoryStatisticsCollectingBufferAllocator::AllocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    BufferAllocator* const originator) {
+  Buffer* buffer = DelegateAllocate(delegate_, requested, minimal, originator);
+  if (buffer != nullptr) {
+    memory_stats_collector_->AllocatedMemoryBytes(buffer->size());
+  } else {
+    memory_stats_collector_->RefusedMemoryBytes(minimal);
+  }
+  return buffer;
+}
+
+bool MemoryStatisticsCollectingBufferAllocator::ReallocateInternal(
+    const size_t requested,
+    const size_t minimal,
+    Buffer* const buffer,
+    BufferAllocator* const originator) {
+  const size_t old_size = buffer->size();
+  bool outcome = DelegateReallocate(delegate_, requested, minimal, buffer,
+                                    originator);
+  if (buffer->size() > old_size) {
+    memory_stats_collector_->AllocatedMemoryBytes(buffer->size() - old_size);
+  } else if (buffer->size() < old_size) {
+    memory_stats_collector_->FreedMemoryBytes(old_size - buffer->size());
+  } else if (!outcome && (minimal > buffer->size())) {
+    memory_stats_collector_->RefusedMemoryBytes(minimal - buffer->size());
+  }
+  return outcome;
+}
+
+void MemoryStatisticsCollectingBufferAllocator::FreeInternal(Buffer* buffer) {
+  DelegateFree(delegate_, buffer);
+  memory_stats_collector_->FreedMemoryBytes(buffer->size());
+}
+
+size_t MemoryTrackingBufferAllocator::Available() const {
+  return enforce_limit_ ? mem_tracker_->SpareCapacity() : std::numeric_limits<int64_t>::max();
+}
+
+bool MemoryTrackingBufferAllocator::TryConsume(int64_t bytes) {
+  // Calls TryConsume first, even if enforce_limit_ is false: this
+  // will cause mem_tracker_ to try to free up more memory by GCing.
+  if (!mem_tracker_->TryConsume(bytes)) {
+    if (enforce_limit_) {
+      return false;
+    } else {
+      // If enforce_limit_ is false, allocate memory anyway.
+      mem_tracker_->Consume(bytes);
+    }
+  }
+  return true;
+}
+
+Buffer* MemoryTrackingBufferAllocator::AllocateInternal(size_t requested,
+                                                        size_t minimal,
+                                                        BufferAllocator* originator) {
+  if (TryConsume(requested)) {
+    Buffer* buffer = DelegateAllocate(delegate_, requested, requested, originator);
+    if (buffer == nullptr) {
+      mem_tracker_->Release(requested);
+    } else {
+      return buffer;
+    }
+  }
+
+  if (TryConsume(minimal)) {
+    Buffer* buffer = DelegateAllocate(delegate_, minimal, minimal, originator);
+    if (buffer == nullptr) {
+      mem_tracker_->Release(minimal);
+    }
+    return buffer;
+  }
+
+  return nullptr;
+}
+
+
+bool MemoryTrackingBufferAllocator::ReallocateInternal(size_t requested,
+                                                       size_t minimal,
+                                                       Buffer* buffer,
+                                                       BufferAllocator* originator) {
+  LOG(FATAL) << "Not implemented";
+  return false;
+}
+
+void MemoryTrackingBufferAllocator::FreeInternal(Buffer* buffer) {
+  DelegateFree(delegate_, buffer);
+  mem_tracker_->Release(buffer->size());
+}
+
+}  // namespace kudu


[17/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/knapsack_solver.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/knapsack_solver.h b/be/src/kudu/util/knapsack_solver.h
new file mode 100644
index 0000000..2c37065
--- /dev/null
+++ b/be/src/kudu/util/knapsack_solver.h
@@ -0,0 +1,269 @@
+// 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.
+#ifndef KUDU_UTIL_KNAPSACK_SOLVER_H
+#define KUDU_UTIL_KNAPSACK_SOLVER_H
+
+#include <glog/logging.h>
+#include <algorithm>
+#include <utility>
+#include <vector>
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+// Solver for the 0-1 knapsack problem. This uses dynamic programming
+// to solve the problem exactly.
+//
+// Given a knapsack capacity of 'W' and a number of potential items 'n',
+// this solver is O(nW) time and space.
+//
+// This implementation is cribbed from wikipedia. The only interesting
+// bit here that doesn't directly match the pseudo-code is that we
+// maintain the "taken" bitmap keeping track of which items were
+// taken, so we can efficiently "trace back" the chosen items.
+template<class Traits>
+class KnapsackSolver {
+ public:
+  typedef typename Traits::item_type item_type;
+  typedef typename Traits::value_type value_type;
+  typedef std::pair<int, value_type> solution_type;
+
+  KnapsackSolver() {}
+  ~KnapsackSolver() {}
+
+  // Solve a knapsack problem in one shot. Finds the set of
+  // items in 'items' such that their weights add up to no
+  // more than 'knapsack_capacity' and maximizes the sum
+  // of their values.
+  // The indexes of the chosen items are stored in 'chosen_items',
+  // and the maximal value is stored in 'optimal_value'.
+  void Solve(std::vector<item_type> &items,
+             int knapsack_capacity,
+             std::vector<int>* chosen_items,
+             value_type* optimal_value);
+
+
+  // The following functions are a more advanced API for solving
+  // knapsack problems, allowing the caller to obtain incremental
+  // results as each item is considered. See the implementation of
+  // Solve() for usage.
+
+  // Prepare to solve a knapsack problem with the given capacity and
+  // item set. The vector of items must remain valid and unchanged
+  // until the next call to Reset().
+  void Reset(int knapsack_capacity,
+             const std::vector<item_type>* items);
+
+  // Process the next item in 'items'. Returns false if there
+  // were no more items to process.
+  bool ProcessNext();
+
+  // Returns the current best solution after the most recent ProcessNext
+  // call. *solution is a pair of (knapsack weight used, value obtained).
+  solution_type GetSolution();
+
+  // Trace the path of item indexes used to achieve the given best
+  // solution as of the latest ProcessNext() call.
+  void TracePath(const solution_type& best,
+                 std::vector<int>* chosen_items);
+
+ private:
+
+  // The state kept by the DP algorithm.
+  class KnapsackBlackboard {
+   public:
+    typedef std::pair<int, value_type> solution_type;
+    KnapsackBlackboard() :
+      n_items_(0),
+      n_weights_(0),
+      cur_item_idx_(0),
+      best_solution_(0, 0) {
+    }
+
+    void ResizeAndClear(int n_items, int max_weight);
+
+    // Current maximum value at the given weight
+    value_type &max_at(int weight) {
+      DCHECK_GE(weight, 0);
+      DCHECK_LT(weight, n_weights_);
+      return max_value_[weight];
+    }
+
+    // Consider the next item to be put into the knapsack
+    // Moves the "state" of the solution forward
+    void Advance(value_type new_val, int new_wt);
+
+    // How many items have been considered
+    int current_item_index() const { return cur_item_idx_; }
+
+    bool item_taken(int item, int weight) const {
+      DCHECK_GE(weight, 0);
+      DCHECK_LT(weight, n_weights_);
+      DCHECK_GE(item, 0);
+      DCHECK_LT(item, n_items_);
+      return item_taken_[index(item, weight)];
+    }
+
+    solution_type best_solution() { return best_solution_; }
+
+    bool done() { return cur_item_idx_ == n_items_; }
+
+   private:
+    void MarkTaken(int item, int weight) {
+      item_taken_[index(item, weight)] = true;
+    }
+
+    // If the dynamic programming matrix has more than this number of cells,
+    // then warn.
+    static const int kWarnDimension = 10000000;
+
+    int index(int item, int weight) const {
+      return n_weights_ * item + weight;
+    }
+
+    // vector with maximum value at the i-th position meaning that it is
+    // the maximum value you can get given a knapsack of weight capacity i
+    // while only considering items 0..cur_item_idx_-1
+    std::vector<value_type> max_value_;
+    std::vector<bool> item_taken_; // TODO: record difference vectors?
+    int n_items_, n_weights_;
+    int cur_item_idx_;
+    // Best current solution
+    solution_type best_solution_;
+
+    DISALLOW_COPY_AND_ASSIGN(KnapsackBlackboard);
+  };
+
+  KnapsackBlackboard bb_;
+  const std::vector<item_type>* items_;
+  int knapsack_capacity_;
+
+  DISALLOW_COPY_AND_ASSIGN(KnapsackSolver);
+};
+
+template<class Traits>
+inline void KnapsackSolver<Traits>::Reset(int knapsack_capacity,
+                                          const std::vector<item_type>* items) {
+  DCHECK_GE(knapsack_capacity, 0);
+  items_ = items;
+  knapsack_capacity_ = knapsack_capacity;
+  bb_.ResizeAndClear(items->size(), knapsack_capacity);
+}
+
+template<class Traits>
+inline bool KnapsackSolver<Traits>::ProcessNext() {
+  if (bb_.done()) return false;
+
+  const item_type& item = (*items_)[bb_.current_item_index()];
+  int item_weight = Traits::get_weight(item);
+  value_type item_value = Traits::get_value(item);
+  bb_.Advance(item_value, item_weight);
+
+  return true;
+}
+
+template<class Traits>
+inline void KnapsackSolver<Traits>::Solve(std::vector<item_type> &items,
+                                          int knapsack_capacity,
+                                          std::vector<int>* chosen_items,
+                                          value_type* optimal_value) {
+  Reset(knapsack_capacity, &items);
+
+  while (ProcessNext()) {
+  }
+
+  solution_type best = GetSolution();
+  *optimal_value = best.second;
+  TracePath(best, chosen_items);
+}
+
+template<class Traits>
+inline typename KnapsackSolver<Traits>::solution_type KnapsackSolver<Traits>::GetSolution() {
+  return bb_.best_solution();
+}
+
+template<class Traits>
+inline void KnapsackSolver<Traits>::TracePath(const solution_type& best,
+                                              std::vector<int>* chosen_items) {
+  chosen_items->clear();
+  // Retrace back which set of items corresponded to this value.
+  int w = best.first;
+  chosen_items->clear();
+  for (int k = bb_.current_item_index() - 1; k >= 0; k--) {
+    if (bb_.item_taken(k, w)) {
+      const item_type& taken = (*items_)[k];
+      chosen_items->push_back(k);
+      w -= Traits::get_weight(taken);
+      DCHECK_GE(w, 0);
+    }
+  }
+}
+
+template<class Traits>
+void KnapsackSolver<Traits>::KnapsackBlackboard::ResizeAndClear(int n_items,
+                                                                int max_weight) {
+  CHECK_GT(n_items, 0);
+  CHECK_GE(max_weight, 0);
+
+  // Rather than zero-indexing the weights, we size the array from
+  // 0 to max_weight. This avoids having to subtract 1 every time
+  // we index into the array.
+  n_weights_ = max_weight + 1;
+  max_value_.resize(n_weights_);
+
+  int dimension = index(n_items, n_weights_);
+  if (dimension > kWarnDimension) {
+    LOG(WARNING) << "Knapsack problem " << n_items << "x" << n_weights_
+                 << " is large: may be inefficient!";
+  }
+  item_taken_.resize(dimension);
+  n_items_ = n_items;
+
+  // Clear
+  std::fill(max_value_.begin(), max_value_.end(), 0);
+  std::fill(item_taken_.begin(), item_taken_.end(), false);
+  best_solution_ = std::make_pair(0, 0);
+
+  cur_item_idx_ = 0;
+}
+
+template<class Traits>
+void KnapsackSolver<Traits>::KnapsackBlackboard::Advance(value_type new_val, int new_wt) {
+  // Use the dynamic programming formula:
+  // Define mv(i, j) as maximum value considering items 0..i-1 with knapsack weight j
+  // Then:
+  // if j - weight(i) >= 0, then:
+  // mv(i, j) = max(mv(i-1, j), mv(i-1, j-weight(i)) + value(j))
+  // else mv(i, j) = mv(i-1, j)
+  // Since the recursive formula requires an access of j-weight(i), we go in reverse.
+  for (int j = n_weights_ - 1; j >= new_wt ; --j) {
+    value_type val_if_taken = max_value_[j - new_wt] + new_val;
+    if (max_value_[j] < val_if_taken) {
+      max_value_[j] = val_if_taken;
+      MarkTaken(cur_item_idx_, j);
+      // Check if new solution found
+      if (best_solution_.second < val_if_taken) {
+        best_solution_ = std::make_pair(j, val_if_taken);
+      }
+    }
+  }
+
+  cur_item_idx_++;
+}
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/locks.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/locks.cc b/be/src/kudu/util/locks.cc
new file mode 100644
index 0000000..380bee4
--- /dev/null
+++ b/be/src/kudu/util/locks.cc
@@ -0,0 +1,47 @@
+// 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 "kudu/util/locks.h"
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/util/malloc.h"
+
+namespace kudu {
+
+using base::subtle::Acquire_CompareAndSwap;
+using base::subtle::NoBarrier_Load;
+using base::subtle::Release_Store;
+
+size_t percpu_rwlock::memory_footprint_excluding_this() const {
+  // Because locks_ is a dynamic array of non-trivially-destructable types,
+  // the returned pointer from new[] isn't guaranteed to point at the start of
+  // a memory block, rendering it useless for malloc_usable_size().
+  //
+  // Rather than replace locks_ with a vector or something equivalent, we'll
+  // just measure the memory footprint using sizeof(), with the understanding
+  // that we might be inaccurate due to malloc "slop".
+  //
+  // See https://code.google.com/p/address-sanitizer/issues/detail?id=395 for
+  // more details.
+  return n_cpus_ * sizeof(padded_lock);
+}
+
+size_t percpu_rwlock::memory_footprint_including_this() const {
+  return kudu_malloc_usable_size(this) + memory_footprint_excluding_this();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/locks.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/locks.h b/be/src/kudu/util/locks.h
new file mode 100644
index 0000000..f70955c
--- /dev/null
+++ b/be/src/kudu/util/locks.h
@@ -0,0 +1,294 @@
+// 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.
+#ifndef KUDU_UTIL_LOCKS_H
+#define KUDU_UTIL_LOCKS_H
+
+#include <sched.h>
+
+#include <algorithm>  // IWYU pragma: keep
+#include <cstddef>
+#include <mutex>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/util/rw_semaphore.h"
+
+namespace kudu {
+
+// Wrapper around the Google SpinLock class to adapt it to the method names
+// expected by Boost.
+class simple_spinlock {
+ public:
+  simple_spinlock() {}
+
+  void lock() {
+    l_.Lock();
+  }
+
+  void unlock() {
+    l_.Unlock();
+  }
+
+  bool try_lock() {
+    return l_.TryLock();
+  }
+
+  // Return whether the lock is currently held.
+  //
+  // This state can change at any instant, so this is only really useful
+  // for assertions where you expect to hold the lock. The success of
+  // such an assertion isn't a guarantee that the current thread is the
+  // holder, but the failure of such an assertion _is_ a guarantee that
+  // the current thread is _not_ holding the lock!
+  bool is_locked() {
+    return l_.IsHeld();
+  }
+
+ private:
+  base::SpinLock l_;
+
+  DISALLOW_COPY_AND_ASSIGN(simple_spinlock);
+};
+
+struct padded_spinlock : public simple_spinlock {
+  char padding[CACHELINE_SIZE - (sizeof(simple_spinlock) % CACHELINE_SIZE)];
+};
+
+// Reader-writer lock.
+// This is functionally equivalent to rw_semaphore in rw_semaphore.h, but should be
+// used whenever the lock is expected to only be acquired on a single thread.
+// It adds TSAN annotations which will detect misuse of the lock, but those
+// annotations also assume that the same thread the takes the lock will unlock it.
+//
+// See rw_semaphore.h for documentation on the individual methods where unclear.
+class rw_spinlock {
+ public:
+  rw_spinlock() {
+    ANNOTATE_RWLOCK_CREATE(this);
+  }
+  ~rw_spinlock() {
+    ANNOTATE_RWLOCK_DESTROY(this);
+  }
+
+  void lock_shared() {
+    sem_.lock_shared();
+    ANNOTATE_RWLOCK_ACQUIRED(this, 0);
+  }
+
+  void unlock_shared() {
+    ANNOTATE_RWLOCK_RELEASED(this, 0);
+    sem_.unlock_shared();
+  }
+
+  bool try_lock() {
+    bool ret = sem_.try_lock();
+    if (ret) {
+      ANNOTATE_RWLOCK_ACQUIRED(this, 1);
+    }
+    return ret;
+  }
+
+  void lock() {
+    sem_.lock();
+    ANNOTATE_RWLOCK_ACQUIRED(this, 1);
+  }
+
+  void unlock() {
+    ANNOTATE_RWLOCK_RELEASED(this, 1);
+    sem_.unlock();
+  }
+
+  bool is_write_locked() const {
+    return sem_.is_write_locked();
+  }
+
+  bool is_locked() const {
+    return sem_.is_locked();
+  }
+
+ private:
+  rw_semaphore sem_;
+};
+
+// A reader-writer lock implementation which is biased for use cases where
+// the write lock is taken infrequently, but the read lock is used often.
+//
+// Internally, this creates N underlying reader-writer locks, one per CPU. When a thread
+// wants to lock in read (shared) mode, it locks only its own CPU's lock in read
+// mode. When it wants to lock in write (exclusive) mode, it locks all CPUs' rwlocks in
+// write mode. The use of reader-writer locks ensures that, even if a thread gets
+// preempted when holding one of the per-CPU locks in read mode, the next thread
+// scheduled onto that CPU will not need to block on the first thread.
+//
+// This means that in the read-mostly case, different readers will not cause any
+// cacheline contention.
+//
+// Usage:
+//   percpu_rwlock mylock;
+//
+//   // Lock shared:
+//   {
+//     kudu::shared_lock<rw_spinlock> lock(mylock.get_lock());
+//     ...
+//   }
+//
+//   // Lock exclusive:
+//
+//   {
+//     std::lock_guard<percpu_rwlock> lock(mylock);
+//     ...
+//   }
+class percpu_rwlock {
+ public:
+  percpu_rwlock() {
+#if defined(__APPLE__) || defined(THREAD_SANITIZER)
+    // OSX doesn't have a way to get the index of the CPU running this thread, so
+    // we'll just use a single lock.
+    //
+    // TSAN limits the number of simultaneous lock acquisitions to 64, so we
+    // can't create one lock per core on machines with lots of cores. So, we'll
+    // also just use a single lock.
+    n_cpus_ = 1;
+#else
+    n_cpus_ = base::MaxCPUIndex() + 1;
+#endif
+    CHECK_GT(n_cpus_, 0);
+    locks_ = new padded_lock[n_cpus_];
+  }
+
+  ~percpu_rwlock() {
+    delete [] locks_;
+  }
+
+  rw_spinlock &get_lock() {
+#if defined(__APPLE__) || defined(THREAD_SANITIZER)
+    int cpu = 0;
+#else
+    int cpu = sched_getcpu();
+    CHECK_LT(cpu, n_cpus_);
+#endif  // defined(__APPLE__)
+    return locks_[cpu].lock;
+  }
+
+  bool try_lock() {
+    for (int i = 0; i < n_cpus_; i++) {
+      if (!locks_[i].lock.try_lock()) {
+        while (i--) {
+          locks_[i].lock.unlock();
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  // Return true if this lock is held on any CPU.
+  // See simple_spinlock::is_locked() for details about where this is useful.
+  bool is_locked() const {
+    for (int i = 0; i < n_cpus_; i++) {
+      if (locks_[i].lock.is_locked()) return true;
+    }
+    return false;
+  }
+
+  bool is_write_locked() const {
+    for (int i = 0; i < n_cpus_; i++) {
+      if (!locks_[i].lock.is_write_locked()) return false;
+    }
+    return true;
+  }
+
+  void lock() {
+    for (int i = 0; i < n_cpus_; i++) {
+      locks_[i].lock.lock();
+    }
+  }
+
+  void unlock() {
+    for (int i = 0; i < n_cpus_; i++) {
+      locks_[i].lock.unlock();
+    }
+  }
+
+  // Returns the memory usage of this object without the object itself. Should
+  // be used when embedded inside another object.
+  size_t memory_footprint_excluding_this() const;
+
+  // Returns the memory usage of this object including the object itself.
+  // Should be used when allocated on the heap.
+  size_t memory_footprint_including_this() const;
+
+ private:
+  struct padded_lock {
+    rw_spinlock lock;
+    char padding[CACHELINE_SIZE - (sizeof(rw_spinlock) % CACHELINE_SIZE)];
+  };
+
+  int n_cpus_;
+  padded_lock *locks_;
+};
+
+// Simple implementation of the std::shared_lock API, which is not available in
+// the standard library until C++14. Defers error checking to the underlying
+// mutex.
+
+template <typename Mutex>
+class shared_lock {
+ public:
+  shared_lock()
+      : m_(nullptr) {
+  }
+
+  explicit shared_lock(Mutex& m)
+      : m_(&m) {
+    m_->lock_shared();
+  }
+
+  shared_lock(Mutex& m, std::try_to_lock_t t)
+      : m_(nullptr) {
+    if (m.try_lock_shared()) {
+      m_ = &m;
+    }
+  }
+
+  bool owns_lock() const {
+    return m_;
+  }
+
+  void swap(shared_lock& other) {
+    std::swap(m_,other.m_);
+  }
+
+  ~shared_lock() {
+    if (m_ != nullptr) {
+      m_->unlock_shared();
+    }
+  }
+
+ private:
+  Mutex* m_;
+  DISALLOW_COPY_AND_ASSIGN(shared_lock<Mutex>);
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/logging-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging-test.cc b/be/src/kudu/util/logging-test.cc
new file mode 100644
index 0000000..cceece8
--- /dev/null
+++ b/be/src/kudu/util/logging-test.cc
@@ -0,0 +1,249 @@
+// 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 <atomic>
+#include <cstdint>
+#include <ctime>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gmock/gmock-matchers.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/async_logger.h"
+#include "kudu/util/barrier.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/logging_test_util.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"  // IWYU pragma: keep
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+// Test the KLOG_EVERY_N_SECS(...) macro.
+TEST(LoggingTest, TestThrottledLogging) {
+  StringVectorSink sink;
+  ScopedRegisterSink srs(&sink);
+
+  for (int i = 0; i < 10000; i++) {
+    KLOG_EVERY_N_SECS(INFO, 1) << "test" << THROTTLE_MSG;
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    if (sink.logged_msgs().size() >= 2) break;
+  }
+  const vector<string>& msgs = sink.logged_msgs();
+  ASSERT_GE(msgs.size(), 2);
+
+  // The first log line shouldn't have a suppression count.
+  EXPECT_THAT(msgs[0], testing::ContainsRegex("test$"));
+  // The second one should have suppressed at least three digits worth of log messages.
+  EXPECT_THAT(msgs[1], testing::ContainsRegex("\\[suppressed [0-9]{3,} similar messages\\]"));
+}
+
+TEST(LoggingTest, TestAdvancedThrottling) {
+  StringVectorSink sink;
+  ScopedRegisterSink srs(&sink);
+
+  logging::LogThrottler throttle_a;
+
+  // First, log only using a single tag and throttler.
+  for (int i = 0; i < 100000; i++) {
+    KLOG_EVERY_N_SECS_THROTTLER(INFO, 1, throttle_a, "tag_a") << "test" << THROTTLE_MSG;
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    if (sink.logged_msgs().size() >= 2) break;
+  }
+  auto& msgs = sink.logged_msgs();
+  ASSERT_GE(msgs.size(), 2);
+
+  // The first log line shouldn't have a suppression count.
+  EXPECT_THAT(msgs[0], testing::ContainsRegex("test$"));
+  // The second one should have suppressed at least three digits worth of log messages.
+  EXPECT_THAT(msgs[1], testing::ContainsRegex("\\[suppressed [0-9]{3,} similar messages\\]"));
+  msgs.clear();
+
+  // Now, try logging using two different tags in rapid succession. This should not
+  // throttle, because the tag is switching.
+  KLOG_EVERY_N_SECS_THROTTLER(INFO, 1, throttle_a, "tag_b") << "test b" << THROTTLE_MSG;
+  KLOG_EVERY_N_SECS_THROTTLER(INFO, 1, throttle_a, "tag_b") << "test b" << THROTTLE_MSG;
+  KLOG_EVERY_N_SECS_THROTTLER(INFO, 1, throttle_a, "tag_c") << "test c" << THROTTLE_MSG;
+  KLOG_EVERY_N_SECS_THROTTLER(INFO, 1, throttle_a, "tag_b") << "test b" << THROTTLE_MSG;
+  ASSERT_EQ(msgs.size(), 3);
+  EXPECT_THAT(msgs[0], testing::ContainsRegex("test b$"));
+  EXPECT_THAT(msgs[1], testing::ContainsRegex("test c$"));
+  EXPECT_THAT(msgs[2], testing::ContainsRegex("test b$"));
+}
+
+// Test Logger implementation that just counts the number of messages
+// and flushes.
+//
+// This is purposefully thread-unsafe because we expect that the
+// AsyncLogger is only accessing the underlying logger from a single
+// thhread.
+class CountingLogger : public google::base::Logger {
+ public:
+  void Write(bool force_flush,
+             time_t /*timestamp*/,
+             const char* /*message*/,
+             int /*message_len*/) override {
+    message_count_++;
+    if (force_flush) {
+      Flush();
+    }
+  }
+
+  void Flush() override {
+    // Simulate a slow disk.
+    SleepFor(MonoDelta::FromMilliseconds(5));
+    flush_count_++;
+  }
+
+  uint32_t LogSize() override {
+    return 0;
+  }
+
+  std::atomic<int> flush_count_ = {0};
+  std::atomic<int> message_count_ = {0};
+};
+
+TEST(LoggingTest, TestAsyncLogger) {
+  const int kNumThreads = 4;
+  const int kNumMessages = 10000;
+  const int kBuffer = 10000;
+  CountingLogger base;
+  AsyncLogger async(&base, kBuffer);
+  async.Start();
+
+  vector<std::thread> threads;
+  Barrier go_barrier(kNumThreads + 1);
+  // Start some threads writing log messages.
+  for (int i = 0; i < kNumThreads; i++) {
+    threads.emplace_back([&]() {
+        go_barrier.Wait();
+        for (int m = 0; m < kNumMessages; m++) {
+          async.Write(true, m, "x", 1);
+        }
+      });
+  }
+
+  // And a thread calling Flush().
+  threads.emplace_back([&]() {
+      go_barrier.Wait();
+      for (int i = 0; i < 10; i++) {
+        async.Flush();
+        SleepFor(MonoDelta::FromMilliseconds(3));
+      }
+    });
+
+  for (auto& t : threads) {
+    t.join();
+  }
+  async.Stop();
+  ASSERT_EQ(base.message_count_, kNumMessages * kNumThreads);
+  // The async logger should only flush once per "batch" rather than
+  // once per message, even though we wrote every message with
+  // 'flush' set to true.
+  ASSERT_LT(base.flush_count_, kNumMessages * kNumThreads);
+  ASSERT_GT(async.app_threads_blocked_count_for_tests(), 0);
+}
+
+TEST(LoggingTest, TestAsyncLoggerAutoFlush) {
+  const int kBuffer = 10000;
+  CountingLogger base;
+  AsyncLogger async(&base, kBuffer);
+
+  FLAGS_logbufsecs = 1;
+  async.Start();
+
+  // Write some log messages with non-force_flush types.
+  async.Write(false, 0, "test-x", 1);
+  async.Write(false, 1, "test-y", 1);
+
+  // The flush wait timeout might take a little bit of time to run.
+  ASSERT_EVENTUALLY([&]() {
+    ASSERT_EQ(base.message_count_, 2);
+    // The AsyncLogger should have flushed at least once by the timer automatically
+    // so there should be no more messages in the buffer.
+    ASSERT_GT(base.flush_count_, 0);
+  });
+  async.Stop();
+}
+
+// Basic test that the redaction utilities work as expected.
+TEST(LoggingTest, TestRedactionBasic) {
+  ASSERT_STREQ("<redacted>", KUDU_REDACT("hello"));
+  {
+    ScopedDisableRedaction no_redaction;
+    ASSERT_STREQ("hello", KUDU_REDACT("hello"));
+  }
+  ASSERT_STREQ("hello", KUDU_DISABLE_REDACTION(KUDU_REDACT("hello")));
+}
+
+// Typically, ToString() methods apply to some complex object with a bunch
+// of fields, some of which are user data (need redaction) and others of which
+// are not. This shows an example of a such a function, which will behave
+// differently based on whether the calling scope has explicitly disabled
+// redaction.
+string SomeComplexStringify(const string& public_data, const string& private_data) {
+  return strings::Substitute("public=$0, private=$1",
+                             public_data,
+                             KUDU_REDACT(private_data));
+}
+
+TEST(LoggingTest, TestRedactionIllustrateUsage) {
+  // By default, the private data will be redacted.
+  ASSERT_EQ("public=abc, private=<redacted>", SomeComplexStringify("abc", "def"));
+
+  // We can wrap the expression in KUDU_DISABLE_REDACTION(...) to evaluate it
+  // with redaction temporarily disabled.
+  ASSERT_EQ("public=abc, private=def", KUDU_DISABLE_REDACTION(SomeComplexStringify("abc", "def")));
+
+  // Or we can execute an entire scope with redaction disabled.
+  KUDU_DISABLE_REDACTION({
+    ASSERT_EQ("public=abc, private=def", SomeComplexStringify("abc", "def"));
+  });
+}
+
+
+TEST(LoggingTest, TestLogTiming) {
+  LOG_TIMING(INFO, "foo") {
+  }
+  {
+    SCOPED_LOG_TIMING(INFO, "bar");
+  }
+  LOG_SLOW_EXECUTION(INFO, 1, "baz") {
+  }
+
+  // Previous implementations of the above macro confused clang-tidy's use-after-move
+  // check and generated false positives.
+  string s1 = "hello";
+  string s2;
+  LOG_SLOW_EXECUTION(INFO, 1, "baz") {
+    LOG(INFO) << s1;
+    s2 = std::move(s1);
+  }
+
+  ASSERT_EQ("hello", s2);
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/logging.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging.cc b/be/src/kudu/util/logging.cc
new file mode 100644
index 0000000..fcf035f
--- /dev/null
+++ b/be/src/kudu/util/logging.cc
@@ -0,0 +1,413 @@
+// 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 "kudu/util/logging.h"
+
+#include <unistd.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <cstdlib>
+#include <ctime>
+#include <fstream>
+#include <mutex>
+#include <utility>
+
+#include <boost/uuid/random_generator.hpp>
+#include <boost/uuid/uuid_io.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/callback.h"  // IWYU pragma: keep
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/async_logger.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging_callback.h"
+#include "kudu/util/minidump.h"
+#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]");
+TAG_FLAG(log_filename, stable);
+
+DEFINE_bool(log_async, true,
+            "Enable asynchronous writing to log files. This improves "
+            "latency and stability.");
+TAG_FLAG(log_async, hidden);
+
+DEFINE_int32(log_async_buffer_bytes_per_level, 2 * 1024 * 1024,
+             "The number of bytes of buffer space used by each log "
+             "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.");
+TAG_FLAG(max_log_files, runtime);
+TAG_FLAG(max_log_files, experimental);
+
+#define PROJ_NAME "kudu"
+
+bool logging_initialized = false;
+
+using namespace std; // NOLINT(*)
+using namespace boost::uuids; // NOLINT(*)
+
+using base::SpinLock;
+using base::SpinLockHolder;
+
+namespace kudu {
+
+__thread bool tls_redact_user_data = true;
+kudu::RedactContext g_should_redact;
+const char* const kRedactionMessage = "<redacted>";
+
+namespace {
+
+class SimpleSink : public google::LogSink {
+ public:
+  explicit SimpleSink(LoggingCallback cb) : cb_(std::move(cb)) {}
+
+  virtual ~SimpleSink() OVERRIDE {
+  }
+
+  virtual void send(google::LogSeverity severity, const char* full_filename,
+                    const char* base_filename, int line,
+                    const struct ::tm* tm_time,
+                    const char* message, size_t message_len) OVERRIDE {
+    LogSeverity kudu_severity;
+    switch (severity) {
+      case google::INFO:
+        kudu_severity = SEVERITY_INFO;
+        break;
+      case google::WARNING:
+        kudu_severity = SEVERITY_WARNING;
+        break;
+      case google::ERROR:
+        kudu_severity = SEVERITY_ERROR;
+        break;
+      case google::FATAL:
+        kudu_severity = SEVERITY_FATAL;
+        break;
+      default:
+        LOG(FATAL) << "Unknown glog severity: " << severity;
+    }
+    cb_.Run(kudu_severity, full_filename, line, tm_time, message, message_len);
+  }
+
+ private:
+
+  LoggingCallback cb_;
+};
+
+SpinLock logging_mutex(base::LINKER_INITIALIZED);
+
+// There can only be a single instance of a SimpleSink.
+//
+// Protected by 'logging_mutex'.
+SimpleSink* registered_sink = nullptr;
+
+// Records the logging severity after the first call to
+// InitGoogleLoggingSafe{Basic}. Calls to UnregisterLoggingCallback()
+// will restore stderr logging back to this severity level.
+//
+// Protected by 'logging_mutex'.
+int initial_stderr_severity;
+
+void EnableAsyncLogging() {
+  debug::ScopedLeakCheckDisabler leaky;
+
+  // Enable Async for every level except for FATAL. Fatal should be synchronous
+  // to ensure that we get the fatal log message written before exiting.
+  for (auto level : { google::INFO, google::WARNING, google::ERROR }) {
+    auto* orig = google::base::GetLogger(level);
+    auto* async = new AsyncLogger(orig, FLAGS_log_async_buffer_bytes_per_level);
+    async->Start();
+    google::base::SetLogger(level, async);
+  }
+}
+
+void UnregisterLoggingCallbackUnlocked() {
+  CHECK(logging_mutex.IsHeld());
+  CHECK(registered_sink);
+
+  // Restore logging to stderr, then remove our sink. This ordering ensures
+  // that no log messages are missed.
+  google::SetStderrLogging(initial_stderr_severity);
+  google::RemoveLogSink(registered_sink);
+  delete registered_sink;
+  registered_sink = nullptr;
+}
+
+void FlushCoverageOnExit() {
+  // Coverage flushing is not re-entrant, but this might be called from a
+  // crash signal context, so avoid re-entrancy.
+  static __thread bool in_call = false;
+  if (in_call) return;
+  in_call = true;
+
+  // The failure writer will be called multiple times per exit.
+  // We only need to flush coverage once. We use a 'once' here so that,
+  // if another thread is already flushing, we'll block and wait for them
+  // to finish before allowing this thread to call abort().
+  static std::once_flag once;
+  std::call_once(once, [] {
+      static const char msg[] = "Flushing coverage data before crash...\n";
+      write(STDERR_FILENO, msg, arraysize(msg));
+      TryFlushCoverage();
+    });
+  in_call = false;
+}
+
+// On SEGVs, etc, glog will call this function to write the error to stderr. This
+// implementation is copied from glog with the exception that we also flush coverage
+// the first time it's called.
+//
+// NOTE: this is only used in coverage builds!
+void FailureWriterWithCoverage(const char* data, int size) {
+  FlushCoverageOnExit();
+
+  // Original implementation from glog:
+  if (write(STDERR_FILENO, data, size) < 0) {
+    // Ignore errors.
+  }
+}
+
+// GLog "failure function". This is called in the case of LOG(FATAL) to
+// ensure that we flush coverage even on crashes.
+//
+// NOTE: this is only used in coverage builds!
+void FlushCoverageAndAbort() {
+  FlushCoverageOnExit();
+  abort();
+}
+} // anonymous namespace
+
+void InitGoogleLoggingSafe(const char* arg) {
+  SpinLockHolder l(&logging_mutex);
+  if (logging_initialized) return;
+
+  google::InstallFailureSignalHandler();
+
+  if (!FLAGS_log_filename.empty()) {
+    for (int severity = google::INFO; severity <= google::FATAL; ++severity) {
+      google::SetLogSymlink(severity, FLAGS_log_filename.c_str());
+    }
+  }
+
+  // This forces our logging to use /tmp rather than looking for a
+  // temporary directory if none is specified. This is done so that we
+  // can reliably construct the log file name without duplicating the
+  // complex logic that glog uses to guess at a temporary dir.
+  if (FLAGS_log_dir.empty()) {
+    FLAGS_log_dir = "/tmp";
+  }
+
+  if (!FLAGS_logtostderr) {
+    // Verify that a log file can be created in log_dir by creating a tmp file.
+    ostringstream ss;
+    random_generator uuid_generator;
+    ss << FLAGS_log_dir << "/" << PROJ_NAME "_test_log." << uuid_generator();
+    const string file_name = ss.str();
+    ofstream test_file(file_name.c_str());
+    if (!test_file.is_open()) {
+      ostringstream error_msg;
+      error_msg << "Could not open file in log_dir " << FLAGS_log_dir;
+      perror(error_msg.str().c_str());
+      // Unlock the mutex before exiting the program to avoid mutex d'tor assert.
+      logging_mutex.Unlock();
+      exit(1);
+    }
+    remove(file_name.c_str());
+  }
+
+  google::InitGoogleLogging(arg);
+
+  // In coverage builds, we should flush coverage before exiting on crash.
+  // This way, fault injection tests still capture coverage of the daemon
+  // that "crashed".
+  if (IsCoverageBuild()) {
+    // We have to use both the "failure writer" and the "FailureFunction".
+    // This allows us to handle both LOG(FATAL) and unintended crashes like
+    // SEGVs.
+    google::InstallFailureWriter(FailureWriterWithCoverage);
+    google::InstallFailureFunction(FlushCoverageAndAbort);
+  }
+
+  // Needs to be done after InitGoogleLogging
+  if (FLAGS_log_filename.empty()) {
+    CHECK_STRNE(google::ProgramInvocationShortName(), "UNKNOWN")
+        << ": must initialize gflags before glog";
+    FLAGS_log_filename = google::ProgramInvocationShortName();
+  }
+
+  // File logging: on.
+  // Stderr logging threshold: FLAGS_stderrthreshold.
+  // Sink logging: off.
+  initial_stderr_severity = FLAGS_stderrthreshold;
+
+  // Ignore SIGPIPE early in the startup process so that threads writing to TLS
+  // sockets do not crash when writing to a closed socket. See KUDU-1910.
+  IgnoreSigPipe();
+
+  // For minidump support. Must be called before logging threads started.
+  CHECK_OK(BlockSigUSR1());
+
+  if (FLAGS_log_async) {
+    EnableAsyncLogging();
+  }
+
+  logging_initialized = true;
+}
+
+void InitGoogleLoggingSafeBasic(const char* arg) {
+  SpinLockHolder l(&logging_mutex);
+  if (logging_initialized) return;
+
+  google::InitGoogleLogging(arg);
+
+  // This also disables file-based logging.
+  google::LogToStderr();
+
+  // File logging: off.
+  // Stderr logging threshold: INFO.
+  // Sink logging: off.
+  initial_stderr_severity = google::INFO;
+  logging_initialized = true;
+}
+
+void RegisterLoggingCallback(const LoggingCallback& cb) {
+  SpinLockHolder l(&logging_mutex);
+  CHECK(logging_initialized);
+
+  if (registered_sink) {
+    LOG(WARNING) << "Cannot register logging callback: one already registered";
+    return;
+  }
+
+  // AddLogSink() claims to take ownership of the sink, but it doesn't
+  // really; it actually expects it to remain valid until
+  // google::ShutdownGoogleLogging() is called.
+  registered_sink = new SimpleSink(cb);
+  google::AddLogSink(registered_sink);
+
+  // Even when stderr logging is ostensibly off, it's still emitting
+  // ERROR-level stuff. This is the default.
+  google::SetStderrLogging(google::ERROR);
+
+  // File logging: yes, if InitGoogleLoggingSafe() was called earlier.
+  // Stderr logging threshold: ERROR.
+  // Sink logging: on.
+}
+
+void UnregisterLoggingCallback() {
+  SpinLockHolder l(&logging_mutex);
+  CHECK(logging_initialized);
+
+  if (!registered_sink) {
+    LOG(WARNING) << "Cannot unregister logging callback: none registered";
+    return;
+  }
+
+  UnregisterLoggingCallbackUnlocked();
+  // File logging: yes, if InitGoogleLoggingSafe() was called earlier.
+  // Stderr logging threshold: initial_stderr_severity.
+  // Sink logging: off.
+}
+
+void GetFullLogFilename(google::LogSeverity severity, string* filename) {
+  ostringstream ss;
+  ss << FLAGS_log_dir << "/" << FLAGS_log_filename << "."
+     << google::GetLogSeverityName(severity);
+  *filename = ss.str();
+}
+
+std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch) {
+  time_t secs_since_epoch = micros_since_epoch / 1000000;
+  int usecs = micros_since_epoch % 1000000;
+  struct tm tm_time;
+  localtime_r(&secs_since_epoch, &tm_time);
+
+  return StringPrintf("%02d%02d %02d:%02d:%02d.%06d",
+                      1 + tm_time.tm_mon,
+                      tm_time.tm_mday,
+                      tm_time.tm_hour,
+                      tm_time.tm_min,
+                      tm_time.tm_sec,
+                      usecs);
+}
+
+void ShutdownLoggingSafe() {
+  SpinLockHolder l(&logging_mutex);
+  if (!logging_initialized) return;
+
+  if (registered_sink) {
+    UnregisterLoggingCallbackUnlocked();
+  }
+
+  google::ShutdownGoogleLogging();
+
+  logging_initialized = false;
+}
+
+Status DeleteExcessLogFiles(Env* env) {
+  int32_t max_log_files = FLAGS_max_log_files;
+  // Ignore bad input or disable log rotation.
+  if (max_log_files <= 0) return Status::OK();
+
+  for (int severity = 0; severity < google::NUM_SEVERITIES; ++severity) {
+    // Build glob pattern for input
+    // e.g. /var/log/kudu/kudu-master.*.INFO.*
+    string pattern = strings::Substitute("$0/$1.*.$2.*", FLAGS_log_dir, FLAGS_log_filename,
+                                         google::GetLogSeverityName(severity));
+
+    // Keep the 'max_log_files' most recent log files, as compared by
+    // modification time. Glog files contain a second-granularity timestamp in
+    // the name, so this could potentially use the filename sort order as
+    // guaranteed by glob, however this code has been adapted from Impala which
+    // uses mtime to determine which files to delete, and there haven't been any
+    // issues in production settings.
+    RETURN_NOT_OK(env_util::DeleteExcessFilesByPattern(env, pattern, max_log_files));
+  }
+  return Status::OK();
+}
+
+// Support for the special THROTTLE_MSG token in a log message stream.
+ostream& operator<<(ostream &os, const PRIVATE_ThrottleMsg& /*unused*/) {
+  using google::LogMessage;
+#ifdef DISABLE_RTTI
+  LogMessage::LogStream *log = static_cast<LogMessage::LogStream*>(&os);
+#else
+  LogMessage::LogStream *log = dynamic_cast<LogMessage::LogStream*>(&os);
+#endif
+  CHECK(log && log == log->self())
+      << "You must not use COUNTER with non-glog ostream";
+  int ctr = log->ctr();
+  if (ctr > 0) {
+    os << " [suppressed " << ctr << " similar messages]";
+  }
+  return os;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/logging.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging.h b/be/src/kudu/util/logging.h
new file mode 100644
index 0000000..428dadc
--- /dev/null
+++ b/be/src/kudu/util/logging.h
@@ -0,0 +1,367 @@
+// 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.
+#ifndef KUDU_UTIL_LOGGING_H
+#define KUDU_UTIL_LOGGING_H
+
+#include <iosfwd>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/logging_callback.h"
+#include "kudu/util/status.h"
+
+////////////////////////////////////////////////////////////////////////////////
+// Redaction support
+////////////////////////////////////////////////////////////////////////////////
+
+// Disable redaction of user data while evaluating the expression 'expr'.
+// This may be used inline as an expression, such as:
+//
+//   LOG(INFO) << KUDU_DISABLE_REDACTION(schema.DebugRow(my_row));
+//
+// or with a block:
+//
+//  KUDU_DISABLE_REDACTION({
+//    LOG(INFO) << schema.DebugRow(my_row);
+//  });
+//
+// Redaction should be disabled in the following cases:
+//
+// 1) Outputting strings to a "secure" endpoint (for example an authenticated and authorized
+//    web UI)
+//
+// 2) Using methods like schema.DebugRow(...) when the parameter is not in fact a user-provided
+//    row, but instead some piece of metadata such as a partition boundary.
+#define KUDU_DISABLE_REDACTION(expr) ([&]() {        \
+      kudu::ScopedDisableRedaction s;                \
+      return (expr);                                 \
+    })()
+
+// Evaluates to 'true' if the caller should redact any user data in the current scope.
+// Most callers should instead use KUDU_REDACT(...) defined below, but this can be useful
+// to short-circuit expensive logic.
+#define KUDU_SHOULD_REDACT() ((kudu::g_should_redact == kudu::RedactContext::ALL ||    \
+  kudu::g_should_redact == kudu::RedactContext::LOG) && kudu::tls_redact_user_data)
+
+// Either evaluate and return 'expr', or return the string "<redacted>", depending on whether
+// redaction is enabled in the current scope.
+#define KUDU_REDACT(expr) \
+  (KUDU_SHOULD_REDACT() ? kRedactionMessage : (expr))
+
+// Like the above, but with the additional condition that redaction will only
+// be performed if 'cond' must be true.
+#define KUDU_MAYBE_REDACT_IF(cond, expr) \
+  ((KUDU_SHOULD_REDACT() && (cond)) ? kudu::kRedactionMessage : (expr))
+
+////////////////////////////////////////
+// Redaction implementation details follow.
+////////////////////////////////////////
+
+namespace kudu {
+
+// Flag which allows redaction to be enabled or disabled for a thread context.
+// Defaults to enabling redaction, since it's the safer default with respect to
+// leaking user data, and it's easier to identify when data is over-redacted
+// than vice-versa.
+extern __thread bool tls_redact_user_data;
+
+// Redacted log messages are replaced with this constant.
+extern const char* const kRedactionMessage;
+
+enum class RedactContext {
+  ALL,
+  LOG,
+  NONE
+};
+
+// Flag to indicate which redaction context is enabled.
+extern kudu::RedactContext g_should_redact;
+
+class ScopedDisableRedaction {
+ public:
+  ScopedDisableRedaction()
+      : old_val_(tls_redact_user_data) {
+    tls_redact_user_data = false;
+  }
+
+  ~ScopedDisableRedaction() {
+    tls_redact_user_data = old_val_;
+  }
+
+ private:
+  bool old_val_;
+};
+
+} // namespace kudu
+
+////////////////////////////////////////////////////////////////////////////////
+// Throttled logging support
+////////////////////////////////////////////////////////////////////////////////
+
+// Logs a message throttled to appear at most once every 'n_secs' seconds to
+// the given severity.
+//
+// The log message may include the special token 'THROTTLE_MSG' which expands
+// to either an empty string or '[suppressed <n> similar messages]'.
+//
+// Example usage:
+//   KLOG_EVERY_N_SECS(WARNING, 1) << "server is low on memory" << THROTTLE_MSG;
+//
+//
+// Advanced per-instance throttling
+// -----------------------------------
+// For cases where the throttling should be scoped to a given class instance,
+// you may define a logging::LogThrottler object and pass it to the
+// KLOG_EVERY_N_SECS_THROTTLER(...) macro. In addition, you must pass a "tag".
+// Only log messages with equal tags (by pointer equality) will be throttled.
+// For example:
+//
+//    struct MyThing {
+//      string name;
+//      LogThrottler throttler;
+//    };
+//
+//    if (...) {
+//      LOG_EVERY_N_SECS_THROTTLER(INFO, 1, my_thing->throttler, "coffee") <<
+//        my_thing->name << " needs coffee!";
+//    } else {
+//      LOG_EVERY_N_SECS_THROTTLER(INFO, 1, my_thing->throttler, "wine") <<
+//        my_thing->name << " needs wine!";
+//    }
+//
+// In this example, the "coffee"-related message will be collapsed into other
+// such messages within the prior one second; however, if the state alternates
+// between the "coffee" message and the "wine" message, then each such alternation
+// will yield a message.
+
+#define KLOG_EVERY_N_SECS_THROTTLER(severity, n_secs, throttler, tag) \
+  int VARNAME_LINENUM(num_suppressed) = 0;                            \
+  if ((throttler).ShouldLog(n_secs, tag, &VARNAME_LINENUM(num_suppressed)))  \
+    google::LogMessage( \
+      __FILE__, __LINE__, google::GLOG_ ## severity, VARNAME_LINENUM(num_suppressed), \
+      &google::LogMessage::SendToLog).stream()
+
+#define KLOG_EVERY_N_SECS(severity, n_secs) \
+  static logging::LogThrottler LOG_THROTTLER;  \
+  KLOG_EVERY_N_SECS_THROTTLER(severity, n_secs, LOG_THROTTLER, "no-tag")
+
+
+namespace kudu {
+enum PRIVATE_ThrottleMsg {THROTTLE_MSG};
+} // namespace kudu
+
+////////////////////////////////////////////////////////////////////////////////
+// Versions of glog macros for "LOG_EVERY" and "LOG_FIRST" that annotate the
+// benign races on their internal static variables.
+////////////////////////////////////////////////////////////////////////////////
+
+// The "base" macros.
+#define KUDU_SOME_KIND_OF_LOG_EVERY_N(severity, n, what_to_do) \
+  static int LOG_OCCURRENCES = 0, LOG_OCCURRENCES_MOD_N = 0; \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES, "Logging every N is approximate"); \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES_MOD_N, "Logging every N is approximate"); \
+  ++LOG_OCCURRENCES; \
+  if (++LOG_OCCURRENCES_MOD_N > n) LOG_OCCURRENCES_MOD_N -= n; \
+  if (LOG_OCCURRENCES_MOD_N == 1) \
+    google::LogMessage( \
+        __FILE__, __LINE__, google::GLOG_ ## severity, LOG_OCCURRENCES, \
+        &what_to_do).stream()
+
+#define KUDU_SOME_KIND_OF_LOG_IF_EVERY_N(severity, condition, n, what_to_do) \
+  static int LOG_OCCURRENCES = 0, LOG_OCCURRENCES_MOD_N = 0; \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES, "Logging every N is approximate"); \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES_MOD_N, "Logging every N is approximate"); \
+  ++LOG_OCCURRENCES; \
+  if (condition && \
+      ((LOG_OCCURRENCES_MOD_N=(LOG_OCCURRENCES_MOD_N + 1) % n) == (1 % n))) \
+    google::LogMessage( \
+        __FILE__, __LINE__, google::GLOG_ ## severity, LOG_OCCURRENCES, \
+                 &what_to_do).stream()
+
+#define KUDU_SOME_KIND_OF_PLOG_EVERY_N(severity, n, what_to_do) \
+  static int LOG_OCCURRENCES = 0, LOG_OCCURRENCES_MOD_N = 0; \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES, "Logging every N is approximate"); \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES_MOD_N, "Logging every N is approximate"); \
+  ++LOG_OCCURRENCES; \
+  if (++LOG_OCCURRENCES_MOD_N > n) LOG_OCCURRENCES_MOD_N -= n; \
+  if (LOG_OCCURRENCES_MOD_N == 1) \
+    google::ErrnoLogMessage( \
+        __FILE__, __LINE__, google::GLOG_ ## severity, LOG_OCCURRENCES, \
+        &what_to_do).stream()
+
+#define KUDU_SOME_KIND_OF_LOG_FIRST_N(severity, n, what_to_do) \
+  static uint64_t LOG_OCCURRENCES = 0; \
+  ANNOTATE_BENIGN_RACE(&LOG_OCCURRENCES, "Logging the first N is approximate"); \
+  if (LOG_OCCURRENCES++ < n) \
+    google::LogMessage( \
+      __FILE__, __LINE__, google::GLOG_ ## severity, LOG_OCCURRENCES, \
+      &what_to_do).stream()
+
+// The direct user-facing macros.
+#define KLOG_EVERY_N(severity, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(google::GLOG_ ## severity < \
+                             google::NUM_SEVERITIES, \
+                             INVALID_REQUESTED_LOG_SEVERITY); \
+  KUDU_SOME_KIND_OF_LOG_EVERY_N(severity, (n), google::LogMessage::SendToLog)
+
+#define KSYSLOG_EVERY_N(severity, n) \
+  KUDU_SOME_KIND_OF_LOG_EVERY_N(severity, (n), google::LogMessage::SendToSyslogAndLog)
+
+#define KPLOG_EVERY_N(severity, n) \
+  KUDU_SOME_KIND_OF_PLOG_EVERY_N(severity, (n), google::LogMessage::SendToLog)
+
+#define KLOG_FIRST_N(severity, n) \
+  KUDU_SOME_KIND_OF_LOG_FIRST_N(severity, (n), google::LogMessage::SendToLog)
+
+#define KLOG_IF_EVERY_N(severity, condition, n) \
+  KUDU_SOME_KIND_OF_LOG_IF_EVERY_N(severity, (condition), (n), google::LogMessage::SendToLog)
+
+// We also disable the un-annotated glog macros for anyone who includes this header.
+#undef LOG_EVERY_N
+#define LOG_EVERY_N(severity, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(false, "LOG_EVERY_N is deprecated. Please use KLOG_EVERY_N.")
+
+#undef SYSLOG_EVERY_N
+#define SYSLOG_EVERY_N(severity, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(false, "SYSLOG_EVERY_N is deprecated. Please use KSYSLOG_EVERY_N.")
+
+#undef PLOG_EVERY_N
+#define PLOG_EVERY_N(severity, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(false, "PLOG_EVERY_N is deprecated. Please use KPLOG_EVERY_N.")
+
+#undef LOG_FIRST_N
+#define LOG_FIRST_N(severity, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(false, "LOG_FIRST_N is deprecated. Please use KLOG_FIRST_N.")
+
+#undef LOG_IF_EVERY_N
+#define LOG_IF_EVERY_N(severity, condition, n) \
+  GOOGLE_GLOG_COMPILE_ASSERT(false, "LOG_IF_EVERY_N is deprecated. Please use KLOG_IF_EVERY_N.")
+
+namespace kudu {
+
+class Env;
+
+// glog doesn't allow multiple invocations of InitGoogleLogging. This method conditionally
+// calls InitGoogleLogging only if it hasn't been called before.
+//
+// It also takes care of installing the google failure signal handler and
+// setting the signal handler for SIGPIPE to SIG_IGN.
+void InitGoogleLoggingSafe(const char* arg);
+
+// Like InitGoogleLoggingSafe() but stripped down: no signal handlers are
+// installed, regular logging is disabled, and log events of any severity
+// will be written to stderr.
+//
+// These properties make it attractive for us in libraries.
+void InitGoogleLoggingSafeBasic(const char* arg);
+
+// Demotes stderr logging to ERROR or higher and registers 'cb' as the
+// recipient for all log events.
+//
+// Subsequent calls to RegisterLoggingCallback no-op (until the callback
+// is unregistered with UnregisterLoggingCallback()).
+void RegisterLoggingCallback(const LoggingCallback& cb);
+
+// Unregisters a callback previously registered with
+// RegisterLoggingCallback() and promotes stderr logging back to all
+// severities.
+//
+// If no callback is registered, this is a no-op.
+void UnregisterLoggingCallback();
+
+// Returns the full pathname of the symlink to the most recent log
+// file corresponding to this severity
+void GetFullLogFilename(google::LogSeverity severity, std::string* filename);
+
+// Format a timestamp in the same format as used by GLog.
+std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch);
+
+// Shuts down the google logging library. Call before exit to ensure that log files are
+// flushed.
+void ShutdownLoggingSafe();
+
+// Deletes excess rotated log files.
+//
+// Keeps at most 'FLAG_max_log_files' of the most recent log files at every
+// severity level, using the file's modified time to determine recency.
+Status DeleteExcessLogFiles(Env* env);
+
+namespace logging {
+
+// A LogThrottler instance tracks the throttling state for a particular
+// log message.
+//
+// This is used internally by KLOG_EVERY_N_SECS, but can also be used
+// explicitly in conjunction with KLOG_EVERY_N_SECS_THROTTLER. See the
+// macro descriptions above for details.
+class LogThrottler {
+ public:
+  LogThrottler() : num_suppressed_(0), last_ts_(0), last_tag_(nullptr) {
+    ANNOTATE_BENIGN_RACE_SIZED(this, sizeof(*this), "OK to be sloppy with log throttling");
+  }
+
+  bool ShouldLog(int n_secs, const char* tag, int* num_suppressed) {
+    MicrosecondsInt64 ts = GetMonoTimeMicros();
+
+    // When we switch tags, we should not show the "suppressed" messages, because
+    // in fact it's a different message that we skipped. So, reset it to zero,
+    // and always log the new message.
+    if (tag != last_tag_) {
+      *num_suppressed = num_suppressed_ = 0;
+      last_tag_ = tag;
+      last_ts_ = ts;
+      return true;
+    }
+
+    if (ts - last_ts_ < n_secs * 1000000) {
+      *num_suppressed = base::subtle::NoBarrier_AtomicIncrement(&num_suppressed_, 1);
+      return false;
+    }
+    last_ts_ = ts;
+    *num_suppressed = base::subtle::NoBarrier_AtomicExchange(&num_suppressed_, 0);
+    return true;
+  }
+ private:
+  Atomic32 num_suppressed_;
+  MicrosecondsInt64 last_ts_;
+  const char* last_tag_;
+};
+} // namespace logging
+
+std::ostream& operator<<(std::ostream &os, const PRIVATE_ThrottleMsg&);
+
+// Convenience macros to prefix log messages with some prefix, these are the unlocked
+// versions and should not obtain a lock (if one is required to obtain the prefix).
+// There must be a LogPrefixUnlocked()/LogPrefixLocked() method available in the current
+// scope in order to use these macros.
+#define LOG_WITH_PREFIX_UNLOCKED(severity) LOG(severity) << LogPrefixUnlocked()
+#define VLOG_WITH_PREFIX_UNLOCKED(verboselevel) LOG_IF(INFO, VLOG_IS_ON(verboselevel)) \
+  << LogPrefixUnlocked()
+
+// Same as the above, but obtain the lock.
+#define LOG_WITH_PREFIX(severity) LOG(severity) << LogPrefix()
+#define VLOG_WITH_PREFIX(verboselevel) LOG_IF(INFO, VLOG_IS_ON(verboselevel)) \
+  << LogPrefix()
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_LOGGING_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/logging_callback.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging_callback.h b/be/src/kudu/util/logging_callback.h
new file mode 100644
index 0000000..83fb973
--- /dev/null
+++ b/be/src/kudu/util/logging_callback.h
@@ -0,0 +1,46 @@
+// 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.
+#ifndef KUDU_UTIL_LOGGING_CALLBACK_H
+#define KUDU_UTIL_LOGGING_CALLBACK_H
+
+#include <ctime>
+#include <string>
+
+#include "kudu/gutil/callback_forward.h"
+
+namespace kudu {
+
+enum LogSeverity {
+  SEVERITY_INFO,
+  SEVERITY_WARNING,
+  SEVERITY_ERROR,
+  SEVERITY_FATAL
+};
+
+// Callback for simple logging.
+//
+// 'message' is NOT terminated with an endline.
+typedef Callback<void(LogSeverity severity,
+                      const char* filename,
+                      int line_number,
+                      const struct ::tm* time,
+                      const char* message,
+                      size_t message_len)> LoggingCallback;
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/logging_test_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/logging_test_util.h b/be/src/kudu/util/logging_test_util.h
new file mode 100644
index 0000000..8102375
--- /dev/null
+++ b/be/src/kudu/util/logging_test_util.h
@@ -0,0 +1,60 @@
+// 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.
+
+#ifndef KUDU_LOGGING_TEST_UTIL_H
+#define KUDU_LOGGING_TEST_UTIL_H
+
+#include <glog/logging.h>
+#include <string>
+#include <vector>
+
+namespace kudu {
+
+// GLog sink that keeps an internal buffer of messages that have been logged.
+class StringVectorSink : public google::LogSink {
+ public:
+  void send(google::LogSeverity severity, const char* full_filename,
+            const char* base_filename, int line,
+            const struct ::tm* tm_time,
+            const char* message, size_t message_len) override {
+    logged_msgs_.push_back(ToString(severity, base_filename, line,
+                                    tm_time, message, message_len));
+  }
+
+  std::vector<std::string>& logged_msgs() {
+    return logged_msgs_;
+  }
+
+ private:
+  std::vector<std::string> logged_msgs_;
+};
+
+// RAII wrapper around registering a LogSink with GLog.
+struct ScopedRegisterSink {
+  explicit ScopedRegisterSink(google::LogSink* s) : s_(s) {
+    google::AddLogSink(s_);
+  }
+  ~ScopedRegisterSink() {
+    google::RemoveLogSink(s_);
+  }
+
+  google::LogSink* s_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/maintenance_manager-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/maintenance_manager-test.cc b/be/src/kudu/util/maintenance_manager-test.cc
new file mode 100644
index 0000000..6777e06
--- /dev/null
+++ b/be/src/kudu/util/maintenance_manager-test.cc
@@ -0,0 +1,369 @@
+// 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 <atomic>
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/maintenance_manager.h"
+#include "kudu/util/maintenance_manager.pb.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+using std::shared_ptr;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+METRIC_DEFINE_entity(test);
+METRIC_DEFINE_gauge_uint32(test, maintenance_ops_running,
+                           "Number of Maintenance Operations Running",
+                           kudu::MetricUnit::kMaintenanceOperations,
+                           "The number of background maintenance operations currently running.");
+METRIC_DEFINE_histogram(test, maintenance_op_duration,
+                        "Maintenance Operation Duration",
+                        kudu::MetricUnit::kSeconds, "", 60000000LU, 2);
+
+DECLARE_int64(log_target_replay_size_mb);
+
+namespace kudu {
+
+static const int kHistorySize = 4;
+static const char kFakeUuid[] = "12345";
+
+class MaintenanceManagerTest : public KuduTest {
+ public:
+  void SetUp() override {
+    MaintenanceManager::Options options;
+    options.num_threads = 2;
+    options.polling_interval_ms = 1;
+    options.history_size = kHistorySize;
+    manager_.reset(new MaintenanceManager(options, kFakeUuid));
+    manager_->set_memory_pressure_func_for_tests(
+        [&](double* consumption) {
+          return indicate_memory_pressure_.load();
+        });
+    ASSERT_OK(manager_->Start());
+  }
+
+  void TearDown() override {
+    manager_->Shutdown();
+  }
+
+ protected:
+  shared_ptr<MaintenanceManager> manager_;
+  std::atomic<bool> indicate_memory_pressure_ { false };
+};
+
+// Just create the MaintenanceManager and then shut it down, to make sure
+// there are no race conditions there.
+TEST_F(MaintenanceManagerTest, TestCreateAndShutdown) {
+}
+
+class TestMaintenanceOp : public MaintenanceOp {
+ public:
+  TestMaintenanceOp(const std::string& name,
+                    IOUsage io_usage)
+    : MaintenanceOp(name, io_usage),
+      ram_anchored_(500),
+      logs_retained_bytes_(0),
+      perf_improvement_(0),
+      metric_entity_(METRIC_ENTITY_test.Instantiate(&metric_registry_, "test")),
+      maintenance_op_duration_(METRIC_maintenance_op_duration.Instantiate(metric_entity_)),
+      maintenance_ops_running_(METRIC_maintenance_ops_running.Instantiate(metric_entity_, 0)),
+      remaining_runs_(1),
+      prepared_runs_(0),
+      sleep_time_(MonoDelta::FromSeconds(0)) {
+  }
+
+  virtual ~TestMaintenanceOp() {}
+
+  virtual bool Prepare() OVERRIDE {
+    std::lock_guard<Mutex> guard(lock_);
+    if (remaining_runs_ == 0) {
+      return false;
+    }
+    remaining_runs_--;
+    prepared_runs_++;
+    DLOG(INFO) << "Prepared op " << name();
+    return true;
+  }
+
+  virtual void Perform() OVERRIDE {
+    {
+      std::lock_guard<Mutex> guard(lock_);
+      DLOG(INFO) << "Performing op " << name();
+
+      // Ensure that we don't call Perform() more times than we returned
+      // success from Prepare().
+      CHECK_GE(prepared_runs_, 1);
+      prepared_runs_--;
+    }
+
+    SleepFor(sleep_time_);
+  }
+
+  virtual void UpdateStats(MaintenanceOpStats* stats) OVERRIDE {
+    std::lock_guard<Mutex> guard(lock_);
+    stats->set_runnable(remaining_runs_ > 0);
+    stats->set_ram_anchored(ram_anchored_);
+    stats->set_logs_retained_bytes(logs_retained_bytes_);
+    stats->set_perf_improvement(perf_improvement_);
+  }
+
+  void set_remaining_runs(int runs) {
+    std::lock_guard<Mutex> guard(lock_);
+    remaining_runs_ = runs;
+  }
+
+  void set_sleep_time(MonoDelta time) {
+    std::lock_guard<Mutex> guard(lock_);
+    sleep_time_ = time;
+  }
+
+  void set_ram_anchored(uint64_t ram_anchored) {
+    std::lock_guard<Mutex> guard(lock_);
+    ram_anchored_ = ram_anchored;
+  }
+
+  void set_logs_retained_bytes(uint64_t logs_retained_bytes) {
+    std::lock_guard<Mutex> guard(lock_);
+    logs_retained_bytes_ = logs_retained_bytes;
+  }
+
+  void set_perf_improvement(uint64_t perf_improvement) {
+    std::lock_guard<Mutex> guard(lock_);
+    perf_improvement_ = perf_improvement;
+  }
+
+  virtual scoped_refptr<Histogram> DurationHistogram() const OVERRIDE {
+    return maintenance_op_duration_;
+  }
+
+  virtual scoped_refptr<AtomicGauge<uint32_t> > RunningGauge() const OVERRIDE {
+    return maintenance_ops_running_;
+  }
+
+ private:
+  Mutex lock_;
+
+  uint64_t ram_anchored_;
+  uint64_t logs_retained_bytes_;
+  uint64_t perf_improvement_;
+  MetricRegistry metric_registry_;
+  scoped_refptr<MetricEntity> metric_entity_;
+  scoped_refptr<Histogram> maintenance_op_duration_;
+  scoped_refptr<AtomicGauge<uint32_t> > maintenance_ops_running_;
+
+  // The number of remaining times this operation will run before disabling
+  // itself.
+  int remaining_runs_;
+  // The number of Prepared() operations which have not yet been Perform()ed.
+  int prepared_runs_;
+
+  // The amount of time each op invocation will sleep.
+  MonoDelta sleep_time_;
+};
+
+// Create an op and wait for it to start running.  Unregister it while it is
+// running and verify that UnregisterOp waits for it to finish before
+// proceeding.
+TEST_F(MaintenanceManagerTest, TestRegisterUnregister) {
+  TestMaintenanceOp op1("1", MaintenanceOp::HIGH_IO_USAGE);
+  op1.set_perf_improvement(10);
+  // Register initially with no remaining runs. We'll later enable it once it's
+  // already registered.
+  op1.set_remaining_runs(0);
+  manager_->RegisterOp(&op1);
+  scoped_refptr<kudu::Thread> thread;
+  CHECK_OK(Thread::Create(
+      "TestThread", "TestRegisterUnregister",
+      boost::bind(&TestMaintenanceOp::set_remaining_runs, &op1, 1), &thread));
+  ASSERT_EVENTUALLY([&]() {
+      ASSERT_EQ(op1.DurationHistogram()->TotalCount(), 1);
+    });
+  manager_->UnregisterOp(&op1);
+  ThreadJoiner(thread.get()).Join();
+}
+
+// Regression test for KUDU-1495: when an operation is being unregistered,
+// new instances of that operation should not be scheduled.
+TEST_F(MaintenanceManagerTest, TestNewOpsDontGetScheduledDuringUnregister) {
+  TestMaintenanceOp op1("1", MaintenanceOp::HIGH_IO_USAGE);
+  op1.set_perf_improvement(10);
+
+  // Set the op to run up to 10 times, and each time should sleep for a second.
+  op1.set_remaining_runs(10);
+  op1.set_sleep_time(MonoDelta::FromSeconds(1));
+  manager_->RegisterOp(&op1);
+
+  // Wait until two instances of the ops start running, since we have two MM threads.
+  ASSERT_EVENTUALLY([&]() {
+      ASSERT_EQ(op1.RunningGauge()->value(), 2);
+    });
+
+  // Trigger Unregister while they are running. This should wait for the currently-
+  // running operations to complete, but no new operations should be scheduled.
+  manager_->UnregisterOp(&op1);
+
+  // Hence, we should have run only the original two that we saw above.
+  ASSERT_LE(op1.DurationHistogram()->TotalCount(), 2);
+}
+
+// Test that we'll run an operation that doesn't improve performance when memory
+// pressure gets high.
+TEST_F(MaintenanceManagerTest, TestMemoryPressure) {
+  TestMaintenanceOp op("op", MaintenanceOp::HIGH_IO_USAGE);
+  op.set_ram_anchored(100);
+  manager_->RegisterOp(&op);
+
+  // At first, we don't want to run this, since there is no perf_improvement.
+  SleepFor(MonoDelta::FromMilliseconds(20));
+  ASSERT_EQ(0, op.DurationHistogram()->TotalCount());
+
+  // Fake that the server is under memory pressure.
+  indicate_memory_pressure_ = true;
+
+  ASSERT_EVENTUALLY([&]() {
+      ASSERT_EQ(op.DurationHistogram()->TotalCount(), 1);
+    });
+  manager_->UnregisterOp(&op);
+}
+
+// Test that ops are prioritized correctly when we add log retention.
+TEST_F(MaintenanceManagerTest, TestLogRetentionPrioritization) {
+  const int64_t kMB = 1024 * 1024;
+
+  manager_->Shutdown();
+
+  TestMaintenanceOp op1("op1", MaintenanceOp::LOW_IO_USAGE);
+  op1.set_ram_anchored(0);
+  op1.set_logs_retained_bytes(100 * kMB);
+
+  TestMaintenanceOp op2("op2", MaintenanceOp::HIGH_IO_USAGE);
+  op2.set_ram_anchored(100);
+  op2.set_logs_retained_bytes(100 * kMB);
+
+  TestMaintenanceOp op3("op3", MaintenanceOp::HIGH_IO_USAGE);
+  op3.set_ram_anchored(200);
+  op3.set_logs_retained_bytes(100 * kMB);
+
+  manager_->RegisterOp(&op1);
+  manager_->RegisterOp(&op2);
+  manager_->RegisterOp(&op3);
+
+  // We want to do the low IO op first since it clears up some log retention.
+  auto op_and_why = manager_->FindBestOp();
+  ASSERT_EQ(&op1, op_and_why.first);
+  EXPECT_EQ(op_and_why.second, "free 104857600 bytes of WAL");
+
+  manager_->UnregisterOp(&op1);
+
+  // Low IO is taken care of, now we find the op that clears the most log retention and ram.
+  // However, with the default settings, we won't bother running any of these operations
+  // which only retain 100MB of logs.
+  op_and_why = manager_->FindBestOp();
+  ASSERT_EQ(nullptr, op_and_why.first);
+  EXPECT_EQ(op_and_why.second, "no ops with positive improvement");
+
+  // If we change the target WAL size, we will select these ops.
+  FLAGS_log_target_replay_size_mb = 50;
+  op_and_why = manager_->FindBestOp();
+  ASSERT_EQ(&op3, op_and_why.first);
+  EXPECT_EQ(op_and_why.second, "104857600 bytes log retention");
+
+  manager_->UnregisterOp(&op3);
+
+  op_and_why = manager_->FindBestOp();
+  ASSERT_EQ(&op2, op_and_why.first);
+  EXPECT_EQ(op_and_why.second, "104857600 bytes log retention");
+
+  manager_->UnregisterOp(&op2);
+}
+
+// Test retrieving a list of an op's running instances
+TEST_F(MaintenanceManagerTest, TestRunningInstances) {
+  TestMaintenanceOp op("op", MaintenanceOp::HIGH_IO_USAGE);
+  op.set_perf_improvement(10);
+  op.set_remaining_runs(2);
+  op.set_sleep_time(MonoDelta::FromSeconds(1));
+  manager_->RegisterOp(&op);
+
+  // Check that running instances are added to the maintenance manager's collection,
+  // and fields are getting filled.
+  ASSERT_EVENTUALLY([&]() {
+      MaintenanceManagerStatusPB status_pb;
+      manager_->GetMaintenanceManagerStatusDump(&status_pb);
+      ASSERT_EQ(status_pb.running_operations_size(), 2);
+      const MaintenanceManagerStatusPB_OpInstancePB& instance1 = status_pb.running_operations(0);
+      const MaintenanceManagerStatusPB_OpInstancePB& instance2 = status_pb.running_operations(1);
+      ASSERT_EQ(instance1.name(), op.name());
+      ASSERT_NE(instance1.thread_id(), instance2.thread_id());
+    });
+
+  // Wait for instances to complete.
+  manager_->UnregisterOp(&op);
+
+  // Check that running instances are removed from collection after completion.
+  MaintenanceManagerStatusPB status_pb;
+  manager_->GetMaintenanceManagerStatusDump(&status_pb);
+  ASSERT_EQ(status_pb.running_operations_size(), 0);
+}
+// Test adding operations and make sure that the history of recently completed operations
+// is correct in that it wraps around and doesn't grow.
+TEST_F(MaintenanceManagerTest, TestCompletedOpsHistory) {
+  for (int i = 0; i < 5; i++) {
+    string name = Substitute("op$0", i);
+    TestMaintenanceOp op(name, MaintenanceOp::HIGH_IO_USAGE);
+    op.set_perf_improvement(1);
+    op.set_ram_anchored(100);
+    manager_->RegisterOp(&op);
+
+    ASSERT_EVENTUALLY([&]() {
+        ASSERT_EQ(op.DurationHistogram()->TotalCount(), 1);
+      });
+    manager_->UnregisterOp(&op);
+
+    MaintenanceManagerStatusPB status_pb;
+    manager_->GetMaintenanceManagerStatusDump(&status_pb);
+    // The size should be at most the history_size.
+    ASSERT_GE(kHistorySize, status_pb.completed_operations_size());
+    // The most recently completed op should always be first, even if we wrap
+    // around.
+    ASSERT_EQ(name, status_pb.completed_operations(0).name());
+  }
+}
+
+} // namespace kudu


[08/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/random_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/random_util.h b/be/src/kudu/util/random_util.h
new file mode 100644
index 0000000..bda8c42
--- /dev/null
+++ b/be/src/kudu/util/random_util.h
@@ -0,0 +1,44 @@
+// 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.
+
+#ifndef KUDU_UTIL_RANDOM_UTIL_H
+#define KUDU_UTIL_RANDOM_UTIL_H
+
+#include <cstdlib>
+#include <stdint.h>
+
+#include <string>
+
+namespace kudu {
+
+class Random;
+
+// Writes exactly n random bytes to dest using the parameter Random generator.
+// Note RandomString() does not null-terminate its strings, though '\0' could
+// be written to dest with the same probability as any other byte.
+void RandomString(void* dest, size_t n, Random* rng);
+
+// Same as the above, but returns the string.
+std::string RandomString(size_t n, Random* rng);
+
+// Generate a 32-bit random seed from several sources, including timestamp,
+// pid & tid.
+uint32_t GetRandomSeed32();
+
+} // namespace kudu
+
+#endif // KUDU_UTIL_RANDOM_UTIL_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rle-encoding.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rle-encoding.h b/be/src/kudu/util/rle-encoding.h
new file mode 100644
index 0000000..5b16fd9
--- /dev/null
+++ b/be/src/kudu/util/rle-encoding.h
@@ -0,0 +1,523 @@
+// 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.
+#ifndef IMPALA_RLE_ENCODING_H
+#define IMPALA_RLE_ENCODING_H
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/bit-stream-utils.inline.h"
+#include "kudu/util/bit-util.h"
+
+namespace kudu {
+
+// Utility classes to do run length encoding (RLE) for fixed bit width values.  If runs
+// are sufficiently long, RLE is used, otherwise, the values are just bit-packed
+// (literal encoding).
+// For both types of runs, there is a byte-aligned indicator which encodes the length
+// of the run and the type of the run.
+// This encoding has the benefit that when there aren't any long enough runs, values
+// are always decoded at fixed (can be precomputed) bit offsets OR both the value and
+// the run length are byte aligned. This allows for very efficient decoding
+// implementations.
+// The encoding is:
+//    encoded-block := run*
+//    run := literal-run | repeated-run
+//    literal-run := literal-indicator < literal bytes >
+//    repeated-run := repeated-indicator < repeated value. padded to byte boundary >
+//    literal-indicator := varint_encode( number_of_groups << 1 | 1)
+//    repeated-indicator := varint_encode( number_of_repetitions << 1 )
+//
+// Each run is preceded by a varint. The varint's least significant bit is
+// used to indicate whether the run is a literal run or a repeated run. The rest
+// of the varint is used to determine the length of the run (eg how many times the
+// value repeats).
+//
+// In the case of literal runs, the run length is always a multiple of 8 (i.e. encode
+// in groups of 8), so that no matter the bit-width of the value, the sequence will end
+// on a byte boundary without padding.
+// Given that we know it is a multiple of 8, we store the number of 8-groups rather than
+// the actual number of encoded ints. (This means that the total number of encoded values
+// can not be determined from the encoded data, since the number of values in the last
+// group may not be a multiple of 8).
+// There is a break-even point when it is more storage efficient to do run length
+// encoding.  For 1 bit-width values, that point is 8 values.  They require 2 bytes
+// for both the repeated encoding or the literal encoding.  This value can always
+// be computed based on the bit-width.
+// TODO: think about how to use this for strings.  The bit packing isn't quite the same.
+//
+// Examples with bit-width 1 (eg encoding booleans):
+// ----------------------------------------
+// 100 1s followed by 100 0s:
+// <varint(100 << 1)> <1, padded to 1 byte> <varint(100 << 1)> <0, padded to 1 byte>
+//  - (total 4 bytes)
+//
+// alternating 1s and 0s (200 total):
+// 200 ints = 25 groups of 8
+// <varint((25 << 1) | 1)> <25 bytes of values, bitpacked>
+// (total 26 bytes, 1 byte overhead)
+//
+
+// Decoder class for RLE encoded data.
+//
+// NOTE: the encoded format does not have any length prefix or any other way of
+// indicating that the encoded sequence ends at a certain point, so the Decoder
+// methods may return some extra bits at the end before the read methods start
+// to return 0/false.
+template<typename T>
+class RleDecoder {
+ public:
+  // Create a decoder object. buffer/buffer_len is the decoded data.
+  // bit_width is the width of each value (before encoding).
+  RleDecoder(const uint8_t* buffer, int buffer_len, int bit_width)
+    : bit_reader_(buffer, buffer_len),
+      bit_width_(bit_width),
+      current_value_(0),
+      repeat_count_(0),
+      literal_count_(0),
+      rewind_state_(CANT_REWIND) {
+    DCHECK_GE(bit_width_, 1);
+    DCHECK_LE(bit_width_, 64);
+  }
+
+  RleDecoder() {}
+
+  // Skip n values, and returns the number of non-zero entries skipped.
+  size_t Skip(size_t to_skip);
+
+  // Gets the next value.  Returns false if there are no more.
+  bool Get(T* val);
+
+  // Seek to the previous value.
+  void RewindOne();
+
+  // Gets the next run of the same 'val'. Returns 0 if there is no
+  // more data to be decoded. Will return a run of at most 'max_run'
+  // values. If there are more values than this, the next call to
+  // GetNextRun will return more from the same run.
+  size_t GetNextRun(T* val, size_t max_run);
+
+ private:
+  bool ReadHeader();
+
+  enum RewindState {
+    REWIND_LITERAL,
+    REWIND_RUN,
+    CANT_REWIND
+  };
+
+  BitReader bit_reader_;
+  int bit_width_;
+  uint64_t current_value_;
+  uint32_t repeat_count_;
+  uint32_t literal_count_;
+  RewindState rewind_state_;
+};
+
+// Class to incrementally build the rle data.
+// The encoding has two modes: encoding repeated runs and literal runs.
+// If the run is sufficiently short, it is more efficient to encode as a literal run.
+// This class does so by buffering 8 values at a time.  If they are not all the same
+// they are added to the literal run.  If they are the same, they are added to the
+// repeated run.  When we switch modes, the previous run is flushed out.
+template<typename T>
+class RleEncoder {
+ public:
+  // buffer: buffer to write bits to.
+  // bit_width: max number of bits for value.
+  // TODO: consider adding a min_repeated_run_length so the caller can control
+  // when values should be encoded as repeated runs.  Currently this is derived
+  // based on the bit_width, which can determine a storage optimal choice.
+  explicit RleEncoder(faststring *buffer, int bit_width)
+    : bit_width_(bit_width),
+      bit_writer_(buffer) {
+    DCHECK_GE(bit_width_, 1);
+    DCHECK_LE(bit_width_, 64);
+    Clear();
+  }
+
+  // Reserve 'num_bytes' bytes for a plain encoded header, set each
+  // byte with 'val': this is used for the RLE-encoded data blocks in
+  // order to be able to able to store the initial ordinal position
+  // and number of elements. This is a part of RleEncoder in order to
+  // maintain the correct offset in 'buffer'.
+  void Reserve(int num_bytes, uint8_t val);
+
+  // Encode value. This value must be representable with bit_width_ bits.
+  void Put(T value, size_t run_length = 1);
+
+  // Flushes any pending values to the underlying buffer.
+  // Returns the total number of bytes written
+  int Flush();
+
+  // Resets all the state in the encoder.
+  void Clear();
+
+  int32_t len() const { return bit_writer_.bytes_written(); }
+
+ private:
+  // Flushes any buffered values.  If this is part of a repeated run, this is largely
+  // a no-op.
+  // If it is part of a literal run, this will call FlushLiteralRun, which writes
+  // out the buffered literal values.
+  // If 'done' is true, the current run would be written even if it would normally
+  // have been buffered more.  This should only be called at the end, when the
+  // encoder has received all values even if it would normally continue to be
+  // buffered.
+  void FlushBufferedValues(bool done);
+
+  // Flushes literal values to the underlying buffer.  If update_indicator_byte,
+  // then the current literal run is complete and the indicator byte is updated.
+  void FlushLiteralRun(bool update_indicator_byte);
+
+  // Flushes a repeated run to the underlying buffer.
+  void FlushRepeatedRun();
+
+  // Number of bits needed to encode the value.
+  const int bit_width_;
+
+  // Underlying buffer.
+  BitWriter bit_writer_;
+
+  // We need to buffer at most 8 values for literals.  This happens when the
+  // bit_width is 1 (so 8 values fit in one byte).
+  // TODO: generalize this to other bit widths
+  uint64_t buffered_values_[8];
+
+  // Number of values in buffered_values_
+  int num_buffered_values_;
+
+  // The current (also last) value that was written and the count of how
+  // many times in a row that value has been seen.  This is maintained even
+  // if we are in a literal run.  If the repeat_count_ get high enough, we switch
+  // to encoding repeated runs.
+  uint64_t current_value_;
+  int repeat_count_;
+
+  // Number of literals in the current run.  This does not include the literals
+  // that might be in buffered_values_.  Only after we've got a group big enough
+  // can we decide if they should part of the literal_count_ or repeat_count_
+  int literal_count_;
+
+  // Index of a byte in the underlying buffer that stores the indicator byte.
+  // This is reserved as soon as we need a literal run but the value is written
+  // when the literal run is complete. We maintain an index rather than a pointer
+  // into the underlying buffer because the pointer value may become invalid if
+  // the underlying buffer is resized.
+  int literal_indicator_byte_idx_;
+};
+
+template<typename T>
+inline bool RleDecoder<T>::ReadHeader() {
+  DCHECK(bit_reader_.is_initialized());
+  if (PREDICT_FALSE(literal_count_ == 0 && repeat_count_ == 0)) {
+    // Read the next run's indicator int, it could be a literal or repeated run
+    // The int is encoded as a vlq-encoded value.
+    int32_t indicator_value = 0;
+    bool result = bit_reader_.GetVlqInt(&indicator_value);
+    if (PREDICT_FALSE(!result)) {
+      return false;
+    }
+
+    // lsb indicates if it is a literal run or repeated run
+    bool is_literal = indicator_value & 1;
+    if (is_literal) {
+      literal_count_ = (indicator_value >> 1) * 8;
+      DCHECK_GT(literal_count_, 0);
+    } else {
+      repeat_count_ = indicator_value >> 1;
+      DCHECK_GT(repeat_count_, 0);
+      bool result = bit_reader_.GetAligned<T>(
+          BitUtil::Ceil(bit_width_, 8), reinterpret_cast<T*>(&current_value_));
+      DCHECK(result);
+    }
+  }
+  return true;
+}
+
+template<typename T>
+inline bool RleDecoder<T>::Get(T* val) {
+  DCHECK(bit_reader_.is_initialized());
+  if (PREDICT_FALSE(!ReadHeader())) {
+    return false;
+  }
+
+  if (PREDICT_TRUE(repeat_count_ > 0)) {
+    *val = current_value_;
+    --repeat_count_;
+    rewind_state_ = REWIND_RUN;
+  } else {
+    DCHECK(literal_count_ > 0);
+    bool result = bit_reader_.GetValue(bit_width_, val);
+    DCHECK(result);
+    --literal_count_;
+    rewind_state_ = REWIND_LITERAL;
+  }
+
+  return true;
+}
+
+template<typename T>
+inline void RleDecoder<T>::RewindOne() {
+  DCHECK(bit_reader_.is_initialized());
+
+  switch (rewind_state_) {
+    case CANT_REWIND:
+      LOG(FATAL) << "Can't rewind more than once after each read!";
+      break;
+    case REWIND_RUN:
+      ++repeat_count_;
+      break;
+    case REWIND_LITERAL:
+      {
+        bit_reader_.Rewind(bit_width_);
+        ++literal_count_;
+        break;
+      }
+  }
+
+  rewind_state_ = CANT_REWIND;
+}
+
+template<typename T>
+inline size_t RleDecoder<T>::GetNextRun(T* val, size_t max_run) {
+  DCHECK(bit_reader_.is_initialized());
+  DCHECK_GT(max_run, 0);
+  size_t ret = 0;
+  size_t rem = max_run;
+  while (ReadHeader()) {
+    if (PREDICT_TRUE(repeat_count_ > 0)) {
+      if (PREDICT_FALSE(ret > 0 && *val != current_value_)) {
+        return ret;
+      }
+      *val = current_value_;
+      if (repeat_count_ >= rem) {
+        // The next run is longer than the amount of remaining data
+        // that the caller wants to read. Only consume it partially.
+        repeat_count_ -= rem;
+        ret += rem;
+        return ret;
+      }
+      ret += repeat_count_;
+      rem -= repeat_count_;
+      repeat_count_ = 0;
+    } else {
+      DCHECK(literal_count_ > 0);
+      if (ret == 0) {
+        bool has_more = bit_reader_.GetValue(bit_width_, val);
+        DCHECK(has_more);
+        literal_count_--;
+        ret++;
+        rem--;
+      }
+
+      while (literal_count_ > 0) {
+        bool result = bit_reader_.GetValue(bit_width_, &current_value_);
+        DCHECK(result);
+        if (current_value_ != *val || rem == 0) {
+          bit_reader_.Rewind(bit_width_);
+          return ret;
+        }
+        ret++;
+        rem--;
+        literal_count_--;
+      }
+    }
+  }
+  return ret;
+ }
+
+template<typename T>
+inline size_t RleDecoder<T>::Skip(size_t to_skip) {
+  DCHECK(bit_reader_.is_initialized());
+
+  size_t set_count = 0;
+  while (to_skip > 0) {
+    bool result = ReadHeader();
+    DCHECK(result);
+
+    if (PREDICT_TRUE(repeat_count_ > 0)) {
+      size_t nskip = (repeat_count_ < to_skip) ? repeat_count_ : to_skip;
+      repeat_count_ -= nskip;
+      to_skip -= nskip;
+      if (current_value_ != 0) {
+        set_count += nskip;
+      }
+    } else {
+      DCHECK(literal_count_ > 0);
+      size_t nskip = (literal_count_ < to_skip) ? literal_count_ : to_skip;
+      literal_count_ -= nskip;
+      to_skip -= nskip;
+      for (; nskip > 0; nskip--) {
+        T value = 0;
+        bool result = bit_reader_.GetValue(bit_width_, &value);
+        DCHECK(result);
+        if (value != 0) {
+          set_count++;
+        }
+      }
+    }
+  }
+  return set_count;
+}
+
+// This function buffers input values 8 at a time.  After seeing all 8 values,
+// it decides whether they should be encoded as a literal or repeated run.
+template<typename T>
+inline void RleEncoder<T>::Put(T value, size_t run_length) {
+  DCHECK(bit_width_ == 64 || value < (1LL << bit_width_));
+
+  // TODO(perf): remove the loop and use the repeat_count_
+  for (; run_length > 0; run_length--) {
+    if (PREDICT_TRUE(current_value_ == value)) {
+      ++repeat_count_;
+      if (repeat_count_ > 8) {
+        // This is just a continuation of the current run, no need to buffer the
+        // values.
+        // Note that this is the fast path for long repeated runs.
+        continue;
+      }
+    } else {
+      if (repeat_count_ >= 8) {
+        // We had a run that was long enough but it has ended.  Flush the
+        // current repeated run.
+        DCHECK_EQ(literal_count_, 0);
+        FlushRepeatedRun();
+      }
+      repeat_count_ = 1;
+      current_value_ = value;
+    }
+
+    buffered_values_[num_buffered_values_] = value;
+    if (++num_buffered_values_ == 8) {
+      DCHECK_EQ(literal_count_ % 8, 0);
+      FlushBufferedValues(false);
+    }
+  }
+}
+
+template<typename T>
+inline void RleEncoder<T>::FlushLiteralRun(bool update_indicator_byte) {
+  if (literal_indicator_byte_idx_ < 0) {
+    // The literal indicator byte has not been reserved yet, get one now.
+    literal_indicator_byte_idx_ = bit_writer_.GetByteIndexAndAdvance(1);
+    DCHECK_GE(literal_indicator_byte_idx_, 0);
+  }
+
+  // Write all the buffered values as bit packed literals
+  for (int i = 0; i < num_buffered_values_; ++i) {
+    bit_writer_.PutValue(buffered_values_[i], bit_width_);
+  }
+  num_buffered_values_ = 0;
+
+  if (update_indicator_byte) {
+    // At this point we need to write the indicator byte for the literal run.
+    // We only reserve one byte, to allow for streaming writes of literal values.
+    // The logic makes sure we flush literal runs often enough to not overrun
+    // the 1 byte.
+    int num_groups = BitUtil::Ceil(literal_count_, 8);
+    int32_t indicator_value = (num_groups << 1) | 1;
+    DCHECK_EQ(indicator_value & 0xFFFFFF00, 0);
+    bit_writer_.buffer()->data()[literal_indicator_byte_idx_] = indicator_value;
+    literal_indicator_byte_idx_ = -1;
+    literal_count_ = 0;
+  }
+}
+
+template<typename T>
+inline void RleEncoder<T>::FlushRepeatedRun() {
+  DCHECK_GT(repeat_count_, 0);
+  // The lsb of 0 indicates this is a repeated run
+  int32_t indicator_value = repeat_count_ << 1 | 0;
+  bit_writer_.PutVlqInt(indicator_value);
+  bit_writer_.PutAligned(current_value_, BitUtil::Ceil(bit_width_, 8));
+  num_buffered_values_ = 0;
+  repeat_count_ = 0;
+}
+
+// Flush the values that have been buffered.  At this point we decide whether
+// we need to switch between the run types or continue the current one.
+template<typename T>
+inline void RleEncoder<T>::FlushBufferedValues(bool done) {
+  if (repeat_count_ >= 8) {
+    // Clear the buffered values.  They are part of the repeated run now and we
+    // don't want to flush them out as literals.
+    num_buffered_values_ = 0;
+    if (literal_count_ != 0) {
+      // There was a current literal run.  All the values in it have been flushed
+      // but we still need to update the indicator byte.
+      DCHECK_EQ(literal_count_ % 8, 0);
+      DCHECK_EQ(repeat_count_, 8);
+      FlushLiteralRun(true);
+    }
+    DCHECK_EQ(literal_count_, 0);
+    return;
+  }
+
+  literal_count_ += num_buffered_values_;
+  int num_groups = BitUtil::Ceil(literal_count_, 8);
+  if (num_groups + 1 >= (1 << 6)) {
+    // We need to start a new literal run because the indicator byte we've reserved
+    // cannot store more values.
+    DCHECK_GE(literal_indicator_byte_idx_, 0);
+    FlushLiteralRun(true);
+  } else {
+    FlushLiteralRun(done);
+  }
+  repeat_count_ = 0;
+}
+
+template<typename T>
+inline void RleEncoder<T>::Reserve(int num_bytes, uint8_t val) {
+  for (int i = 0; i < num_bytes; ++i) {
+    bit_writer_.PutValue(val, 8);
+  }
+}
+
+template<typename T>
+inline int RleEncoder<T>::Flush() {
+  if (literal_count_ > 0 || repeat_count_ > 0 || num_buffered_values_ > 0) {
+    bool all_repeat = literal_count_ == 0 &&
+        (repeat_count_ == num_buffered_values_ || num_buffered_values_ == 0);
+    // There is something pending, figure out if it's a repeated or literal run
+    if (repeat_count_ > 0 && all_repeat) {
+      FlushRepeatedRun();
+    } else  {
+      literal_count_ += num_buffered_values_;
+      FlushLiteralRun(true);
+      repeat_count_ = 0;
+    }
+  }
+  bit_writer_.Flush();
+  DCHECK_EQ(num_buffered_values_, 0);
+  DCHECK_EQ(literal_count_, 0);
+  DCHECK_EQ(repeat_count_, 0);
+  return bit_writer_.bytes_written();
+}
+
+template<typename T>
+inline void RleEncoder<T>::Clear() {
+  current_value_ = 0;
+  repeat_count_ = 0;
+  num_buffered_values_ = 0;
+  literal_count_ = 0;
+  literal_indicator_byte_idx_ = -1;
+  bit_writer_.Clear();
+}
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rle-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rle-test.cc b/be/src/kudu/util/rle-test.cc
new file mode 100644
index 0000000..20d0a27
--- /dev/null
+++ b/be/src/kudu/util/rle-test.cc
@@ -0,0 +1,546 @@
+// 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 <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <ostream>
+#include <string>
+#include <vector>
+
+// Must come before gtest.h.
+#include "kudu/gutil/mathlimits.h"
+
+#include <boost/utility/binary.hpp>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/bit-stream-utils.h"
+#include "kudu/util/bit-stream-utils.inline.h"
+#include "kudu/util/bit-util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/hexdump.h"
+#include "kudu/util/rle-encoding.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+const int kMaxWidth = 64;
+
+class TestRle : public KuduTest {};
+
+TEST(BitArray, TestBool) {
+  const int len_bytes = 2;
+  faststring buffer(len_bytes);
+
+  BitWriter writer(&buffer);
+
+  // Write alternating 0's and 1's
+  for (int i = 0; i < 8; ++i) {
+    writer.PutValue(i % 2, 1);
+  }
+  writer.Flush();
+  EXPECT_EQ(buffer[0], BOOST_BINARY(1 0 1 0 1 0 1 0));
+
+  // Write 00110011
+  for (int i = 0; i < 8; ++i) {
+    switch (i) {
+      case 0:
+      case 1:
+      case 4:
+      case 5:
+        writer.PutValue(0, 1);
+        break;
+      default:
+        writer.PutValue(1, 1);
+        break;
+    }
+  }
+  writer.Flush();
+
+  // Validate the exact bit value
+  EXPECT_EQ(buffer[0], BOOST_BINARY(1 0 1 0 1 0 1 0));
+  EXPECT_EQ(buffer[1], BOOST_BINARY(1 1 0 0 1 1 0 0));
+
+  // Use the reader and validate
+  BitReader reader(buffer.data(), buffer.size());
+  for (int i = 0; i < 8; ++i) {
+    bool val = false;
+    bool result = reader.GetValue(1, &val);
+    EXPECT_TRUE(result);
+    EXPECT_EQ(val, i % 2);
+  }
+
+  for (int i = 0; i < 8; ++i) {
+    bool val = false;
+    bool result = reader.GetValue(1, &val);
+    EXPECT_TRUE(result);
+    switch (i) {
+      case 0:
+      case 1:
+      case 4:
+      case 5:
+        EXPECT_EQ(val, false);
+        break;
+      default:
+        EXPECT_EQ(val, true);
+        break;
+    }
+  }
+}
+
+// Writes 'num_vals' values with width 'bit_width' and reads them back.
+void TestBitArrayValues(int bit_width, int num_vals) {
+  const int kTestLen = BitUtil::Ceil(bit_width * num_vals, 8);
+  const uint64_t mod = bit_width == 64? 1 : 1LL << bit_width;
+
+  faststring buffer(kTestLen);
+  BitWriter writer(&buffer);
+  for (int i = 0; i < num_vals; ++i) {
+    writer.PutValue(i % mod, bit_width);
+  }
+  writer.Flush();
+  EXPECT_EQ(writer.bytes_written(), kTestLen);
+
+  BitReader reader(buffer.data(), kTestLen);
+  for (int i = 0; i < num_vals; ++i) {
+    int64_t val = 0;
+    bool result = reader.GetValue(bit_width, &val);
+    EXPECT_TRUE(result);
+    EXPECT_EQ(val, i % mod);
+  }
+  EXPECT_EQ(reader.bytes_left(), 0);
+}
+
+TEST(BitArray, TestValues) {
+  for (int width = 1; width <= kMaxWidth; ++width) {
+    TestBitArrayValues(width, 1);
+    TestBitArrayValues(width, 2);
+    // Don't write too many values
+    TestBitArrayValues(width, (width < 12) ? (1 << width) : 4096);
+    TestBitArrayValues(width, 1024);
+  }
+}
+
+// Test some mixed values
+TEST(BitArray, TestMixed) {
+  const int kTestLenBits = 1024;
+  faststring buffer(kTestLenBits / 8);
+  bool parity = true;
+
+  BitWriter writer(&buffer);
+  for (int i = 0; i < kTestLenBits; ++i) {
+    if (i % 2 == 0) {
+      writer.PutValue(parity, 1);
+      parity = !parity;
+    } else {
+      writer.PutValue(i, 10);
+    }
+  }
+  writer.Flush();
+
+  parity = true;
+  BitReader reader(buffer.data(), buffer.size());
+  for (int i = 0; i < kTestLenBits; ++i) {
+    bool result;
+    if (i % 2 == 0) {
+      bool val = false;
+      result = reader.GetValue(1, &val);
+      EXPECT_EQ(val, parity);
+      parity = !parity;
+    } else {
+      int val;
+      result = reader.GetValue(10, &val);
+      EXPECT_EQ(val, i);
+    }
+    EXPECT_TRUE(result);
+  }
+}
+
+// Validates encoding of values by encoding and decoding them.  If
+// expected_encoding != NULL, also validates that the encoded buffer is
+// exactly 'expected_encoding'.
+// if expected_len is not -1, it will validate the encoded size is correct.
+template<typename T>
+void ValidateRle(const vector<T>& values, int bit_width,
+    uint8_t* expected_encoding, int expected_len) {
+  faststring buffer;
+  RleEncoder<T> encoder(&buffer, bit_width);
+
+  for (const auto& value : values) {
+    encoder.Put(value);
+  }
+  int encoded_len = encoder.Flush();
+
+  if (expected_len != -1) {
+    EXPECT_EQ(encoded_len, expected_len);
+  }
+  if (expected_encoding != nullptr) {
+    EXPECT_EQ(memcmp(buffer.data(), expected_encoding, expected_len), 0)
+      << "\n"
+      << "Expected: " << HexDump(Slice(expected_encoding, expected_len)) << "\n"
+      << "Got:      " << HexDump(Slice(buffer));
+  }
+
+  // Verify read
+  RleDecoder<T> decoder(buffer.data(), encoded_len, bit_width);
+  for (const auto& value : values) {
+    T val = 0;
+    bool result = decoder.Get(&val);
+    EXPECT_TRUE(result);
+    EXPECT_EQ(value, val);
+  }
+}
+
+TEST(Rle, SpecificSequences) {
+  const int kTestLen = 1024;
+  uint8_t expected_buffer[kTestLen];
+  vector<uint64_t> values;
+
+  // Test 50 0' followed by 50 1's
+  values.resize(100);
+  for (int i = 0; i < 50; ++i) {
+    values[i] = 0;
+  }
+  for (int i = 50; i < 100; ++i) {
+    values[i] = 1;
+  }
+
+  // expected_buffer valid for bit width <= 1 byte
+  expected_buffer[0] = (50 << 1);
+  expected_buffer[1] = 0;
+  expected_buffer[2] = (50 << 1);
+  expected_buffer[3] = 1;
+  for (int width = 1; width <= 8; ++width) {
+    ValidateRle(values, width, expected_buffer, 4);
+  }
+
+  for (int width = 9; width <= kMaxWidth; ++width) {
+    ValidateRle(values, width, nullptr, 2 * (1 + BitUtil::Ceil(width, 8)));
+  }
+
+  // Test 100 0's and 1's alternating
+  for (int i = 0; i < 100; ++i) {
+    values[i] = i % 2;
+  }
+  int num_groups = BitUtil::Ceil(100, 8);
+  expected_buffer[0] = (num_groups << 1) | 1;
+  for (int i = 0; i < 100/8; ++i) {
+    expected_buffer[i + 1] = BOOST_BINARY(1 0 1 0 1 0 1 0); // 0xaa
+  }
+  // Values for the last 4 0 and 1's
+  expected_buffer[1 + 100/8] = BOOST_BINARY(0 0 0 0 1 0 1 0); // 0x0a
+
+  // num_groups and expected_buffer only valid for bit width = 1
+  ValidateRle(values, 1, expected_buffer, 1 + num_groups);
+  for (int width = 2; width <= kMaxWidth; ++width) {
+    ValidateRle(values, width, nullptr, 1 + BitUtil::Ceil(width * 100, 8));
+  }
+}
+
+// ValidateRle on 'num_vals' values with width 'bit_width'. If 'value' != -1, that value
+// is used, otherwise alternating values are used.
+void TestRleValues(int bit_width, int num_vals, int value = -1) {
+  const uint64_t mod = bit_width == 64 ? 1ULL : 1ULL << bit_width;
+  vector<uint64_t> values;
+  for (uint64_t v = 0; v < num_vals; ++v) {
+    values.push_back((value != -1) ? value : (bit_width == 64 ? v : (v % mod)));
+  }
+  ValidateRle(values, bit_width, nullptr, -1);
+}
+
+TEST(Rle, TestValues) {
+  for (int width = 1; width <= kMaxWidth; ++width) {
+    TestRleValues(width, 1);
+    TestRleValues(width, 1024);
+    TestRleValues(width, 1024, 0);
+    TestRleValues(width, 1024, 1);
+  }
+}
+
+class BitRle : public KuduTest {
+};
+
+// Tests all true/false values
+TEST_F(BitRle, AllSame) {
+  const int kTestLen = 1024;
+  vector<bool> values;
+
+  for (int v = 0; v < 2; ++v) {
+    values.clear();
+    for (int i = 0; i < kTestLen; ++i) {
+      values.push_back(v ? true : false);
+    }
+
+    ValidateRle(values, 1, nullptr, 3);
+  }
+}
+
+// Test that writes out a repeated group and then a literal
+// group but flush before finishing.
+TEST_F(BitRle, Flush) {
+  vector<bool> values;
+  for (int i = 0; i < 16; ++i) values.push_back(1);
+  values.push_back(false);
+  ValidateRle(values, 1, nullptr, -1);
+  values.push_back(true);
+  ValidateRle(values, 1, nullptr, -1);
+  values.push_back(true);
+  ValidateRle(values, 1, nullptr, -1);
+  values.push_back(true);
+  ValidateRle(values, 1, nullptr, -1);
+}
+
+// Test some random bool sequences.
+TEST_F(BitRle, RandomBools) {
+  int iters = 0;
+  const int n_iters = AllowSlowTests() ? 1000 : 20;
+  while (iters < n_iters) {
+    srand(iters++);
+    if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters;
+    vector<uint64_t > values;
+    bool parity = 0;
+    for (int i = 0; i < 1000; ++i) {
+      int group_size = rand() % 20 + 1; // NOLINT(*)
+      if (group_size > 16) {
+        group_size = 1;
+      }
+      for (int i = 0; i < group_size; ++i) {
+        values.push_back(parity);
+      }
+      parity = !parity;
+    }
+    ValidateRle(values, (iters % kMaxWidth) + 1, nullptr, -1);
+  }
+}
+
+// Test some random 64-bit sequences.
+TEST_F(BitRle, Random64Bit) {
+  int iters = 0;
+  const int n_iters = AllowSlowTests() ? 1000 : 20;
+  while (iters < n_iters) {
+    srand(iters++);
+    if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters;
+    vector<uint64_t > values;
+    for (int i = 0; i < 1000; ++i) {
+      int group_size = rand() % 20 + 1; // NOLINT(*)
+      uint64_t cur_value = (static_cast<uint64_t>(rand()) << 32) + static_cast<uint64_t>(rand());
+      if (group_size > 16) {
+        group_size = 1;
+      }
+      for (int i = 0; i < group_size; ++i) {
+        values.push_back(cur_value);
+      }
+
+    }
+    ValidateRle(values, 64, nullptr, -1);
+  }
+}
+
+// Test a sequence of 1 0's, 2 1's, 3 0's. etc
+// e.g. 011000111100000
+TEST_F(BitRle, RepeatedPattern) {
+  vector<bool> values;
+  const int min_run = 1;
+  const int max_run = 32;
+
+  for (int i = min_run; i <= max_run; ++i) {
+    int v = i % 2;
+    for (int j = 0; j < i; ++j) {
+      values.push_back(v);
+    }
+  }
+
+  // And go back down again
+  for (int i = max_run; i >= min_run; --i) {
+    int v = i % 2;
+    for (int j = 0; j < i; ++j) {
+      values.push_back(v);
+    }
+  }
+
+  ValidateRle(values, 1, nullptr, -1);
+}
+
+TEST_F(TestRle, TestBulkPut) {
+  size_t run_length;
+  bool val = false;
+
+  faststring buffer(1);
+  RleEncoder<bool> encoder(&buffer, 1);
+  encoder.Put(true, 10);
+  encoder.Put(false, 7);
+  encoder.Put(true, 5);
+  encoder.Put(true, 15);
+  encoder.Flush();
+
+  RleDecoder<bool> decoder(buffer.data(), encoder.len(), 1);
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_TRUE(val);
+  ASSERT_EQ(10, run_length);
+
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_FALSE(val);
+  ASSERT_EQ(7, run_length);
+
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_TRUE(val);
+  ASSERT_EQ(20, run_length);
+
+  ASSERT_EQ(0, decoder.GetNextRun(&val, MathLimits<size_t>::kMax));
+}
+
+TEST_F(TestRle, TestGetNextRun) {
+  // Repeat the test with different number of items
+  for (int num_items = 7; num_items < 200; num_items += 13) {
+    // Test different block patterns
+    //    1: 01010101 01010101
+    //    2: 00110011 00110011
+    //    3: 00011100 01110001
+    //    ...
+    for (int block = 1; block <= 20; ++block) {
+      faststring buffer(1);
+      RleEncoder<bool> encoder(&buffer, 1);
+      for (int j = 0; j < num_items; ++j) {
+        encoder.Put(!!(j & 1), block);
+      }
+      encoder.Flush();
+
+      RleDecoder<bool> decoder(buffer.data(), encoder.len(), 1);
+      size_t count = num_items * block;
+      for (int j = 0; j < num_items; ++j) {
+        size_t run_length;
+        bool val = false;
+        DCHECK_GT(count, 0);
+        run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+        run_length = std::min(run_length, count);
+
+        ASSERT_EQ(!!(j & 1), val);
+        ASSERT_EQ(block, run_length);
+        count -= run_length;
+      }
+      DCHECK_EQ(count, 0);
+    }
+  }
+}
+
+// Generate a random bit string which consists of 'num_runs' runs,
+// each with a random length between 1 and 100. Returns the number
+// of values encoded (i.e the sum run length).
+static size_t GenerateRandomBitString(int num_runs, faststring* enc_buf, string* string_rep) {
+  RleEncoder<bool> enc(enc_buf, 1);
+  int num_bits = 0;
+  for (int i = 0; i < num_runs; i++) {
+    int run_length = random() % 100;
+    bool value = static_cast<bool>(i & 1);
+    enc.Put(value, run_length);
+    string_rep->append(run_length, value ? '1' : '0');
+    num_bits += run_length;
+  }
+  enc.Flush();
+  return num_bits;
+}
+
+TEST_F(TestRle, TestRoundTripRandomSequencesWithRuns) {
+  SeedRandom();
+
+  // Test the limiting function of GetNextRun.
+  const int kMaxToReadAtOnce = (random() % 20) + 1;
+
+  // Generate a bunch of random bit sequences, and "round-trip" them
+  // through the encode/decode sequence.
+  for (int rep = 0; rep < 100; rep++) {
+    faststring buf;
+    string string_rep;
+    int num_bits = GenerateRandomBitString(10, &buf, &string_rep);
+    RleDecoder<bool> decoder(buf.data(), buf.size(), 1);
+    string roundtrip_str;
+    int rem_to_read = num_bits;
+    size_t run_len;
+    bool val;
+    while (rem_to_read > 0 &&
+           (run_len = decoder.GetNextRun(&val, std::min(kMaxToReadAtOnce, rem_to_read))) != 0) {
+      ASSERT_LE(run_len, kMaxToReadAtOnce);
+      roundtrip_str.append(run_len, val ? '1' : '0');
+      rem_to_read -= run_len;
+    }
+
+    ASSERT_EQ(string_rep, roundtrip_str);
+  }
+}
+TEST_F(TestRle, TestSkip) {
+  faststring buffer(1);
+  RleEncoder<bool> encoder(&buffer, 1);
+
+  // 0101010[1] 01010101 01
+  //        "A"
+  for (int j = 0; j < 18; ++j) {
+    encoder.Put(!!(j & 1));
+  }
+
+  // 0011[00] 11001100 11001100 11001100 11001100
+  //      "B"
+  for (int j = 0; j < 19; ++j) {
+    encoder.Put(!!(j & 1), 2);
+  }
+
+  // 000000000000 11[1111111111] 000000000000 111111111111
+  //                   "C"
+  // 000000000000 111111111111 0[00000000000] 111111111111
+  //                                  "D"
+  // 000000000000 111111111111 000000000000 111111111111
+  for (int j = 0; j < 12; ++j) {
+    encoder.Put(!!(j & 1), 12);
+  }
+  encoder.Flush();
+
+  bool val = false;
+  size_t run_length;
+  RleDecoder<bool> decoder(buffer.data(), encoder.len(), 1);
+
+  // position before "A"
+  ASSERT_EQ(3, decoder.Skip(7));
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_TRUE(val);
+  ASSERT_EQ(1, run_length);
+
+  // position before "B"
+  ASSERT_EQ(7, decoder.Skip(14));
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_FALSE(val);
+  ASSERT_EQ(2, run_length);
+
+  // position before "C"
+  ASSERT_EQ(18, decoder.Skip(46));
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_TRUE(val);
+  ASSERT_EQ(10, run_length);
+
+  // position before "D"
+  ASSERT_EQ(24, decoder.Skip(49));
+  run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  ASSERT_FALSE(val);
+  ASSERT_EQ(11, run_length);
+
+  encoder.Flush();
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rolling_log-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rolling_log-test.cc b/be/src/kudu/util/rolling_log-test.cc
new file mode 100644
index 0000000..f4f8186
--- /dev/null
+++ b/be/src/kudu/util/rolling_log-test.cc
@@ -0,0 +1,147 @@
+// 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 "kudu/util/rolling_log.h"
+
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include <glog/stl_logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+class RollingLogTest : public KuduTest {
+ public:
+  RollingLogTest()
+    : log_dir_(GetTestPath("log_dir")) {
+  }
+
+  virtual void SetUp() OVERRIDE {
+    ASSERT_OK(env_->CreateDir(log_dir_));
+  }
+
+ protected:
+  void AssertLogCount(int expected_count, vector<string>* children) {
+    vector<string> dir_entries;
+    ASSERT_OK(env_->GetChildren(log_dir_, &dir_entries));
+    children->clear();
+
+    for (const string& child : dir_entries) {
+      if (child == "." || child == "..") continue;
+      children->push_back(child);
+      ASSERT_TRUE(HasPrefixString(child, "rolling_log-test."));
+      ASSERT_STR_CONTAINS(child, ".mylog.");
+
+      string pid_suffix = Substitute("$0", getpid());
+      ASSERT_TRUE(HasSuffixString(child, pid_suffix) ||
+                  HasSuffixString(child, pid_suffix + ".gz")) << "bad child: " << child;
+    }
+    std::sort(children->begin(), children->end());
+    ASSERT_EQ(children->size(), expected_count) << *children;
+  }
+
+  const string log_dir_;
+};
+
+// Test with compression off.
+TEST_F(RollingLogTest, TestLog) {
+  RollingLog log(env_, log_dir_, "mylog");
+  log.SetCompressionEnabled(false);
+  log.SetRollThresholdBytes(100);
+
+  // Before writing anything, we shouldn't open a log file.
+  vector<string> children;
+  NO_FATALS(AssertLogCount(0, &children));
+
+  // Appending some data should write a new segment.
+  const string kTestString = "Hello world\n";
+  ASSERT_OK(log.Append(kTestString));
+  NO_FATALS(AssertLogCount(1, &children));
+
+  for (int i = 0; i < 10; i++) {
+    ASSERT_OK(log.Append(kTestString));
+  }
+  NO_FATALS(AssertLogCount(2, &children));
+
+  faststring data;
+  string path = JoinPathSegments(log_dir_, children[0]);
+  ASSERT_OK(ReadFileToString(env_, path, &data));
+  ASSERT_TRUE(HasPrefixString(data.ToString(), kTestString))
+    << "Data missing";
+  ASSERT_LE(data.size(), 100 + kTestString.length())
+      << "Roll threshold not respected";
+}
+
+// Test with compression on.
+TEST_F(RollingLogTest, TestCompression) {
+  RollingLog log(env_, log_dir_, "mylog");
+  ASSERT_OK(log.Open());
+
+  StringPiece data = "Hello world\n";
+  int raw_size = 0;
+  for (int i = 0; i < 1000; i++) {
+    ASSERT_OK(log.Append(data));
+    raw_size += data.size();
+  }
+  ASSERT_OK(log.Close());
+
+  vector<string> children;
+  NO_FATALS(AssertLogCount(1, &children));
+  ASSERT_TRUE(HasSuffixString(children[0], ".gz"));
+
+  // Ensure that the output is actually gzipped.
+  uint64_t size;
+  ASSERT_OK(env_->GetFileSize(JoinPathSegments(log_dir_, children[0]), &size));
+  ASSERT_LT(size, raw_size / 10);
+  ASSERT_GT(size, 0);
+}
+
+TEST_F(RollingLogTest, TestFileCountLimit) {
+  RollingLog log(env_, log_dir_, "mylog");
+  ASSERT_OK(log.Open());
+  log.SetRollThresholdBytes(100);
+  log.SetMaxNumSegments(3);
+
+  for (int i = 0; i < 100; i++) {
+    ASSERT_OK(log.Append("hello world\n"));
+  }
+  ASSERT_OK(log.Close());
+
+  vector<string> children;
+  NO_FATALS(AssertLogCount(3, &children));
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rolling_log.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rolling_log.cc b/be/src/kudu/util/rolling_log.cc
new file mode 100644
index 0000000..50f9fbd
--- /dev/null
+++ b/be/src/kudu/util/rolling_log.cc
@@ -0,0 +1,285 @@
+// 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 "kudu/util/rolling_log.h"
+
+#include <unistd.h>
+
+#include <ctime>
+#include <iomanip>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <zlib.h>
+
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/user.h"
+
+using std::ostringstream;
+using std::setw;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+static const int kDefaultRollThresholdBytes = 64 * 1024 * 1024; // 64MB
+
+DECLARE_int32(max_log_files);
+
+namespace kudu {
+
+RollingLog::RollingLog(Env* env, string log_dir, string log_name)
+    : env_(env),
+      log_dir_(std::move(log_dir)),
+      log_name_(std::move(log_name)),
+      roll_threshold_bytes_(kDefaultRollThresholdBytes),
+      max_num_segments_(FLAGS_max_log_files),
+      compress_after_close_(true) {}
+
+RollingLog::~RollingLog() {
+  WARN_NOT_OK(Close(), "Unable to close RollingLog");
+}
+
+void RollingLog::SetRollThresholdBytes(int64_t size) {
+  CHECK_GT(size, 0);
+  roll_threshold_bytes_ = size;
+}
+
+void RollingLog::SetMaxNumSegments(int num_segments) {
+  CHECK_GT(num_segments, 0);
+  max_num_segments_ = num_segments;
+}
+
+void RollingLog::SetCompressionEnabled(bool compress) {
+  compress_after_close_ = compress;
+}
+
+namespace {
+
+string HostnameOrUnknown() {
+  string hostname;
+  Status s = GetHostname(&hostname);
+  if (!s.ok()) {
+    return "unknown_host";
+  }
+  return hostname;
+}
+
+string UsernameOrUnknown() {
+  string user_name;
+  Status s = GetLoggedInUser(&user_name);
+  if (!s.ok()) {
+    return "unknown_user";
+  }
+  return user_name;
+}
+
+string FormattedTimestamp() {
+  // Implementation cribbed from glog/logging.cc
+  time_t time = static_cast<time_t>(WallTime_Now());
+  struct ::tm tm_time;
+  localtime_r(&time, &tm_time);
+
+  ostringstream str;
+  str.fill('0');
+  str << 1900+tm_time.tm_year
+      << setw(2) << 1+tm_time.tm_mon
+      << setw(2) << tm_time.tm_mday
+      << '-'
+      << setw(2) << tm_time.tm_hour
+      << setw(2) << tm_time.tm_min
+      << setw(2) << tm_time.tm_sec;
+  return str.str();
+}
+
+} // anonymous namespace
+
+string RollingLog::GetLogFileName(int sequence) const {
+  return Substitute("$0.$1.$2.$3.$4.$5.$6",
+                    google::ProgramInvocationShortName(),
+                    HostnameOrUnknown(),
+                    UsernameOrUnknown(),
+                    log_name_,
+                    FormattedTimestamp(),
+                    sequence,
+                    getpid());
+}
+
+string RollingLog::GetLogFilePattern() const {
+  return Substitute("$0.$1.$2.$3.$4.$5.$6",
+                    google::ProgramInvocationShortName(),
+                    HostnameOrUnknown(),
+                    UsernameOrUnknown(),
+                    log_name_,
+                    /* any timestamp */'*',
+                    /* any sequence number */'*',
+                    /* any pid */'*');
+}
+
+Status RollingLog::Open() {
+  CHECK(!file_);
+
+  for (int sequence = 0; ; sequence++) {
+
+    string path = JoinPathSegments(log_dir_, GetLogFileName(sequence));
+    // Don't reuse an existing path if there is already a log
+    // or a compressed log with the same name.
+    if (env_->FileExists(path) ||
+        env_->FileExists(path + ".gz")) {
+      continue;
+    }
+
+    WritableFileOptions opts;
+    // Logs aren't worth the performance cost of durability.
+    opts.sync_on_close = false;
+    opts.mode = Env::CREATE_NON_EXISTING;
+
+    RETURN_NOT_OK(env_->NewWritableFile(opts, path, &file_));
+
+    VLOG(1) << "Rolled " << log_name_ << " log to new file: " << path;
+    break;
+  }
+  return Status::OK();
+}
+
+Status RollingLog::Close() {
+  if (!file_) {
+    return Status::OK();
+  }
+  string path = file_->filename();
+  RETURN_NOT_OK_PREPEND(file_->Close(),
+                        Substitute("Unable to close $0", path));
+  file_.reset();
+  if (compress_after_close_) {
+    WARN_NOT_OK(CompressFile(path), "Unable to compress old log file");
+  }
+  auto glob = JoinPathSegments(log_dir_, GetLogFilePattern());
+  WARN_NOT_OK(env_util::DeleteExcessFilesByPattern(env_, glob, max_num_segments_),
+              Substitute("failed to delete old $0 log files", log_name_));
+  return Status::OK();
+}
+
+Status RollingLog::Append(StringPiece s) {
+  if (!file_) {
+    RETURN_NOT_OK_PREPEND(Open(), "Unable to open log");
+  }
+
+  RETURN_NOT_OK(file_->Append(s));
+  if (file_->Size() > roll_threshold_bytes_) {
+    RETURN_NOT_OK_PREPEND(Close(), "Unable to roll log");
+    roll_count_++;
+    RETURN_NOT_OK_PREPEND(Open(), "Unable to roll log");
+  }
+  return Status::OK();
+}
+
+namespace {
+
+Status GzClose(gzFile f) {
+  int err = gzclose(f);
+  switch (err) {
+    case Z_OK:
+      return Status::OK();
+    case Z_STREAM_ERROR:
+      return Status::InvalidArgument("Stream not valid");
+    case Z_ERRNO:
+      return Status::IOError("IO Error closing stream");
+    case Z_MEM_ERROR:
+      return Status::RuntimeError("Out of memory");
+    case Z_BUF_ERROR:
+      return Status::IOError("read ended in the middle of a stream");
+    default:
+      return Status::IOError("Unknown zlib error", SimpleItoa(err));
+  }
+}
+
+class ScopedGzipCloser {
+ public:
+  explicit ScopedGzipCloser(gzFile f)
+    : file_(f) {
+  }
+
+  ~ScopedGzipCloser() {
+    if (file_) {
+      WARN_NOT_OK(GzClose(file_), "Unable to close gzip stream");
+    }
+  }
+
+  void Cancel() {
+    file_ = nullptr;
+  }
+
+ private:
+  gzFile file_;
+};
+} // anonymous namespace
+
+// We implement CompressFile() manually using zlib APIs rather than forking
+// out to '/bin/gzip' since fork() can be expensive on processes that use a large
+// amount of memory. During the time of the fork, other threads could end up
+// blocked. Implementing it using the zlib stream APIs isn't too much code
+// and is less likely to be problematic.
+Status RollingLog::CompressFile(const std::string& path) const {
+  unique_ptr<SequentialFile> in_file;
+  RETURN_NOT_OK_PREPEND(env_->NewSequentialFile(path, &in_file),
+                        "Unable to open input file to compress");
+
+  string gz_path = path + ".gz";
+  gzFile gzf = gzopen(gz_path.c_str(), "w");
+  if (!gzf) {
+    return Status::IOError("Unable to open gzip stream");
+  }
+
+  ScopedGzipCloser closer(gzf);
+
+  // Loop reading data from the input file and writing to the gzip stream.
+  uint8_t buf[32 * 1024];
+  while (true) {
+    Slice result(buf, arraysize(buf));
+    RETURN_NOT_OK_PREPEND(in_file->Read(&result),
+                          "Unable to read from gzip input");
+    if (result.size() == 0) {
+      break;
+    }
+    int n = gzwrite(gzf, result.data(), result.size());
+    if (n == 0) {
+      int errnum;
+      return Status::IOError("Unable to write to gzip output",
+                             gzerror(gzf, &errnum));
+    }
+  }
+  closer.Cancel();
+  RETURN_NOT_OK_PREPEND(GzClose(gzf),
+                        "Unable to close gzip output");
+
+  WARN_NOT_OK(env_->DeleteFile(path),
+              "Unable to delete gzip input file after compression");
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rolling_log.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rolling_log.h b/be/src/kudu/util/rolling_log.h
new file mode 100644
index 0000000..0bb6755
--- /dev/null
+++ b/be/src/kudu/util/rolling_log.h
@@ -0,0 +1,128 @@
+// 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.
+#ifndef KUDU_UTIL_ROLLING_LOG_H
+#define KUDU_UTIL_ROLLING_LOG_H
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Env;
+class WritableFile;
+
+// A simple rolling log.
+//
+// This creates a log which spans multiple files in a specified directory.
+// After a log file reaches a specified size threshold, it automatically rolls
+// to the next file in the sequence.
+//
+// The files are named similarly to glog log files and use the following pattern:
+//
+// <log_dir>/<program-name>.<hostname>.<user-name>.<log-name>.<timestamp>.<sequence>.<pid>
+//   log_dir:      the log_dir specified in the constructor
+//   program-name: argv[0], as determined by google::ProgramInvocationShortName()
+//   hostname:     the local machine hostname
+//   user-name:    the current user name
+//   log-name:     the log_name specified in the constructor
+//   timestamp:    the wall clock time when the log file was created, in
+//                 YYYYmmdd-HHMMSS fixed-length format.
+//   sequence:     a sequence number which is used to disambiguate when the log file is
+//                 rolled multiple times within a second
+//   pid:          the pid of the daemon
+//
+// The log implementation does not ensure durability of the log or its files in any way.
+// This class is not thread-safe and must be externally synchronized.
+class RollingLog {
+ public:
+  RollingLog(Env* env, std::string log_dir, std::string log_name);
+
+  ~RollingLog();
+
+  // Open the log.
+  // It is optional to call this function. Append() will automatically open
+  // the log as necessary if it is not open.
+  Status Open();
+
+  // Set the pre-compression size threshold at which the log file will be rolled.
+  // If the log is already open, this applies for the the current and any future
+  // log file.
+  //
+  // NOTE: This is the limit on a single segment of the log, not a limit on the total
+  // size of the log.
+  //
+  // NOTE: The threshold is checked _after_ each call to Append(). So, the size of
+  // the log may overshoot this threshold by as much as the size of a single appended
+  // message.
+  void SetRollThresholdBytes(int64_t size);
+
+  // Set the total number of log segments to be retained. When the log is rolled,
+  // old segments are removed to achieve the targeted number of segments.
+  void SetMaxNumSegments(int num_segments);
+
+  // If compression is enabled, log files are compressed.
+  // NOTE: this requires that the passed-in Env instance is the local file system.
+  void SetCompressionEnabled(bool compress);
+
+  // Append the given data to the current log file.
+  //
+  // If, after appending this data, the file size has crossed the configured roll
+  // threshold, a new empty log file is created. Note that this is a synchronous API and
+  // causes potentially-blocking IO on the current thread. However, this does not fsync()
+  // or otherwise ensure durability of the appended data.
+  Status Append(StringPiece data) WARN_UNUSED_RESULT;
+
+  // Close the log.
+  Status Close();
+
+  // Return the number of times this log has rolled since it was first opened.
+  int roll_count() const {
+    return roll_count_;
+  }
+
+ private:
+  std::string GetLogFileName(int sequence) const;
+
+  // Get a glob pattern matching all log files written by this instance.
+  std::string GetLogFilePattern() const;
+
+  // Compress the given path, writing a new file '<path>.gz'.
+  Status CompressFile(const std::string& path) const;
+
+  Env* const env_;
+  const std::string log_dir_;
+  const std::string log_name_;
+
+  int64_t roll_threshold_bytes_;
+  int max_num_segments_;
+
+  std::unique_ptr<WritableFile> file_;
+  bool compress_after_close_;
+
+  int roll_count_ = 0;
+
+  DISALLOW_COPY_AND_ASSIGN(RollingLog);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_ROLLING_LOG_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rw_mutex-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rw_mutex-test.cc b/be/src/kudu/util/rw_mutex-test.cc
new file mode 100644
index 0000000..c2cb394
--- /dev/null
+++ b/be/src/kudu/util/rw_mutex-test.cc
@@ -0,0 +1,185 @@
+// 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 <cstdint>
+#include <mutex>
+#include <ostream>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/atomic.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/test_util.h"
+
+using std::lock_guard;
+using std::thread;
+using std::try_to_lock;
+using std::unique_lock;
+using std::vector;
+
+namespace kudu {
+
+class RWMutexTest : public KuduTest,
+                    public ::testing::WithParamInterface<RWMutex::Priority> {
+ public:
+  RWMutexTest()
+     : lock_(GetParam()) {
+  }
+ protected:
+  RWMutex lock_;
+};
+
+// Instantiate every test for each kind of RWMutex priority.
+INSTANTIATE_TEST_CASE_P(Priorities, RWMutexTest,
+                        ::testing::Values(RWMutex::Priority::PREFER_READING,
+                                          RWMutex::Priority::PREFER_WRITING));
+
+// Multi-threaded test that tries to find deadlocks in the RWMutex wrapper.
+TEST_P(RWMutexTest, TestDeadlocks) {
+  uint64_t number_of_writes = 0;
+  AtomicInt<uint64_t> number_of_reads(0);
+
+  AtomicBool done(false);
+  vector<thread> threads;
+
+  // Start several blocking and non-blocking read-write workloads.
+  for (int i = 0; i < 2; i++) {
+    threads.emplace_back([&](){
+      while (!done.Load()) {
+        lock_guard<RWMutex> l(lock_);
+        number_of_writes++;
+      }
+    });
+    threads.emplace_back([&](){
+      while (!done.Load()) {
+        unique_lock<RWMutex> l(lock_, try_to_lock);
+        if (l.owns_lock()) {
+          number_of_writes++;
+        }
+      }
+    });
+  }
+
+  // Start several blocking and non-blocking read-only workloads.
+  for (int i = 0; i < 2; i++) {
+    threads.emplace_back([&](){
+      while (!done.Load()) {
+        shared_lock<RWMutex> l(lock_);
+        number_of_reads.Increment();
+      }
+    });
+    threads.emplace_back([&](){
+      while (!done.Load()) {
+        shared_lock<RWMutex> l(lock_, try_to_lock);
+        if (l.owns_lock()) {
+          number_of_reads.Increment();
+        }
+      }
+    });
+  }
+
+  SleepFor(MonoDelta::FromSeconds(1));
+  done.Store(true);
+  for (auto& t : threads) {
+    t.join();
+  }
+
+  shared_lock<RWMutex> l(lock_);
+  LOG(INFO) << "Number of writes: " << number_of_writes;
+  LOG(INFO) << "Number of reads: " << number_of_reads.Load();
+}
+
+#ifndef NDEBUG
+// Tests that the RWMutex wrapper catches basic usage errors. This checking is
+// only enabled in debug builds.
+TEST_P(RWMutexTest, TestLockChecking) {
+  EXPECT_DEATH({
+    lock_.ReadLock();
+    lock_.ReadLock();
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryReadLock());
+    CHECK(lock_.TryReadLock());
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    lock_.ReadLock();
+    lock_.WriteLock();
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryReadLock());
+    CHECK(lock_.TryWriteLock());
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    lock_.WriteLock();
+    lock_.ReadLock();
+  }, "already holding lock for writing");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryWriteLock());
+    CHECK(lock_.TryReadLock());
+  }, "already holding lock for writing");
+
+  EXPECT_DEATH({
+    lock_.WriteLock();
+    lock_.WriteLock();
+  }, "already holding lock for writing");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryWriteLock());
+    CHECK(lock_.TryWriteLock());
+  }, "already holding lock for writing");
+
+  EXPECT_DEATH({
+    lock_.ReadUnlock();
+  }, "wasn't holding lock for reading");
+
+  EXPECT_DEATH({
+    lock_.WriteUnlock();
+  }, "wasn't holding lock for writing");
+
+  EXPECT_DEATH({
+    lock_.ReadLock();
+    lock_.WriteUnlock();
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryReadLock());
+    lock_.WriteUnlock();
+  }, "already holding lock for reading");
+
+  EXPECT_DEATH({
+    lock_.WriteLock();
+    lock_.ReadUnlock();
+  }, "already holding lock for writing");
+
+  EXPECT_DEATH({
+    CHECK(lock_.TryWriteLock());
+    lock_.ReadUnlock();
+  }, "already holding lock for writing");
+}
+#endif
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rw_mutex.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rw_mutex.cc b/be/src/kudu/util/rw_mutex.cc
new file mode 100644
index 0000000..75fc1e7
--- /dev/null
+++ b/be/src/kudu/util/rw_mutex.cc
@@ -0,0 +1,207 @@
+// 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 "kudu/util/rw_mutex.h"
+
+#include <cerrno>
+#include <cstring>
+#include <mutex>
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/util/env.h"
+
+using std::lock_guard;
+
+namespace {
+
+void unlock_rwlock(pthread_rwlock_t* rwlock) {
+  int rv = pthread_rwlock_unlock(rwlock);
+  DCHECK_EQ(0, rv) << strerror(rv);
+}
+
+} // anonymous namespace
+
+namespace kudu {
+
+RWMutex::RWMutex()
+#ifndef NDEBUG
+    : writer_tid_(0)
+#endif
+{
+  Init(Priority::PREFER_READING);
+}
+
+RWMutex::RWMutex(Priority prio)
+#ifndef NDEBUG
+    : writer_tid_(0)
+#endif
+{
+  Init(prio);
+}
+
+void RWMutex::Init(Priority prio) {
+#ifdef __linux__
+  // Adapt from priority to the pthread type.
+  int kind = PTHREAD_RWLOCK_PREFER_READER_NP;
+  switch (prio) {
+    case Priority::PREFER_READING:
+      kind = PTHREAD_RWLOCK_PREFER_READER_NP;
+      break;
+    case Priority::PREFER_WRITING:
+      kind = PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP;
+      break;
+  }
+
+  // Initialize the new rwlock with the user's preference.
+  pthread_rwlockattr_t attr;
+  int rv = pthread_rwlockattr_init(&attr);
+  DCHECK_EQ(0, rv) << strerror(rv);
+  rv = pthread_rwlockattr_setkind_np(&attr, kind);
+  DCHECK_EQ(0, rv) << strerror(rv);
+  rv = pthread_rwlock_init(&native_handle_, &attr);
+  DCHECK_EQ(0, rv) << strerror(rv);
+  rv = pthread_rwlockattr_destroy(&attr);
+  DCHECK_EQ(0, rv) << strerror(rv);
+#else
+  int rv = pthread_rwlock_init(&native_handle_, NULL);
+  DCHECK_EQ(0, rv) << strerror(rv);
+#endif
+}
+
+RWMutex::~RWMutex() {
+  int rv = pthread_rwlock_destroy(&native_handle_);
+  DCHECK_EQ(0, rv) << strerror(rv);
+}
+
+void RWMutex::ReadLock() {
+  CheckLockState(LockState::NEITHER);
+  int rv = pthread_rwlock_rdlock(&native_handle_);
+  DCHECK_EQ(0, rv) << strerror(rv);
+  MarkForReading();
+}
+
+void RWMutex::ReadUnlock() {
+  CheckLockState(LockState::READER);
+  UnmarkForReading();
+  unlock_rwlock(&native_handle_);
+}
+
+bool RWMutex::TryReadLock() {
+  CheckLockState(LockState::NEITHER);
+  int rv = pthread_rwlock_tryrdlock(&native_handle_);
+  if (rv == EBUSY) {
+    return false;
+  }
+  DCHECK_EQ(0, rv) << strerror(rv);
+  MarkForReading();
+  return true;
+}
+
+void RWMutex::WriteLock() {
+  CheckLockState(LockState::NEITHER);
+  int rv = pthread_rwlock_wrlock(&native_handle_);
+  DCHECK_EQ(0, rv) << strerror(rv);
+  MarkForWriting();
+}
+
+void RWMutex::WriteUnlock() {
+  CheckLockState(LockState::WRITER);
+  UnmarkForWriting();
+  unlock_rwlock(&native_handle_);
+}
+
+bool RWMutex::TryWriteLock() {
+  CheckLockState(LockState::NEITHER);
+  int rv = pthread_rwlock_trywrlock(&native_handle_);
+  if (rv == EBUSY) {
+    return false;
+  }
+  DCHECK_EQ(0, rv) << strerror(rv);
+  MarkForWriting();
+  return true;
+}
+
+#ifndef NDEBUG
+
+void RWMutex::AssertAcquired() const {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  CHECK(ContainsKey(reader_tids_, Env::Default()->gettid()) ||
+        Env::Default()->gettid() == writer_tid_);
+}
+
+void RWMutex::AssertAcquiredForReading() const {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  CHECK(ContainsKey(reader_tids_, Env::Default()->gettid()));
+}
+
+void RWMutex::AssertAcquiredForWriting() const {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  CHECK_EQ(Env::Default()->gettid(), writer_tid_);
+}
+
+void RWMutex::CheckLockState(LockState state) const {
+  pid_t my_tid = Env::Default()->gettid();
+  bool is_reader;
+  bool is_writer;
+  {
+    lock_guard<simple_spinlock> l(tid_lock_);
+    is_reader = ContainsKey(reader_tids_, my_tid);
+    is_writer = writer_tid_ == my_tid;
+  }
+
+  switch (state) {
+    case LockState::NEITHER:
+      CHECK(!is_reader) << "Invalid state, already holding lock for reading";
+      CHECK(!is_writer) << "Invalid state, already holding lock for writing";
+      break;
+    case LockState::READER:
+      CHECK(!is_writer) << "Invalid state, already holding lock for writing";
+      CHECK(is_reader) << "Invalid state, wasn't holding lock for reading";
+      break;
+    case LockState::WRITER:
+      CHECK(!is_reader) << "Invalid state, already holding lock for reading";
+      CHECK(is_writer) << "Invalid state, wasn't holding lock for writing";
+      break;
+  }
+}
+
+void RWMutex::MarkForReading() {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  reader_tids_.insert(Env::Default()->gettid());
+}
+
+void RWMutex::MarkForWriting() {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  writer_tid_ = Env::Default()->gettid();
+}
+
+void RWMutex::UnmarkForReading() {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  reader_tids_.erase(Env::Default()->gettid());
+}
+
+void RWMutex::UnmarkForWriting() {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  writer_tid_ = 0;
+}
+
+#endif
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rw_mutex.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rw_mutex.h b/be/src/kudu/util/rw_mutex.h
new file mode 100644
index 0000000..bb6168c
--- /dev/null
+++ b/be/src/kudu/util/rw_mutex.h
@@ -0,0 +1,123 @@
+// 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.
+
+#pragma once
+
+#include <pthread.h>
+#include <sys/types.h>
+
+#include <unordered_set>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/locks.h"
+
+namespace kudu {
+
+// Read/write mutex. Implemented as a thin wrapper around pthread_rwlock_t.
+//
+// Although pthread_rwlock_t allows recursive acquisition, this wrapper does
+// not, and will crash in debug mode if recursive acquisition is detected.
+class RWMutex {
+ public:
+
+  // Possible fairness policies for the RWMutex.
+  enum class Priority {
+    // The lock will prioritize readers at the expense of writers.
+    PREFER_READING,
+
+    // The lock will prioritize writers at the expense of readers.
+    //
+    // Care should be taken when using this fairness policy, as it can lead to
+    // unexpected deadlocks (e.g. a writer waiting on the lock will prevent
+    // additional readers from acquiring it).
+    PREFER_WRITING,
+  };
+
+  // Create an RWMutex that prioritizes readers.
+  RWMutex();
+
+  // Create an RWMutex with customized priority. This is a best effort; the
+  // underlying platform may not support custom priorities.
+  explicit RWMutex(Priority prio);
+
+  ~RWMutex();
+
+  void ReadLock();
+  void ReadUnlock();
+  bool TryReadLock();
+
+  void WriteLock();
+  void WriteUnlock();
+  bool TryWriteLock();
+
+#ifndef NDEBUG
+  void AssertAcquired() const;
+  void AssertAcquiredForReading() const;
+  void AssertAcquiredForWriting() const;
+#else
+  void AssertAcquired() const {}
+  void AssertAcquiredForReading() const {}
+  void AssertAcquiredForWriting() const {}
+#endif
+
+  // Aliases for use with std::lock_guard and kudu::shared_lock.
+  void lock() { WriteLock(); }
+  void unlock() { WriteUnlock(); }
+  bool try_lock() { return TryWriteLock(); }
+  void lock_shared() { ReadLock(); }
+  void unlock_shared() { ReadUnlock(); }
+  bool try_lock_shared() { return TryReadLock(); }
+
+ private:
+  void Init(Priority prio);
+
+  enum class LockState {
+    NEITHER,
+    READER,
+    WRITER,
+  };
+#ifndef NDEBUG
+  void CheckLockState(LockState state) const;
+  void MarkForReading();
+  void MarkForWriting();
+  void UnmarkForReading();
+  void UnmarkForWriting();
+#else
+  void CheckLockState(LockState state) const {}
+  void MarkForReading() {}
+  void MarkForWriting() {}
+  void UnmarkForReading() {}
+  void UnmarkForWriting() {}
+#endif
+
+  pthread_rwlock_t native_handle_;
+
+#ifndef NDEBUG
+  // Protects reader_tids_ and writer_tid_.
+  mutable simple_spinlock tid_lock_;
+
+  // Tracks all current readers by tid.
+  std::unordered_set<pid_t> reader_tids_;
+
+  // Tracks the current writer (if one exists) by tid.
+  pid_t writer_tid_;
+#endif
+
+  DISALLOW_COPY_AND_ASSIGN(RWMutex);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rw_semaphore-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rw_semaphore-test.cc b/be/src/kudu/util/rw_semaphore-test.cc
new file mode 100644
index 0000000..7717608
--- /dev/null
+++ b/be/src/kudu/util/rw_semaphore-test.cc
@@ -0,0 +1,94 @@
+// 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 <cstdint>
+#include <mutex>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/rw_semaphore.h"
+
+using std::thread;
+using std::vector;
+
+namespace kudu {
+struct SharedState {
+  SharedState() : done(false), int_var(0) {}
+
+  bool done;
+  int64_t int_var;
+  rw_semaphore sem;
+};
+
+// Thread which increases the value in the shared state under the write lock.
+void Writer(SharedState* state) {
+  int i = 0;
+  while (true) {
+    std::lock_guard<rw_semaphore> l(state->sem);
+    state->int_var += (i++);
+    if (state->done) {
+      break;
+    }
+  }
+}
+
+// Thread which verifies that the value in the shared state only increases.
+void Reader(SharedState* state) {
+  int prev_val = 0;
+  while (true) {
+    shared_lock<rw_semaphore> l(state->sem);
+    // The int var should only be seen to increase.
+    CHECK_GE(state->int_var, prev_val);
+    prev_val = state->int_var;
+    if (state->done) {
+      break;
+    }
+  }
+}
+
+// Test which verifies basic functionality of the semaphore.
+// When run under TSAN this also verifies the barriers.
+TEST(RWSemaphoreTest, TestBasicOperation) {
+  SharedState s;
+  vector<thread*> threads;
+  // Start 5 readers and writers.
+  for (int i = 0; i < 5; i++) {
+    threads.push_back(new thread(Reader, &s));
+    threads.push_back(new thread(Writer, &s));
+  }
+
+  // Let them contend for a short amount of time.
+  SleepFor(MonoDelta::FromMilliseconds(50));
+
+  // Signal them to stop.
+  {
+    std::lock_guard<rw_semaphore> l(s.sem);
+    s.done = true;
+  }
+
+  for (thread* t : threads) {
+    t->join();
+    delete t;
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rw_semaphore.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rw_semaphore.h b/be/src/kudu/util/rw_semaphore.h
new file mode 100644
index 0000000..20c0b57
--- /dev/null
+++ b/be/src/kudu/util/rw_semaphore.h
@@ -0,0 +1,206 @@
+// 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.
+#ifndef KUDU_UTIL_RW_SEMAPHORE_H
+#define KUDU_UTIL_RW_SEMAPHORE_H
+
+// Uncomment for extra debugging information. See below for details.
+//   #define RW_SEMAPHORE_TRACK_HOLDER 1
+
+#include <boost/smart_ptr/detail/yield_k.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#ifdef RW_SEMAPHORE_TRACK_HOLDER
+#include "kudu/util/debug-util.h"
+#endif
+#include "kudu/util/thread.h"
+
+namespace kudu {
+
+// Read-Write semaphore. 32bit uint that contains the number of readers.
+// When someone wants to write, tries to set the 32bit, and waits until
+// the readers have finished. Readers are spinning while the write flag is set.
+//
+// This rw-semaphore makes no attempt at fairness, though it does avoid write
+// starvation (no new readers may obtain the lock if a write is waiting).
+//
+// NOTE: this means that it is not safe to reentrantly acquire the read lock,
+// due to the following deadlock:
+//   - T1: acquire read lock
+//   - T2: wait for write lock
+//     (blocks waiting for readers)
+//   - T1: try to acquire read-lock reentrantly
+//     (blocks to avoid starving writers)
+//
+// Given that this is currently based only on spinning (and not futex),
+// it should only be used in cases where the lock is held for very short
+// time intervals.
+//
+// If the semaphore is expected to always be released from the same thread
+// that acquired it, use rw_spinlock instead.
+//
+// In order to support easier debugging of leaked locks, this class can track
+// the stack trace of the last thread to lock it in write mode. To do so,
+// uncomment the definition of RW_SEMAPHORE_TRACK_HOLDER at the top of this
+// file. Then, in gdb, print the contents of the semaphore, and you should see
+// the collected stack trace.
+class rw_semaphore {
+ public:
+  rw_semaphore() : state_(0) {
+  }
+  ~rw_semaphore() {}
+
+  void lock_shared() {
+    int loop_count = 0;
+    Atomic32 cur_state = base::subtle::NoBarrier_Load(&state_);
+    while (true) {
+      Atomic32 expected = cur_state & kNumReadersMask;   // I expect no write lock
+      Atomic32 try_new_state = expected + 1;          // Add me as reader
+      cur_state = base::subtle::Acquire_CompareAndSwap(&state_, expected, try_new_state);
+      if (cur_state == expected)
+        break;
+      // Either was already locked by someone else, or CAS failed.
+      boost::detail::yield(loop_count++);
+    }
+  }
+
+  void unlock_shared() {
+    int loop_count = 0;
+    Atomic32 cur_state = base::subtle::NoBarrier_Load(&state_);
+    while (true) {
+      DCHECK_GT(cur_state & kNumReadersMask, 0)
+        << "unlock_shared() called when there are no shared locks held";
+      Atomic32 expected = cur_state;           // I expect a write lock and other readers
+      Atomic32 try_new_state = expected - 1;   // Drop me as reader
+      cur_state = base::subtle::Release_CompareAndSwap(&state_, expected, try_new_state);
+      if (cur_state == expected)
+        break;
+      // Either was already locked by someone else, or CAS failed.
+      boost::detail::yield(loop_count++);
+    }
+  }
+
+  // Tries to acquire a write lock, if no one else has it.
+  // This function retries on CAS failure and waits for readers to complete.
+  bool try_lock() {
+    int loop_count = 0;
+    Atomic32 cur_state = base::subtle::NoBarrier_Load(&state_);
+    while (true) {
+      // someone else has already the write lock
+      if (cur_state & kWriteFlag)
+        return false;
+
+      Atomic32 expected = cur_state & kNumReadersMask;   // I expect some 0+ readers
+      Atomic32 try_new_state = kWriteFlag | expected;    // I want to lock the other writers
+      cur_state = base::subtle::Acquire_CompareAndSwap(&state_, expected, try_new_state);
+      if (cur_state == expected)
+        break;
+      // Either was already locked by someone else, or CAS failed.
+      boost::detail::yield(loop_count++);
+    }
+
+    WaitPendingReaders();
+    RecordLockHolderStack();
+    return true;
+  }
+
+  void lock() {
+    int loop_count = 0;
+    Atomic32 cur_state = base::subtle::NoBarrier_Load(&state_);
+    while (true) {
+      Atomic32 expected = cur_state & kNumReadersMask;   // I expect some 0+ readers
+      Atomic32 try_new_state = kWriteFlag | expected;    // I want to lock the other writers
+      // Note: we use NoBarrier here because we'll do the Acquire barrier down below
+      // in WaitPendingReaders
+      cur_state = base::subtle::NoBarrier_CompareAndSwap(&state_, expected, try_new_state);
+      if (cur_state == expected)
+        break;
+      // Either was already locked by someone else, or CAS failed.
+      boost::detail::yield(loop_count++);
+    }
+
+    WaitPendingReaders();
+
+#ifndef NDEBUG
+    writer_tid_ = Thread::CurrentThreadId();
+#endif // NDEBUG
+    RecordLockHolderStack();
+  }
+
+  void unlock() {
+    // I expect to be the only writer
+    DCHECK_EQ(base::subtle::NoBarrier_Load(&state_), kWriteFlag);
+
+#ifndef NDEBUG
+    writer_tid_ = -1; // Invalid tid.
+#endif // NDEBUG
+
+    ResetLockHolderStack();
+    // Reset: no writers & no readers.
+    Release_Store(&state_, 0);
+  }
+
+  // Return true if the lock is currently held for write by any thread.
+  // See simple_semaphore::is_locked() for details about where this is useful.
+  bool is_write_locked() const {
+    return base::subtle::NoBarrier_Load(&state_) & kWriteFlag;
+  }
+
+  // Return true if the lock is currently held, either for read or write
+  // by any thread.
+  // See simple_semaphore::is_locked() for details about where this is useful.
+  bool is_locked() const {
+    return base::subtle::NoBarrier_Load(&state_);
+  }
+
+ private:
+  static const uint32_t kNumReadersMask = 0x7fffffff;
+  static const uint32_t kWriteFlag = 1 << 31;
+
+#ifdef RW_SEMAPHORE_TRACK_HOLDER
+  StackTrace writer_stack_;
+  void RecordLockHolderStack() {
+    writer_stack_.Collect();
+  }
+  void ResetLockHolderStack() {
+    writer_stack_.Reset();
+  }
+#else
+  void RecordLockHolderStack() {
+  }
+  void ResetLockHolderStack() {
+  }
+#endif
+
+  void WaitPendingReaders() {
+    int loop_count = 0;
+    while ((base::subtle::Acquire_Load(&state_) & kNumReadersMask) > 0) {
+      boost::detail::yield(loop_count++);
+    }
+  }
+
+ private:
+  volatile Atomic32 state_;
+#ifndef NDEBUG
+  int64_t writer_tid_;
+#endif // NDEBUG
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_RW_SEMAPHORE_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rwc_lock-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rwc_lock-test.cc b/be/src/kudu/util/rwc_lock-test.cc
new file mode 100644
index 0000000..17c2b19
--- /dev/null
+++ b/be/src/kudu/util/rwc_lock-test.cc
@@ -0,0 +1,147 @@
+// 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 <mutex>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/rwc_lock.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+using base::subtle::NoBarrier_Load;
+using base::subtle::Release_Store;
+using std::string;
+using std::thread;
+using std::vector;
+
+class RWCLockTest : public KuduTest {};
+
+// Holds counters of how many threads hold the lock in each of the
+// provided modes.
+struct LockHoldersCount {
+  LockHoldersCount()
+    : num_readers(0),
+      num_writers(0),
+      num_committers(0) {
+  }
+
+  // Check the invariants of the lock counts.
+  void CheckInvariants() {
+    // At no time should we have more than one writer or committer.
+    CHECK_LE(num_writers, 1);
+    CHECK_LE(num_committers, 1);
+
+    // If we have any readers, then we should not have any committers.
+    if (num_readers > 0) {
+      CHECK_EQ(num_committers, 0);
+    }
+  }
+
+  void AdjustReaders(int delta) {
+    std::lock_guard<simple_spinlock> l(lock);
+    num_readers += delta;
+    CheckInvariants();
+  }
+
+  void AdjustWriters(int delta) {
+    std::lock_guard<simple_spinlock> l(lock);
+    num_writers += delta;
+    CheckInvariants();
+  }
+
+  void AdjustCommitters(int delta) {
+    std::lock_guard<simple_spinlock> l(lock);
+    num_committers += delta;
+    CheckInvariants();
+  }
+
+  int num_readers;
+  int num_writers;
+  int num_committers;
+  simple_spinlock lock;
+};
+
+struct SharedState {
+  LockHoldersCount counts;
+  RWCLock rwc_lock;
+  Atomic32 stop;
+};
+
+void ReaderThread(SharedState* state) {
+  while (!NoBarrier_Load(&state->stop)) {
+    state->rwc_lock.ReadLock();
+    state->counts.AdjustReaders(1);
+    state->counts.AdjustReaders(-1);
+    state->rwc_lock.ReadUnlock();
+  }
+}
+
+void WriterThread(SharedState* state) {
+  string local_str;
+  while (!NoBarrier_Load(&state->stop)) {
+    state->rwc_lock.WriteLock();
+    state->counts.AdjustWriters(1);
+
+    state->rwc_lock.UpgradeToCommitLock();
+    state->counts.AdjustWriters(-1);
+    state->counts.AdjustCommitters(1);
+
+    state->counts.AdjustCommitters(-1);
+    state->rwc_lock.CommitUnlock();
+  }
+}
+
+
+TEST_F(RWCLockTest, TestCorrectBehavior) {
+  SharedState state;
+  Release_Store(&state.stop, 0);
+
+  vector<thread> threads;
+
+  const int kNumWriters = 5;
+  const int kNumReaders = 5;
+
+  for (int i = 0; i < kNumWriters; i++) {
+    threads.emplace_back(WriterThread, &state);
+  }
+  for (int i = 0; i < kNumReaders; i++) {
+    threads.emplace_back(ReaderThread, &state);
+  }
+
+  if (AllowSlowTests()) {
+    SleepFor(MonoDelta::FromSeconds(1));
+  } else {
+    SleepFor(MonoDelta::FromMilliseconds(100));
+  }
+
+  Release_Store(&state.stop, 1);
+
+  for (thread& t : threads) {
+    t.join();
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rwc_lock.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rwc_lock.cc b/be/src/kudu/util/rwc_lock.cc
new file mode 100644
index 0000000..ffe4cbb
--- /dev/null
+++ b/be/src/kudu/util/rwc_lock.cc
@@ -0,0 +1,136 @@
+// 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 "kudu/util/rwc_lock.h"
+
+#include <glog/logging.h>
+
+#ifndef NDEBUG
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/thread.h"
+#endif // NDEBUG
+
+namespace kudu {
+
+RWCLock::RWCLock()
+  : no_mutators_(&lock_),
+    no_readers_(&lock_),
+    reader_count_(0),
+#ifdef NDEBUG
+    write_locked_(false) {
+#else
+    write_locked_(false),
+    writer_tid_(0),
+    last_writelock_acquire_time_(0) {
+  last_writer_backtrace_[0] = '\0';
+#endif // NDEBUG
+}
+
+RWCLock::~RWCLock() {
+  DCHECK(!HasReaders());
+  DCHECK(!HasWriteLock());
+}
+
+void RWCLock::ReadLock() {
+  MutexLock l(lock_);
+  reader_count_++;
+}
+
+void RWCLock::ReadUnlock() {
+  MutexLock l(lock_);
+  DCHECK(HasReadersUnlocked());
+  reader_count_--;
+  if (reader_count_ == 0) {
+    no_readers_.Signal();
+  }
+}
+
+bool RWCLock::HasReaders() const {
+  MutexLock l(lock_);
+  return HasReadersUnlocked();
+}
+
+bool RWCLock::HasReadersUnlocked() const {
+  lock_.AssertAcquired();
+  return reader_count_ > 0;
+}
+
+bool RWCLock::HasWriteLock() const {
+  MutexLock l(lock_);
+  return HasWriteLockUnlocked();
+}
+
+bool RWCLock::HasWriteLockUnlocked() const {
+  lock_.AssertAcquired();
+#ifndef NDEBUG
+  return writer_tid_ == Thread::CurrentThreadId();
+#else
+  return write_locked_;
+#endif
+}
+
+void RWCLock::WriteLock() {
+  MutexLock l(lock_);
+  // Wait for any other mutations to finish.
+  while (write_locked_) {
+    no_mutators_.Wait();
+  }
+#ifndef NDEBUG
+  last_writelock_acquire_time_ = GetCurrentTimeMicros();
+  writer_tid_ = Thread::CurrentThreadId();
+  HexStackTraceToString(last_writer_backtrace_, kBacktraceBufSize);
+#endif // NDEBUG
+  write_locked_ = true;
+}
+
+void RWCLock::WriteUnlock() {
+  MutexLock l(lock_);
+  DCHECK(HasWriteLockUnlocked());
+  write_locked_ = false;
+#ifndef NDEBUG
+  writer_tid_ = 0;
+  last_writer_backtrace_[0] = '\0';
+#endif // NDEBUG
+  no_mutators_.Signal();
+}
+
+void RWCLock::UpgradeToCommitLock() {
+  lock_.lock();
+  DCHECK(HasWriteLockUnlocked());
+  while (reader_count_ > 0) {
+    no_readers_.Wait();
+  }
+  DCHECK(HasWriteLockUnlocked());
+
+  // Leaves the lock held, which prevents any new readers
+  // or writers.
+}
+
+void RWCLock::CommitUnlock() {
+  DCHECK(!HasReadersUnlocked());
+  DCHECK(HasWriteLockUnlocked());
+  write_locked_ = false;
+#ifndef NDEBUG
+  writer_tid_ = 0;
+  last_writer_backtrace_[0] = '\0';
+#endif // NDEBUG
+  no_mutators_.Broadcast();
+  lock_.unlock();
+}
+
+} // namespace kudu


[29/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bit-stream-utils.inline.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bit-stream-utils.inline.h b/be/src/kudu/util/bit-stream-utils.inline.h
new file mode 100644
index 0000000..d168bda
--- /dev/null
+++ b/be/src/kudu/util/bit-stream-utils.inline.h
@@ -0,0 +1,211 @@
+// 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.
+#ifndef IMPALA_UTIL_BIT_STREAM_UTILS_INLINE_H
+#define IMPALA_UTIL_BIT_STREAM_UTILS_INLINE_H
+
+#include <algorithm>
+
+#include "glog/logging.h"
+#include "kudu/util/bit-stream-utils.h"
+#include "kudu/util/alignment.h"
+
+namespace kudu {
+
+inline void BitWriter::PutValue(uint64_t v, int num_bits) {
+  DCHECK_LE(num_bits, 64);
+  // Truncate the higher-order bits. This is necessary to
+  // support signed values.
+  v &= ~0ULL >> (64 - num_bits);
+
+
+  buffered_values_ |= v << bit_offset_;
+  bit_offset_ += num_bits;
+
+  if (PREDICT_FALSE(bit_offset_ >= 64)) {
+    // Flush buffered_values_ and write out bits of v that did not fit
+    buffer_->reserve(KUDU_ALIGN_UP(byte_offset_ + 8, 8));
+    buffer_->resize(byte_offset_ + 8);
+    DCHECK_LE(byte_offset_ + 8, buffer_->capacity());
+    memcpy(buffer_->data() + byte_offset_, &buffered_values_, 8);
+    buffered_values_ = 0;
+    byte_offset_ += 8;
+    bit_offset_ -= 64;
+    buffered_values_ = BitUtil::ShiftRightZeroOnOverflow(v, (num_bits - bit_offset_));
+  }
+  DCHECK_LT(bit_offset_, 64);
+}
+
+inline void BitWriter::Flush(bool align) {
+  int num_bytes = BitUtil::Ceil(bit_offset_, 8);
+  buffer_->reserve(KUDU_ALIGN_UP(byte_offset_ + num_bytes, 8));
+  buffer_->resize(byte_offset_ + num_bytes);
+  DCHECK_LE(byte_offset_ + num_bytes, buffer_->capacity());
+  memcpy(buffer_->data() + byte_offset_, &buffered_values_, num_bytes);
+
+  if (align) {
+    buffered_values_ = 0;
+    byte_offset_ += num_bytes;
+    bit_offset_ = 0;
+  }
+}
+
+inline uint8_t* BitWriter::GetNextBytePtr(int num_bytes) {
+  Flush(/* align */ true);
+  buffer_->reserve(KUDU_ALIGN_UP(byte_offset_ + num_bytes, 8));
+  buffer_->resize(byte_offset_ + num_bytes);
+  uint8_t* ptr = buffer_->data() + byte_offset_;
+  byte_offset_ += num_bytes;
+  DCHECK_LE(byte_offset_, buffer_->capacity());
+  return ptr;
+}
+
+template<typename T>
+inline void BitWriter::PutAligned(T val, int num_bytes) {
+  DCHECK_LE(num_bytes, sizeof(T));
+  uint8_t* ptr = GetNextBytePtr(num_bytes);
+  memcpy(ptr, &val, num_bytes);
+}
+
+inline void BitWriter::PutVlqInt(int32_t v) {
+  while ((v & 0xFFFFFF80) != 0L) {
+    PutAligned<uint8_t>((v & 0x7F) | 0x80, 1);
+    v >>= 7;
+  }
+  PutAligned<uint8_t>(v & 0x7F, 1);
+}
+
+
+inline BitReader::BitReader(const uint8_t* buffer, int buffer_len)
+  : buffer_(buffer),
+    max_bytes_(buffer_len),
+    buffered_values_(0),
+    byte_offset_(0),
+    bit_offset_(0) {
+  int num_bytes = std::min(8, max_bytes_);
+  memcpy(&buffered_values_, buffer_ + byte_offset_, num_bytes);
+}
+
+inline void BitReader::BufferValues() {
+  int bytes_remaining = max_bytes_ - byte_offset_;
+  if (PREDICT_TRUE(bytes_remaining >= 8)) {
+    memcpy(&buffered_values_, buffer_ + byte_offset_, 8);
+  } else {
+    memcpy(&buffered_values_, buffer_ + byte_offset_, bytes_remaining);
+  }
+}
+
+template<typename T>
+inline bool BitReader::GetValue(int num_bits, T* v) {
+  DCHECK_LE(num_bits, 64);
+  DCHECK_LE(num_bits, sizeof(T) * 8);
+
+  if (PREDICT_FALSE(byte_offset_ * 8 + bit_offset_ + num_bits > max_bytes_ * 8)) return false;
+
+  *v = BitUtil::TrailingBits(buffered_values_, bit_offset_ + num_bits) >> bit_offset_;
+
+  bit_offset_ += num_bits;
+  if (bit_offset_ >= 64) {
+    byte_offset_ += 8;
+    bit_offset_ -= 64;
+    BufferValues();
+    // Read bits of v that crossed into new buffered_values_
+    *v |= BitUtil::ShiftLeftZeroOnOverflow(
+        BitUtil::TrailingBits(buffered_values_, bit_offset_),
+        (num_bits - bit_offset_));
+  }
+  DCHECK_LE(bit_offset_, 64);
+  return true;
+}
+
+inline void BitReader::Rewind(int num_bits) {
+  bit_offset_ -= num_bits;
+  if (bit_offset_ >= 0) {
+    return;
+  }
+  while (bit_offset_ < 0) {
+    int seek_back = std::min(byte_offset_, 8);
+    byte_offset_ -= seek_back;
+    bit_offset_ += seek_back * 8;
+  }
+  // This should only be executed *if* rewinding by 'num_bits'
+  // make the existing buffered_values_ invalid
+  DCHECK_GE(byte_offset_, 0); // Check for underflow
+  memcpy(&buffered_values_, buffer_ + byte_offset_, 8);
+}
+
+inline void BitReader::SeekToBit(uint stream_position) {
+  DCHECK_LE(stream_position, max_bytes_ * 8);
+
+  int delta = static_cast<int>(stream_position) - position();
+  if (delta == 0) {
+    return;
+  } else if (delta < 0) {
+    Rewind(position() - stream_position);
+  } else {
+    bit_offset_ += delta;
+    while (bit_offset_ >= 64) {
+      byte_offset_ +=8;
+      bit_offset_ -= 64;
+      if (bit_offset_ < 64) {
+        // This should only be executed if seeking to
+        // 'stream_position' makes the existing buffered_values_
+        // invalid.
+        BufferValues();
+      }
+    }
+  }
+}
+
+template<typename T>
+inline bool BitReader::GetAligned(int num_bytes, T* v) {
+  DCHECK_LE(num_bytes, sizeof(T));
+  int bytes_read = BitUtil::Ceil(bit_offset_, 8);
+  if (PREDICT_FALSE(byte_offset_ + bytes_read + num_bytes > max_bytes_)) return false;
+
+  // Advance byte_offset to next unread byte and read num_bytes
+  byte_offset_ += bytes_read;
+  memcpy(v, buffer_ + byte_offset_, num_bytes);
+  byte_offset_ += num_bytes;
+
+  // Reset buffered_values_
+  bit_offset_ = 0;
+  int bytes_remaining = max_bytes_ - byte_offset_;
+  if (PREDICT_TRUE(bytes_remaining >= 8)) {
+    memcpy(&buffered_values_, buffer_ + byte_offset_, 8);
+  } else {
+    memcpy(&buffered_values_, buffer_ + byte_offset_, bytes_remaining);
+  }
+  return true;
+}
+
+inline bool BitReader::GetVlqInt(int32_t* v) {
+  *v = 0;
+  int shift = 0;
+  int num_bytes = 0;
+  uint8_t byte = 0;
+  do {
+    if (!GetAligned<uint8_t>(1, &byte)) return false;
+    *v |= (byte & 0x7F) << shift;
+    shift += 7;
+    DCHECK_LE(++num_bytes, MAX_VLQ_BYTE_LEN);
+  } while ((byte & 0x80) != 0);
+  return true;
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bit-util-test.cc b/be/src/kudu/util/bit-util-test.cc
new file mode 100644
index 0000000..0d8eab4
--- /dev/null
+++ b/be/src/kudu/util/bit-util-test.cc
@@ -0,0 +1,45 @@
+// 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 <boost/utility/binary.hpp>
+#include <gtest/gtest.h>
+#include "kudu/util/bit-util.h"
+
+namespace kudu {
+
+TEST(BitUtil, TrailingBits) {
+  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 0), 0);
+  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 1), 1);
+  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 64),
+            BOOST_BINARY(1 1 1 1 1 1 1 1));
+  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 100),
+            BOOST_BINARY(1 1 1 1 1 1 1 1));
+  EXPECT_EQ(BitUtil::TrailingBits(0, 1), 0);
+  EXPECT_EQ(BitUtil::TrailingBits(0, 64), 0);
+  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 0), 0);
+  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 63), 0);
+  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 64), 1LL << 63);
+
+}
+
+TEST(BitUtil, ShiftBits) {
+  EXPECT_EQ(BitUtil::ShiftLeftZeroOnOverflow(1ULL, 64), 0ULL);
+  EXPECT_EQ(BitUtil::ShiftLeftZeroOnOverflow(0xFFFFFFFFFFFFFFFFULL, 32), 0xFFFFFFFF00000000ULL);
+  EXPECT_EQ(BitUtil::ShiftRightZeroOnOverflow(1ULL, 64), 0ULL);
+  EXPECT_EQ(BitUtil::ShiftRightZeroOnOverflow(0xFFFFFFFFFFFFFFFFULL, 32), 0x00000000FFFFFFFFULL);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bit-util.h b/be/src/kudu/util/bit-util.h
new file mode 100644
index 0000000..5f36887
--- /dev/null
+++ b/be/src/kudu/util/bit-util.h
@@ -0,0 +1,57 @@
+// 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.
+#ifndef IMPALA_BIT_UTIL_H
+#define IMPALA_BIT_UTIL_H
+
+#include <stdint.h>
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+// Utility class to do standard bit tricks
+// TODO: is this in boost or something else like that?
+class BitUtil {
+ public:
+  // Returns the ceil of value/divisor
+  static inline int Ceil(int value, int divisor) {
+    return value / divisor + (value % divisor != 0);
+  }
+
+  // Returns the 'num_bits' least-significant bits of 'v'.
+  static inline uint64_t TrailingBits(uint64_t v, int num_bits) {
+    if (PREDICT_FALSE(num_bits == 0)) return 0;
+    if (PREDICT_FALSE(num_bits >= 64)) return v;
+    int n = 64 - num_bits;
+    return (v << n) >> n;
+  }
+
+  static inline uint64_t ShiftLeftZeroOnOverflow(uint64_t v, int num_bits) {
+    if (PREDICT_FALSE(num_bits >= 64)) return 0;
+    return v << num_bits;
+  }
+
+  static inline uint64_t ShiftRightZeroOnOverflow(uint64_t v, int num_bits) {
+    if (PREDICT_FALSE(num_bits >= 64)) return 0;
+    return v >> num_bits;
+  }
+
+
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bitmap-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bitmap-test.cc b/be/src/kudu/util/bitmap-test.cc
new file mode 100644
index 0000000..089ed3b
--- /dev/null
+++ b/be/src/kudu/util/bitmap-test.cc
@@ -0,0 +1,230 @@
+// 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 <cstdint>
+#include <cstring>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/join.h"
+#include "kudu/util/bitmap.h"
+
+namespace kudu {
+
+static int ReadBackBitmap(uint8_t *bm, size_t bits,
+                           std::vector<size_t> *result) {
+  int iters = 0;
+  for (TrueBitIterator iter(bm, bits);
+       !iter.done();
+       ++iter) {
+    size_t val = *iter;
+    result->push_back(val);
+
+    iters++;
+  }
+  return iters;
+}
+
+TEST(TestBitMap, TestIteration) {
+  uint8_t bm[8];
+  memset(bm, 0, sizeof(bm));
+  BitmapSet(bm, 0);
+  BitmapSet(bm, 8);
+  BitmapSet(bm, 31);
+  BitmapSet(bm, 32);
+  BitmapSet(bm, 33);
+  BitmapSet(bm, 63);
+
+  EXPECT_EQ("   0: 10000000 10000000 00000000 00000001 11000000 00000000 00000000 00000001 \n",
+            BitmapToString(bm, sizeof(bm) * 8));
+
+  std::vector<size_t> read_back;
+
+  int iters = ReadBackBitmap(bm, sizeof(bm)*8, &read_back);
+  ASSERT_EQ(6, iters);
+  ASSERT_EQ("0,8,31,32,33,63", JoinElements(read_back, ","));
+}
+
+
+TEST(TestBitMap, TestIteration2) {
+  uint8_t bm[1];
+  memset(bm, 0, sizeof(bm));
+  BitmapSet(bm, 1);
+
+  std::vector<size_t> read_back;
+
+  int iters = ReadBackBitmap(bm, 3, &read_back);
+  ASSERT_EQ(1, iters);
+  ASSERT_EQ("1", JoinElements(read_back, ","));
+}
+
+TEST(TestBitmap, TestSetAndTestBits) {
+  uint8_t bm[1];
+  memset(bm, 0, sizeof(bm));
+
+  size_t num_bits = sizeof(bm) * 8;
+  for (size_t i = 0; i < num_bits; i++) {
+    ASSERT_FALSE(BitmapTest(bm, i));
+
+    BitmapSet(bm, i);
+    ASSERT_TRUE(BitmapTest(bm, i));
+
+    BitmapClear(bm, i);
+    ASSERT_FALSE(BitmapTest(bm, i));
+
+    BitmapChange(bm, i, true);
+    ASSERT_TRUE(BitmapTest(bm, i));
+
+    BitmapChange(bm, i, false);
+    ASSERT_FALSE(BitmapTest(bm, i));
+  }
+
+  // Set the other bit: 01010101
+  for (size_t i = 0; i < num_bits; ++i) {
+    ASSERT_FALSE(BitmapTest(bm, i));
+    if (i & 1) BitmapSet(bm, i);
+  }
+
+  // Check and Clear the other bit: 0000000
+  for (size_t i = 0; i < num_bits; ++i) {
+    ASSERT_EQ(!!(i & 1), BitmapTest(bm, i));
+    if (i & 1) BitmapClear(bm, i);
+  }
+
+  // Check if bits are zero and change the other to one
+  for (size_t i = 0; i < num_bits; ++i) {
+    ASSERT_FALSE(BitmapTest(bm, i));
+    BitmapChange(bm, i, i & 1);
+  }
+
+  // Check the bits change them again
+  for (size_t i = 0; i < num_bits; ++i) {
+    ASSERT_EQ(!!(i & 1), BitmapTest(bm, i));
+    BitmapChange(bm, i, !(i & 1));
+  }
+
+  // Check the last setup
+  for (size_t i = 0; i < num_bits; ++i) {
+    ASSERT_EQ(!(i & 1), BitmapTest(bm, i));
+  }
+}
+
+TEST(TestBitMap, TestBulkSetAndTestBits) {
+  uint8_t bm[16];
+  size_t total_size = sizeof(bm) * 8;
+
+  // Test Bulk change bits and test bits
+  for (int i = 0; i < 4; ++i) {
+    bool value = i & 1;
+    size_t num_bits = total_size;
+    while (num_bits > 0) {
+      for (size_t offset = 0; offset < num_bits; ++offset) {
+        BitmapChangeBits(bm, 0, total_size, !value);
+        BitmapChangeBits(bm, offset, num_bits - offset, value);
+
+        ASSERT_EQ(value, BitMapIsAllSet(bm, offset, num_bits));
+        ASSERT_EQ(!value, BitmapIsAllZero(bm, offset, num_bits));
+
+        if (offset > 1) {
+          ASSERT_EQ(value, BitmapIsAllZero(bm, 0, offset - 1));
+          ASSERT_EQ(!value, BitMapIsAllSet(bm, 0, offset - 1));
+        }
+
+        if ((offset + num_bits) < total_size) {
+          ASSERT_EQ(value, BitmapIsAllZero(bm, num_bits, total_size));
+          ASSERT_EQ(!value, BitMapIsAllSet(bm, num_bits, total_size));
+        }
+      }
+      num_bits--;
+    }
+  }
+}
+
+TEST(TestBitMap, TestFindBit) {
+  uint8_t bm[16];
+
+  size_t num_bits = sizeof(bm) * 8;
+  BitmapChangeBits(bm, 0, num_bits, false);
+  while (num_bits > 0) {
+    for (size_t offset = 0; offset < num_bits; ++offset) {
+      size_t idx;
+      ASSERT_FALSE(BitmapFindFirstSet(bm, offset, num_bits, &idx));
+      ASSERT_TRUE(BitmapFindFirstZero(bm, offset, num_bits, &idx));
+      ASSERT_EQ(idx, offset);
+    }
+    num_bits--;
+  }
+
+  num_bits = sizeof(bm) * 8;
+  for (int i = 0; i < num_bits; ++i) {
+    BitmapChange(bm, i, i & 3);
+  }
+
+  for (; num_bits > 0; num_bits--) {
+    for (size_t offset = 0; offset < num_bits; ++offset) {
+      size_t idx;
+
+      // Find a set bit
+      bool res = BitmapFindFirstSet(bm, offset, num_bits, &idx);
+      size_t expected_set_idx = (offset + !(offset & 3));
+      bool expect_set_found = (expected_set_idx < num_bits);
+      ASSERT_EQ(expect_set_found, res);
+      if (expect_set_found) {
+        ASSERT_EQ(expected_set_idx, idx);
+      }
+
+      // Find a zero bit
+      res = BitmapFindFirstZero(bm, offset, num_bits, &idx);
+      size_t expected_zero_idx = offset + ((offset & 3) ? (4 - (offset & 3)) : 0);
+      bool expect_zero_found = (expected_zero_idx < num_bits);
+      ASSERT_EQ(expect_zero_found, res);
+      if (expect_zero_found) {
+        ASSERT_EQ(expected_zero_idx, idx);
+      }
+    }
+  }
+}
+
+TEST(TestBitMap, TestBitmapIteration) {
+  uint8_t bm[8];
+  memset(bm, 0, sizeof(bm));
+  BitmapSet(bm, 0);
+  BitmapSet(bm, 8);
+  BitmapSet(bm, 31);
+  BitmapSet(bm, 32);
+  BitmapSet(bm, 33);
+  BitmapSet(bm, 63);
+
+  BitmapIterator biter(bm, sizeof(bm) * 8);
+
+  size_t i = 0;
+  size_t size;
+  bool value = false;
+  bool expected_value = true;
+  size_t expected_sizes[] = {1, 7, 1, 22, 3, 29, 1, 0};
+  while ((size = biter.Next(&value)) > 0) {
+    ASSERT_LT(i, 8);
+    ASSERT_EQ(expected_value, value);
+    ASSERT_EQ(expected_sizes[i], size);
+    expected_value = !expected_value;
+    i++;
+  }
+  ASSERT_EQ(expected_sizes[i], size);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bitmap.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bitmap.cc b/be/src/kudu/util/bitmap.cc
new file mode 100644
index 0000000..eed7880
--- /dev/null
+++ b/be/src/kudu/util/bitmap.cc
@@ -0,0 +1,136 @@
+// 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 "kudu/util/bitmap.h"
+
+#include <cstring>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/stringprintf.h"
+
+namespace kudu {
+
+void BitmapChangeBits(uint8_t *bitmap, size_t offset, size_t num_bits, bool value) {
+  DCHECK_GT(num_bits, 0);
+
+  size_t start_byte = (offset >> 3);
+  size_t end_byte = (offset + num_bits - 1) >> 3;
+  int single_byte = (start_byte == end_byte);
+
+  // Change the last bits of the first byte
+  size_t left = offset & 0x7;
+  size_t right = (single_byte) ? (left + num_bits) : 8;
+  uint8_t mask = ((0xff << left) & (0xff >> (8 - right)));
+  if (value) {
+    bitmap[start_byte++] |= mask;
+  } else {
+    bitmap[start_byte++] &= ~mask;
+  }
+
+  // Nothing left... I'm done
+  if (single_byte) {
+    return;
+  }
+
+  // change the middle bits
+  if (end_byte > start_byte) {
+    const uint8_t pattern8[2] = { 0x00, 0xff };
+    memset(bitmap + start_byte, pattern8[value], end_byte - start_byte);
+  }
+
+  // change the first bits of the last byte
+  right = offset + num_bits - (end_byte << 3);
+  mask = (0xff >> (8 - right));
+  if (value) {
+    bitmap[end_byte] |= mask;
+  } else {
+    bitmap[end_byte] &= ~mask;
+  }
+}
+
+bool BitmapFindFirst(const uint8_t *bitmap, size_t offset, size_t bitmap_size,
+                     bool value, size_t *idx) {
+  const uint64_t pattern64[2] = { 0xffffffffffffffff, 0x0000000000000000 };
+  const uint8_t pattern8[2] = { 0xff, 0x00 };
+  size_t bit;
+
+  DCHECK_LE(offset, bitmap_size);
+
+  // Jump to the byte at specified offset
+  const uint8_t *p = bitmap + (offset >> 3);
+  size_t num_bits = bitmap_size - offset;
+
+  // Find a 'value' bit at the end of the first byte
+  if ((bit = offset & 0x7)) {
+    for (; bit < 8 && num_bits > 0; ++bit) {
+      if (BitmapTest(p, bit) == value) {
+        *idx = ((p - bitmap) << 3) + bit;
+        return true;
+      }
+
+      num_bits--;
+    }
+
+    p++;
+  }
+
+  // check 64bit at the time for a 'value' bit
+  const uint64_t *u64 = (const uint64_t *)p;
+  while (num_bits >= 64 && *u64 == pattern64[value]) {
+    num_bits -= 64;
+    u64++;
+  }
+
+  // check 8bit at the time for a 'value' bit
+  p = (const uint8_t *)u64;
+  while (num_bits >= 8 && *p == pattern8[value]) {
+    num_bits -= 8;
+    p++;
+  }
+
+  // Find a 'value' bit at the beginning of the last byte
+  for (bit = 0; num_bits > 0; ++bit) {
+    if (BitmapTest(p, bit) == value) {
+      *idx = ((p - bitmap) << 3) + bit;
+      return true;
+    }
+    num_bits--;
+  }
+
+  return false;
+}
+
+std::string BitmapToString(const uint8_t *bitmap, size_t num_bits) {
+  std::string s;
+  size_t index = 0;
+  while (index < num_bits) {
+    StringAppendF(&s, "%4zu: ", index);
+    for (int i = 0; i < 8 && index < num_bits; ++i) {
+      for (int j = 0; j < 8 && index < num_bits; ++j) {
+        StringAppendF(&s, "%d", BitmapTest(bitmap, index));
+        index++;
+      }
+      StringAppendF(&s, " ");
+    }
+    StringAppendF(&s, "\n");
+  }
+  return s;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bitmap.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bitmap.h b/be/src/kudu/util/bitmap.h
new file mode 100644
index 0000000..d9f5260
--- /dev/null
+++ b/be/src/kudu/util/bitmap.h
@@ -0,0 +1,219 @@
+// 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.
+//
+// Utility functions for dealing with a byte array as if it were a bitmap.
+#ifndef KUDU_UTIL_BITMAP_H
+#define KUDU_UTIL_BITMAP_H
+
+#include <cstddef>
+#include <cstdint>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/bits.h"
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+// Return the number of bytes necessary to store the given number of bits.
+inline size_t BitmapSize(size_t num_bits) {
+  return (num_bits + 7) / 8;
+}
+
+// Set the given bit.
+inline void BitmapSet(uint8_t *bitmap, size_t idx) {
+  bitmap[idx >> 3] |= 1 << (idx & 7);
+}
+
+// Switch the given bit to the specified value.
+inline void BitmapChange(uint8_t *bitmap, size_t idx, bool value) {
+  bitmap[idx >> 3] = (bitmap[idx >> 3] & ~(1 << (idx & 7))) | ((!!value) << (idx & 7));
+}
+
+// Clear the given bit.
+inline void BitmapClear(uint8_t *bitmap, size_t idx) {
+  bitmap[idx >> 3] &= ~(1 << (idx & 7));
+}
+
+// Test/get the given bit.
+inline bool BitmapTest(const uint8_t *bitmap, size_t idx) {
+  return bitmap[idx >> 3] & (1 << (idx & 7));
+}
+
+// Merge the two bitmaps using bitwise or. Both bitmaps should have at least
+// n_bits valid bits.
+inline void BitmapMergeOr(uint8_t *dst, const uint8_t *src, size_t n_bits) {
+  size_t n_bytes = BitmapSize(n_bits);
+  for (size_t i = 0; i < n_bytes; i++) {
+    *dst++ |= *src++;
+  }
+}
+
+// Set bits from offset to (offset + num_bits) to the specified value
+void BitmapChangeBits(uint8_t *bitmap, size_t offset, size_t num_bits, bool value);
+
+// Find the first bit of the specified value, starting from the specified offset.
+bool BitmapFindFirst(const uint8_t *bitmap, size_t offset, size_t bitmap_size,
+                     bool value, size_t *idx);
+
+// Find the first set bit in the bitmap, at the specified offset.
+inline bool BitmapFindFirstSet(const uint8_t *bitmap, size_t offset,
+                               size_t bitmap_size, size_t *idx) {
+  return BitmapFindFirst(bitmap, offset, bitmap_size, true, idx);
+}
+
+// Find the first zero bit in the bitmap, at the specified offset.
+inline bool BitmapFindFirstZero(const uint8_t *bitmap, size_t offset,
+                                size_t bitmap_size, size_t *idx) {
+  return BitmapFindFirst(bitmap, offset, bitmap_size, false, idx);
+}
+
+// Returns true if the bitmap contains only ones.
+inline bool BitMapIsAllSet(const uint8_t *bitmap, size_t offset, size_t bitmap_size) {
+  DCHECK_LT(offset, bitmap_size);
+  size_t idx;
+  return !BitmapFindFirstZero(bitmap, offset, bitmap_size, &idx);
+}
+
+// Returns true if the bitmap contains only zeros.
+inline bool BitmapIsAllZero(const uint8_t *bitmap, size_t offset, size_t bitmap_size) {
+  DCHECK_LT(offset, bitmap_size);
+  size_t idx;
+  return !BitmapFindFirstSet(bitmap, offset, bitmap_size, &idx);
+}
+
+std::string BitmapToString(const uint8_t *bitmap, size_t num_bits);
+
+// Iterator which yields ranges of set and unset bits.
+// Example usage:
+//   bool value;
+//   size_t size;
+//   BitmapIterator iter(bitmap, n_bits);
+//   while ((size = iter.Next(&value))) {
+//      printf("bitmap block len=%lu value=%d\n", size, value);
+//   }
+class BitmapIterator {
+ public:
+  BitmapIterator(const uint8_t *map, size_t num_bits)
+    : offset_(0), num_bits_(num_bits), map_(map)
+  {}
+
+  bool done() const {
+    return (num_bits_ - offset_) == 0;
+  }
+
+  void SeekTo(size_t bit) {
+    DCHECK_LE(bit, num_bits_);
+    offset_ = bit;
+  }
+
+  size_t Next(bool *value) {
+    size_t len = num_bits_ - offset_;
+    if (PREDICT_FALSE(len == 0))
+      return(0);
+
+    *value = BitmapTest(map_, offset_);
+
+    size_t index;
+    if (BitmapFindFirst(map_, offset_, num_bits_, !(*value), &index)) {
+      len = index - offset_;
+    } else {
+      index = num_bits_;
+    }
+
+    offset_ = index;
+    return len;
+  }
+
+ private:
+  size_t offset_;
+  size_t num_bits_;
+  const uint8_t *map_;
+};
+
+// Iterator which yields the set bits in a bitmap.
+// Example usage:
+//   for (TrueBitIterator iter(bitmap, n_bits);
+//        !iter.done();
+//        ++iter) {
+//     int next_onebit_position = *iter;
+//   }
+class TrueBitIterator {
+ public:
+  TrueBitIterator(const uint8_t *bitmap, size_t n_bits)
+    : bitmap_(bitmap),
+      cur_byte_(0),
+      cur_byte_idx_(0),
+      n_bits_(n_bits),
+      n_bytes_(BitmapSize(n_bits_)),
+      bit_idx_(0) {
+    if (n_bits_ == 0) {
+      cur_byte_idx_ = 1; // sets done
+    } else {
+      cur_byte_ = bitmap[0];
+      AdvanceToNextOneBit();
+    }
+  }
+
+  TrueBitIterator &operator ++() {
+    DCHECK(!done());
+    DCHECK(cur_byte_ & 1);
+    cur_byte_ &= (~1);
+    AdvanceToNextOneBit();
+    return *this;
+  }
+
+  bool done() const {
+    return cur_byte_idx_ >= n_bytes_;
+  }
+
+  size_t operator *() const {
+    DCHECK(!done());
+    return bit_idx_;
+  }
+
+ private:
+  void AdvanceToNextOneBit() {
+    while (cur_byte_ == 0) {
+      cur_byte_idx_++;
+      if (cur_byte_idx_ >= n_bytes_) return;
+      cur_byte_ = bitmap_[cur_byte_idx_];
+      bit_idx_ = cur_byte_idx_ * 8;
+    }
+    DVLOG(2) << "Found next nonzero byte at " << cur_byte_idx_
+             << " val=" << cur_byte_;
+
+    DCHECK_NE(cur_byte_, 0);
+    int set_bit = Bits::FindLSBSetNonZero(cur_byte_);
+    bit_idx_ += set_bit;
+    cur_byte_ >>= set_bit;
+  }
+
+  const uint8_t *bitmap_;
+  uint8_t cur_byte_;
+  uint8_t cur_byte_idx_;
+
+  const size_t n_bits_;
+  const size_t n_bytes_;
+  size_t bit_idx_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/blocking_queue-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/blocking_queue-test.cc b/be/src/kudu/util/blocking_queue-test.cc
new file mode 100644
index 0000000..a2271ff
--- /dev/null
+++ b/be/src/kudu/util/blocking_queue-test.cc
@@ -0,0 +1,249 @@
+// 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 <cstddef>
+#include <cstdint>
+#include <map>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/blocking_queue.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using std::string;
+using std::thread;
+using std::vector;
+
+namespace kudu {
+
+BlockingQueue<int32_t> test1_queue(5);
+
+void InsertSomeThings() {
+  ASSERT_EQ(test1_queue.Put(1), QUEUE_SUCCESS);
+  ASSERT_EQ(test1_queue.Put(2), QUEUE_SUCCESS);
+  ASSERT_EQ(test1_queue.Put(3), QUEUE_SUCCESS);
+}
+
+TEST(BlockingQueueTest, Test1) {
+  thread inserter_thread(InsertSomeThings);
+  int32_t i;
+  ASSERT_TRUE(test1_queue.BlockingGet(&i));
+  ASSERT_EQ(1, i);
+  ASSERT_TRUE(test1_queue.BlockingGet(&i));
+  ASSERT_EQ(2, i);
+  ASSERT_TRUE(test1_queue.BlockingGet(&i));
+  ASSERT_EQ(3, i);
+  inserter_thread.join();
+}
+
+TEST(BlockingQueueTest, TestBlockingDrainTo) {
+  BlockingQueue<int32_t> test_queue(3);
+  ASSERT_EQ(test_queue.Put(1), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put(2), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put(3), QUEUE_SUCCESS);
+  vector<int32_t> out;
+  ASSERT_OK(test_queue.BlockingDrainTo(&out, MonoTime::Now() + MonoDelta::FromSeconds(30)));
+  ASSERT_EQ(1, out[0]);
+  ASSERT_EQ(2, out[1]);
+  ASSERT_EQ(3, out[2]);
+
+  // Set a deadline in the past and ensure we time out.
+  Status s = test_queue.BlockingDrainTo(&out, MonoTime::Now() - MonoDelta::FromSeconds(1));
+  ASSERT_TRUE(s.IsTimedOut());
+
+  // Ensure that if the queue is shut down, we get Aborted status.
+  test_queue.Shutdown();
+  s = test_queue.BlockingDrainTo(&out, MonoTime::Now() - MonoDelta::FromSeconds(1));
+  ASSERT_TRUE(s.IsAborted());
+}
+
+// Test that, when the queue is shut down with elements still pending,
+// Drain still returns OK until the elements are all gone.
+TEST(BlockingQueueTest, TestGetAndDrainAfterShutdown) {
+  // Put some elements into the queue and then shut it down.
+  BlockingQueue<int32_t> q(3);
+  ASSERT_EQ(q.Put(1), QUEUE_SUCCESS);
+  ASSERT_EQ(q.Put(2), QUEUE_SUCCESS);
+
+  q.Shutdown();
+
+  // Get() should still return an element.
+  int i;
+  ASSERT_TRUE(q.BlockingGet(&i));
+  ASSERT_EQ(1, i);
+
+  // Drain should still return OK, since it yielded elements.
+  vector<int32_t> out;
+  ASSERT_OK(q.BlockingDrainTo(&out));
+  ASSERT_EQ(2, out[0]);
+
+  // Now that it's empty, it should return Aborted.
+  Status s = q.BlockingDrainTo(&out);
+  ASSERT_TRUE(s.IsAborted()) << s.ToString();
+  ASSERT_FALSE(q.BlockingGet(&i));
+}
+
+TEST(BlockingQueueTest, TestTooManyInsertions) {
+  BlockingQueue<int32_t> test_queue(2);
+  ASSERT_EQ(test_queue.Put(123), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put(123), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put(123), QUEUE_FULL);
+}
+
+namespace {
+
+struct LengthLogicalSize {
+  static size_t logical_size(const string& s) {
+    return s.length();
+  }
+};
+
+} // anonymous namespace
+
+TEST(BlockingQueueTest, TestLogicalSize) {
+  BlockingQueue<string, LengthLogicalSize> test_queue(4);
+  ASSERT_EQ(test_queue.Put("a"), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put("bcd"), QUEUE_SUCCESS);
+  ASSERT_EQ(test_queue.Put("e"), QUEUE_FULL);
+}
+
+TEST(BlockingQueueTest, TestNonPointerParamsMayBeNonEmptyOnDestruct) {
+  BlockingQueue<int32_t> test_queue(1);
+  ASSERT_EQ(test_queue.Put(123), QUEUE_SUCCESS);
+  // No DCHECK failure on destruct.
+}
+
+#ifndef NDEBUG
+TEST(BlockingQueueDeathTest, TestPointerParamsMustBeEmptyOnDestruct) {
+  ::testing::FLAGS_gtest_death_test_style = "threadsafe";
+  ASSERT_DEATH({
+      BlockingQueue<int32_t*> test_queue(1);
+      int32_t element = 123;
+      ASSERT_EQ(test_queue.Put(&element), QUEUE_SUCCESS);
+      // Debug assertion triggered on queue destruction since type is a pointer.
+    },
+    "BlockingQueue holds bare pointers");
+}
+#endif // NDEBUG
+
+TEST(BlockingQueueTest, TestGetFromShutdownQueue) {
+  BlockingQueue<int64_t> test_queue(2);
+  ASSERT_EQ(test_queue.Put(123), QUEUE_SUCCESS);
+  test_queue.Shutdown();
+  ASSERT_EQ(test_queue.Put(456), QUEUE_SHUTDOWN);
+  int64_t i;
+  ASSERT_TRUE(test_queue.BlockingGet(&i));
+  ASSERT_EQ(123, i);
+  ASSERT_FALSE(test_queue.BlockingGet(&i));
+}
+
+TEST(BlockingQueueTest, TestGscopedPtrMethods) {
+  BlockingQueue<int*> test_queue(2);
+  gscoped_ptr<int> input_int(new int(123));
+  ASSERT_EQ(test_queue.Put(&input_int), QUEUE_SUCCESS);
+  gscoped_ptr<int> output_int;
+  ASSERT_TRUE(test_queue.BlockingGet(&output_int));
+  ASSERT_EQ(123, *output_int.get());
+  test_queue.Shutdown();
+}
+
+class MultiThreadTest {
+ public:
+  MultiThreadTest()
+   :  puts_(4),
+      blocking_puts_(4),
+      nthreads_(5),
+      queue_(nthreads_ * puts_),
+      num_inserters_(nthreads_),
+      sync_latch_(nthreads_) {
+  }
+
+  void InserterThread(int arg) {
+    for (int i = 0; i < puts_; i++) {
+      ASSERT_EQ(queue_.Put(arg), QUEUE_SUCCESS);
+    }
+    sync_latch_.CountDown();
+    sync_latch_.Wait();
+    for (int i = 0; i < blocking_puts_; i++) {
+      ASSERT_TRUE(queue_.BlockingPut(arg));
+    }
+    MutexLock guard(lock_);
+    if (--num_inserters_ == 0) {
+      queue_.Shutdown();
+    }
+  }
+
+  void RemoverThread() {
+    for (int i = 0; i < puts_ + blocking_puts_; i++) {
+      int32_t arg = 0;
+      bool got = queue_.BlockingGet(&arg);
+      if (!got) {
+        arg = -1;
+      }
+      MutexLock guard(lock_);
+      gotten_[arg] = gotten_[arg] + 1;
+    }
+  }
+
+  void Run() {
+    for (int i = 0; i < nthreads_; i++) {
+      threads_.emplace_back(&MultiThreadTest::InserterThread, this, i);
+      threads_.emplace_back(&MultiThreadTest::RemoverThread, this);
+    }
+    // We add an extra thread to ensure that there aren't enough elements in
+    // the queue to go around.  This way, we test removal after Shutdown.
+    threads_.emplace_back(&MultiThreadTest::RemoverThread, this);
+    for (auto& thread : threads_) {
+      thread.join();
+    }
+    // Let's check to make sure we got what we should have.
+    MutexLock guard(lock_);
+    for (int i = 0; i < nthreads_; i++) {
+      ASSERT_EQ(puts_ + blocking_puts_, gotten_[i]);
+    }
+    // And there were nthreads_ * (puts_ + blocking_puts_)
+    // elements removed, but only nthreads_ * puts_ +
+    // blocking_puts_ elements added.  So some removers hit the
+    // shutdown case.
+    ASSERT_EQ(puts_ + blocking_puts_, gotten_[-1]);
+  }
+
+  int puts_;
+  int blocking_puts_;
+  int nthreads_;
+  BlockingQueue<int32_t> queue_;
+  Mutex lock_;
+  std::map<int32_t, int> gotten_;
+  vector<thread> threads_;
+  int num_inserters_;
+  CountDownLatch sync_latch_;
+};
+
+TEST(BlockingQueueTest, TestMultipleThreads) {
+  MultiThreadTest test;
+  test.Run();
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/blocking_queue.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/blocking_queue.h b/be/src/kudu/util/blocking_queue.h
new file mode 100644
index 0000000..7331c12
--- /dev/null
+++ b/be/src/kudu/util/blocking_queue.h
@@ -0,0 +1,256 @@
+// 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.
+#ifndef KUDU_UTIL_BLOCKING_QUEUE_H
+#define KUDU_UTIL_BLOCKING_QUEUE_H
+
+#include <list>
+#include <string>
+#include <type_traits>
+#include <unistd.h>
+#include <vector>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Return values for BlockingQueue::Put()
+enum QueueStatus {
+  QUEUE_SUCCESS = 0,
+  QUEUE_SHUTDOWN = 1,
+  QUEUE_FULL = 2
+};
+
+// Default logical length implementation: always returns 1.
+struct DefaultLogicalSize {
+  template<typename T>
+  static size_t logical_size(const T& /* unused */) {
+    return 1;
+  }
+};
+
+template <typename T, class LOGICAL_SIZE = DefaultLogicalSize>
+class BlockingQueue {
+ public:
+  // If T is a pointer, this will be the base type.  If T is not a pointer, you
+  // can ignore this and the functions which make use of it.
+  // Template substitution failure is not an error.
+  typedef typename std::remove_pointer<T>::type T_VAL;
+
+  explicit BlockingQueue(size_t max_size)
+    : shutdown_(false),
+      size_(0),
+      max_size_(max_size),
+      not_empty_(&lock_),
+      not_full_(&lock_) {
+  }
+
+  // If the queue holds a bare pointer, it must be empty on destruction, since
+  // it may have ownership of the pointer.
+  ~BlockingQueue() {
+    DCHECK(list_.empty() || !std::is_pointer<T>::value)
+        << "BlockingQueue holds bare pointers at destruction time";
+  }
+
+  // Get an element from the queue.  Returns false if we were shut down prior to
+  // getting the element.
+  bool BlockingGet(T *out) {
+    MutexLock l(lock_);
+    while (true) {
+      if (!list_.empty()) {
+        *out = list_.front();
+        list_.pop_front();
+        decrement_size_unlocked(*out);
+        not_full_.Signal();
+        return true;
+      }
+      if (shutdown_) {
+        return false;
+      }
+      not_empty_.Wait();
+    }
+  }
+
+  // Get an element from the queue.  Returns false if the queue is empty and
+  // we were shut down prior to getting the element.
+  bool BlockingGet(gscoped_ptr<T_VAL> *out) {
+    T t = NULL;
+    bool got_element = BlockingGet(&t);
+    if (!got_element) {
+      return false;
+    }
+    out->reset(t);
+    return true;
+  }
+
+  // Get all elements from the queue and append them to a vector.
+  //
+  // If 'deadline' passes and no elements have been returned from the
+  // queue, returns Status::TimedOut(). If 'deadline' is uninitialized,
+  // no deadline is used.
+  //
+  // If the queue has been shut down, but there are still elements waiting,
+  // then it returns those elements as if the queue were not yet shut down.
+  //
+  // Returns:
+  // - OK if successful
+  // - TimedOut if the deadline passed
+  // - Aborted if the queue shut down
+  Status BlockingDrainTo(std::vector<T>* out, MonoTime deadline = MonoTime()) {
+    MutexLock l(lock_);
+    while (true) {
+      if (!list_.empty()) {
+        out->reserve(list_.size());
+        for (const T& elt : list_) {
+          out->push_back(elt);
+          decrement_size_unlocked(elt);
+        }
+        list_.clear();
+        not_full_.Signal();
+        return Status::OK();
+      }
+      if (PREDICT_FALSE(shutdown_)) {
+        return Status::Aborted("");
+      }
+      if (!deadline.Initialized()) {
+        not_empty_.Wait();
+      } else if (PREDICT_FALSE(!not_empty_.WaitUntil(deadline))) {
+        return Status::TimedOut("");
+      }
+    }
+  }
+
+  // Attempts to put the given value in the queue.
+  // Returns:
+  //   QUEUE_SUCCESS: if successfully inserted
+  //   QUEUE_FULL: if the queue has reached max_size
+  //   QUEUE_SHUTDOWN: if someone has already called Shutdown()
+  QueueStatus Put(const T &val) {
+    MutexLock l(lock_);
+    if (size_ >= max_size_) {
+      return QUEUE_FULL;
+    }
+    if (shutdown_) {
+      return QUEUE_SHUTDOWN;
+    }
+    list_.push_back(val);
+    increment_size_unlocked(val);
+    l.Unlock();
+    not_empty_.Signal();
+    return QUEUE_SUCCESS;
+  }
+
+  // Returns the same as the other Put() overload above.
+  // If the element was inserted, the gscoped_ptr releases its contents.
+  QueueStatus Put(gscoped_ptr<T_VAL> *val) {
+    QueueStatus s = Put(val->get());
+    if (s == QUEUE_SUCCESS) {
+      ignore_result<>(val->release());
+    }
+    return s;
+  }
+
+  // Gets an element for the queue; if the queue is full, blocks until
+  // space becomes available. Returns false if we were shutdown prior
+  // to enqueueing the element.
+  bool BlockingPut(const T& val) {
+    MutexLock l(lock_);
+    while (true) {
+      if (shutdown_) {
+        return false;
+      }
+      if (size_ < max_size_) {
+        list_.push_back(val);
+        increment_size_unlocked(val);
+        l.Unlock();
+        not_empty_.Signal();
+        return true;
+      }
+      not_full_.Wait();
+    }
+  }
+
+  // Same as other BlockingPut() overload above. If the element was
+  // enqueued, gscoped_ptr releases its contents.
+  bool BlockingPut(gscoped_ptr<T_VAL>* val) {
+    bool ret = Put(val->get());
+    if (ret) {
+      ignore_result(val->release());
+    }
+    return ret;
+  }
+
+  // Shut down the queue.
+  // When a blocking queue is shut down, no more elements can be added to it,
+  // and Put() will return QUEUE_SHUTDOWN.
+  // Existing elements will drain out of it, and then BlockingGet will start
+  // returning false.
+  void Shutdown() {
+    MutexLock l(lock_);
+    shutdown_ = true;
+    not_full_.Broadcast();
+    not_empty_.Broadcast();
+  }
+
+  bool empty() const {
+    MutexLock l(lock_);
+    return list_.empty();
+  }
+
+  size_t max_size() const {
+    return max_size_;
+  }
+
+  std::string ToString() const {
+    std::string ret;
+
+    MutexLock l(lock_);
+    for (const T& t : list_) {
+      ret.append(t->ToString());
+      ret.append("\n");
+    }
+    return ret;
+  }
+
+ private:
+
+  // Increments queue size. Must be called when 'lock_' is held.
+  void increment_size_unlocked(const T& t) {
+    size_ += LOGICAL_SIZE::logical_size(t);
+  }
+
+  // Decrements queue size. Must be called when 'lock_' is held.
+  void decrement_size_unlocked(const T& t) {
+    size_ -= LOGICAL_SIZE::logical_size(t);
+  }
+
+  bool shutdown_;
+  size_t size_;
+  size_t max_size_;
+  mutable Mutex lock_;
+  ConditionVariable not_empty_;
+  ConditionVariable not_full_;
+  std::list<T> list_;
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bloom_filter-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bloom_filter-test.cc b/be/src/kudu/util/bloom_filter-test.cc
new file mode 100644
index 0000000..788ec36
--- /dev/null
+++ b/be/src/kudu/util/bloom_filter-test.cc
@@ -0,0 +1,92 @@
+// 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 <cstdint>
+#include <cstdlib>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/bloom_filter.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+static const int kRandomSeed = 0xdeadbeef;
+
+static void AddRandomKeys(int random_seed, int n_keys, BloomFilterBuilder *bf) {
+  srandom(random_seed);
+  for (int i = 0; i < n_keys; i++) {
+    uint64_t key = random();
+    Slice key_slice(reinterpret_cast<const uint8_t *>(&key), sizeof(key));
+    BloomKeyProbe probe(key_slice);
+    bf->AddKey(probe);
+  }
+}
+
+static void CheckRandomKeys(int random_seed, int n_keys, const BloomFilter &bf) {
+  srandom(random_seed);
+  for (int i = 0; i < n_keys; i++) {
+    uint64_t key = random();
+    Slice key_slice(reinterpret_cast<const uint8_t *>(&key), sizeof(key));
+    BloomKeyProbe probe(key_slice);
+    ASSERT_TRUE(bf.MayContainKey(probe));
+  }
+}
+
+TEST(TestBloomFilter, TestInsertAndProbe) {
+  int n_keys = 2000;
+  BloomFilterBuilder bfb(
+    BloomFilterSizing::ByCountAndFPRate(n_keys, 0.01));
+
+  // Check that the desired false positive rate is achieved.
+  double expected_fp_rate = bfb.false_positive_rate();
+  ASSERT_NEAR(expected_fp_rate, 0.01, 0.002);
+
+  // 1% FP rate should need about 9 bits per key
+  ASSERT_EQ(9, bfb.n_bits() / n_keys);
+
+  // Enter n_keys random keys into the bloom filter
+  AddRandomKeys(kRandomSeed, n_keys, &bfb);
+
+  // Verify that the keys we inserted all return true when queried.
+  BloomFilter bf(bfb.slice(), bfb.n_hashes());
+  CheckRandomKeys(kRandomSeed, n_keys, bf);
+
+  // Query a bunch of other keys, and verify the false positive rate
+  // is within reasonable bounds.
+  uint32_t num_queries = 100000;
+  uint32_t num_positives = 0;
+  for (int i = 0; i < num_queries; i++) {
+    uint64_t key = random();
+    Slice key_slice(reinterpret_cast<const uint8_t *>(&key), sizeof(key));
+    BloomKeyProbe probe(key_slice);
+    if (bf.MayContainKey(probe)) {
+      num_positives++;
+    }
+  }
+
+  double fp_rate = static_cast<double>(num_positives) / static_cast<double>(num_queries);
+  LOG(INFO) << "FP rate: " << fp_rate << " (" << num_positives << "/" << num_queries << ")";
+  LOG(INFO) << "Expected FP rate: " << expected_fp_rate;
+
+  // Actual FP rate should be within 20% of the estimated FP rate
+  ASSERT_NEAR(fp_rate, expected_fp_rate, 0.20*expected_fp_rate);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bloom_filter.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bloom_filter.cc b/be/src/kudu/util/bloom_filter.cc
new file mode 100644
index 0000000..b1a2055
--- /dev/null
+++ b/be/src/kudu/util/bloom_filter.cc
@@ -0,0 +1,89 @@
+// 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 "kudu/util/bloom_filter.h"
+
+#include <cmath>
+#include <cstring>
+#include <ostream>
+
+#include <glog/logging.h>
+
+namespace kudu {
+
+static double kNaturalLog2 = 0.69314;
+
+static int ComputeOptimalHashCount(size_t n_bits, size_t elems) {
+  int n_hashes = n_bits * kNaturalLog2 / elems;
+  if (n_hashes < 1) n_hashes = 1;
+  return n_hashes;
+}
+
+BloomFilterSizing BloomFilterSizing::ByCountAndFPRate(
+  size_t expected_count, double fp_rate) {
+  CHECK_GT(fp_rate, 0);
+  CHECK_LT(fp_rate, 1);
+
+  double n_bits = -static_cast<double>(expected_count) * log(fp_rate)
+    / kNaturalLog2 / kNaturalLog2;
+  int n_bytes = static_cast<int>(ceil(n_bits / 8));
+  CHECK_GT(n_bytes, 0)
+    << "expected_count: " << expected_count
+    << " fp_rate: " << fp_rate;
+  return BloomFilterSizing(n_bytes, expected_count);
+}
+
+BloomFilterSizing BloomFilterSizing::BySizeAndFPRate(size_t n_bytes, double fp_rate) {
+  size_t n_bits = n_bytes * 8;
+  double expected_elems = -static_cast<double>(n_bits) * kNaturalLog2 * kNaturalLog2 /
+    log(fp_rate);
+  DCHECK_GT(expected_elems, 1);
+  return BloomFilterSizing(n_bytes, (size_t)ceil(expected_elems));
+}
+
+
+BloomFilterBuilder::BloomFilterBuilder(const BloomFilterSizing &sizing)
+  : n_bits_(sizing.n_bytes() * 8),
+    bitmap_(new uint8_t[sizing.n_bytes()]),
+    n_hashes_(ComputeOptimalHashCount(n_bits_, sizing.expected_count())),
+    expected_count_(sizing.expected_count()),
+    n_inserted_(0) {
+  Clear();
+}
+
+void BloomFilterBuilder::Clear() {
+  memset(&bitmap_[0], 0, n_bytes());
+  n_inserted_ = 0;
+}
+
+double BloomFilterBuilder::false_positive_rate() const {
+  CHECK_NE(expected_count_, 0)
+    << "expected_count_ not initialized: can't call this function on "
+    << "a BloomFilter initialized from external data";
+
+  return pow(1 - exp(-static_cast<double>(n_hashes_) * expected_count_ / n_bits_), n_hashes_);
+}
+
+BloomFilter::BloomFilter(const Slice &data, size_t n_hashes)
+  : n_bits_(data.size() * 8),
+    bitmap_(reinterpret_cast<const uint8_t *>(data.data())),
+    n_hashes_(n_hashes)
+{}
+
+
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bloom_filter.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bloom_filter.h b/be/src/kudu/util/bloom_filter.h
new file mode 100644
index 0000000..ad4e3eb
--- /dev/null
+++ b/be/src/kudu/util/bloom_filter.h
@@ -0,0 +1,254 @@
+// 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.
+#ifndef KUDU_UTIL_BLOOM_FILTER_H
+#define KUDU_UTIL_BLOOM_FILTER_H
+
+#include <cstddef>
+#include <cstdint>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/hash/city.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/bitmap.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+// Probe calculated from a given key. This caches the calculated
+// hash values which are necessary for probing into a Bloom Filter,
+// so that when many bloom filters have to be consulted for a given
+// key, we only need to calculate the hashes once.
+//
+// This is implemented based on the idea of double-hashing from the following paper:
+//   "Less Hashing, Same Performance: Building a Better Bloom Filter"
+//   Kirsch and Mitzenmacher, ESA 2006
+//   https://www.eecs.harvard.edu/~michaelm/postscripts/tr-02-05.pdf
+//
+// Currently, the implementation uses the 64-bit City Hash.
+// TODO: an SSE CRC32 hash is probably ~20% faster. Come back to this
+// at some point.
+class BloomKeyProbe {
+ public:
+  // Default constructor - this is only used to instantiate an object
+  // and later reassign by assignment from another instance
+  BloomKeyProbe() {}
+
+  // Construct a probe from the given key.
+  //
+  // NOTE: proper operation requires that the referenced memory remain
+  // valid for the lifetime of this object.
+  explicit BloomKeyProbe(const Slice &key) : key_(key) {
+    uint64_t h = util_hash::CityHash64(
+      reinterpret_cast<const char *>(key.data()),
+      key.size());
+
+    // Use the top and bottom halves of the 64-bit hash
+    // as the two independent hash functions for mixing.
+    h_1_ = static_cast<uint32_t>(h);
+    h_2_ = static_cast<uint32_t>(h >> 32);
+  }
+
+  const Slice &key() const { return key_; }
+
+  // The initial hash value. See MixHash() for usage example.
+  uint32_t initial_hash() const {
+    return h_1_;
+  }
+
+  // Mix the given hash function with the second calculated hash
+  // value. A sequence of independent hashes can be calculated
+  // by repeatedly calling MixHash() on its previous result.
+  ATTRIBUTE_NO_SANITIZE_INTEGER
+  uint32_t MixHash(uint32_t h) const {
+    return h + h_2_;
+  }
+
+ private:
+  Slice key_;
+
+  // The two hashes.
+  uint32_t h_1_;
+  uint32_t h_2_;
+};
+
+// Sizing parameters for the constructor to BloomFilterBuilder.
+// This is simply to provide a nicer API than a bunch of overloaded
+// constructors.
+class BloomFilterSizing {
+ public:
+  // Size the bloom filter by a fixed size and false positive rate.
+  //
+  // Picks the number of entries to achieve the above.
+  static BloomFilterSizing BySizeAndFPRate(size_t n_bytes, double fp_rate);
+
+  // Size the bloom filer by an expected count and false positive rate.
+  //
+  // Picks the number of bytes to achieve the above.
+  static BloomFilterSizing ByCountAndFPRate(size_t expected_count, double fp_rate);
+
+  size_t n_bytes() const { return n_bytes_; }
+  size_t expected_count() const { return expected_count_; }
+
+ private:
+  BloomFilterSizing(size_t n_bytes, size_t expected_count) :
+    n_bytes_(n_bytes),
+    expected_count_(expected_count)
+  {}
+
+  size_t n_bytes_;
+  size_t expected_count_;
+};
+
+
+// Builder for a BloomFilter structure.
+class BloomFilterBuilder {
+ public:
+  // Create a bloom filter.
+  // See BloomFilterSizing static methods to specify this argument.
+  explicit BloomFilterBuilder(const BloomFilterSizing &sizing);
+
+  // Clear all entries, reset insertion count.
+  void Clear();
+
+  // Add the given key to the bloom filter.
+  void AddKey(const BloomKeyProbe &probe);
+
+  // Return an estimate of the false positive rate.
+  double false_positive_rate() const;
+
+  int n_bytes() const {
+    return n_bits_ / 8;
+  }
+
+  int n_bits() const {
+    return n_bits_;
+  }
+
+  // Return a slice view into this Bloom Filter, suitable for
+  // writing out to a file.
+  const Slice slice() const {
+    return Slice(&bitmap_[0], n_bytes());
+  }
+
+  // Return the number of hashes that are calculated for each entry
+  // in the bloom filter.
+  size_t n_hashes() const { return n_hashes_; }
+
+  size_t expected_count() const { return expected_count_; }
+
+  // Return the number of keys inserted.
+  size_t count() const { return n_inserted_; }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterBuilder);
+
+  size_t n_bits_;
+  gscoped_array<uint8_t> bitmap_;
+
+  // The number of hash functions to compute.
+  size_t n_hashes_;
+
+  // The expected number of elements, for which the bloom is optimized.
+  size_t expected_count_;
+
+  // The number of elements inserted so far since the last Reset.
+  size_t n_inserted_;
+};
+
+
+// Wrapper around a byte array for reading it as a bloom filter.
+class BloomFilter {
+ public:
+  BloomFilter() : bitmap_(nullptr) {}
+  BloomFilter(const Slice &data, size_t n_hashes);
+
+  // Return true if the filter may contain the given key.
+  bool MayContainKey(const BloomKeyProbe &probe) const;
+
+ private:
+  friend class BloomFilterBuilder;
+  static uint32_t PickBit(uint32_t hash, size_t n_bits);
+
+  size_t n_bits_;
+  const uint8_t *bitmap_;
+
+  size_t n_hashes_;
+};
+
+
+////////////////////////////////////////////////////////////
+// Inline implementations
+////////////////////////////////////////////////////////////
+
+inline uint32_t BloomFilter::PickBit(uint32_t hash, size_t n_bits) {
+  switch (n_bits) {
+    // Fast path for the default bloom filter block size. Bitwise math
+    // is much faster than division.
+    case 4096 * 8:
+      return hash & (n_bits - 1);
+
+    default:
+      return hash % n_bits;
+  }
+}
+
+inline void BloomFilterBuilder::AddKey(const BloomKeyProbe &probe) {
+  uint32_t h = probe.initial_hash();
+  for (size_t i = 0; i < n_hashes_; i++) {
+    uint32_t bitpos = BloomFilter::PickBit(h, n_bits_);
+    BitmapSet(&bitmap_[0], bitpos);
+    h = probe.MixHash(h);
+  }
+  n_inserted_++;
+}
+
+inline bool BloomFilter::MayContainKey(const BloomKeyProbe &probe) const {
+  uint32_t h = probe.initial_hash();
+
+  // Basic unrolling by 2s gives a small benefit here since the two bit positions
+  // can be calculated in parallel -- it's a 50% chance that the first will be
+  // set even if it's a bloom miss, in which case we can parallelize the load.
+  int rem_hashes = n_hashes_;
+  while (rem_hashes >= 2) {
+    uint32_t bitpos1 = PickBit(h, n_bits_);
+    h = probe.MixHash(h);
+    uint32_t bitpos2 = PickBit(h, n_bits_);
+    h = probe.MixHash(h);
+
+    if (!BitmapTest(&bitmap_[0], bitpos1) ||
+        !BitmapTest(&bitmap_[0], bitpos2)) {
+      return false;
+    }
+
+    rem_hashes -= 2;
+  }
+
+  while (rem_hashes) {
+    uint32_t bitpos = PickBit(h, n_bits_);
+    if (!BitmapTest(&bitmap_[0], bitpos)) {
+      return false;
+    }
+    h = probe.MixHash(h);
+    rem_hashes--;
+  }
+  return true;
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/boost_mutex_utils.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/boost_mutex_utils.h b/be/src/kudu/util/boost_mutex_utils.h
new file mode 100644
index 0000000..6f6390b
--- /dev/null
+++ b/be/src/kudu/util/boost_mutex_utils.h
@@ -0,0 +1,45 @@
+// 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.
+#ifndef KUDU_BOOST_MUTEX_UTILS_H
+#define KUDU_BOOST_MUTEX_UTILS_H
+
+
+// Similar to std::lock_guard except that it takes
+// a lock pointer, and checks against nullptr. If the
+// pointer is NULL, does nothing. Otherwise guards
+// with the lock.
+template<class LockType>
+class lock_guard_maybe {
+ public:
+  explicit lock_guard_maybe(LockType *l) :
+    lock_(l) {
+    if (l != nullptr) {
+      l->lock();
+    }
+  }
+
+  ~lock_guard_maybe() {
+    if (lock_ != nullptr) {
+      lock_->unlock();
+    }
+  }
+
+ private:
+  LockType *lock_;
+};
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache-bench.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache-bench.cc b/be/src/kudu/util/cache-bench.cc
new file mode 100644
index 0000000..1e705be
--- /dev/null
+++ b/be/src/kudu/util/cache-bench.cc
@@ -0,0 +1,191 @@
+// 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 <string.h>
+
+#include <atomic>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/bits.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/test_util.h"
+
+DEFINE_int32(num_threads, 16, "The number of threads to access the cache concurrently.");
+DEFINE_int32(run_seconds, 1, "The number of seconds to run the benchmark");
+
+using std::atomic;
+using std::pair;
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+
+// Benchmark a 1GB cache.
+static constexpr int kCacheCapacity = 1024 * 1024 * 1024;
+// Use 4kb entries.
+static constexpr int kEntrySize = 4 * 1024;
+
+// Test parameterization.
+struct BenchSetup {
+  enum class Pattern {
+    // Zipfian distribution -- a small number of items make up the
+    // vast majority of lookups.
+    ZIPFIAN,
+    // Every item is equally likely to be looked up.
+    UNIFORM
+  };
+  Pattern pattern;
+
+  // The ratio between the size of the dataset and the cache.
+  //
+  // A value smaller than 1 will ensure that the whole dataset fits
+  // in the cache.
+  double dataset_cache_ratio;
+
+  string ToString() const {
+    string ret;
+    switch (pattern) {
+      case Pattern::ZIPFIAN: ret += "ZIPFIAN"; break;
+      case Pattern::UNIFORM: ret += "UNIFORM"; break;
+    }
+    ret += StringPrintf(" ratio=%.2fx n_unique=%d", dataset_cache_ratio, max_key());
+    return ret;
+  }
+
+  // Return the maximum cache key to be generated for a lookup.
+  uint32_t max_key() const {
+    return static_cast<int64_t>(kCacheCapacity * dataset_cache_ratio) / kEntrySize;
+  }
+};
+
+class CacheBench : public KuduTest,
+                   public testing::WithParamInterface<BenchSetup>{
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    cache_.reset(NewLRUCache(DRAM_CACHE, kCacheCapacity, "test-cache"));
+  }
+
+  // Run queries against the cache until '*done' becomes true.
+  // Returns a pair of the number of cache hits and lookups.
+  pair<int64_t, int64_t> DoQueries(const atomic<bool>* done) {
+    const BenchSetup& setup = GetParam();
+    Random r(GetRandomSeed32());
+    int64_t lookups = 0;
+    int64_t hits = 0;
+    while (!*done) {
+      uint32_t int_key;
+      if (setup.pattern == BenchSetup::Pattern::ZIPFIAN) {
+        int_key = r.Skewed(Bits::Log2Floor(setup.max_key()));
+      } else {
+        int_key = r.Uniform(setup.max_key());
+      }
+      char key_buf[sizeof(int_key)];
+      memcpy(key_buf, &int_key, sizeof(int_key));
+      Slice key_slice(key_buf, arraysize(key_buf));
+      Cache::Handle* h = cache_->Lookup(key_slice, Cache::EXPECT_IN_CACHE);
+      if (h) {
+        hits++;
+      } else {
+        Cache::PendingHandle* ph = cache_->Allocate(
+            key_slice, /* val_len=*/kEntrySize, /* charge=*/kEntrySize);
+        h = cache_->Insert(ph, nullptr);
+      }
+
+      cache_->Release(h);
+      lookups++;
+    }
+    return {hits, lookups};
+  }
+
+  // Starts the given number of threads to concurrently call DoQueries.
+  // Returns the aggregated number of cache hits and lookups.
+  pair<int64_t, int64_t> RunQueryThreads(int n_threads, int n_seconds) {
+    vector<thread> threads(n_threads);
+    atomic<bool> done(false);
+    atomic<int64_t> total_lookups(0);
+    atomic<int64_t> total_hits(0);
+    for (int i = 0; i < n_threads; i++) {
+      threads[i] = thread([&]() {
+          pair<int64_t, int64_t> hits_lookups = DoQueries(&done);
+          total_hits += hits_lookups.first;
+          total_lookups += hits_lookups.second;
+        });
+    }
+    SleepFor(MonoDelta::FromSeconds(n_seconds));
+    done = true;
+    for (auto& t : threads) {
+      t.join();
+    }
+    return {total_hits, total_lookups};
+  }
+
+ protected:
+  unique_ptr<Cache> cache_;
+};
+
+// Test both distributions, and for each, test both the case where the data
+// fits in the cache and where it is a bit larger.
+INSTANTIATE_TEST_CASE_P(Patterns, CacheBench, testing::ValuesIn(std::vector<BenchSetup>{
+      {BenchSetup::Pattern::ZIPFIAN, 1.0},
+      {BenchSetup::Pattern::ZIPFIAN, 3.0},
+      {BenchSetup::Pattern::UNIFORM, 1.0},
+      {BenchSetup::Pattern::UNIFORM, 3.0}
+    }));
+
+TEST_P(CacheBench, RunBench) {
+  const BenchSetup& setup = GetParam();
+
+  // Run a short warmup phase to try to populate the cache. Otherwise even if the
+  // dataset is smaller than the cache capacity, we would count a bunch of misses
+  // during the warm-up phase.
+  LOG(INFO) << "Warming up...";
+  RunQueryThreads(FLAGS_num_threads, 1);
+
+  LOG(INFO) << "Running benchmark...";
+  pair<int64_t, int64_t> hits_lookups = RunQueryThreads(FLAGS_num_threads, FLAGS_run_seconds);
+  int64_t hits = hits_lookups.first;
+  int64_t lookups = hits_lookups.second;
+
+  int64_t l_per_sec = lookups / FLAGS_run_seconds;
+  double hit_rate = static_cast<double>(hits) / lookups;
+  string test_case = setup.ToString();
+  LOG(INFO) << test_case << ": " << HumanReadableNum::ToString(l_per_sec) << " lookups/sec";
+  LOG(INFO) << test_case << ": " << StringPrintf("%.1f", hit_rate * 100.0) << "% hit rate";
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache-test.cc b/be/src/kudu/util/cache-test.cc
new file mode 100644
index 0000000..3fd1d5f
--- /dev/null
+++ b/be/src/kudu/util/cache-test.cc
@@ -0,0 +1,246 @@
+// Some portions Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include <cassert>
+#include <cstring>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/cache.h"
+#include "kudu/util/coding.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+#if defined(__linux__)
+DECLARE_string(nvm_cache_path);
+#endif // defined(__linux__)
+
+DECLARE_double(cache_memtracker_approximation_ratio);
+
+namespace kudu {
+
+// Conversions between numeric keys/values and the types expected by Cache.
+static std::string EncodeInt(int k) {
+  faststring result;
+  PutFixed32(&result, k);
+  return result.ToString();
+}
+static int DecodeInt(const Slice& k) {
+  assert(k.size() == 4);
+  return DecodeFixed32(k.data());
+}
+
+class CacheTest : public KuduTest,
+                  public ::testing::WithParamInterface<CacheType>,
+                  public Cache::EvictionCallback {
+ public:
+
+  // Implementation of the EvictionCallback interface
+  void EvictedEntry(Slice key, Slice val) override {
+    evicted_keys_.push_back(DecodeInt(key));
+    evicted_values_.push_back(DecodeInt(val));
+  }
+  std::vector<int> evicted_keys_;
+  std::vector<int> evicted_values_;
+  std::shared_ptr<MemTracker> mem_tracker_;
+  gscoped_ptr<Cache> cache_;
+  MetricRegistry metric_registry_;
+
+  static const int kCacheSize = 14*1024*1024;
+
+  virtual void SetUp() OVERRIDE {
+
+#if defined(HAVE_LIB_VMEM)
+    if (google::GetCommandLineFlagInfoOrDie("nvm_cache_path").is_default) {
+      FLAGS_nvm_cache_path = GetTestPath("nvm-cache");
+      ASSERT_OK(Env::Default()->CreateDir(FLAGS_nvm_cache_path));
+    }
+#endif // defined(HAVE_LIB_VMEM)
+
+    // Disable approximate tracking of cache memory since we make specific
+    // assertions on the MemTracker in this test.
+    FLAGS_cache_memtracker_approximation_ratio = 0;
+
+    cache_.reset(NewLRUCache(GetParam(), kCacheSize, "cache_test"));
+
+    MemTracker::FindTracker("cache_test-sharded_lru_cache", &mem_tracker_);
+    // Since nvm cache does not have memtracker due to the use of
+    // tcmalloc for this we only check for it in the DRAM case.
+    if (GetParam() == DRAM_CACHE) {
+      ASSERT_TRUE(mem_tracker_.get());
+    }
+
+    scoped_refptr<MetricEntity> entity = METRIC_ENTITY_server.Instantiate(
+        &metric_registry_, "test");
+    cache_->SetMetrics(entity);
+  }
+
+  int Lookup(int key) {
+    Cache::Handle* handle = cache_->Lookup(EncodeInt(key), Cache::EXPECT_IN_CACHE);
+    const int r = (handle == nullptr) ? -1 : DecodeInt(cache_->Value(handle));
+    if (handle != nullptr) {
+      cache_->Release(handle);
+    }
+    return r;
+  }
+
+  void Insert(int key, int value, int charge = 1) {
+    std::string key_str = EncodeInt(key);
+    std::string val_str = EncodeInt(value);
+    Cache::PendingHandle* handle = CHECK_NOTNULL(cache_->Allocate(key_str, val_str.size(), charge));
+    memcpy(cache_->MutableValue(handle), val_str.data(), val_str.size());
+
+    cache_->Release(cache_->Insert(handle, this));
+  }
+
+  void Erase(int key) {
+    cache_->Erase(EncodeInt(key));
+  }
+};
+
+#if defined(__linux__)
+INSTANTIATE_TEST_CASE_P(CacheTypes, CacheTest, ::testing::Values(DRAM_CACHE, NVM_CACHE));
+#else
+INSTANTIATE_TEST_CASE_P(CacheTypes, CacheTest, ::testing::Values(DRAM_CACHE));
+#endif // defined(__linux__)
+
+TEST_P(CacheTest, TrackMemory) {
+  if (mem_tracker_) {
+    Insert(100, 100, 1);
+    ASSERT_EQ(1, mem_tracker_->consumption());
+    Erase(100);
+    ASSERT_EQ(0, mem_tracker_->consumption());
+    ASSERT_EQ(1, mem_tracker_->peak_consumption());
+  }
+}
+
+TEST_P(CacheTest, HitAndMiss) {
+  ASSERT_EQ(-1, Lookup(100));
+
+  Insert(100, 101);
+  ASSERT_EQ(101, Lookup(100));
+  ASSERT_EQ(-1,  Lookup(200));
+  ASSERT_EQ(-1,  Lookup(300));
+
+  Insert(200, 201);
+  ASSERT_EQ(101, Lookup(100));
+  ASSERT_EQ(201, Lookup(200));
+  ASSERT_EQ(-1,  Lookup(300));
+
+  Insert(100, 102);
+  ASSERT_EQ(102, Lookup(100));
+  ASSERT_EQ(201, Lookup(200));
+  ASSERT_EQ(-1,  Lookup(300));
+
+  ASSERT_EQ(1, evicted_keys_.size());
+  ASSERT_EQ(100, evicted_keys_[0]);
+  ASSERT_EQ(101, evicted_values_[0]);
+}
+
+TEST_P(CacheTest, Erase) {
+  Erase(200);
+  ASSERT_EQ(0, evicted_keys_.size());
+
+  Insert(100, 101);
+  Insert(200, 201);
+  Erase(100);
+  ASSERT_EQ(-1,  Lookup(100));
+  ASSERT_EQ(201, Lookup(200));
+  ASSERT_EQ(1, evicted_keys_.size());
+  ASSERT_EQ(100, evicted_keys_[0]);
+  ASSERT_EQ(101, evicted_values_[0]);
+
+  Erase(100);
+  ASSERT_EQ(-1,  Lookup(100));
+  ASSERT_EQ(201, Lookup(200));
+  ASSERT_EQ(1, evicted_keys_.size());
+}
+
+TEST_P(CacheTest, EntriesArePinned) {
+  Insert(100, 101);
+  Cache::Handle* h1 = cache_->Lookup(EncodeInt(100), Cache::EXPECT_IN_CACHE);
+  ASSERT_EQ(101, DecodeInt(cache_->Value(h1)));
+
+  Insert(100, 102);
+  Cache::Handle* h2 = cache_->Lookup(EncodeInt(100), Cache::EXPECT_IN_CACHE);
+  ASSERT_EQ(102, DecodeInt(cache_->Value(h2)));
+  ASSERT_EQ(0, evicted_keys_.size());
+
+  cache_->Release(h1);
+  ASSERT_EQ(1, evicted_keys_.size());
+  ASSERT_EQ(100, evicted_keys_[0]);
+  ASSERT_EQ(101, evicted_values_[0]);
+
+  Erase(100);
+  ASSERT_EQ(-1, Lookup(100));
+  ASSERT_EQ(1, evicted_keys_.size());
+
+  cache_->Release(h2);
+  ASSERT_EQ(2, evicted_keys_.size());
+  ASSERT_EQ(100, evicted_keys_[1]);
+  ASSERT_EQ(102, evicted_values_[1]);
+}
+
+TEST_P(CacheTest, EvictionPolicy) {
+  Insert(100, 101);
+  Insert(200, 201);
+
+  const int kNumElems = 1000;
+  const int kSizePerElem = kCacheSize / kNumElems;
+
+  // Loop adding and looking up new entries, but repeatedly accessing key 101. This
+  // frequently-used entry should not be evicted.
+  for (int i = 0; i < kNumElems + 1000; i++) {
+    Insert(1000+i, 2000+i, kSizePerElem);
+    ASSERT_EQ(2000+i, Lookup(1000+i));
+    ASSERT_EQ(101, Lookup(100));
+  }
+  ASSERT_EQ(101, Lookup(100));
+  // Since '200' wasn't accessed in the loop above, it should have
+  // been evicted.
+  ASSERT_EQ(-1, Lookup(200));
+}
+
+TEST_P(CacheTest, HeavyEntries) {
+  // Add a bunch of light and heavy entries and then count the combined
+  // size of items still in the cache, which must be approximately the
+  // same as the total capacity.
+  const int kLight = kCacheSize/1000;
+  const int kHeavy = kCacheSize/100;
+  int added = 0;
+  int index = 0;
+  while (added < 2*kCacheSize) {
+    const int weight = (index & 1) ? kLight : kHeavy;
+    Insert(index, 1000+index, weight);
+    added += weight;
+    index++;
+  }
+
+  int cached_weight = 0;
+  for (int i = 0; i < index; i++) {
+    const int weight = (i & 1 ? kLight : kHeavy);
+    int r = Lookup(i);
+    if (r >= 0) {
+      cached_weight += weight;
+      ASSERT_EQ(1000+i, r);
+    }
+  }
+  ASSERT_LE(cached_weight, kCacheSize + kCacheSize/10);
+}
+
+}  // namespace kudu


[07/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/rwc_lock.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/rwc_lock.h b/be/src/kudu/util/rwc_lock.h
new file mode 100644
index 0000000..7b78e35
--- /dev/null
+++ b/be/src/kudu/util/rwc_lock.h
@@ -0,0 +1,142 @@
+// 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.
+#ifndef KUDU_UTIL_RWC_LOCK_H
+#define KUDU_UTIL_RWC_LOCK_H
+
+#include <cstdint>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/mutex.h"
+
+namespace kudu {
+
+// A read-write-commit lock.
+//
+// This lock has three modes: read, write, and commit.
+// The lock compatibility matrix is as follows:
+//
+//           Read    Write    Commit
+//  Read      X        X
+//  Write     X
+//  Commit
+//
+// An 'X' indicates that the two types of locks may be
+// held at the same time.
+//
+// In prose:
+// - Multiple threads may hold the Read lock at the same time.
+// - A single thread may hold the Write lock, potentially at the
+//   same time as any number of readers.
+// - A single thread may hold the Commit lock, but this lock is completely
+//   exclusive (no concurrent readers or writers).
+//
+// A typical use case for this type of lock is when a structure is read often,
+// occasionally updated, and the update operation can take a long time. In this
+// use case, the readers simply use ReadLock() and ReadUnlock(), while the
+// writer uses a copy-on-write technique like:
+//
+//   obj->lock.WriteLock();
+//   // NOTE: cannot safely mutate obj->state directly here, since readers
+//   // may be concurrent! So, we make a local copy to mutate.
+//   my_local_copy = obj->state;
+//   SomeLengthyMutation(my_local_copy);
+//   obj->lock.UpgradeToCommitLock();
+//   obj->state = my_local_copy;
+//   obj->lock.CommitUnlock();
+//
+// This is more efficient than a standard Reader-Writer lock since the lengthy
+// mutation is only protected against other concurrent mutators, and readers
+// may continue to run with no contention.
+//
+// For the common pattern described above, the 'CowObject<>' template class defined
+// in cow_object.h is more convenient than manual locking.
+//
+// NOTE: this implementation currently does not implement any starvation protection
+// or fairness. If the read lock is being constantly acquired (i.e reader count
+// never drops to 0) then UpgradeToCommitLock() may block arbitrarily long.
+class RWCLock {
+ public:
+  RWCLock();
+  ~RWCLock();
+
+  // Acquire the lock in read mode. Upon return, guarantees that:
+  // - Other threads may concurrently hold the lock for Read.
+  // - Either zero or one thread may hold the lock for Write.
+  // - No threads hold the lock for Commit.
+  void ReadLock();
+  void ReadUnlock();
+
+  // Return true if there are any readers currently holding the lock.
+  // Useful for debug assertions.
+  bool HasReaders() const;
+
+  // Return true if the current thread holds the write lock.
+  //
+  // In DEBUG mode this is accurate -- we track the current holder's tid.
+  // In non-DEBUG mode, this may sometimes return true even if another thread
+  // is in fact the holder.
+  // Thus, this is only really useful in the context of a DCHECK assertion.
+  bool HasWriteLock() const;
+
+  // Boost-like wrappers, so boost lock guards work
+  void lock_shared() { ReadLock(); }
+  void unlock_shared() { ReadUnlock(); }
+
+  // Acquire the lock in write mode. Upon return, guarantees that:
+  // - Other threads may concurrently hold the lock for Read.
+  // - No other threads hold the lock for Write or Commit.
+  void WriteLock();
+  void WriteUnlock();
+
+  // Boost-like wrappers
+  void lock() { WriteLock(); }
+  void unlock() { WriteUnlock(); }
+
+  // Upgrade the lock from Write mode to Commit mode.
+  // Requires that the current thread holds the lock in Write mode.
+  // Upon return, guarantees:
+  // - No other thread holds the lock in any mode.
+  void UpgradeToCommitLock();
+  void CommitUnlock();
+
+ private:
+  // Variants of the functions above that must be called with lock_ held.
+  bool HasReadersUnlocked() const;
+  bool HasWriteLockUnlocked() const;
+
+  // Lock which protects reader_count_ and write_locked_.
+  // Additionally, while the commit lock is held, the
+  // locking thread holds this mutex, which prevents any new
+  // threads from obtaining the lock in any mode.
+  mutable Mutex lock_;
+  ConditionVariable no_mutators_, no_readers_;
+  int reader_count_;
+  bool write_locked_;
+
+#ifndef NDEBUG
+  static const int kBacktraceBufSize = 1024;
+  int64_t writer_tid_;
+  int64_t last_writelock_acquire_time_;
+  char last_writer_backtrace_[kBacktraceBufSize];
+#endif // NDEBUG
+
+  DISALLOW_COPY_AND_ASSIGN(RWCLock);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_RWC_LOCK_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/safe_math-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/safe_math-test.cc b/be/src/kudu/util/safe_math-test.cc
new file mode 100644
index 0000000..d3a81c6
--- /dev/null
+++ b/be/src/kudu/util/safe_math-test.cc
@@ -0,0 +1,56 @@
+// 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 <stdint.h>
+
+// Must come before gtest.h.
+#include "kudu/gutil/mathlimits.h"
+
+#include <gtest/gtest.h>
+#include "kudu/util/safe_math.h"
+
+namespace kudu {
+template<typename T>
+static void DoTest(T a, T b, bool expected) {
+  SCOPED_TRACE(a);
+  SCOPED_TRACE(b);
+  bool overflow = false;
+  T ret = AddWithOverflowCheck(a, b, &overflow);
+  EXPECT_EQ(overflow, expected);
+  if (!overflow) {
+    EXPECT_EQ(ret, a + b);
+  }
+}
+
+TEST(TestSafeMath, TestSignedInts) {
+  // Overflow above max of range.
+  DoTest<int32_t>(MathLimits<int32_t>::kMax - 10, 15, true);
+  DoTest<int32_t>(MathLimits<int32_t>::kMax - 10, 10, false);
+
+  // Underflow around negative
+  DoTest<int32_t>(MathLimits<int32_t>::kMin + 10, -15, true);
+  DoTest<int32_t>(MathLimits<int32_t>::kMin + 10, -5, false);
+
+}
+
+TEST(TestSafeMath, TestUnsignedInts) {
+  // Overflow above max
+  DoTest<uint32_t>(MathLimits<uint32_t>::kMax - 10, 15, true);
+  DoTest<uint32_t>(MathLimits<uint32_t>::kMax - 10, 10, false);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/safe_math.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/safe_math.h b/be/src/kudu/util/safe_math.h
new file mode 100644
index 0000000..4c126dd
--- /dev/null
+++ b/be/src/kudu/util/safe_math.h
@@ -0,0 +1,69 @@
+// 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.
+//
+// Inline functions for doing overflow-safe operations on integers.
+// These should be used when doing bounds checks on user-provided data,
+// for example.
+// See also: https://www.securecoding.cert.org/confluence/display/cplusplus/INT32-CPP.+Ensure+that+operations+on+signed+integers+do+not+result+in+overflow
+#ifndef KUDU_UTIL_SAFE_MATH_H
+#define KUDU_UTIL_SAFE_MATH_H
+
+#include "kudu/gutil/mathlimits.h"
+
+namespace kudu {
+
+namespace safe_math_internal {
+
+// Template which is specialized for signed and unsigned types separately.
+template<typename Type, bool is_signed>
+struct WithOverflowCheck {
+};
+
+
+// Specialization for signed types.
+template<typename Type>
+struct WithOverflowCheck<Type, true> {
+  static inline Type Add(Type a, Type b, bool *overflowed) {
+    // Implementation from the CERT article referenced in the file header.
+    *overflowed = (((a > 0) && (b > 0) && (a > (MathLimits<Type>::kMax - b))) ||
+                   ((a < 0) && (b < 0) && (a < (MathLimits<Type>::kMin - b))));
+    return a + b;
+  }
+};
+
+// Specialization for unsigned types.
+template<typename Type>
+struct WithOverflowCheck<Type, false> {
+  static inline Type Add(Type a, Type b, bool *overflowed) {
+    Type ret = a + b;
+    *overflowed = ret < a;
+    return a + b;
+  }
+};
+
+} // namespace safe_math_internal
+
+// Add 'a' and 'b', and set *overflowed to true if overflow occured.
+template<typename Type>
+inline Type AddWithOverflowCheck(Type a, Type b, bool *overflowed) {
+  // Pick the right specialization based on whether Type is signed.
+  typedef safe_math_internal::WithOverflowCheck<Type, MathLimits<Type>::kIsSigned> my_struct;
+  return my_struct::Add(a, b, overflowed);
+}
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/scoped_cleanup-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/scoped_cleanup-test.cc b/be/src/kudu/util/scoped_cleanup-test.cc
new file mode 100644
index 0000000..2e77705
--- /dev/null
+++ b/be/src/kudu/util/scoped_cleanup-test.cc
@@ -0,0 +1,56 @@
+// 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 "kudu/util/scoped_cleanup.h"
+
+#include <gtest/gtest.h>
+
+namespace kudu {
+
+TEST(ScopedCleanup, TestCleanup) {
+  int var = 0;
+  {
+    auto saved = var;
+    auto cleanup = MakeScopedCleanup([&] () { var = saved; });
+    var = 42;
+  }
+  ASSERT_EQ(0, var);
+}
+
+TEST(ScopedCleanup, TestCleanupMacro) {
+  int var = 0;
+  {
+    auto saved = var;
+    SCOPED_CLEANUP({ var = saved; });
+    var = 42;
+  }
+  ASSERT_EQ(0, var);
+}
+
+
+TEST(ScopedCleanup, TestCancelCleanup) {
+  int var = 0;
+  {
+    auto saved = var;
+    auto cleanup = MakeScopedCleanup([&] () { var = saved; });
+    var = 42;
+    cleanup.cancel();
+  }
+  ASSERT_EQ(42, var);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/scoped_cleanup.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/scoped_cleanup.h b/be/src/kudu/util/scoped_cleanup.h
new file mode 100644
index 0000000..8ecfbcb
--- /dev/null
+++ b/be/src/kudu/util/scoped_cleanup.h
@@ -0,0 +1,67 @@
+// 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.
+
+#pragma once
+
+#include <utility>
+
+#include "kudu/gutil/macros.h"
+
+// Run the given function body (which is typically a block of code surrounded by
+// curly-braces) when the current scope exits.
+//
+// Example:
+//   int fd = open(...);
+//   SCOPED_CLEANUP({ close(fd); });
+//
+// NOTE: in the case that you want to cancel the cleanup, use the more verbose
+// (non-macro) form below.
+#define SCOPED_CLEANUP(func_body) \
+  auto VARNAME_LINENUM(scoped_cleanup) = MakeScopedCleanup([&] { func_body });
+
+namespace kudu {
+
+// A scoped object which runs a cleanup function when going out of scope. Can
+// be used for scoped resource cleanup.
+//
+// Use 'MakeScopedCleanup()' below to instantiate.
+template<typename F>
+class ScopedCleanup {
+ public:
+  explicit ScopedCleanup(F f)
+      : cancelled_(false),
+        f_(std::move(f)) {
+  }
+  ~ScopedCleanup() {
+    if (!cancelled_) {
+      f_();
+    }
+  }
+  void cancel() { cancelled_ = true; }
+
+ private:
+  bool cancelled_;
+  F f_;
+};
+
+// Creates a new scoped cleanup instance with the provided function.
+template<typename F>
+ScopedCleanup<F> MakeScopedCleanup(F f) {
+  return ScopedCleanup<F>(f);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/semaphore.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/semaphore.cc b/be/src/kudu/util/semaphore.cc
new file mode 100644
index 0000000..72ff214
--- /dev/null
+++ b/be/src/kudu/util/semaphore.cc
@@ -0,0 +1,105 @@
+// 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 "kudu/util/semaphore.h"
+
+#include <semaphore.h>
+
+#include <cerrno>
+#include <cstdint>
+#include <cstdlib>
+#include <ctime>
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+
+Semaphore::Semaphore(int capacity) {
+  DCHECK_GE(capacity, 0);
+  if (sem_init(&sem_, 0, capacity) != 0) {
+    Fatal("init");
+  }
+}
+
+Semaphore::~Semaphore() {
+  if (sem_destroy(&sem_) != 0) {
+    Fatal("destroy");
+  }
+}
+
+void Semaphore::Acquire() {
+  while (true) {
+    int ret;
+    RETRY_ON_EINTR(ret, sem_wait(&sem_));
+    if (ret == 0) {
+      // TODO(todd): would be nice to track acquisition time, etc.
+      return;
+    }
+    Fatal("wait");
+  }
+}
+
+bool Semaphore::TryAcquire() {
+  int ret;
+  RETRY_ON_EINTR(ret, sem_trywait(&sem_));
+  if (ret == 0) {
+    return true;
+  }
+  if (errno == EAGAIN) {
+    return false;
+  }
+  Fatal("trywait");
+}
+
+bool Semaphore::TimedAcquire(const MonoDelta& timeout) {
+  int64_t microtime = GetCurrentTimeMicros();
+  microtime += timeout.ToMicroseconds();
+
+  struct timespec abs_timeout;
+  MonoDelta::NanosToTimeSpec(microtime * MonoTime::kNanosecondsPerMicrosecond,
+                             &abs_timeout);
+
+  while (true) {
+    int ret;
+    RETRY_ON_EINTR(ret, sem_timedwait(&sem_, &abs_timeout));
+    if (ret == 0) return true;
+    if (errno == ETIMEDOUT) return false;
+    Fatal("timedwait");
+  }
+}
+
+void Semaphore::Release() {
+  PCHECK(sem_post(&sem_) == 0);
+}
+
+int Semaphore::GetValue() {
+  int val;
+  PCHECK(sem_getvalue(&sem_, &val) == 0);
+  return val;
+}
+
+void Semaphore::Fatal(const char* action) {
+  PLOG(FATAL) << "Could not " << action << " semaphore "
+              << reinterpret_cast<void*>(&sem_);
+  abort(); // unnecessary, but avoids gcc complaining
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/semaphore.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/semaphore.h b/be/src/kudu/util/semaphore.h
new file mode 100644
index 0000000..4f12658
--- /dev/null
+++ b/be/src/kudu/util/semaphore.h
@@ -0,0 +1,77 @@
+// 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.
+#ifndef KUDU_UTIL_SEMAPHORE_H
+#define KUDU_UTIL_SEMAPHORE_H
+
+#include <semaphore.h>
+#if defined(__APPLE__)
+#include <dispatch/dispatch.h>
+#include "kudu/util/atomic.h"
+#endif  // define(__APPLE__)
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+class MonoDelta;
+
+// Wrapper for POSIX semaphores.
+class Semaphore {
+ public:
+  // Initialize the semaphore with the specified capacity.
+  explicit Semaphore(int capacity);
+  ~Semaphore();
+
+  // Acquire the semaphore.
+  void Acquire();
+
+  // Acquire the semaphore within the given timeout. Returns true if successful.
+  bool TimedAcquire(const MonoDelta& timeout);
+
+  // Try to acquire the semaphore immediately. Returns false if unsuccessful.
+  bool TryAcquire();
+
+  // Release the semaphore.
+  void Release();
+
+  // Get the current value of the semaphore.
+  int GetValue();
+
+  // Boost-compatible wrappers.
+  void lock() { Acquire(); }
+  void unlock() { Release(); }
+  bool try_lock() { return TryAcquire(); }
+
+ private:
+#if !defined(__APPLE__)
+  // Log a fatal error message. Separated out to keep the main functions
+  // as small as possible in terms of code size.
+  void Fatal(const char* action) ATTRIBUTE_NORETURN;
+#endif  // !define(__APPLE__)
+
+#if defined(__APPLE__)
+  dispatch_semaphore_t sem_;
+  AtomicInt<int32_t> count_;
+#else
+  sem_t sem_;
+#endif  // define(__APPLE__)
+  DISALLOW_COPY_AND_ASSIGN(Semaphore);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_SEMAPHORE_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/semaphore_macosx.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/semaphore_macosx.cc b/be/src/kudu/util/semaphore_macosx.cc
new file mode 100644
index 0000000..e2d235c
--- /dev/null
+++ b/be/src/kudu/util/semaphore_macosx.cc
@@ -0,0 +1,75 @@
+// 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 "kudu/util/semaphore.h"
+
+#include <semaphore.h>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+
+Semaphore::Semaphore(int capacity)
+  : count_(capacity) {
+  DCHECK_GE(capacity, 0);
+  sem_ = dispatch_semaphore_create(capacity);
+  CHECK_NOTNULL(sem_);
+}
+
+Semaphore::~Semaphore() {
+  dispatch_release(sem_);
+}
+
+void Semaphore::Acquire() {
+  // If the timeout is DISPATCH_TIME_FOREVER, then dispatch_semaphore_wait()
+  // waits forever and always returns zero.
+  CHECK(dispatch_semaphore_wait(sem_, DISPATCH_TIME_FOREVER) == 0);
+  count_.IncrementBy(-1);
+}
+
+bool Semaphore::TryAcquire() {
+  // The dispatch_semaphore_wait() function returns zero upon success and
+  // non-zero after the timeout expires.
+  if (dispatch_semaphore_wait(sem_, DISPATCH_TIME_NOW) == 0) {
+    count_.IncrementBy(-1);
+    return true;
+  }
+  return false;
+}
+
+bool Semaphore::TimedAcquire(const MonoDelta& timeout) {
+  dispatch_time_t t = dispatch_time(DISPATCH_TIME_NOW, timeout.ToNanoseconds());
+  if (dispatch_semaphore_wait(sem_, t) == 0) {
+    count_.IncrementBy(-1);
+    return true;
+  }
+  return false;
+}
+
+void Semaphore::Release() {
+  dispatch_semaphore_signal(sem_);
+  count_.IncrementBy(1);
+}
+
+int Semaphore::GetValue() {
+  return count_.Load();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/signal.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/signal.cc b/be/src/kudu/util/signal.cc
new file mode 100644
index 0000000..e8b6e79
--- /dev/null
+++ b/be/src/kudu/util/signal.cc
@@ -0,0 +1,47 @@
+// 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 "kudu/util/signal.h"
+
+#include <glog/logging.h>
+
+namespace kudu {
+
+void SetSignalHandler(int signal, SignalHandlerCallback handler) {
+  struct sigaction act;
+  act.sa_handler = handler;
+  sigemptyset(&act.sa_mask);
+  act.sa_flags = 0;
+  PCHECK(sigaction(signal, &act, nullptr) == 0);
+}
+
+void IgnoreSigPipe() {
+  SetSignalHandler(SIGPIPE, SIG_IGN);
+}
+
+void ResetSigPipeHandlerToDefault() {
+  SetSignalHandler(SIGPIPE, SIG_DFL);
+}
+
+// We unblock all signal masks since they are inherited.
+void ResetAllSignalMasksToUnblocked() {
+  sigset_t signals;
+  PCHECK(sigfillset(&signals) == 0);
+  PCHECK(sigprocmask(SIG_UNBLOCK, &signals, nullptr) == 0);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/signal.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/signal.h b/be/src/kudu/util/signal.h
new file mode 100644
index 0000000..0c88a80
--- /dev/null
+++ b/be/src/kudu/util/signal.h
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include <signal.h>
+
+namespace kudu {
+
+#if defined(__linux__)
+typedef sighandler_t SignalHandlerCallback;
+#else
+typedef sig_t SignalHandlerCallback;
+#endif
+
+// Set a process-wide signal handler.
+void SetSignalHandler(int signal, SignalHandlerCallback handler);
+
+// Set the disposition of SIGPIPE to SIG_IGN.
+void IgnoreSigPipe();
+
+// Set the disposition of SIGPIPE to SIG_DFL.
+void ResetSigPipeHandlerToDefault();
+
+// Unblock all signal masks.
+void ResetAllSignalMasksToUnblocked();
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/slice-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/slice-test.cc b/be/src/kudu/util/slice-test.cc
new file mode 100644
index 0000000..0f7a893
--- /dev/null
+++ b/be/src/kudu/util/slice-test.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 "kudu/util/slice.h"
+
+#include <cstdint>
+#include <map>
+#include <string>
+#include <utility>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/map-util.h"
+
+using std::string;
+
+namespace kudu {
+
+typedef SliceMap<int>::type MySliceMap;
+
+TEST(SliceTest, TestSliceMap) {
+  MySliceMap my_map;
+  Slice a("a");
+  Slice b("b");
+  Slice c("c");
+
+  // Insertion is deliberately out-of-order; the map should restore order.
+  InsertOrDie(&my_map, c, 3);
+  InsertOrDie(&my_map, a, 1);
+  InsertOrDie(&my_map, b, 2);
+
+  int expectedValue = 0;
+  for (const MySliceMap::value_type& pair : my_map) {
+    int data = 'a' + expectedValue++;
+    ASSERT_EQ(Slice(reinterpret_cast<uint8_t*>(&data), 1), pair.first);
+    ASSERT_EQ(expectedValue, pair.second);
+  }
+
+  expectedValue = 0;
+  for (auto iter = my_map.begin(); iter != my_map.end(); iter++) {
+    int data = 'a' + expectedValue++;
+    ASSERT_EQ(Slice(reinterpret_cast<uint8_t*>(&data), 1), iter->first);
+    ASSERT_EQ(expectedValue, iter->second);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/slice.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/slice.cc b/be/src/kudu/util/slice.cc
new file mode 100644
index 0000000..775d54a
--- /dev/null
+++ b/be/src/kudu/util/slice.cc
@@ -0,0 +1,97 @@
+// 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 "kudu/util/slice.h"
+
+#include <cctype>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/util/status.h"
+#include "kudu/util/logging.h"
+
+namespace kudu {
+
+Status Slice::check_size(size_t expected_size) const {
+  if (PREDICT_FALSE(size() != expected_size)) {
+    return Status::Corruption(StringPrintf("Unexpected Slice size. "
+        "Expected %zu but got %zu.", expected_size, size()), KUDU_REDACT(ToDebugString(100)));
+  }
+  return Status::OK();
+}
+
+// Return a string that contains the copy of the referenced data.
+std::string Slice::ToString() const {
+  return std::string(reinterpret_cast<const char *>(data_), size_);
+}
+
+std::string Slice::ToDebugString(size_t max_len) const {
+  size_t bytes_to_print = size_;
+  bool abbreviated = false;
+  if (max_len != 0 && bytes_to_print > max_len) {
+    bytes_to_print = max_len;
+    abbreviated = true;
+  }
+
+  int size = 0;
+  for (int i = 0; i < bytes_to_print; i++) {
+    if (!isgraph(data_[i])) {
+      size += 4;
+    } else {
+      size++;
+    }
+  }
+  if (abbreviated) {
+    size += 20;  // extra padding
+  }
+
+  std::string ret;
+  ret.reserve(size);
+  for (int i = 0; i < bytes_to_print; i++) {
+    if (!isgraph(data_[i])) {
+      StringAppendF(&ret, "\\x%02x", data_[i] & 0xff);
+    } else {
+      ret.push_back(data_[i]);
+    }
+  }
+  if (abbreviated) {
+    StringAppendF(&ret, "...<%zd bytes total>", size_);
+  }
+  return ret;
+}
+
+bool IsAllZeros(const Slice& s) {
+  // Walk a pointer through the slice instead of using s[i]
+  // since this is way faster in debug mode builds. We also do some
+  // manual unrolling for the same purpose.
+  const uint8_t* p = &s[0];
+  int rem = s.size();
+
+  while (rem >= 8) {
+    if (UNALIGNED_LOAD64(p) != 0) return false;
+    rem -= 8;
+    p += 8;
+  }
+
+  while (rem > 0) {
+    if (*p++ != '\0') return false;
+    rem--;
+  }
+  return true;
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/slice.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/slice.h b/be/src/kudu/util/slice.h
new file mode 100644
index 0000000..d34c744
--- /dev/null
+++ b/be/src/kudu/util/slice.h
@@ -0,0 +1,332 @@
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+
+#ifndef KUDU_UTIL_SLICE_H_
+#define KUDU_UTIL_SLICE_H_
+
+// NOTE: using stdint.h instead of cstdint because this file is supposed
+//       to be processed by a compiler lacking C++11 support.
+#include <stdint.h>
+
+#include <cassert>
+#include <cstddef>
+#include <cstring>
+#include <iosfwd>
+#include <map>
+#include <string>
+
+#ifdef KUDU_HEADERS_USE_RICH_SLICE
+#include "kudu/gutil/strings/fastmem.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/faststring.h"
+#endif
+#ifdef KUDU_HEADERS_NO_STUBS
+#include "kudu/gutil/port.h"
+#endif
+#include "kudu/util/kudu_export.h"
+
+namespace kudu {
+
+class Status;
+
+/// @brief A wrapper around externally allocated data.
+///
+/// Slice is a simple structure containing a pointer into some external
+/// storage and a size. The user of a Slice must ensure that the slice
+/// is not used after the corresponding external storage has been
+/// deallocated.
+///
+/// Multiple threads can invoke const methods on a Slice without
+/// external synchronization, but if any of the threads may call a
+/// non-const method, all threads accessing the same Slice must use
+/// external synchronization.
+///
+/// Slices can be built around faststrings and StringPieces using constructors
+/// with implicit casts. Both StringPieces and faststrings depend on a great
+/// deal of gutil code.
+class KUDU_EXPORT Slice {
+ public:
+  /// Create an empty slice.
+  Slice() : data_(reinterpret_cast<const uint8_t *>("")),
+            size_(0) { }
+
+  /// Create a slice that refers to a @c uint8_t byte array.
+  ///
+  /// @param [in] d
+  ///   The input array.
+  /// @param [in] n
+  ///   Number of bytes in the array.
+  Slice(const uint8_t* d, size_t n) : data_(d), size_(n) { }
+
+  /// Create a slice that refers to a @c char byte array.
+  ///
+  /// @param [in] d
+  ///   The input array.
+  /// @param [in] n
+  ///   Number of bytes in the array.
+  Slice(const char* d, size_t n) :
+    data_(reinterpret_cast<const uint8_t *>(d)),
+    size_(n) { }
+
+  /// Create a slice that refers to the contents of the given string.
+  ///
+  /// @param [in] s
+  ///   The input string.
+  Slice(const std::string& s) : // NOLINT(runtime/explicit)
+    data_(reinterpret_cast<const uint8_t *>(s.data())),
+    size_(s.size()) { }
+
+  /// Create a slice that refers to a C-string s[0,strlen(s)-1].
+  ///
+  /// @param [in] s
+  ///   The input C-string.
+  Slice(const char* s) : // NOLINT(runtime/explicit)
+    data_(reinterpret_cast<const uint8_t *>(s)),
+    size_(strlen(s)) { }
+
+#ifdef KUDU_HEADERS_USE_RICH_SLICE
+  /// Create a slice that refers to the contents of a faststring.
+  ///
+  /// @note Further appends to the faststring may invalidate this slice.
+  ///
+  /// @param [in] s
+  ///   The input faststring.
+  Slice(const faststring &s) // NOLINT(runtime/explicit)
+    : data_(s.data()),
+      size_(s.size()) {
+  }
+
+  /// Create a slice that refers to the contents of a string piece.
+  ///
+  /// @param [in] s
+  ///   The input StringPiece.
+  Slice(const StringPiece& s) // NOLINT(runtime/explicit)
+    : data_(reinterpret_cast<const uint8_t*>(s.data())),
+      size_(s.size()) {
+  }
+#endif
+
+  /// @return A pointer to the beginning of the referenced data.
+  const uint8_t* data() const { return data_; }
+
+  /// @return A mutable pointer to the beginning of the referenced data.
+  uint8_t *mutable_data() { return const_cast<uint8_t *>(data_); }
+
+  /// @return The length (in bytes) of the referenced data.
+  size_t size() const { return size_; }
+
+  /// @return @c true iff the length of the referenced data is zero.
+  bool empty() const { return size_ == 0; }
+
+  /// @pre n < size()
+  ///
+  /// @param [in] n
+  ///   The index of the byte.
+  /// @return the n-th byte in the referenced data.
+  const uint8_t &operator[](size_t n) const {
+    assert(n < size());
+    return data_[n];
+  }
+
+  /// Change this slice to refer to an empty array.
+  void clear() {
+    data_ = reinterpret_cast<const uint8_t *>("");
+    size_ = 0;
+  }
+
+  /// Drop the first "n" bytes from this slice.
+  ///
+  /// @pre n <= size()
+  ///
+  /// @note Only the base and bounds of the slice are changed;
+  ///   the data is not modified.
+  ///
+  /// @param [in] n
+  ///   Number of bytes that should be dropped from the beginning.
+  void remove_prefix(size_t n) {
+    assert(n <= size());
+    data_ += n;
+    size_ -= n;
+  }
+
+  /// Truncate the slice to the given number of bytes.
+  ///
+  /// @pre n <= size()
+  ///
+  /// @note Only the base and bounds of the slice are changed;
+  ///   the data is not modified.
+  ///
+  /// @param [in] n
+  ///   The new size of the slice.
+  void truncate(size_t n) {
+    assert(n <= size());
+    size_ = n;
+  }
+
+  /// Check that the slice has the expected size.
+  ///
+  /// @param [in] expected_size
+  /// @return Status::Corruption() iff size() != @c expected_size
+  Status check_size(size_t expected_size) const;
+
+  /// @return A string that contains a copy of the referenced data.
+  std::string ToString() const;
+
+  /// Get printable representation of the data in the slice.
+  ///
+  /// @param [in] max_len
+  ///   The maximum number of bytes to output in the printable format;
+  ///   @c 0 means no limit.
+  /// @return A string with printable representation of the data.
+  std::string ToDebugString(size_t max_len = 0) const;
+
+  /// Do a three-way comparison of the slice's data.
+  ///
+  /// @param [in] b
+  ///   The other slice to compare with.
+  /// @return Values are
+  ///   @li <  0 iff "*this" <  "b"
+  ///   @li == 0 iff "*this" == "b"
+  ///   @li >  0 iff "*this" >  "b"
+  int compare(const Slice& b) const;
+
+  /// Check whether the slice starts with the given prefix.
+  /// @param [in] x
+  ///   The slice in question.
+  /// @return @c true iff "x" is a prefix of "*this"
+  bool starts_with(const Slice& x) const {
+    return ((size_ >= x.size_) &&
+            (MemEqual(data_, x.data_, x.size_)));
+  }
+
+  /// @brief Comparator struct, useful for ordered collections (like STL maps).
+  struct Comparator {
+    /// Compare two slices using Slice::compare()
+    ///
+    /// @param [in] a
+    ///   The slice to call Slice::compare() at.
+    /// @param [in] b
+    ///   The slice to use as a parameter for Slice::compare().
+    /// @return @c true iff @c a is less than @c b by Slice::compare().
+    bool operator()(const Slice& a, const Slice& b) const {
+      return a.compare(b) < 0;
+    }
+  };
+
+  /// Relocate/copy the slice's data into a new location.
+  ///
+  /// @param [in] d
+  ///   The new location for the data. If it's the same location, then no
+  ///   relocation is done. It is assumed that the new location is
+  ///   large enough to fit the data.
+  void relocate(uint8_t* d) {
+    if (data_ != d) {
+      memcpy(d, data_, size_);
+      data_ = d;
+    }
+  }
+
+ private:
+  friend bool operator==(const Slice& x, const Slice& y);
+
+  static bool MemEqual(const void* a, const void* b, size_t n) {
+#ifdef KUDU_HEADERS_USE_RICH_SLICE
+    return strings::memeq(a, b, n);
+#else
+    return memcmp(a, b, n) == 0;
+#endif
+  }
+
+  static int MemCompare(const void* a, const void* b, size_t n) {
+#ifdef KUDU_HEADERS_USE_RICH_SLICE
+    return strings::fastmemcmp_inlined(a, b, n);
+#else
+    return memcmp(a, b, n);
+#endif
+  }
+
+  const uint8_t* data_;
+  size_t size_;
+
+  // Intentionally copyable
+};
+
+/// Check whether two slices are identical.
+///
+/// @param [in] x
+///   One slice.
+/// @param [in] y
+///   Another slice.
+/// @return @c true iff two slices contain byte-for-byte identical data.
+inline bool operator==(const Slice& x, const Slice& y) {
+  return ((x.size() == y.size()) &&
+          (Slice::MemEqual(x.data(), y.data(), x.size())));
+}
+
+/// Check whether two slices are not identical.
+///
+/// @param [in] x
+///   One slice.
+/// @param [in] y
+///   Another slice.
+/// @return @c true iff slices contain different data.
+inline bool operator!=(const Slice& x, const Slice& y) {
+  return !(x == y);
+}
+
+/// Output printable representation of the slice into the given output stream.
+///
+/// @param [out] o
+///   The output stream to print the info.
+/// @param [in] s
+///   The slice to print.
+/// @return Reference to the updated output stream.
+inline std::ostream& operator<<(std::ostream& o, const Slice& s) {
+  return o << s.ToDebugString(16); // should be enough for anyone...
+}
+
+inline int Slice::compare(const Slice& b) const {
+  const int min_len = (size_ < b.size_) ? size_ : b.size_;
+  int r = MemCompare(data_, b.data_, min_len);
+  if (r == 0) {
+    if (size_ < b.size_) r = -1;
+    else if (size_ > b.size_) r = +1;
+  }
+  return r;
+}
+
+// We don't run TSAN on this function because it makes it really slow and causes some
+// test timeouts. This is only used on local buffers anyway, so we don't lose much
+// by not checking it.
+#ifdef KUDU_HEADERS_NO_STUBS
+ATTRIBUTE_NO_SANITIZE_THREAD
+#endif
+bool IsAllZeros(const Slice& s);
+
+/// @brief STL map whose keys are Slices.
+///
+/// An example of usage:
+/// @code
+///   typedef SliceMap<int>::type MySliceMap;
+///
+///   MySliceMap my_map;
+///   my_map.insert(MySliceMap::value_type(a, 1));
+///   my_map.insert(MySliceMap::value_type(b, 2));
+///   my_map.insert(MySliceMap::value_type(c, 3));
+///
+///   for (const MySliceMap::value_type& pair : my_map) {
+///     ...
+///   }
+/// @endcode
+template <typename T>
+struct SliceMap {
+  /// A handy typedef for the slice map with appropriate comparison operator.
+  typedef std::map<Slice, T, Slice::Comparator> type;
+};
+
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_SLICE_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/sorted_disjoint_interval_list-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/sorted_disjoint_interval_list-test.cc b/be/src/kudu/util/sorted_disjoint_interval_list-test.cc
new file mode 100644
index 0000000..8e0fe70
--- /dev/null
+++ b/be/src/kudu/util/sorted_disjoint_interval_list-test.cc
@@ -0,0 +1,98 @@
+// 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 <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/sorted_disjoint_interval_list.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::vector;
+
+namespace kudu {
+
+class TestSortedDisjointIntervalList : public KuduTest {
+};
+
+typedef int PointType;
+typedef std::pair<PointType, PointType> ClosedInterval;
+
+TEST_F(TestSortedDisjointIntervalList, TestBasic) {
+  // Coalesce an empty interval list.
+  vector<ClosedInterval> intervals = {};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  vector<ClosedInterval> expected = {};
+  ASSERT_EQ(expected, intervals);
+
+  // Coalesce an interval list with length 0 interval.
+  intervals = {{26, 26}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  expected = {{26, 26}};
+  ASSERT_EQ(expected, intervals);
+
+  // Coalesce an interval list with a single interval.
+  intervals = {{33, 69}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  expected = {{33, 69}};
+  ASSERT_EQ(expected, intervals);
+
+  // Coalesce an interval list with adjacent ranges.
+  intervals = {{4, 7}, {3, 4}, {1, 2}, {-23, 1}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  expected = {{-23, 2}, {3, 7}};
+  ASSERT_EQ(expected, intervals);
+}
+
+TEST_F(TestSortedDisjointIntervalList, TestOverlappedIntervals) {
+  vector<ClosedInterval> intervals = {{4, 7}, {3, 9}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  vector<ClosedInterval> expected = {{3, 9}};
+  ASSERT_EQ(expected, intervals);
+
+  intervals = {{4, 7}, {3, 9}, {-23, 1},
+               {4, 350}, {369, 400}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  expected = {{-23, 1}, {3, 350}, {369, 400}};
+  ASSERT_EQ(expected, intervals);
+}
+
+TEST_F(TestSortedDisjointIntervalList, TestDuplicateIntervals) {
+  vector<ClosedInterval> intervals = {{1, 2}, {4, 7},
+                                      {1, 2}, {1, 2}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  const vector<ClosedInterval> expected = {{1, 2}, {4, 7}};
+  ASSERT_EQ(expected, intervals);
+}
+
+TEST_F(TestSortedDisjointIntervalList, TestInvalidIntervals) {
+  vector<ClosedInterval> intervals = {{1, 2}, {10, 2},
+                                      {4, 7}, {40, 7}};
+  ASSERT_TRUE(CoalesceIntervals<PointType>(&intervals).IsInvalidArgument());
+}
+
+TEST_F(TestSortedDisjointIntervalList, TestSingleElementIntervals) {
+  vector<ClosedInterval> intervals = {{0, 0}, {0, 1}, {1, 2}};
+  ASSERT_OK(CoalesceIntervals<PointType>(&intervals));
+  const vector<ClosedInterval> expected = {{0, 2}};
+  ASSERT_EQ(expected, intervals);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/sorted_disjoint_interval_list.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/sorted_disjoint_interval_list.h b/be/src/kudu/util/sorted_disjoint_interval_list.h
new file mode 100644
index 0000000..d3180a9
--- /dev/null
+++ b/be/src/kudu/util/sorted_disjoint_interval_list.h
@@ -0,0 +1,95 @@
+// 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.
+
+#pragma once
+
+#include <algorithm>
+#include <cstdint>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+// Constructs a sorted disjoint interval list in-place given a list of intervals.
+// The result is written back to the given 'intervals'.
+//
+// Returns an error if the list contains any invalid intervals.
+//
+// Sorted disjoint interval list is a data structure holding a group of sorted
+// non-overlapping ranges. The operation to construct such one is O(nlg n + n)
+// where 'n' is the number of intervals.
+//
+// For example, given the input interval list:
+//
+//   [------2-------)         [-----1-----)
+//       [--3--)    [---5--)    [----4----)
+//
+// The output sorted disjoint interval list:
+//
+//   [----------1----------)  [-----2-----)
+//
+//
+// This method assumes that all intervals are "half-open" intervals -- the
+// intervals are inclusive of their start point and exclusive of end point,
+// e.g., [3, 6). Note that interval with the same start and end point is
+// considered to be valid in this implementation.
+// It also assumes 'PointType' has a proper defined comparator.
+template<typename PointType>
+Status CoalesceIntervals(std::vector<std::pair<PointType, PointType>>* intervals) {
+  if (intervals->empty()) return Status::OK();
+
+  // Sort the intervals to prepare for coalescing overlapped ranges.
+  for (const auto& interval : *intervals) {
+    if (interval.first > interval.second) {
+      return Status::InvalidArgument(strings::Substitute("invalid interval: [$0, $1)",
+                                                         interval.first,
+                                                         interval.second));
+    }
+  }
+  std::sort(intervals->begin(), intervals->end());
+
+  // Traverse the intervals to coalesce overlapped intervals. During the process,
+  // uses 'head', 'tail' to track the start and end point of the current disjoint
+  // interval.
+  auto head = intervals->begin();
+  auto tail = head;
+  while (++tail != intervals->end()) {
+    // If interval 'head' and 'tail' overlap with each other, coalesce them and move
+    // to next. Otherwise, the two intervals are disjoint.
+    if (head->second >= tail->first) {
+      if (tail->second > head->second) head->second = std::move(tail->second);
+    } else {
+      // The two intervals are disjoint. If the 'head' previously already coalesced
+      // some intervals, 'head' and 'tail' will not be adjacent. If so, move 'tail'
+      // to the next 'head' to make sure we do not include any of the previously-coalesced
+      // intervals.
+      ++head;
+      if (head != tail) *head = std::move(*tail);
+    }
+  }
+
+  // Truncate the rest useless elements, if any.
+  intervals->erase(++head, tail);
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/spinlock_profiling-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/spinlock_profiling-test.cc b/be/src/kudu/util/spinlock_profiling-test.cc
new file mode 100644
index 0000000..d0ef2b4
--- /dev/null
+++ b/be/src/kudu/util/spinlock_profiling-test.cc
@@ -0,0 +1,81 @@
+// 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 <cstdint>
+#include <ostream>
+#include <string>
+
+#include <gtest/gtest.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/util/spinlock_profiling.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/trace.h"
+
+// Can't include gutil/synchronization_profiling.h directly as it'll
+// declare a weak symbol directly in this unit test, which the runtime
+// linker will prefer over equivalent strong symbols for some reason. By
+// declaring the symbol without providing an empty definition, the strong
+// symbols are chosen when provided via shared libraries.
+//
+// Further reading:
+// - http://stackoverflow.com/questions/20658809/dynamic-loading-and-weak-symbol-resolution
+// - http://notmysock.org/blog/php/weak-symbols-arent.html
+namespace gutil {
+extern void SubmitSpinLockProfileData(const void *, int64);
+} // namespace gutil
+
+namespace kudu {
+
+class SpinLockProfilingTest : public KuduTest {};
+
+TEST_F(SpinLockProfilingTest, TestSpinlockProfiling) {
+  scoped_refptr<Trace> t(new Trace);
+  base::SpinLock lock;
+  {
+    ADOPT_TRACE(t.get());
+    gutil::SubmitSpinLockProfileData(&lock, 4000000);
+  }
+  std::string result = t->DumpToString();
+  LOG(INFO) << "trace: " << result;
+  ASSERT_STR_CONTAINS(result, "\"spinlock_wait_cycles\":4000000");
+  // We can't assert more specifically because the CyclesPerSecond
+  // on different machines might be different.
+  ASSERT_STR_CONTAINS(result, "Waited ");
+  ASSERT_STR_CONTAINS(result, "on lock ");
+
+  ASSERT_GT(GetSpinLockContentionMicros(), 0);
+}
+
+TEST_F(SpinLockProfilingTest, TestStackCollection) {
+  StartSynchronizationProfiling();
+  base::SpinLock lock;
+  gutil::SubmitSpinLockProfileData(&lock, 12345);
+  StopSynchronizationProfiling();
+  std::ostringstream str;
+  int64_t dropped = 0;
+  FlushSynchronizationProfile(&str, &dropped);
+  std::string s = str.str();
+  ASSERT_STR_CONTAINS(s, "12345 1 @ ");
+  ASSERT_EQ(0, dropped);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/spinlock_profiling.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/spinlock_profiling.cc b/be/src/kudu/util/spinlock_profiling.cc
new file mode 100644
index 0000000..e7f93b0
--- /dev/null
+++ b/be/src/kudu/util/spinlock_profiling.cc
@@ -0,0 +1,308 @@
+// 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 "kudu/util/spinlock_profiling.h"
+
+#include <sstream>
+#include <string>
+
+#include <glog/logging.h>
+#include <gflags/gflags.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/casts.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/spinlock.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/striped64.h"
+#include "kudu/util/trace.h"
+
+DEFINE_int32(lock_contention_trace_threshold_cycles,
+             2000000, // 2M cycles should be about 1ms
+             "If acquiring a spinlock takes more than this number of "
+             "cycles, and a Trace is currently active, then the current "
+             "stack trace is logged to the trace buffer.");
+TAG_FLAG(lock_contention_trace_threshold_cycles, hidden);
+
+METRIC_DEFINE_gauge_uint64(server, spinlock_contention_time,
+    "Spinlock Contention Time", kudu::MetricUnit::kMicroseconds,
+    "Amount of time consumed by contention on internal spinlocks since the server "
+    "started. If this increases rapidly, it may indicate a performance issue in Kudu "
+    "internals triggered by a particular workload and warrant investigation.",
+    kudu::EXPOSE_AS_COUNTER);
+
+
+using base::SpinLock;
+using base::SpinLockHolder;
+
+namespace kudu {
+
+static const double kMicrosPerSecond = 1000000.0;
+
+static LongAdder* g_contended_cycles = nullptr;
+
+namespace {
+
+// Implements a very simple linear-probing hashtable of stack traces with
+// a fixed number of entries.
+//
+// Threads experiencing contention record their stacks into this hashtable,
+// or increment an already-existing entry. Each entry has its own lock,
+// but we can "skip" an entry under contention, and spread out a single stack
+// into multiple buckets if necessary.
+//
+// A thread collecting a profile collects stack traces out of the hash table
+// and resets the counts to 0 as they are collected.
+class ContentionStacks {
+ public:
+  ContentionStacks()
+    : dropped_samples_(0) {
+  }
+
+  // Add a stack trace to the table.
+  void AddStack(const StackTrace& s, int64_t cycles);
+
+  // Flush stacks from the buffer to 'out'. See the docs for FlushSynchronizationProfile()
+  // in spinlock_profiling.h for details on format.
+  //
+  // On return, guarantees that any stack traces that were present at the beginning of
+  // the call have been flushed. However, new stacks can be added concurrently with this call.
+  void Flush(std::ostringstream* out, int64_t* dropped);
+
+ private:
+
+  // Collect the next sample from the underlying buffer, and set it back to 0 count
+  // (thus marking it as "empty").
+  //
+  // 'iterator' serves as a way to keep track of the current position in the buffer.
+  // Callers should initially set it to 0, and then pass the same pointer to each
+  // call to CollectSample. This serves to loop through the collected samples.
+  bool CollectSample(uint64_t* iterator, StackTrace* s, int64_t* trip_count, int64_t* cycles);
+
+  // Hashtable entry.
+  struct Entry {
+    Entry() : trip_count(0),
+              cycle_count(0) {
+    }
+
+    // Protects all other entries.
+    SpinLock lock;
+
+    // The number of times we've experienced contention with a stack trace equal
+    // to 'trace'.
+    //
+    // If this is 0, then the entry is "unclaimed" and the other fields are not
+    // considered valid.
+    int64_t trip_count;
+
+    // The total number of cycles spent waiting at this stack trace.
+    int64_t cycle_count;
+
+    // A cached hashcode of the trace.
+    uint64_t hash;
+
+    // The actual stack trace.
+    StackTrace trace;
+  };
+
+  enum {
+    kNumEntries = 1024,
+    kNumLinearProbeAttempts = 4
+  };
+  Entry entries_[kNumEntries];
+
+  // The number of samples which were dropped due to contention on this structure or
+  // due to the hashtable being too full.
+  AtomicInt<int64_t> dropped_samples_;
+};
+
+Atomic32 g_profiling_enabled = 0;
+ContentionStacks* g_contention_stacks = nullptr;
+
+void ContentionStacks::AddStack(const StackTrace& s, int64_t cycles) {
+  uint64_t hash = s.HashCode();
+
+  // Linear probe up to 4 attempts before giving up
+  for (int i = 0; i < kNumLinearProbeAttempts; i++) {
+    Entry* e = &entries_[(hash + i) % kNumEntries];
+    if (!e->lock.TryLock()) {
+      // If we fail to lock it, we can safely just use a different slot.
+      // It's OK if a single stack shows up multiple times, because pprof
+      // aggregates them in the end anyway.
+      continue;
+    }
+
+    if (e->trip_count == 0) {
+      // It's an un-claimed slot. Claim it.
+      e->hash = hash;
+      e->trace.CopyFrom(s);
+    } else if (e->hash != hash || !e->trace.Equals(s)) {
+      // It's claimed by a different stack trace.
+      e->lock.Unlock();
+      continue;
+    }
+
+    // Contribute to the stats for this stack.
+    e->cycle_count += cycles;
+    e->trip_count++;
+    e->lock.Unlock();
+    return;
+  }
+
+  // If we failed to find a matching hashtable slot, or we hit lock contention
+  // trying to record our sample, add it to the dropped sample count.
+  dropped_samples_.Increment();
+}
+
+void ContentionStacks::Flush(std::ostringstream* out, int64_t* dropped) {
+  uint64_t iterator = 0;
+  StackTrace t;
+  int64_t cycles;
+  int64_t count;
+  while (g_contention_stacks->CollectSample(&iterator, &t, &count, &cycles)) {
+    *out << cycles << " " << count
+         << " @ " << t.ToHexString(StackTrace::NO_FIX_CALLER_ADDRESSES |
+                                   StackTrace::HEX_0X_PREFIX)
+         << std::endl;
+  }
+
+  *dropped += dropped_samples_.Exchange(0);
+}
+
+bool ContentionStacks::CollectSample(uint64_t* iterator, StackTrace* s, int64_t* trip_count,
+                                     int64_t* cycles) {
+  while (*iterator < kNumEntries) {
+    Entry* e = &entries_[(*iterator)++];
+    SpinLockHolder l(&e->lock);
+    if (e->trip_count == 0) continue;
+
+    *trip_count = e->trip_count;
+    *cycles = e->cycle_count;
+    s->CopyFrom(e->trace);
+
+    e->trip_count = 0;
+    e->cycle_count = 0;
+    return true;
+  }
+
+  // Looped through the whole array and found nothing.
+  return false;
+}
+
+
+void SubmitSpinLockProfileData(const void *contendedlock, int64_t wait_cycles) {
+  TRACE_COUNTER_INCREMENT("spinlock_wait_cycles", wait_cycles);
+  bool profiling_enabled = base::subtle::Acquire_Load(&g_profiling_enabled);
+  bool long_wait_time = wait_cycles > FLAGS_lock_contention_trace_threshold_cycles;
+  // Short circuit this function quickly in the common case.
+  if (PREDICT_TRUE(!profiling_enabled && !long_wait_time)) {
+    return;
+  }
+
+  static __thread bool in_func = false;
+  if (in_func) return; // non-re-entrant
+  in_func = true;
+
+  StackTrace stack;
+  stack.Collect();
+
+  if (profiling_enabled) {
+    DCHECK_NOTNULL(g_contention_stacks)->AddStack(stack, wait_cycles);
+  }
+
+  if (PREDICT_FALSE(long_wait_time)) {
+    Trace* t = Trace::CurrentTrace();
+    if (t) {
+      double seconds = static_cast<double>(wait_cycles) / base::CyclesPerSecond();
+      char backtrace_buffer[1024];
+      stack.StringifyToHex(backtrace_buffer, arraysize(backtrace_buffer));
+      TRACE_TO(t, "Waited $0 on lock $1. stack: $2",
+               HumanReadableElapsedTime::ToShortString(seconds), contendedlock,
+               backtrace_buffer);
+    }
+  }
+
+  LongAdder* la = reinterpret_cast<LongAdder*>(
+      base::subtle::Acquire_Load(reinterpret_cast<AtomicWord*>(&g_contended_cycles)));
+  if (la) {
+    la->IncrementBy(wait_cycles);
+  }
+
+  in_func = false;
+}
+
+void DoInit() {
+  base::subtle::Release_Store(reinterpret_cast<AtomicWord*>(&g_contention_stacks),
+                              reinterpret_cast<uintptr_t>(new ContentionStacks()));
+  base::subtle::Release_Store(reinterpret_cast<AtomicWord*>(&g_contended_cycles),
+                              reinterpret_cast<uintptr_t>(new LongAdder()));
+}
+
+} // anonymous namespace
+
+void InitSpinLockContentionProfiling() {
+  static GoogleOnceType once = GOOGLE_ONCE_INIT;
+  GoogleOnceInit(&once, DoInit);
+}
+
+
+void RegisterSpinLockContentionMetrics(const scoped_refptr<MetricEntity>& entity) {
+  InitSpinLockContentionProfiling();
+  entity->NeverRetire(
+      METRIC_spinlock_contention_time.InstantiateFunctionGauge(
+          entity, Bind(&GetSpinLockContentionMicros)));
+}
+
+uint64_t GetSpinLockContentionMicros() {
+  int64_t wait_cycles = DCHECK_NOTNULL(g_contended_cycles)->Value();
+  double micros = static_cast<double>(wait_cycles) / base::CyclesPerSecond()
+    * kMicrosPerSecond;
+  return implicit_cast<int64_t>(micros);
+}
+
+void StartSynchronizationProfiling() {
+  InitSpinLockContentionProfiling();
+  base::subtle::Barrier_AtomicIncrement(&g_profiling_enabled, 1);
+}
+
+void FlushSynchronizationProfile(std::ostringstream* out,
+                                 int64_t* drop_count) {
+  CHECK_NOTNULL(g_contention_stacks)->Flush(out, drop_count);
+}
+
+void StopSynchronizationProfiling() {
+  InitSpinLockContentionProfiling();
+  CHECK_GE(base::subtle::Barrier_AtomicIncrement(&g_profiling_enabled, -1), 0);
+}
+
+} // namespace kudu
+
+// The hook expected by gutil is in the gutil namespace. Simply forward into the
+// kudu namespace so we don't need to qualify everything.
+namespace gutil {
+void SubmitSpinLockProfileData(const void *contendedlock, int64_t wait_cycles) {
+  kudu::SubmitSpinLockProfileData(contendedlock, wait_cycles);
+}
+} // namespace gutil

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/spinlock_profiling.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/spinlock_profiling.h b/be/src/kudu/util/spinlock_profiling.h
new file mode 100644
index 0000000..702eb18
--- /dev/null
+++ b/be/src/kudu/util/spinlock_profiling.h
@@ -0,0 +1,72 @@
+// 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.
+#ifndef KUDU_UTIL_SPINLOCK_PROFILING_H
+#define KUDU_UTIL_SPINLOCK_PROFILING_H
+
+#include <cstdint>
+#include <iosfwd>
+
+#include "kudu/gutil/ref_counted.h"
+
+namespace kudu {
+
+class MetricEntity;
+
+// Enable instrumentation of spinlock contention.
+//
+// Calling this method currently does nothing, except for ensuring
+// that the spinlock_profiling.cc object file gets linked into your
+// executable. It needs to be somewhere reachable in your code,
+// just so that gcc doesn't omit the underlying module from the binary.
+void InitSpinLockContentionProfiling();
+
+// Return the total number of microseconds spent in spinlock contention
+// since the server started.
+uint64_t GetSpinLockContentionMicros();
+
+// Register metrics in the given server entity which measure the amount of
+// spinlock contention.
+void RegisterSpinLockContentionMetrics(const scoped_refptr<MetricEntity>& entity);
+
+// Enable process-wide synchronization profiling.
+//
+// While profiling is enabled, spinlock contention will be recorded in a buffer.
+// The caller should periodically call FlushSynchronizationProfile() to empty
+// the buffer, or else profiles may be dropped.
+void StartSynchronizationProfiling();
+
+// Flush the current buffer of contention profile samples to the given stream.
+//
+// Each stack trace that has been observed results in at least one line of the
+// following format:
+//   <cycles> <trip count> @ <hex stack trace>
+//
+// Flushing the data also clears the current buffer of trace samples.
+// This may be called while synchronization profiling is enabled or after it has
+// been disabled.
+//
+// *dropped_samples will be incremented by the number of samples which were dropped
+// due to the contention buffer overflowing. If profiling is enabled during this
+// call, then the 'drop_count' may be slightly out-of-date with respect to the
+// returned samples.
+void FlushSynchronizationProfile(std::ostringstream* out, int64_t* drop_count);
+
+// Stop collecting contention profiles.
+void StopSynchronizationProfiling();
+
+} // namespace kudu
+#endif /* KUDU_UTIL_SPINLOCK_PROFILING_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/stack_watchdog-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/stack_watchdog-test.cc b/be/src/kudu/util/stack_watchdog-test.cc
new file mode 100644
index 0000000..aefe220
--- /dev/null
+++ b/be/src/kudu/util/stack_watchdog-test.cc
@@ -0,0 +1,152 @@
+// 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 "kudu/util/kernel_stack_watchdog.h"
+
+#include <ostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::thread;
+using std::vector;
+using strings::Substitute;
+
+DECLARE_int32(hung_task_check_interval_ms);
+DECLARE_int32(inject_latency_on_kernel_stack_lookup_ms);
+
+namespace kudu {
+
+class StackWatchdogTest : public KuduTest {
+ public:
+  virtual void SetUp() OVERRIDE {
+    KuduTest::SetUp();
+    KernelStackWatchdog::GetInstance()->SaveLogsForTests(true);
+    ANNOTATE_BENIGN_RACE(&FLAGS_hung_task_check_interval_ms, "");
+    ANNOTATE_BENIGN_RACE(&FLAGS_inject_latency_on_kernel_stack_lookup_ms, "");
+    FLAGS_hung_task_check_interval_ms = 10;
+  }
+};
+
+// The KernelStackWatchdog is only enabled on Linux, since we can't get kernel
+// stack traces on other platforms.
+#if defined(__linux__)
+TEST_F(StackWatchdogTest, TestWatchdog) {
+  vector<string> log;
+  {
+    SCOPED_WATCH_STACK(20);
+    for (int i = 0; i < 50; i++) {
+      SleepFor(MonoDelta::FromMilliseconds(100));
+      log = KernelStackWatchdog::GetInstance()->LoggedMessagesForTests();
+      // Wait for several samples, since it's possible that we get unlucky
+      // and the watchdog sees us just before or after a sleep.
+      if (log.size() > 5) {
+        break;
+      }
+    }
+  }
+  string s = JoinStrings(log, "\n");
+  ASSERT_STR_CONTAINS(s, "TestWatchdog_Test::TestBody()");
+  ASSERT_STR_CONTAINS(s, "nanosleep");
+}
+#endif
+
+// Test that SCOPED_WATCH_STACK scopes can be nested.
+TEST_F(StackWatchdogTest, TestNestedScopes) {
+  vector<string> log;
+  int line1;
+  int line2;
+  {
+    SCOPED_WATCH_STACK(20); line1 = __LINE__;
+    {
+      SCOPED_WATCH_STACK(20); line2 = __LINE__;
+      for (int i = 0; i < 50; i++) {
+        SleepFor(MonoDelta::FromMilliseconds(100));
+        log = KernelStackWatchdog::GetInstance()->LoggedMessagesForTests();
+        if (log.size() > 3) {
+          break;
+        }
+      }
+    }
+  }
+
+  // Verify that both nested scopes were collected.
+  string s = JoinStrings(log, "\n");
+  ASSERT_STR_CONTAINS(s, Substitute("stack_watchdog-test.cc:$0", line1));
+  ASSERT_STR_CONTAINS(s, Substitute("stack_watchdog-test.cc:$0", line2));
+}
+
+TEST_F(StackWatchdogTest, TestPerformance) {
+  // Reset the check interval to be reasonable. Otherwise the benchmark
+  // wastes a lot of CPU running the watchdog thread too often.
+  FLAGS_hung_task_check_interval_ms = 500;
+  LOG_TIMING(INFO, "1M SCOPED_WATCH_STACK()s") {
+    for (int i = 0; i < 1000000; i++) {
+      SCOPED_WATCH_STACK(100);
+    }
+  }
+}
+
+// Stress test to ensure that we properly handle the case where threads are short-lived
+// and the watchdog may try to grab a stack of a thread that has already exited.
+//
+// This also serves as a benchmark -- we make the stack-grabbing especially slow and
+// ensure that we can still start and join threads quickly.
+TEST_F(StackWatchdogTest, TestShortLivedThreadsStress) {
+  // Run the stack watchdog continuously.
+  FLAGS_hung_task_check_interval_ms = 0;
+
+  // Make the actual stack trace collection slow. In practice we find that
+  // stack trace collection can often take quite some time due to symbolization, etc.
+  FLAGS_inject_latency_on_kernel_stack_lookup_ms = 1000;
+
+  MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(5);
+  vector<thread> threads(100);
+  int started = 0;
+  while (MonoTime::Now() < deadline) {
+    thread* t = &threads[started % threads.size()];
+    if (t->joinable()) {
+      t->join();
+    }
+    *t = thread([&]() {
+        // Trigger watchdog at 1ms, but then sleep for 2ms, to ensure that
+        // the watchdog has plenty of work to do.
+        SCOPED_WATCH_STACK(1);
+        SleepFor(MonoDelta::FromMilliseconds(2));
+      });
+    started++;
+  }
+  for (auto& t : threads) {
+    if (t.joinable()) t.join();
+  }
+  LOG(INFO) << "started and joined " << started << " threads";
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/status-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/status-test.cc b/be/src/kudu/util/status-test.cc
new file mode 100644
index 0000000..a0aef3d
--- /dev/null
+++ b/be/src/kudu/util/status-test.cc
@@ -0,0 +1,119 @@
+// Some portions Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include <cerrno>
+#include <string>
+#include <utility>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(StatusTest, TestPosixCode) {
+  Status ok = Status::OK();
+  ASSERT_EQ(0, ok.posix_code());
+  Status file_error = Status::IOError("file error", Slice(), ENOTDIR);
+  ASSERT_EQ(ENOTDIR, file_error.posix_code());
+}
+
+TEST(StatusTest, TestToString) {
+  Status file_error = Status::IOError("file error", Slice(), ENOTDIR);
+  ASSERT_EQ(string("IO error: file error (error 20)"), file_error.ToString());
+}
+
+TEST(StatusTest, TestClonePrepend) {
+  Status file_error = Status::IOError("file error", "msg2", ENOTDIR);
+  Status appended = file_error.CloneAndPrepend("Heading");
+  ASSERT_EQ(string("IO error: Heading: file error: msg2 (error 20)"), appended.ToString());
+}
+
+TEST(StatusTest, TestCloneAppend) {
+  Status remote_error = Status::RemoteError("Application error");
+  Status appended = remote_error.CloneAndAppend(Status::NotFound("Unknown tablet").ToString());
+  ASSERT_EQ(string("Remote error: Application error: Not found: Unknown tablet"),
+            appended.ToString());
+}
+
+TEST(StatusTest, TestMemoryUsage) {
+  ASSERT_EQ(0, Status::OK().memory_footprint_excluding_this());
+  ASSERT_GT(Status::IOError(
+      "file error", "some other thing", ENOTDIR).memory_footprint_excluding_this(), 0);
+}
+
+TEST(StatusTest, TestMoveConstructor) {
+  // OK->OK move should do nothing.
+  {
+    Status src = Status::OK();
+    Status dst = std::move(src);
+    ASSERT_OK(src); // NOLINT(bugprone-use-after-move)
+    ASSERT_OK(dst);
+  }
+
+  // Moving a not-OK status into a new one should make the moved status
+  // "OK".
+  {
+    Status src = Status::NotFound("foo");
+    Status dst = std::move(src);
+    ASSERT_OK(src); // NOLINT(bugprone-use-after-move)
+    ASSERT_EQ("Not found: foo", dst.ToString());
+  }
+}
+
+TEST(StatusTest, TestMoveAssignment) {
+  // OK->Bad move should clear the source status and also make the
+  // destination status OK.
+  {
+    Status src = Status::OK();
+    Status dst = Status::NotFound("orig dst");
+    dst = std::move(src);
+    ASSERT_OK(src); // NOLINT(bugprone-use-after-move)
+    ASSERT_OK(dst);
+  }
+
+  // Bad->Bad move.
+  {
+    Status src = Status::NotFound("orig src");
+    Status dst = Status::NotFound("orig dst");
+    dst = std::move(src);
+    ASSERT_OK(src); // NOLINT(bugprone-use-after-move)
+    ASSERT_EQ("Not found: orig src", dst.ToString());
+  }
+
+  // Bad->OK move
+  {
+    Status src = Status::NotFound("orig src");
+    Status dst = Status::OK();
+    dst = std::move(src);
+    ASSERT_OK(src); // NOLINT(bugprone-use-after-move)
+    ASSERT_EQ("Not found: orig src", dst.ToString());
+  }
+}
+
+TEST(StatusTest, TestAndThen) {
+  ASSERT_OK(Status::OK().AndThen(Status::OK)
+                        .AndThen(Status::OK)
+                        .AndThen(Status::OK));
+
+  ASSERT_TRUE(Status::InvalidArgument("").AndThen([] { return Status::IllegalState(""); })
+                                         .IsInvalidArgument());
+  ASSERT_TRUE(Status::InvalidArgument("").AndThen(Status::OK)
+                                         .IsInvalidArgument());
+  ASSERT_TRUE(Status::OK().AndThen([] { return Status::InvalidArgument(""); })
+                          .AndThen(Status::OK)
+                          .IsInvalidArgument());
+
+  ASSERT_EQ("foo: bar",
+            Status::OK().CloneAndPrepend("baz")
+                        .AndThen([] {
+                          return Status::InvalidArgument("bar").CloneAndPrepend("foo");
+                        }).message());
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/status.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/status.cc b/be/src/kudu/util/status.cc
new file mode 100644
index 0000000..1197682
--- /dev/null
+++ b/be/src/kudu/util/status.cc
@@ -0,0 +1,170 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "kudu/util/status.h"
+
+#include <cstdio>
+#include <cstring>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/fastmem.h"
+#include "kudu/util/malloc.h"
+
+namespace kudu {
+
+const char* Status::CopyState(const char* state) {
+  uint32_t size;
+  strings::memcpy_inlined(&size, state, sizeof(size));
+  auto result = new char[size + 7];
+  strings::memcpy_inlined(result, state, size + 7);
+  return result;
+}
+
+Status::Status(Code code, const Slice& msg, const Slice& msg2,
+               int16_t posix_code) {
+  DCHECK(code != kOk);
+  const uint32_t len1 = msg.size();
+  const uint32_t len2 = msg2.size();
+  const uint32_t size = len1 + (len2 ? (2 + len2) : 0);
+  auto result = new char[size + 7];
+  memcpy(result, &size, sizeof(size));
+  result[4] = static_cast<char>(code);
+  memcpy(result + 5, &posix_code, sizeof(posix_code));
+  memcpy(result + 7, msg.data(), len1);
+  if (len2) {
+    result[7 + len1] = ':';
+    result[8 + len1] = ' ';
+    memcpy(result + 9 + len1, msg2.data(), len2);
+  }
+  state_ = result;
+}
+
+std::string Status::CodeAsString() const {
+  if (state_ == nullptr) {
+    return "OK";
+  }
+
+  const char* type;
+  switch (code()) {
+    case kOk:
+      type = "OK";
+      break;
+    case kNotFound:
+      type = "Not found";
+      break;
+    case kCorruption:
+      type = "Corruption";
+      break;
+    case kNotSupported:
+      type = "Not implemented";
+      break;
+    case kInvalidArgument:
+      type = "Invalid argument";
+      break;
+    case kIOError:
+      type = "IO error";
+      break;
+    case kAlreadyPresent:
+      type = "Already present";
+      break;
+    case kRuntimeError:
+      type = "Runtime error";
+      break;
+    case kNetworkError:
+      type = "Network error";
+      break;
+    case kIllegalState:
+      type = "Illegal state";
+      break;
+    case kNotAuthorized:
+      type = "Not authorized";
+      break;
+    case kAborted:
+      type = "Aborted";
+      break;
+    case kRemoteError:
+      type = "Remote error";
+      break;
+    case kServiceUnavailable:
+      type = "Service unavailable";
+      break;
+    case kTimedOut:
+      type = "Timed out";
+      break;
+    case kUninitialized:
+      type = "Uninitialized";
+      break;
+    case kConfigurationError:
+      type = "Configuration error";
+      break;
+    case kIncomplete:
+      type = "Incomplete";
+      break;
+    case kEndOfFile:
+      type = "End of file";
+      break;
+  }
+  return std::string(type);
+}
+
+std::string Status::ToString() const {
+  std::string result(CodeAsString());
+  if (state_ == nullptr) {
+    return result;
+  }
+
+  result.append(": ");
+  Slice msg = message();
+  result.append(reinterpret_cast<const char*>(msg.data()), msg.size());
+  int16_t posix = posix_code();
+  if (posix != -1) {
+    char buf[64];
+    snprintf(buf, sizeof(buf), " (error %d)", posix);
+    result.append(buf);
+  }
+  return result;
+}
+
+Slice Status::message() const {
+  if (state_ == nullptr) {
+    return Slice();
+  }
+
+  uint32_t length;
+  memcpy(&length, state_, sizeof(length));
+  return Slice(state_ + 7, length);
+}
+
+int16_t Status::posix_code() const {
+  if (state_ == nullptr) {
+    return 0;
+  }
+  int16_t posix_code;
+  memcpy(&posix_code, state_ + 5, sizeof(posix_code));
+  return posix_code;
+}
+
+Status Status::CloneAndPrepend(const Slice& msg) const {
+  if (ok()) {
+    return *this;
+  }
+  return Status(code(), msg, message(), posix_code());
+}
+
+Status Status::CloneAndAppend(const Slice& msg) const {
+  if (ok()) {
+    return *this;
+  }
+  return Status(code(), message(), msg, posix_code());
+}
+
+size_t Status::memory_footprint_excluding_this() const {
+  return state_ ? kudu_malloc_usable_size(state_) : 0;
+}
+
+size_t Status::memory_footprint_including_this() const {
+  return kudu_malloc_usable_size(this) + memory_footprint_excluding_this();
+}
+}  // namespace kudu


[06/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/status.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/status.h b/be/src/kudu/util/status.h
new file mode 100644
index 0000000..3c8a1d9
--- /dev/null
+++ b/be/src/kudu/util/status.h
@@ -0,0 +1,493 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// A Status encapsulates the result of an operation.  It may indicate success,
+// or it may indicate an error with an associated error message.
+//
+// Multiple threads can invoke const methods on a Status without
+// external synchronization, but if any of the threads may call a
+// non-const method, all threads accessing the same Status must use
+// external synchronization.
+
+#ifndef KUDU_UTIL_STATUS_H_
+#define KUDU_UTIL_STATUS_H_
+
+// NOTE: using stdint.h instead of cstdint and errno.h instead of errno because
+// this file is supposed to be processed by a compiler lacking C++11 support.
+#include <errno.h>
+#include <stdint.h>
+
+#include <cstddef>
+#include <string>
+
+#ifdef KUDU_HEADERS_NO_STUBS
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#else
+#include "kudu/client/stubs.h"
+#endif
+
+#include "kudu/util/kudu_export.h"
+#include "kudu/util/slice.h"
+
+/// @brief Return the given status if it is not @c OK.
+#define KUDU_RETURN_NOT_OK(s) do { \
+    const ::kudu::Status& _s = (s);             \
+    if (PREDICT_FALSE(!_s.ok())) return _s;     \
+  } while (0);
+
+/// @brief Return the given status if it is not OK, but first clone it and
+///   prepend the given message.
+#define KUDU_RETURN_NOT_OK_PREPEND(s, msg) do { \
+    const ::kudu::Status& _s = (s);                              \
+    if (PREDICT_FALSE(!_s.ok())) return _s.CloneAndPrepend(msg); \
+  } while (0);
+
+/// @brief Return @c to_return if @c to_call returns a bad status.
+///   The substitution for 'to_return' may reference the variable
+///   @c s for the bad status.
+#define KUDU_RETURN_NOT_OK_RET(to_call, to_return) do { \
+    const ::kudu::Status& s = (to_call);                \
+    if (PREDICT_FALSE(!s.ok())) return (to_return);  \
+  } while (0);
+
+/// @brief Return the given status if it is not OK, evaluating `on_error` if so.
+#define KUDU_RETURN_NOT_OK_EVAL(s, on_error) do { \
+    const ::kudu::Status& _s = (s); \
+    if (PREDICT_FALSE(!_s.ok())) { \
+      (on_error); \
+      return _s; \
+    } \
+  } while (0);
+
+/// @brief Emit a warning if @c to_call returns a bad status.
+#define KUDU_WARN_NOT_OK(to_call, warning_prefix) do { \
+    const ::kudu::Status& _s = (to_call);              \
+    if (PREDICT_FALSE(!_s.ok())) { \
+      KUDU_LOG(WARNING) << (warning_prefix) << ": " << _s.ToString();  \
+    } \
+  } while (0);
+
+/// @brief Log the given status and return immediately.
+#define KUDU_LOG_AND_RETURN(level, status) do { \
+    const ::kudu::Status& _s = (status);        \
+    KUDU_LOG(level) << _s.ToString(); \
+    return _s; \
+  } while (0);
+
+/// @brief If the given status is not OK, log it and 'msg' at 'level' and return the status.
+#define KUDU_RETURN_NOT_OK_LOG(s, level, msg) do { \
+    const ::kudu::Status& _s = (s);             \
+    if (PREDICT_FALSE(!_s.ok())) { \
+      KUDU_LOG(level) << "Status: " << _s.ToString() << " " << (msg); \
+      return _s;     \
+    } \
+  } while (0);
+
+/// @brief If @c to_call returns a bad status, CHECK immediately with
+///   a logged message of @c msg followed by the status.
+#define KUDU_CHECK_OK_PREPEND(to_call, msg) do { \
+    const ::kudu::Status& _s = (to_call);                   \
+    KUDU_CHECK(_s.ok()) << (msg) << ": " << _s.ToString();  \
+  } while (0);
+
+/// @brief If the status is bad, CHECK immediately, appending the status to the
+///   logged message.
+#define KUDU_CHECK_OK(s) KUDU_CHECK_OK_PREPEND(s, "Bad status")
+
+/// @brief If @c to_call returns a bad status, DCHECK immediately with
+///   a logged message of @c msg followed by the status.
+#define KUDU_DCHECK_OK_PREPEND(to_call, msg) do { \
+    const ::kudu::Status& _s = (to_call);                   \
+    KUDU_DCHECK(_s.ok()) << (msg) << ": " << _s.ToString();  \
+  } while (0);
+
+/// @brief If the status is bad, DCHECK immediately, appending the status to the
+///   logged 'Bad status' message.
+#define KUDU_DCHECK_OK(s) KUDU_DCHECK_OK_PREPEND(s, "Bad status")
+
+/// @file status.h
+///
+/// This header is used in both the Kudu build as well as in builds of
+/// applications that use the Kudu C++ client. In the latter we need to be
+/// careful to "namespace" our macros, to avoid colliding or overriding with
+/// similarly named macros belonging to the application.
+///
+/// KUDU_HEADERS_USE_SHORT_STATUS_MACROS handles this behavioral change. When
+/// defined, we're building Kudu and:
+/// @li Non-namespaced macros are allowed and mapped to the namespaced versions
+///   defined above.
+/// @li Namespaced versions of glog macros are mapped to the real glog macros
+///   (otherwise the macros are defined in the C++ client stubs).
+#ifdef KUDU_HEADERS_USE_SHORT_STATUS_MACROS
+#define RETURN_NOT_OK         KUDU_RETURN_NOT_OK
+#define RETURN_NOT_OK_PREPEND KUDU_RETURN_NOT_OK_PREPEND
+#define RETURN_NOT_OK_RET     KUDU_RETURN_NOT_OK_RET
+#define RETURN_NOT_OK_EVAL    KUDU_RETURN_NOT_OK_EVAL
+#define WARN_NOT_OK           KUDU_WARN_NOT_OK
+#define LOG_AND_RETURN        KUDU_LOG_AND_RETURN
+#define RETURN_NOT_OK_LOG     KUDU_RETURN_NOT_OK_LOG
+#define CHECK_OK_PREPEND      KUDU_CHECK_OK_PREPEND
+#define CHECK_OK              KUDU_CHECK_OK
+#define DCHECK_OK_PREPEND     KUDU_DCHECK_OK_PREPEND
+#define DCHECK_OK             KUDU_DCHECK_OK
+
+// These are standard glog macros.
+#define KUDU_LOG              LOG
+#define KUDU_CHECK            CHECK
+#define KUDU_DCHECK           DCHECK
+#endif
+
+namespace kudu {
+
+/// @brief A representation of an operation's outcome.
+class KUDU_EXPORT Status {
+ public:
+  /// Create an object representing success status.
+  Status() : state_(NULL) { }
+
+  ~Status() { delete[] state_; }
+
+  /// Copy the specified status.
+  ///
+  /// @param [in] s
+  ///   The status object to copy from.
+  Status(const Status& s);
+
+  /// Assign the specified status.
+  ///
+  /// @param [in] s
+  ///   The status object to assign from.
+  /// @return The reference to the modified object.
+  Status& operator=(const Status& s);
+
+#if __cplusplus >= 201103L
+  /// Move the specified status (C++11).
+  ///
+  /// @param [in] s
+  ///   rvalue reference to a Status object.
+  Status(Status&& s) noexcept;
+
+  /// Assign the specified status using move semantics (C++11).
+  ///
+  /// @param [in] s
+  ///   rvalue reference to a Status object.
+  /// @return The reference to the modified object.
+  Status& operator=(Status&& s) noexcept;
+
+  /// If this status is OK, calls 'op' and returns the result, otherwise returns
+  /// this status.
+  ///
+  /// This method can be used to chain together multiple Status-returning
+  /// operations, short circuiting after the first one to fail.
+  ///
+  /// Example:
+  ///
+  /// @code
+  /// unique_ptr<SequentialFile> file;
+  /// Status s = Env::Default()
+  ///               ->NewSequentialFile("/tmp/example.txt", &file)
+  ///               .AndThen([&] {
+  ///                 return file->Write(0, "some data")
+  ///                             .CloneAndPrepend("failed to write to example file");
+  ///               });
+  /// @endcode
+  ///
+  /// @param [in] op
+  ///   Status-returning closure or function to run.
+  /// @return 'this', if this is not OK, or the result of running op.
+  template<typename F>
+  Status AndThen(F op) {
+    if (ok()) {
+      return op();
+    }
+    return *this;
+  }
+#endif
+
+  /// @return A success status.
+  static Status OK() { return Status(); }
+
+
+  /// @name Methods to build status objects for various types of errors.
+  ///
+  /// @param [in] msg
+  ///   The informational message on the error.
+  /// @param [in] msg2
+  ///   Additional information on the error (optional).
+  /// @param [in] posix_code
+  ///   POSIX error code, if applicable (optional).
+  /// @return The error status of an appropriate type.
+  ///
+  ///@{
+  static Status NotFound(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kNotFound, msg, msg2, posix_code);
+  }
+  static Status Corruption(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kCorruption, msg, msg2, posix_code);
+  }
+  static Status NotSupported(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kNotSupported, msg, msg2, posix_code);
+  }
+  static Status InvalidArgument(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kInvalidArgument, msg, msg2, posix_code);
+  }
+  static Status IOError(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kIOError, msg, msg2, posix_code);
+  }
+  static Status AlreadyPresent(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kAlreadyPresent, msg, msg2, posix_code);
+  }
+  static Status RuntimeError(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kRuntimeError, msg, msg2, posix_code);
+  }
+  static Status NetworkError(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kNetworkError, msg, msg2, posix_code);
+  }
+  static Status IllegalState(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kIllegalState, msg, msg2, posix_code);
+  }
+  static Status NotAuthorized(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kNotAuthorized, msg, msg2, posix_code);
+  }
+  static Status Aborted(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kAborted, msg, msg2, posix_code);
+  }
+  static Status RemoteError(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kRemoteError, msg, msg2, posix_code);
+  }
+  static Status ServiceUnavailable(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kServiceUnavailable, msg, msg2, posix_code);
+  }
+  static Status TimedOut(const Slice& msg, const Slice& msg2 = Slice(),
+                         int16_t posix_code = -1) {
+    return Status(kTimedOut, msg, msg2, posix_code);
+  }
+  static Status Uninitialized(const Slice& msg, const Slice& msg2 = Slice(),
+                              int16_t posix_code = -1) {
+    return Status(kUninitialized, msg, msg2, posix_code);
+  }
+  static Status ConfigurationError(const Slice& msg, const Slice& msg2 = Slice(),
+                                   int16_t posix_code = -1) {
+    return Status(kConfigurationError, msg, msg2, posix_code);
+  }
+  static Status Incomplete(const Slice& msg, const Slice& msg2 = Slice(),
+                           int64_t posix_code = -1) {
+    return Status(kIncomplete, msg, msg2, posix_code);
+  }
+  static Status EndOfFile(const Slice& msg, const Slice& msg2 = Slice(),
+                          int64_t posix_code = -1) {
+    return Status(kEndOfFile, msg, msg2, posix_code);
+  }
+  ///@}
+
+  /// @return @c true iff the status indicates success.
+  bool ok() const { return (state_ == NULL); }
+
+  /// @return @c true iff the status indicates a NotFound error.
+  bool IsNotFound() const { return code() == kNotFound; }
+
+  /// @return @c true iff the status indicates a Corruption error.
+  bool IsCorruption() const { return code() == kCorruption; }
+
+  /// @return @c true iff the status indicates a NotSupported error.
+  bool IsNotSupported() const { return code() == kNotSupported; }
+
+  /// @return @c true iff the status indicates an IOError.
+  bool IsIOError() const { return code() == kIOError; }
+
+  /// @return @c true iff the status indicates an InvalidArgument error.
+  bool IsInvalidArgument() const { return code() == kInvalidArgument; }
+
+  /// @return @c true iff the status indicates an AlreadyPresent error.
+  bool IsAlreadyPresent() const { return code() == kAlreadyPresent; }
+
+  /// @return @c true iff the status indicates a RuntimeError.
+  bool IsRuntimeError() const { return code() == kRuntimeError; }
+
+  /// @return @c true iff the status indicates a NetworkError.
+  bool IsNetworkError() const { return code() == kNetworkError; }
+
+  /// @return @c true iff the status indicates an IllegalState error.
+  bool IsIllegalState() const { return code() == kIllegalState; }
+
+  /// @return @c true iff the status indicates a NotAuthorized error.
+  bool IsNotAuthorized() const { return code() == kNotAuthorized; }
+
+  /// @return @c true iff the status indicates an Aborted error.
+  bool IsAborted() const { return code() == kAborted; }
+
+  /// @return @c true iff the status indicates a RemoteError.
+  bool IsRemoteError() const { return code() == kRemoteError; }
+
+  /// @return @c true iff the status indicates ServiceUnavailable.
+  bool IsServiceUnavailable() const { return code() == kServiceUnavailable; }
+
+  /// @return @c true iff the status indicates TimedOut.
+  bool IsTimedOut() const { return code() == kTimedOut; }
+
+  /// @return @c true iff the status indicates Uninitialized.
+  bool IsUninitialized() const { return code() == kUninitialized; }
+
+  /// @return @c true iff the status indicates ConfigurationError.
+  bool IsConfigurationError() const { return code() == kConfigurationError; }
+
+  /// @return @c true iff the status indicates Incomplete.
+  bool IsIncomplete() const { return code() == kIncomplete; }
+
+  /// @return @c true iff the status indicates end of file.
+  bool IsEndOfFile() const { return code() == kEndOfFile; }
+
+  /// @return @c true iff the status indicates a disk failure.
+  bool IsDiskFailure() const {
+    switch (posix_code()) {
+      case EIO:
+      case ENODEV:
+      case ENXIO:
+      case EROFS:
+        return true;
+    }
+    return false;
+  }
+
+  /// @return A string representation of this status suitable for printing.
+  ///   Returns the string "OK" for success.
+  std::string ToString() const;
+
+  /// @return A string representation of the status code, without the message
+  ///   text or POSIX code information.
+  std::string CodeAsString() const;
+
+  /// This is similar to ToString, except that it does not include
+  /// the stringified error code or POSIX code.
+  ///
+  /// @note The returned Slice is only valid as long as this Status object
+  ///   remains live and unchanged.
+  ///
+  /// @return The message portion of the Status. For @c OK statuses,
+  ///   this returns an empty string.
+  Slice message() const;
+
+  /// @return The POSIX code associated with this Status object,
+  ///   or @c -1 if there is none.
+  int16_t posix_code() const;
+
+  /// Clone this status and add the specified prefix to the message.
+  ///
+  /// If this status is OK, then an OK status will be returned.
+  ///
+  /// @param [in] msg
+  ///   The message to prepend.
+  /// @return A new Status object with the same state plus an additional
+  ///   leading message.
+  Status CloneAndPrepend(const Slice& msg) const;
+
+  /// Clone this status and add the specified suffix to the message.
+  ///
+  /// If this status is OK, then an OK status will be returned.
+  ///
+  /// @param [in] msg
+  ///   The message to append.
+  /// @return A new Status object with the same state plus an additional
+  ///   trailing message.
+  Status CloneAndAppend(const Slice& msg) const;
+
+  /// @return The memory usage of this object without the object itself.
+  ///   Should be used when embedded inside another object.
+  size_t memory_footprint_excluding_this() const;
+
+  /// @return The memory usage of this object including the object itself.
+  ///   Should be used when allocated on the heap.
+  size_t memory_footprint_including_this() const;
+
+ private:
+  // OK status has a NULL state_.  Otherwise, state_ is a new[] array
+  // of the following form:
+  //    state_[0..3] == length of message
+  //    state_[4]    == code
+  //    state_[5..6] == posix_code
+  //    state_[7..]  == message
+  const char* state_;
+
+  enum Code {
+    kOk = 0,
+    kNotFound = 1,
+    kCorruption = 2,
+    kNotSupported = 3,
+    kInvalidArgument = 4,
+    kIOError = 5,
+    kAlreadyPresent = 6,
+    kRuntimeError = 7,
+    kNetworkError = 8,
+    kIllegalState = 9,
+    kNotAuthorized = 10,
+    kAborted = 11,
+    kRemoteError = 12,
+    kServiceUnavailable = 13,
+    kTimedOut = 14,
+    kUninitialized = 15,
+    kConfigurationError = 16,
+    kIncomplete = 17,
+    kEndOfFile = 18,
+    // NOTE: Remember to duplicate these constants into wire_protocol.proto and
+    // and to add StatusTo/FromPB ser/deser cases in wire_protocol.cc !
+    // Also remember to make the same changes to the java client in Status.java.
+    //
+    // TODO: Move error codes into an error_code.proto or something similar.
+  };
+  COMPILE_ASSERT(sizeof(Code) == 4, code_enum_size_is_part_of_abi);
+
+  Code code() const {
+    return (state_ == NULL) ? kOk : static_cast<Code>(state_[4]);
+  }
+
+  Status(Code code, const Slice& msg, const Slice& msg2, int16_t posix_code);
+  static const char* CopyState(const char* s);
+};
+
+inline Status::Status(const Status& s) {
+  state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+}
+
+inline Status& Status::operator=(const Status& s) {
+  // The following condition catches both aliasing (when this == &s),
+  // and the common case where both s and *this are OK.
+  if (state_ != s.state_) {
+    delete[] state_;
+    state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+  }
+  return *this;
+}
+
+#if __cplusplus >= 201103L
+inline Status::Status(Status&& s) noexcept : state_(s.state_) {
+  s.state_ = nullptr;
+}
+
+inline Status& Status::operator=(Status&& s) noexcept {
+  if (state_ != s.state_) {
+    delete[] state_;
+    state_ = s.state_;
+    s.state_ = nullptr;
+  }
+  return *this;
+}
+#endif
+
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_STATUS_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/status_callback.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/status_callback.cc b/be/src/kudu/util/status_callback.cc
new file mode 100644
index 0000000..a3932b5
--- /dev/null
+++ b/be/src/kudu/util/status_callback.cc
@@ -0,0 +1,41 @@
+// 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 "kudu/util/status_callback.h"
+
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+
+void DoNothingStatusCB(const Status& status) {}
+
+void CrashIfNotOkStatusCB(const string& message, const Status& status) {
+  if (PREDICT_FALSE(!status.ok())) {
+    LOG(FATAL) << message << ": " << status.ToString();
+  }
+}
+
+Status DoNothingStatusClosure() { return Status::OK(); }
+
+} // end namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/status_callback.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/status_callback.h b/be/src/kudu/util/status_callback.h
new file mode 100644
index 0000000..70bbb97
--- /dev/null
+++ b/be/src/kudu/util/status_callback.h
@@ -0,0 +1,54 @@
+// 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.
+#ifndef KUDU_UTIL_STATUS_CALLBACK_H
+#define KUDU_UTIL_STATUS_CALLBACK_H
+
+#include <functional>
+#include <string>
+
+#include "kudu/gutil/callback_forward.h"
+
+namespace kudu {
+
+class Status;
+
+// A callback which takes a Status. This is typically used for functions which
+// produce asynchronous results and may fail.
+typedef Callback<void(const Status& status)> StatusCallback;
+
+// Like StatusCallback but uses the STL function objects.
+//
+// TODO(adar): should eventually replace all StatusCallback usage with this.
+typedef std::function<void(const Status& status)> StdStatusCallback;
+
+// To be used when a function signature requires a StatusCallback but none
+// is needed.
+extern void DoNothingStatusCB(const Status& status);
+
+// A callback that crashes with a FATAL log message if the given Status is not OK.
+extern void CrashIfNotOkStatusCB(const std::string& message, const Status& status);
+
+// A closure (callback without arguments) that returns a Status indicating
+// whether it was successful or not.
+typedef Callback<Status(void)> StatusClosure;
+
+// To be used when setting a StatusClosure is optional.
+extern Status DoNothingStatusClosure();
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/stopwatch.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/stopwatch.h b/be/src/kudu/util/stopwatch.h
new file mode 100644
index 0000000..f15b597
--- /dev/null
+++ b/be/src/kudu/util/stopwatch.h
@@ -0,0 +1,364 @@
+// 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.
+#ifndef KUDU_UTIL_STOPWATCH_H
+#define KUDU_UTIL_STOPWATCH_H
+
+#include <glog/logging.h>
+#include <sys/resource.h>
+#include <sys/time.h>
+#include <time.h>
+#include <string>
+#if defined(__APPLE__)
+#include <mach/clock.h>
+#include <mach/mach.h>
+#include <mach/thread_info.h>
+#endif  // defined(__APPLE__)
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/walltime.h"
+
+namespace kudu {
+
+// Macro for logging timing of a block. Usage:
+//   LOG_TIMING_PREFIX_IF(INFO, FLAGS_should_record_time, "Tablet X: ", "doing some task") {
+//     ... some task which takes some time
+//   }
+// If FLAGS_should_record_time is true, yields a log like:
+// I1102 14:35:51.726186 23082 file.cc:167] Tablet X: Time spent doing some task:
+//   real 3.729s user 3.570s sys 0.150s
+// The task will always execute regardless of whether the timing information is
+// printed.
+#define LOG_TIMING_PREFIX_IF(severity, condition, prefix, description) \
+  for (kudu::sw_internal::LogTiming _l(__FILE__, __LINE__, google::severity, prefix, description, \
+          -1, (condition)); !_l.HasRun(); _l.MarkHasRun())
+
+// Conditionally log, no prefix.
+#define LOG_TIMING_IF(severity, condition, description) \
+  LOG_TIMING_PREFIX_IF(severity, (condition), "", (description))
+
+// Always log, including prefix.
+#define LOG_TIMING_PREFIX(severity, prefix, description) \
+  LOG_TIMING_PREFIX_IF(severity, true, (prefix), (description))
+
+// Always log, no prefix.
+#define LOG_TIMING(severity, description) \
+  LOG_TIMING_IF(severity, true, (description))
+
+// Macro to log the time spent in the rest of the block.
+#define SCOPED_LOG_TIMING(severity, description) \
+  kudu::sw_internal::LogTiming VARNAME_LINENUM(_log_timing)(__FILE__, __LINE__, \
+      google::severity, "", description, -1, true);
+
+// Scoped version of LOG_SLOW_EXECUTION().
+#define SCOPED_LOG_SLOW_EXECUTION(severity, max_expected_millis, description) \
+  kudu::sw_internal::LogTiming VARNAME_LINENUM(_log_timing)(__FILE__, __LINE__, \
+      google::severity, "", description, max_expected_millis, true)
+
+// Scoped version of LOG_SLOW_EXECUTION() but with a prefix.
+#define SCOPED_LOG_SLOW_EXECUTION_PREFIX(severity, max_expected_millis, prefix, description) \
+  kudu::sw_internal::LogTiming VARNAME_LINENUM(_log_timing)(__FILE__, __LINE__, \
+      google::severity, prefix, description, max_expected_millis, true)
+
+// Macro for logging timing of a block. Usage:
+//   LOG_SLOW_EXECUTION(INFO, 5, "doing some task") {
+//     ... some task which takes some time
+//   }
+// when slower than 5 milliseconds, yields a log like:
+// I1102 14:35:51.726186 23082 file.cc:167] Time spent doing some task:
+//   real 3.729s user 3.570s sys 0.150s
+#define LOG_SLOW_EXECUTION(severity, max_expected_millis, description) \
+  for (kudu::sw_internal::LogTiming _l(__FILE__, __LINE__, google::severity, "", description, \
+          max_expected_millis, true); !_l.HasRun(); _l.MarkHasRun())
+
+// Macro for vlogging timing of a block. The execution happens regardless of the vlog_level,
+// it's only the logging that's affected.
+// Usage:
+//   VLOG_TIMING(1, "doing some task") {
+//     ... some task which takes some time
+//   }
+// Yields a log just like LOG_TIMING's.
+#define VLOG_TIMING(vlog_level, description) \
+  for (kudu::sw_internal::LogTiming _l(__FILE__, __LINE__, google::INFO, "", description, \
+          -1, VLOG_IS_ON(vlog_level)); !_l.HasRun(); _l.MarkHasRun())
+
+// Macro to log the time spent in the rest of the block.
+#define SCOPED_VLOG_TIMING(vlog_level, description) \
+  kudu::sw_internal::LogTiming VARNAME_LINENUM(_log_timing)(__FILE__, __LINE__, \
+      google::INFO, "", description, -1, VLOG_IS_ON(vlog_level));
+
+
+// Workaround for the clang analyzer being confused by the above loop-based macros.
+// The analyzer thinks the macros might loop more than once, and thus generates
+// false positives. So, for its purposes, just make them empty.
+#if defined(CLANG_TIDY) || defined(__clang_analyzer__)
+
+#undef LOG_TIMING_PREFIX_IF
+#define LOG_TIMING_PREFIX_IF(severity, condition, prefix, description)
+
+#undef VLOG_TIMING
+#define VLOG_TIMING(vlog_level, description)
+
+#undef LOG_SLOW_EXECUTION
+#define LOG_SLOW_EXECUTION(severity, max_expected_millis, description)
+#endif
+
+
+#define NANOS_PER_SECOND 1000000000.0
+#define NANOS_PER_MILLISECOND 1000000.0
+
+class Stopwatch;
+
+typedef int64_t nanosecond_type;
+
+// Structure which contains an elapsed amount of wall/user/sys time.
+struct CpuTimes {
+  nanosecond_type wall;
+  nanosecond_type user;
+  nanosecond_type system;
+  int64_t context_switches;
+
+  void clear() { wall = user = system = context_switches = 0LL; }
+
+  // Return a string formatted similar to the output of the "time" shell command.
+  std::string ToString() const {
+    return StringPrintf(
+      "real %.3fs\tuser %.3fs\tsys %.3fs",
+      wall_seconds(), user_cpu_seconds(), system_cpu_seconds());
+  }
+
+  double wall_millis() const {
+    return static_cast<double>(wall) / NANOS_PER_MILLISECOND;
+  }
+
+  double wall_seconds() const {
+    return static_cast<double>(wall) / NANOS_PER_SECOND;
+  }
+
+  double user_cpu_seconds() const {
+    return static_cast<double>(user) / NANOS_PER_SECOND;
+  }
+
+  double system_cpu_seconds() const {
+    return static_cast<double>(system) / NANOS_PER_SECOND;
+  }
+};
+
+// A Stopwatch is a convenient way of timing a given operation.
+//
+// Wall clock time is based on a monotonic timer, so can be reliably used for
+// determining durations.
+// CPU time is based on either current thread's usage or the usage of the whole
+// process, depending on the value of 'Mode' passed to the constructor.
+//
+// The implementation relies on several syscalls, so should not be used for
+// hot paths, but is useful for timing anything on the granularity of seconds
+// or more.
+//
+// NOTE: the user time reported by this class is based on Linux scheduler ticks
+// and thus has low precision. Use GetThreadCpuTimeMicros() from walltime.h if
+// more accurate per-thread CPU usage timing is required.
+class Stopwatch {
+ public:
+
+  enum Mode {
+    // Collect usage only about the calling thread.
+    // This may not be supported on older versions of Linux.
+    THIS_THREAD,
+    // Collect usage of all threads.
+    ALL_THREADS
+  };
+
+  // Construct a new stopwatch. The stopwatch is initially stopped.
+  explicit Stopwatch(Mode mode = THIS_THREAD)
+      : mode_(mode),
+        stopped_(true) {
+    times_.clear();
+  }
+
+  // Start counting. If the stopwatch is already counting, then resets the
+  // start point at the current time.
+  void start() {
+    stopped_ = false;
+    GetTimes(&times_);
+  }
+
+  // Stop counting. If the stopwatch is already stopped, has no effect.
+  void stop() {
+    if (stopped_) return;
+    stopped_ = true;
+
+    CpuTimes current;
+    GetTimes(&current);
+    times_.wall = current.wall - times_.wall;
+    times_.user = current.user - times_.user;
+    times_.system = current.system - times_.system;
+    times_.context_switches = current.context_switches - times_.context_switches;
+  }
+
+  // Return the elapsed amount of time. If the stopwatch is running, then returns
+  // the amount of time since it was started. If it is stopped, returns the amount
+  // of time between the most recent start/stop pair. If the stopwatch has never been
+  // started, the elapsed time is considered to be zero.
+  CpuTimes elapsed() const {
+    if (stopped_) return times_;
+
+    CpuTimes current;
+    GetTimes(&current);
+    current.wall -= times_.wall;
+    current.user -= times_.user;
+    current.system -= times_.system;
+    current.context_switches -= times_.context_switches;
+    return current;
+  }
+
+  // Resume a stopped stopwatch, such that the elapsed time continues to grow from
+  // the point where it was last stopped.
+  // For example:
+  //   Stopwatch s;
+  //   s.start();
+  //   sleep(1); // elapsed() is now ~1sec
+  //   s.stop();
+  //   sleep(1);
+  //   s.resume();
+  //   sleep(1); // elapsed() is now ~2sec
+  void resume() {
+    if (!stopped_) return;
+
+    CpuTimes current(times_);
+    start();
+    times_.wall   -= current.wall;
+    times_.user   -= current.user;
+    times_.system -= current.system;
+    times_.context_switches -= current.context_switches;
+  }
+
+  bool is_stopped() const {
+    return stopped_;
+  }
+
+ private:
+  void GetTimes(CpuTimes *times) const {
+    struct rusage usage;
+    struct timespec wall;
+
+#if defined(__APPLE__)
+    if (mode_ == THIS_THREAD) {
+      // Adapted from https://codereview.chromium.org/16818003
+      thread_basic_info_data_t t_info;
+      mach_msg_type_number_t count = THREAD_BASIC_INFO_COUNT;
+      CHECK_EQ(KERN_SUCCESS, thread_info(mach_thread_self(), THREAD_BASIC_INFO,
+                                         (thread_info_t)&t_info, &count));
+      usage.ru_utime.tv_sec = t_info.user_time.seconds;
+      usage.ru_utime.tv_usec = t_info.user_time.microseconds;
+      usage.ru_stime.tv_sec = t_info.system_time.seconds;
+      usage.ru_stime.tv_usec = t_info.system_time.microseconds;
+      usage.ru_nivcsw = t_info.suspend_count;
+      usage.ru_nvcsw = 0;
+    } else {
+      CHECK_EQ(0, getrusage(RUSAGE_SELF, &usage));
+    }
+
+    mach_timespec_t ts;
+    walltime_internal::GetCurrentTime(&ts);
+    wall.tv_sec = ts.tv_sec;
+    wall.tv_nsec = ts.tv_nsec;
+#else
+    CHECK_EQ(0, getrusage((mode_ == THIS_THREAD) ? RUSAGE_THREAD : RUSAGE_SELF, &usage));
+    CHECK_EQ(0, clock_gettime(CLOCK_MONOTONIC, &wall));
+#endif  // defined(__APPLE__)
+    times->wall   = wall.tv_sec * 1000000000L + wall.tv_nsec;
+    times->user   = usage.ru_utime.tv_sec * 1000000000L + usage.ru_utime.tv_usec * 1000L;
+    times->system = usage.ru_stime.tv_sec * 1000000000L + usage.ru_stime.tv_usec * 1000L;
+    times->context_switches = usage.ru_nvcsw + usage.ru_nivcsw;
+  }
+
+  const Mode mode_;
+  bool stopped_;
+  CpuTimes times_;
+};
+
+
+namespace sw_internal {
+
+// Internal class used by the LOG_TIMING macro.
+class LogTiming {
+ public:
+  LogTiming(const char *file, int line, google::LogSeverity severity,
+            std::string prefix, std::string description,
+            int64_t max_expected_millis, bool should_print)
+      : file_(file),
+        line_(line),
+        severity_(severity),
+        prefix_(std::move(prefix)),
+        description_(std::move(description)),
+        max_expected_millis_(max_expected_millis),
+        should_print_(should_print),
+        has_run_(false) {
+    stopwatch_.start();
+  }
+
+  ~LogTiming() {
+    if (should_print_) {
+      Print(max_expected_millis_);
+    }
+  }
+
+  // Allows this object to be used as the loop variable in for-loop macros.
+  // Call HasRun() in the conditional check in the for-loop.
+  bool HasRun() {
+    return has_run_;
+  }
+
+  // Allows this object to be used as the loop variable in for-loop macros.
+  // Call MarkHasRun() in the "increment" section of the for-loop.
+  void MarkHasRun() {
+    has_run_ = true;
+  }
+
+ private:
+  Stopwatch stopwatch_;
+  const char *file_;
+  const int line_;
+  const google::LogSeverity severity_;
+  const std::string prefix_;
+  const std::string description_;
+  const int64_t max_expected_millis_;
+  const bool should_print_;
+  bool has_run_;
+
+  // Print if the number of expected millis exceeds the max.
+  // Passing a negative number implies "always print".
+  void Print(int64_t max_expected_millis) {
+    stopwatch_.stop();
+    CpuTimes times = stopwatch_.elapsed();
+    // TODO(todd): for some reason, times.wall_millis() sometimes ends up negative
+    // on rare occasion, for unclear reasons, so we have to check max_expected_millis
+    // < 0 to be sure we always print when requested.
+    if (max_expected_millis < 0 || times.wall_millis() > max_expected_millis) {
+      google::LogMessage(file_, line_, severity_).stream()
+        << prefix_ << "Time spent " << description_ << ": "
+        << times.ToString();
+    }
+  }
+
+};
+
+} // namespace sw_internal
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/string_case-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/string_case-test.cc b/be/src/kudu/util/string_case-test.cc
new file mode 100644
index 0000000..96831a1
--- /dev/null
+++ b/be/src/kudu/util/string_case-test.cc
@@ -0,0 +1,65 @@
+// 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 <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/string_case.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(TestStringCase, TestSnakeToCamel) {
+  string out;
+  SnakeToCamelCase("foo_bar", &out);
+  ASSERT_EQ("FooBar", out);
+
+
+  SnakeToCamelCase("foo-bar", &out);
+  ASSERT_EQ("FooBar", out);
+
+  SnakeToCamelCase("foobar", &out);
+  ASSERT_EQ("Foobar", out);
+}
+
+TEST(TestStringCase, TestToUpperCase) {
+  string out;
+  ToUpperCase(string("foo"), &out);
+  ASSERT_EQ("FOO", out);
+  ToUpperCase(string("foo bar-BaZ"), &out);
+  ASSERT_EQ("FOO BAR-BAZ", out);
+}
+
+TEST(TestStringCase, TestToUpperCaseInPlace) {
+  string in_out = "foo";
+  ToUpperCase(in_out, &in_out);
+  ASSERT_EQ("FOO", in_out);
+}
+
+TEST(TestStringCase, TestCapitalize) {
+  string word = "foo";
+  Capitalize(&word);
+  ASSERT_EQ("Foo", word);
+
+  word = "HiBerNATe";
+  Capitalize(&word);
+  ASSERT_EQ("Hibernate", word);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/string_case.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/string_case.cc b/be/src/kudu/util/string_case.cc
new file mode 100644
index 0000000..7cf60ab
--- /dev/null
+++ b/be/src/kudu/util/string_case.cc
@@ -0,0 +1,76 @@
+// 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 "kudu/util/string_case.h"
+
+#include <cctype>
+#include <cstdint>
+#include <ostream>
+
+#include <glog/logging.h>
+
+namespace kudu {
+
+using std::string;
+
+void SnakeToCamelCase(const std::string &snake_case,
+                      std::string *camel_case) {
+  DCHECK_NE(camel_case, &snake_case) << "Does not support in-place operation";
+  camel_case->clear();
+  camel_case->reserve(snake_case.size());
+
+  bool uppercase_next = true;
+  for (char c : snake_case) {
+    if ((c == '_') ||
+        (c == '-')) {
+      uppercase_next = true;
+      continue;
+    }
+    if (uppercase_next) {
+      camel_case->push_back(toupper(c));
+    } else {
+      camel_case->push_back(c);
+    }
+    uppercase_next = false;
+  }
+}
+
+void ToUpperCase(const std::string &string,
+                 std::string *out) {
+  if (out != &string) {
+    *out = string;
+  }
+
+  for (char& c : *out) {
+    c = toupper(c);
+  }
+}
+
+void Capitalize(string *word) {
+  uint32_t size = word->size();
+  if (size == 0) {
+    return;
+  }
+
+  (*word)[0] = toupper((*word)[0]);
+
+  for (int i = 1; i < size; i++) {
+    (*word)[i] = tolower((*word)[i]);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/string_case.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/string_case.h b/be/src/kudu/util/string_case.h
new file mode 100644
index 0000000..98f5828
--- /dev/null
+++ b/be/src/kudu/util/string_case.h
@@ -0,0 +1,48 @@
+// 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.
+//
+// Utility methods for dealing with string case.
+#ifndef KUDU_UTIL_STRING_CASE_H
+#define KUDU_UTIL_STRING_CASE_H
+
+#include <string>
+
+namespace kudu {
+
+// Convert the given snake_case string to camel case.
+// Also treats '-' in a string like a '_'
+// For example:
+// - 'foo_bar' -> FooBar
+// - 'foo-bar' -> FooBar
+//
+// This function cannot operate in-place -- i.e. 'camel_case' must not
+// point to 'snake_case'.
+void SnakeToCamelCase(const std::string &snake_case,
+                      std::string *camel_case);
+
+// Upper-case all of the characters in the given string.
+// 'string' and 'out' may refer to the same string to replace in-place.
+void ToUpperCase(const std::string &string,
+                 std::string *out);
+
+// Capitalizes a string containing a word in place.
+// For example:
+// - 'hiBerNATe' -> 'Hibernate'
+void Capitalize(std::string *word);
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/striped64-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/striped64-test.cc b/be/src/kudu/util/striped64-test.cc
new file mode 100644
index 0000000..c74e165
--- /dev/null
+++ b/be/src/kudu/util/striped64-test.cc
@@ -0,0 +1,163 @@
+// 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 <cstdint>
+#include <ostream>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/striped64.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+// These flags are used by the multi-threaded tests, can be used for microbenchmarking.
+DEFINE_int32(num_operations, 10*1000, "Number of operations to perform");
+DEFINE_int32(num_threads, 2, "Number of worker threads");
+
+namespace kudu {
+
+// Test some basic operations
+TEST(Striped64Test, TestBasic) {
+  LongAdder adder;
+  ASSERT_EQ(adder.Value(), 0);
+  adder.IncrementBy(100);
+  ASSERT_EQ(adder.Value(), 100);
+  adder.Increment();
+  ASSERT_EQ(adder.Value(), 101);
+  adder.Decrement();
+  ASSERT_EQ(adder.Value(), 100);
+  adder.IncrementBy(-200);
+  ASSERT_EQ(adder.Value(), -100);
+  adder.Reset();
+  ASSERT_EQ(adder.Value(), 0);
+}
+
+template <class Adder>
+class MultiThreadTest {
+ public:
+  typedef std::vector<scoped_refptr<Thread> > thread_vec_t;
+
+  MultiThreadTest(int64_t num_operations, int64_t num_threads)
+   :  num_operations_(num_operations),
+      num_threads_(num_threads) {
+  }
+
+  void IncrementerThread(const int64_t num) {
+    for (int i = 0; i < num; i++) {
+      adder_.Increment();
+    }
+  }
+
+  void DecrementerThread(const int64_t num) {
+    for (int i = 0; i < num; i++) {
+      adder_.Decrement();
+    }
+  }
+
+  void Run() {
+    // Increment
+    for (int i = 0; i < num_threads_; i++) {
+      scoped_refptr<Thread> ref;
+      Thread::Create("Striped64", "Incrementer", &MultiThreadTest::IncrementerThread, this,
+                     num_operations_, &ref);
+      threads_.push_back(ref);
+    }
+    for (const scoped_refptr<Thread> &t : threads_) {
+      t->Join();
+    }
+    ASSERT_EQ(num_threads_*num_operations_, adder_.Value());
+    threads_.clear();
+
+    // Decrement back to zero
+    for (int i = 0; i < num_threads_; i++) {
+      scoped_refptr<Thread> ref;
+      Thread::Create("Striped64", "Decrementer", &MultiThreadTest::DecrementerThread, this,
+                     num_operations_, &ref);
+      threads_.push_back(ref);
+    }
+    for (const scoped_refptr<Thread> &t : threads_) {
+      t->Join();
+    }
+    ASSERT_EQ(0, adder_.Value());
+  }
+
+  Adder adder_;
+
+  int64_t num_operations_;
+  // This is rounded down to the nearest even number
+  int32_t num_threads_;
+  thread_vec_t threads_;
+};
+
+// Test adder implemented by a single AtomicInt for comparison
+class BasicAdder {
+ public:
+  BasicAdder() : value_(0) {}
+  void IncrementBy(int64_t x) { value_.IncrementBy(x); }
+  inline void Increment() { IncrementBy(1); }
+  inline void Decrement() { IncrementBy(-1); }
+  int64_t Value() { return value_.Load(); }
+ private:
+  AtomicInt<int64_t> value_;
+};
+
+void RunMultiTest(int64_t num_operations, int64_t num_threads) {
+  MonoTime start = MonoTime::Now();
+  MultiThreadTest<BasicAdder> basicTest(num_operations, num_threads);
+  basicTest.Run();
+  MonoTime end1 = MonoTime::Now();
+  MultiThreadTest<LongAdder> test(num_operations, num_threads);
+  test.Run();
+  MonoTime end2 = MonoTime::Now();
+  MonoDelta basic = end1 - start;
+  MonoDelta striped = end2 - end1;
+  LOG(INFO) << "Basic counter took   " << basic.ToMilliseconds() << "ms.";
+  LOG(INFO) << "Striped counter took " << striped.ToMilliseconds() << "ms.";
+}
+
+// Compare a single-thread workload. Demonstrates the overhead of LongAdder over AtomicInt.
+TEST(Striped64Test, TestSingleIncrDecr) {
+  OverrideFlagForSlowTests(
+      "num_operations",
+      strings::Substitute("$0", (FLAGS_num_operations * 100)));
+  RunMultiTest(FLAGS_num_operations, 1);
+}
+
+// Compare a multi-threaded workload. LongAdder should show improvements here.
+TEST(Striped64Test, TestMultiIncrDecr) {
+  OverrideFlagForSlowTests(
+      "num_operations",
+      strings::Substitute("$0", (FLAGS_num_operations * 100)));
+  OverrideFlagForSlowTests(
+      "num_threads",
+      strings::Substitute("$0", (FLAGS_num_threads * 4)));
+  RunMultiTest(FLAGS_num_operations, FLAGS_num_threads);
+}
+
+TEST(Striped64Test, TestSize) {
+  ASSERT_EQ(16, sizeof(LongAdder));
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/striped64.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/striped64.cc b/be/src/kudu/util/striped64.cc
new file mode 100644
index 0000000..789a395
--- /dev/null
+++ b/be/src/kudu/util/striped64.cc
@@ -0,0 +1,191 @@
+// 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 "kudu/util/striped64.h"
+
+#include <mm_malloc.h>
+#include <unistd.h>
+
+#include <cstdlib>
+#include <new>
+#include <ostream>
+#include <glog/logging.h>
+
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+
+using kudu::striped64::internal::Cell;
+
+namespace kudu {
+
+namespace striped64 {
+namespace internal {
+
+//
+// Cell
+//
+
+Cell::Cell()
+    : value_(0) {
+}
+} // namespace internal
+} // namespace striped64
+
+//
+// Striped64
+//
+__thread uint64_t Striped64::tls_hashcode_ = 0;
+
+namespace {
+const uint32_t kNumCpus = sysconf(_SC_NPROCESSORS_ONLN);
+uint32_t ComputeNumCells() {
+  uint32_t n = 1;
+  // Calculate the size. Nearest power of two >= NCPU.
+  // Also handle a negative NCPU, can happen if sysconf name is unknown
+  while (kNumCpus > n) {
+    n <<= 1;
+  }
+  return n;
+}
+const uint32_t kNumCells = ComputeNumCells();
+const uint32_t kCellMask = kNumCells - 1;
+
+striped64::internal::Cell* const kCellsLocked =
+      reinterpret_cast<striped64::internal::Cell*>(-1L);
+
+} // anonymous namespace
+
+uint64_t Striped64::get_tls_hashcode() {
+  if (PREDICT_FALSE(tls_hashcode_ == 0)) {
+    Random r((MonoTime::Now() - MonoTime::Min()).ToNanoseconds());
+    const uint64_t hash = r.Next64();
+    // Avoid zero to allow xorShift rehash, and because 0 indicates an unset
+    // hashcode above.
+    tls_hashcode_ = (hash == 0) ? 1 : hash;
+  }
+  return tls_hashcode_;
+}
+
+
+Striped64::~Striped64() {
+  // Cell is a POD, so no need to destruct each one.
+  free(cells_);
+}
+
+template<class Updater>
+void Striped64::RetryUpdate(Rehash to_rehash, Updater updater) {
+  uint64_t h = get_tls_hashcode();
+  // There are three operations in this loop.
+  //
+  // 1. Try to add to the Cell hash table entry for the thread if the table exists.
+  //    When there's contention, rehash to try a different Cell.
+  // 2. Try to initialize the hash table.
+  // 3. Try to update the base counter.
+  //
+  // These are predicated on successful CAS operations, which is why it's all wrapped in an
+  // infinite retry loop.
+  while (true) {
+    Cell* cells = cells_.load(std::memory_order_acquire);
+    if (cells && cells != kCellsLocked) {
+      if (to_rehash == kRehash) {
+        // CAS failed already, rehash before trying to increment.
+        to_rehash = kNoRehash;
+      } else {
+        Cell *cell = &(cells_[h & kCellMask]);
+        int64_t v = cell->value_.load(std::memory_order_relaxed);
+        if (cell->CompareAndSet(v, updater(v))) {
+          // Successfully CAS'd the corresponding cell, done.
+          break;
+        }
+      }
+      // Rehash since we failed to CAS, either previously or just now.
+      h ^= h << 13;
+      h ^= h >> 17;
+      h ^= h << 5;
+    } else if (cells == nullptr &&
+               cells_.compare_exchange_weak(cells, kCellsLocked)) {
+      // Allocate cache-aligned memory for use by the cells_ table.
+      void* cell_buffer = nullptr;
+      int err = posix_memalign(&cell_buffer, CACHELINE_SIZE, sizeof(Cell) * kNumCells);
+      CHECK_EQ(0, err) << "error calling posix_memalign" << std::endl;
+      // Initialize the table
+      cells = new (cell_buffer) Cell[kNumCells];
+      cells_.store(cells, std::memory_order_release);
+    } else {
+      // Fallback to adding to the base value.
+      // Means the table wasn't initialized or we failed to init it.
+      int64_t v = base_.load(std::memory_order_relaxed);
+      if (CasBase(v, updater(v))) {
+        break;
+      }
+    }
+  }
+  // Record index for next time
+  tls_hashcode_ = h;
+}
+
+void Striped64::InternalReset(int64_t initial_value) {
+  base_.store(initial_value);
+  Cell* c;
+  do {
+    c = cells_.load(std::memory_order_acquire);
+  } while (c == kCellsLocked);
+  if (c) {
+    for (int i = 0; i < kNumCells; i++) {
+      c[i].value_.store(initial_value);
+    }
+  }
+}
+void LongAdder::IncrementBy(int64_t x) {
+  // Use hash table if present. If that fails, call RetryUpdate to rehash and retry.
+  // If no hash table, try to CAS the base counter. If that fails, RetryUpdate to init the table.
+  Cell* cells = cells_.load(std::memory_order_acquire);
+  if (cells && cells != kCellsLocked) {
+    Cell *cell = &(cells[get_tls_hashcode() & kCellMask]);
+    DCHECK_EQ(0, reinterpret_cast<const uintptr_t>(cell) & (sizeof(Cell) - 1))
+        << " unaligned Cell not allowed for Striped64" << std::endl;
+    const int64_t old = cell->value_.load(std::memory_order_relaxed);
+    if (!cell->CompareAndSet(old, old + x)) {
+      // When we hit a hash table contention, signal RetryUpdate to rehash.
+      RetryUpdate(kRehash, [x](int64_t old) { return old + x; });
+    }
+  } else {
+    int64_t b = base_.load(std::memory_order_relaxed);
+    if (!CasBase(b, b + x)) {
+      // Attempt to initialize the table. No need to rehash since the contention was for the
+      // base counter, not the hash table.
+      RetryUpdate(kNoRehash, [x](int64_t old) { return old + x; });
+    }
+  }
+}
+
+//
+// LongAdder
+//
+
+int64_t LongAdder::Value() const {
+  int64_t sum = base_.load(std::memory_order_relaxed);
+  Cell* c = cells_.load(std::memory_order_acquire);
+  if (c && c != kCellsLocked) {
+    for (int i = 0; i < kNumCells; i++) {
+      sum += c[i].value_.load(std::memory_order_relaxed);
+    }
+  }
+  return sum;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/striped64.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/striped64.h b/be/src/kudu/util/striped64.h
new file mode 100644
index 0000000..48332e1
--- /dev/null
+++ b/be/src/kudu/util/striped64.h
@@ -0,0 +1,168 @@
+// 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.
+
+#ifndef KUDU_UTIL_STRIPED64_H_
+#define KUDU_UTIL_STRIPED64_H_
+
+#include <atomic>
+#include <cstdint>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+namespace striped64 {
+namespace internal {
+
+// Padded POD container for atomic<int64_t>. This prevents false sharing of cache lines.
+class Cell {
+ public:
+  static constexpr int kAtomicInt64Size = sizeof(std::atomic<int64_t>);
+
+  Cell();
+  inline bool CompareAndSet(int64_t cmp, int64_t value) {
+    return value_.compare_exchange_weak(cmp, value);
+  }
+
+  // Padding advice from Herb Sutter:
+  // http://www.drdobbs.com/parallel/eliminate-false-sharing/217500206?pgno=4
+  std::atomic<int64_t> value_;
+  char pad[CACHELINE_SIZE > kAtomicInt64Size ?
+           CACHELINE_SIZE - kAtomicInt64Size : 1];
+
+  DISALLOW_COPY_AND_ASSIGN(Cell);
+} CACHELINE_ALIGNED;
+#undef ATOMIC_INT_SIZE
+
+} // namespace internal
+} // namespace striped64
+
+// This set of classes is heavily derived from JSR166e, released into the public domain
+// by Doug Lea and the other authors.
+//
+// See: http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/jsr166e/Striped64.java?view=co
+// See: http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/jsr166e/LongAdder.java?view=co
+//
+// The Striped64 and LongAdder implementations here are simplified versions of what's present in
+// JSR166e. However, the core ideas remain the same.
+//
+// Updating a single AtomicInteger in a multi-threaded environment can be quite slow:
+//
+//   1. False sharing of cache lines with other counters.
+//   2. Cache line bouncing from high update rates, especially with many cores.
+//
+// These two problems are addressed by Striped64. When there is no contention, it uses CAS on a
+// single base counter to store updates. However, when Striped64 detects contention
+// (via a failed CAS operation), it will allocate a small, fixed size hashtable of Cells.
+// A Cell is a simple POD that pads out an atomic<int64_t> to 64 bytes to prevent
+// sharing a cache line.
+//
+// Reading the value of a Striped64 requires traversing the hashtable to calculate the true sum.
+//
+// Each updating thread uses a thread-local hashcode to determine its Cell in the hashtable.
+// If a thread fails to CAS its hashed Cell, it will do a lightweight rehash operation to try
+// and find an uncontended bucket. Because the hashcode is thread-local, this rehash affects all
+// Striped64's accessed by the thread. This is good, since contention on one Striped64 is
+// indicative of contention elsewhere too.
+//
+// The hashtable is statically sized to the nearest power of 2 greater than or equal to the
+// number of CPUs. This is sufficient, since this guarantees the existence of a perfect hash
+// function. Due to the random rehashing, the threads should eventually converge to this function.
+// In practice, this scheme has shown to be sufficient.
+//
+// The biggest simplification of this implementation compared to JSR166e is that we do not
+// dynamically grow the table, instead immediately allocating it to the full size.
+// We also do not lazily allocate each Cell, instead allocating the entire array at once.
+// This means we waste some additional memory in low contention scenarios, and initial allocation
+// will also be slower. Some of the micro-optimizations were also elided for readability.
+class Striped64 {
+ public:
+  Striped64() = default;
+
+ protected:
+  // NOTE: the destructor is not virtual so that we can ensure that Striped64
+  // has no vtable, thus reducing its size. We make it protected to ensure that
+  // no one attempts to delete a Striped64* and invokes the wrong destructor.
+  ~Striped64();
+
+  enum Rehash {
+    kRehash,
+    kNoRehash
+  };
+
+  // CAS the base field.
+  bool CasBase(int64_t cmp, int64_t val) { return base_.compare_exchange_weak(cmp, val); }
+
+  // Handles cases of updates involving initialization, resizing, creating new Cells, and/or
+  // contention. See above for further explanation.
+  //
+  // 'Updater' should be a function which takes the current value and returns
+  // the new value.
+  template<class Updater>
+  void RetryUpdate(Rehash to_rehash, Updater updater);
+
+  // Sets base and all cells to the given value.
+  void InternalReset(int64_t initial_value);
+
+  // Base value, used mainly when there is no contention, but also as a fallback during
+  // table initialization races. Updated via CAS.
+  std::atomic<int64_t> base_ { 0 };
+
+  // Table of cells. When non-null, size is the nearest power of 2 >= NCPU.
+  // If this is set to -1, the pointer is 'locked' and some thread is in the
+  // process of allocating the array.
+  std::atomic<striped64::internal::Cell*> cells_ { nullptr };
+
+ protected:
+  static uint64_t get_tls_hashcode();
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(Striped64);
+
+  // Static hash code per-thread. Shared across all instances to limit thread-local pollution.
+  // Also, if a thread hits a collision on one Striped64, it's also likely to collide on
+  // other Striped64s too.
+  static __thread uint64_t tls_hashcode_;
+};
+
+// A 64-bit number optimized for high-volume concurrent updates.
+// See Striped64 for a longer explanation of the inner workings.
+class LongAdder : Striped64 {
+ public:
+  LongAdder() {}
+  void IncrementBy(int64_t x);
+  void Increment() { IncrementBy(1); }
+  void Decrement() { IncrementBy(-1); }
+
+  // Returns the current value.
+  // Note this is not an atomic snapshot in the presence of concurrent updates.
+  int64_t Value() const;
+
+  // Resets the counter state to zero.
+  void Reset() { InternalReset(0); }
+
+ protected:
+  int64_t CombineValue(int64_t current_value, int64_t new_value) {
+    return current_value + new_value;
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(LongAdder);
+};
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/subprocess-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/subprocess-test.cc b/be/src/kudu/util/subprocess-test.cc
new file mode 100644
index 0000000..24d7cb3
--- /dev/null
+++ b/be/src/kudu/util/subprocess-test.cc
@@ -0,0 +1,381 @@
+// 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 "kudu/util/subprocess.h"
+
+#include <errno.h>
+#include <pthread.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <atomic>
+#include <csignal>
+#include <cstdio>
+#include <cstdlib>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::atomic;
+using std::string;
+using std::thread;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+class SubprocessTest : public KuduTest {};
+
+TEST_F(SubprocessTest, TestSimplePipe) {
+  Subprocess p({ "/usr/bin/tr", "a-z", "A-Z" });
+  p.ShareParentStdout(false);
+  ASSERT_OK(p.Start());
+
+  FILE* out = fdopen(p.ReleaseChildStdinFd(), "w");
+  PCHECK(out);
+  FILE* in = fdopen(p.from_child_stdout_fd(), "r");
+  PCHECK(in);
+
+  fprintf(out, "hello world\n");
+  // We have to close 'out' or else tr won't write any output, since
+  // it enters a buffered mode if it detects that its input is a FIFO.
+  int err;
+  RETRY_ON_EINTR(err, fclose(out));
+
+  char buf[1024];
+  ASSERT_EQ(buf, fgets(buf, sizeof(buf), in));
+  ASSERT_STREQ("HELLO WORLD\n", &buf[0]);
+
+  int wait_status = 0;
+  ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFEXITED(wait_status));
+  ASSERT_EQ(0, WEXITSTATUS(wait_status));
+}
+
+TEST_F(SubprocessTest, TestErrPipe) {
+  Subprocess p({ "/usr/bin/tee", "/dev/stderr" });
+  p.ShareParentStderr(false);
+  ASSERT_OK(p.Start());
+
+  FILE* out = fdopen(p.ReleaseChildStdinFd(), "w");
+  PCHECK(out);
+
+  fprintf(out, "Hello, World\n");
+
+  // Same reasoning as above, flush to prevent tee buffering.
+  int err;
+  RETRY_ON_EINTR(err, fclose(out));
+
+  FILE* in = fdopen(p.from_child_stderr_fd(), "r");
+  PCHECK(in);
+
+  char buf[1024];
+  ASSERT_EQ(buf, fgets(buf, sizeof(buf), in));
+  ASSERT_STREQ("Hello, World\n", &buf[0]);
+
+  int wait_status = 0;
+  ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFEXITED(wait_status));
+  ASSERT_EQ(0, WEXITSTATUS(wait_status));
+}
+
+TEST_F(SubprocessTest, TestKill) {
+  Subprocess p({ "/bin/cat" });
+  ASSERT_OK(p.Start());
+
+  ASSERT_OK(p.Kill(SIGKILL));
+
+  int wait_status = 0;
+  ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFSIGNALED(wait_status));
+  ASSERT_EQ(SIGKILL, WTERMSIG(wait_status));
+
+  // Test that calling Wait() a second time returns the same
+  // cached value instead of trying to wait on some other process
+  // that was assigned the same pid.
+  wait_status = 0;
+  ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFSIGNALED(wait_status));
+  ASSERT_EQ(SIGKILL, WTERMSIG(wait_status));
+}
+
+// Writes enough bytes to stdout and stderr concurrently that if Call() were
+// fully reading them one at a time, the test would deadlock.
+TEST_F(SubprocessTest, TestReadFromStdoutAndStderr) {
+  // Set an alarm to break out of any potential deadlocks (if the implementation
+  // regresses).
+  alarm(60);
+
+  string stdout;
+  string stderr;
+  ASSERT_OK(Subprocess::Call({
+    "/bin/bash",
+    "-c",
+    "dd if=/dev/urandom of=/dev/stdout bs=512 count=2048 &"
+    "dd if=/dev/urandom of=/dev/stderr bs=512 count=2048 &"
+    "wait"
+  }, "", &stdout, &stderr));
+
+  // Reset the alarm when the test is done
+  SCOPED_CLEANUP({ alarm(0); })
+}
+
+// Test that environment variables can be passed to the subprocess.
+TEST_F(SubprocessTest, TestEnvVars) {
+  Subprocess p({ "/bin/bash", "-c", "echo $FOO" });
+  p.SetEnvVars({{"FOO", "bar"}});
+  p.ShareParentStdout(false);
+  ASSERT_OK(p.Start());
+  FILE* in = fdopen(p.from_child_stdout_fd(), "r");
+  PCHECK(in);
+  char buf[1024];
+  ASSERT_EQ(buf, fgets(buf, sizeof(buf), in));
+  ASSERT_STREQ("bar\n", &buf[0]);
+  ASSERT_OK(p.Wait());
+}
+
+// Test that the the subprocesses CWD can be set.
+TEST_F(SubprocessTest, TestCurrentDir) {
+  string dir_path = GetTestPath("d");
+  string file_path = JoinPathSegments(dir_path, "f");
+  ASSERT_OK(Env::Default()->CreateDir(dir_path));
+  std::unique_ptr<WritableFile> file;
+  ASSERT_OK(Env::Default()->NewWritableFile(file_path, &file));
+
+  Subprocess p({ "/bin/ls", "f" });
+  p.SetCurrentDir(dir_path);
+  p.ShareParentStdout(false);
+  ASSERT_OK(p.Start());
+  ASSERT_OK(p.Wait());
+
+  int rc;
+  ASSERT_OK(p.GetExitStatus(&rc, nullptr));
+  EXPECT_EQ(0, rc);
+}
+
+// Tests writing to the subprocess stdin.
+TEST_F(SubprocessTest, TestCallWithStdin) {
+  string stdout;
+  ASSERT_OK(Subprocess::Call({ "/bin/bash" },
+                             "echo \"quick brown fox\"",
+                             &stdout));
+  EXPECT_EQ("quick brown fox\n", stdout);
+}
+
+// Test KUDU-1674: '/bin/bash -c "echo"' command below is expected to
+// capture a string on stderr. This test validates that passing
+// stderr alone doesn't result in SIGSEGV as reported in the bug and
+// also check for sanity of stderr in the output.
+TEST_F(SubprocessTest, TestReadSingleFD) {
+  string stderr;
+  const string str = "ApacheKudu";
+  const string cmd_str = Substitute("/bin/echo -n $0 1>&2", str);
+  ASSERT_OK(Subprocess::Call({"/bin/sh", "-c", cmd_str}, "", nullptr, &stderr));
+  ASSERT_EQ(stderr, str);
+
+  // Also sanity check other combinations.
+  string stdout;
+  ASSERT_OK(Subprocess::Call({"/bin/ls", "/dev/null"}, "", &stdout, nullptr));
+  ASSERT_STR_CONTAINS(stdout, "/dev/null");
+
+  ASSERT_OK(Subprocess::Call({"/bin/ls", "/dev/zero"}, "", nullptr, nullptr));
+}
+
+TEST_F(SubprocessTest, TestGetExitStatusExitSuccess) {
+  Subprocess p({ "/bin/sh", "-c", "exit 0" });
+  ASSERT_OK(p.Start());
+  ASSERT_OK(p.Wait());
+  int exit_status;
+  string exit_info;
+  ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+  ASSERT_EQ(0, exit_status);
+  ASSERT_STR_CONTAINS(exit_info, "process successfully exited");
+}
+
+TEST_F(SubprocessTest, TestGetExitStatusExitFailure) {
+  static const vector<int> kStatusCodes = { 1, 255 };
+  for (auto code : kStatusCodes) {
+    Subprocess p({ "/bin/sh", "-c", Substitute("exit $0", code) });
+    ASSERT_OK(p.Start());
+    ASSERT_OK(p.Wait());
+    int exit_status;
+    string exit_info;
+    ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+    ASSERT_EQ(code, exit_status);
+    ASSERT_STR_CONTAINS(exit_info,
+                        Substitute("process exited with non-zero status $0",
+                                   exit_status));
+  }
+}
+
+TEST_F(SubprocessTest, TestGetExitStatusSignaled) {
+  static const vector<int> kSignals = {
+    SIGHUP,
+    SIGABRT,
+    SIGKILL,
+    SIGTERM,
+    SIGUSR2,
+  };
+  for (auto signum : kSignals) {
+    Subprocess p({ "/bin/cat" });
+    ASSERT_OK(p.Start());
+    ASSERT_OK(p.Kill(signum));
+    ASSERT_OK(p.Wait());
+    int exit_status;
+    string exit_info;
+    ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+    EXPECT_EQ(signum, exit_status);
+    ASSERT_STR_CONTAINS(exit_info, Substitute("process exited on signal $0",
+                                              signum));
+  }
+}
+
+TEST_F(SubprocessTest, TestSubprocessDestroyWithCustomSignal) {
+  string kTestFile = GetTestPath("foo");
+
+  // Start a subprocess that creates kTestFile immediately and deletes it on exit.
+  //
+  // Note: it's important that the shell not invoke a command while waiting
+  // to be killed (i.e. "sleep 60"); if it did, the signal could be delivered
+  // just after the command starts but just before the shell decides to forward
+  // signals to it, and we wind up with a deadlock.
+  vector<string> argv = {
+      "/bin/bash",
+      "-c",
+      Substitute(
+          // Delete kTestFile on exit.
+          "trap \"rm $0\" EXIT;"
+          // Create kTestFile on start.
+          "touch $0;"
+          // Spin in a tight loop waiting to be killed.
+          "while true;"
+          "  do FOO=$$((FOO + 1));"
+          "done", kTestFile)
+  };
+
+  {
+    Subprocess s(argv);
+    ASSERT_OK(s.Start());
+    AssertEventually([&]{
+        ASSERT_TRUE(env_->FileExists(kTestFile));
+    });
+  }
+
+  // The subprocess went out of scope and was killed with SIGKILL, so it left
+  // kTestFile behind.
+  ASSERT_TRUE(env_->FileExists(kTestFile));
+
+  ASSERT_OK(env_->DeleteFile(kTestFile));
+  {
+    Subprocess s(argv, SIGTERM);
+    ASSERT_OK(s.Start());
+    AssertEventually([&]{
+        ASSERT_TRUE(env_->FileExists(kTestFile));
+    });
+  }
+
+  // The subprocess was killed with SIGTERM, giving it a chance to delete kTestFile.
+  ASSERT_FALSE(env_->FileExists(kTestFile));
+}
+
+// TEST KUDU-2208: Test subprocess interruption handling
+void handler(int /* signal */) {
+}
+
+TEST_F(SubprocessTest, TestSubprocessInterruptionHandling) {
+  // Create Subprocess thread
+  pthread_t t;
+  Subprocess p({ "/bin/sleep", "1" });
+  atomic<bool> t_started(false);
+  atomic<bool> t_finished(false);
+  thread subprocess_thread([&]() {
+    t = pthread_self();
+    t_started = true;
+    SleepFor(MonoDelta::FromMilliseconds(50));
+    CHECK_OK(p.Start());
+    CHECK_OK(p.Wait());
+    t_finished = true;
+  });
+
+  // Set up a no-op signal handler for SIGUSR2.
+  struct sigaction sa, sa_old;
+  memset(&sa, 0, sizeof(sa));
+  sa.sa_handler = &handler;
+  sigaction(SIGUSR2, &sa, &sa_old);
+
+  SCOPED_CLEANUP({ sigaction(SIGUSR2, &sa_old, nullptr); });
+  SCOPED_CLEANUP({ subprocess_thread.join(); });
+
+  // Send kill signals to Subprocess thread
+  LOG(INFO) << "Start sending kill signals to Subprocess thread";
+  while (!t_finished) {
+    if (t_started) {
+      int err = pthread_kill(t, SIGUSR2);
+      ASSERT_TRUE(err == 0 || err == ESRCH);
+      if (err == ESRCH) {
+        LOG(INFO) << "Async kill signal failed with err=" << err <<
+            " because it tried to kill vanished subprocess_thread";
+        ASSERT_TRUE(t_finished);
+      }
+      // Add microseconds delay to make the unit test runs faster and more reliable
+      SleepFor(MonoDelta::FromMicroseconds(rand() % 1));
+    }
+  }
+}
+
+#ifdef __linux__
+// This test requires a system with /proc/<pid>/stat.
+TEST_F(SubprocessTest, TestGetProcfsState) {
+  // This test should be RUNNING.
+  Subprocess::ProcfsState state;
+  ASSERT_OK(Subprocess::GetProcfsState(getpid(), &state));
+  ASSERT_EQ(Subprocess::ProcfsState::RUNNING, state);
+
+  // When started, /bin/sleep will be RUNNING (even though it's asleep).
+  Subprocess sleep({"/bin/sleep", "1000"});
+  ASSERT_OK(sleep.Start());
+  ASSERT_OK(Subprocess::GetProcfsState(sleep.pid(), &state));
+  ASSERT_EQ(Subprocess::ProcfsState::RUNNING, state);
+
+  // After a SIGSTOP, it should be PAUSED.
+  ASSERT_OK(sleep.Kill(SIGSTOP));
+  ASSERT_OK(Subprocess::GetProcfsState(sleep.pid(), &state));
+  ASSERT_EQ(Subprocess::ProcfsState::PAUSED, state);
+
+  // After a SIGCONT, it should be RUNNING again.
+  ASSERT_OK(sleep.Kill(SIGCONT));
+  ASSERT_OK(Subprocess::GetProcfsState(sleep.pid(), &state));
+  ASSERT_EQ(Subprocess::ProcfsState::RUNNING, state);
+}
+#endif
+
+} // namespace kudu


[22/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env_posix.cc b/be/src/kudu/util/env_posix.cc
new file mode 100644
index 0000000..fe47bfd
--- /dev/null
+++ b/be/src/kudu/util/env_posix.cc
@@ -0,0 +1,1852 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include <dirent.h>
+#include <fcntl.h>
+#include <fnmatch.h>
+#include <fts.h>
+#include <glob.h>
+#include <pthread.h>
+#include <sys/resource.h>
+#include <sys/stat.h>
+#include <sys/statvfs.h>
+#include <sys/time.h>
+#include <sys/uio.h>
+#include <sys/utsname.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cerrno>
+#include <cstdint>
+#include <cstdio>
+#include <cstdlib>
+#include <cstring>
+#include <ctime>
+#include <map>
+#include <memory>
+#include <numeric>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/array_view.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/env.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/fault_injection.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/malloc.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/thread_restrictions.h"
+#include "kudu/util/trace.h"
+
+#if defined(__APPLE__)
+#include <mach-o/dyld.h>
+#include <sys/sysctl.h>
+#else
+#include <linux/falloc.h>
+#include <linux/fiemap.h>
+#include <linux/fs.h>
+#include <linux/ioctl.h>
+#include <linux/kernel.h>
+#include <linux/magic.h>
+#include <linux/types.h>
+#include <sys/ioctl.h>
+#include <sys/sysinfo.h>
+#include <sys/vfs.h>
+#endif  // defined(__APPLE__)
+
+using base::subtle::Atomic64;
+using base::subtle::Barrier_AtomicIncrement;
+using std::accumulate;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+// Copied from falloc.h. Useful for older kernels that lack support for
+// hole punching; fallocate(2) will return EOPNOTSUPP.
+#ifndef FALLOC_FL_KEEP_SIZE
+#define FALLOC_FL_KEEP_SIZE 0x01 /* default is extend size */
+#endif
+#ifndef FALLOC_FL_PUNCH_HOLE
+#define FALLOC_FL_PUNCH_HOLE  0x02 /* de-allocates range */
+#endif
+
+
+#ifndef __APPLE__
+// These struct and ioctl definitions were copied verbatim from xfsprogs.
+typedef struct xfs_flock64 {
+        __s16           l_type;
+        __s16           l_whence;
+        __s64           l_start;
+        __s64           l_len;          /* len == 0 means until end of file */
+        __s32           l_sysid;
+        __u32           l_pid;
+        __s32           l_pad[4];       /* reserve area                     */
+} xfs_flock64_t;
+#define XFS_IOC_UNRESVSP64      _IOW ('X', 43, struct xfs_flock64)
+#endif
+
+// OSX does not have fdatasync or fread_unlocked.
+#ifdef __APPLE__
+#define fdatasync fsync
+#define fread_unlocked fread
+#endif
+
+// With some probability, if 'filename_expr' matches the glob pattern specified
+// by the 'env_inject_eio_globs' flag, calls RETURN_NOT_OK on 'error_expr'.
+#define MAYBE_RETURN_EIO(filename_expr, error_expr) do { \
+  const string& f_ = (filename_expr); \
+  MAYBE_RETURN_FAILURE(FLAGS_env_inject_eio, \
+      ShouldInject(f_, FLAGS_env_inject_eio_globs) ? (error_expr) : Status::OK()) \
+} while (0);
+
+bool ShouldInject(const string& candidate, const string& glob_patterns) {
+  // Never inject on /proc/ file accesses regardless of the configured flag,
+  // since it's not possible for /proc to "go bad".
+  //
+  // NB: it's important that this is done here _before_ consulting glob_patterns
+  // since some background threads read /proc/ after gflags have already been
+  // destructed.
+  if (HasPrefixString(candidate, "/proc/")) {
+    return false;
+  }
+
+  vector<string> globs = strings::Split(glob_patterns, ",", strings::SkipEmpty());
+  for (const auto& glob : globs) {
+    if (fnmatch(glob.c_str(), candidate.c_str(), 0) == 0) {
+      return true;
+    }
+  }
+  return false;
+}
+
+// See KUDU-588 for details.
+DEFINE_bool(env_use_fsync, false,
+            "Use fsync(2) instead of fdatasync(2) for synchronizing dirty "
+            "data to disk.");
+TAG_FLAG(env_use_fsync, advanced);
+TAG_FLAG(env_use_fsync, evolving);
+
+// See KUDU-2052 for details.
+DEFINE_bool(env_use_ioctl_hole_punch_on_xfs, true,
+            "Use the XFS_IOC_UNRESVSP64 ioctl instead of fallocate(2) to "
+            "punch holes on XFS filesystems.");
+TAG_FLAG(env_use_ioctl_hole_punch_on_xfs, advanced);
+TAG_FLAG(env_use_ioctl_hole_punch_on_xfs, experimental);
+
+DEFINE_bool(crash_on_eio, false,
+            "Kill the process if an I/O operation results in EIO. If false, "
+            "I/O resulting in EIOs will return the status IOError and leave "
+            "error-handling up to the caller.");
+TAG_FLAG(crash_on_eio, advanced);
+TAG_FLAG(crash_on_eio, experimental);
+
+DEFINE_bool(never_fsync, false,
+            "Never fsync() anything to disk. This is used by certain test cases to "
+            "speed up runtime. This is very unsafe to use in production.");
+TAG_FLAG(never_fsync, advanced);
+TAG_FLAG(never_fsync, unsafe);
+
+DEFINE_int32(env_inject_short_read_bytes, 0,
+             "The number of bytes less than the requested bytes to read");
+TAG_FLAG(env_inject_short_read_bytes, hidden);
+DEFINE_int32(env_inject_short_write_bytes, 0,
+             "The number of bytes less than the requested bytes to write");
+TAG_FLAG(env_inject_short_write_bytes, hidden);
+
+DEFINE_double(env_inject_eio, 0.0,
+              "Fraction of the time that operations on certain files will fail "
+              "with the posix code EIO.");
+TAG_FLAG(env_inject_eio, hidden);
+DEFINE_string(env_inject_eio_globs, "*",
+              "Comma-separated list of glob patterns specifying files on which "
+              "I/O will fail. By default, all files may cause a failure.");
+TAG_FLAG(env_inject_eio_globs, hidden);
+
+DEFINE_string(env_inject_lock_failure_globs, "",
+              "Comma-separated list of glob patterns specifying files on which "
+              "attempts to obtain a file lock will fail. By default, no files "
+              "will fail.");
+TAG_FLAG(env_inject_lock_failure_globs, hidden);
+
+static __thread uint64_t thread_local_id;
+static Atomic64 cur_thread_local_id_;
+
+namespace kudu {
+
+const char* const Env::kInjectedFailureStatusMsg = "INJECTED FAILURE";
+
+namespace {
+
+#if defined(__APPLE__)
+// Simulates Linux's fallocate file preallocation API on OS X.
+int fallocate(int fd, int mode, off_t offset, off_t len) {
+  CHECK_EQ(mode, 0);
+  off_t size = offset + len;
+
+  struct stat stat;
+  int ret = fstat(fd, &stat);
+  if (ret < 0) {
+    return ret;
+  }
+
+  if (stat.st_blocks * 512 < size) {
+    // The offset field seems to have no effect; the file is always allocated
+    // with space from 0 to the size. This is probably because OS X does not
+    // support sparse files.
+    fstore_t store = {F_ALLOCATECONTIG, F_PEOFPOSMODE, 0, size};
+    if (fcntl(fd, F_PREALLOCATE, &store) < 0) {
+      LOG(INFO) << "Unable to allocate contiguous disk space, attempting non-contiguous allocation";
+      store.fst_flags = F_ALLOCATEALL;
+      ret = fcntl(fd, F_PREALLOCATE, &store);
+      if (ret < 0) {
+        return ret;
+      }
+    }
+  }
+
+  if (stat.st_size < size) {
+    // fcntl does not change the file size, so set it if necessary.
+    int ret;
+    RETRY_ON_EINTR(ret, ftruncate(fd, size));
+    return ret;
+  }
+  return 0;
+}
+
+// Simulates Linux's preadv API on OS X.
+ssize_t preadv(int fd, const struct iovec* iovec, int count, off_t offset) {
+  ssize_t total_read_bytes = 0;
+  for (int i = 0; i < count; i++) {
+    ssize_t r;
+    RETRY_ON_EINTR(r, pread(fd, iovec[i].iov_base, iovec[i].iov_len, offset));
+    if (r < 0) {
+      return r;
+    }
+    total_read_bytes += r;
+    if (static_cast<size_t>(r) < iovec[i].iov_len) {
+      break;
+    }
+    offset += iovec[i].iov_len;
+  }
+  return total_read_bytes;
+}
+
+// Simulates Linux's pwritev API on OS X.
+ssize_t pwritev(int fd, const struct iovec* iovec, int count, off_t offset) {
+  ssize_t total_written_bytes = 0;
+  for (int i = 0; i < count; i++) {
+    ssize_t r;
+    RETRY_ON_EINTR(r, pwrite(fd, iovec[i].iov_base, iovec[i].iov_len, offset));
+    if (r < 0) {
+      return r;
+    }
+    total_written_bytes += r;
+    if (static_cast<size_t>(r) < iovec[i].iov_len) {
+      break;
+    }
+    offset += iovec[i].iov_len;
+  }
+  return total_written_bytes;
+}
+#endif
+
+
+// Close file descriptor when object goes out of scope.
+class ScopedFdCloser {
+ public:
+  explicit ScopedFdCloser(int fd)
+    : fd_(fd) {
+  }
+
+  ~ScopedFdCloser() {
+    ThreadRestrictions::AssertIOAllowed();
+    int err;
+    RETRY_ON_EINTR(err, ::close(fd_));
+    if (PREDICT_FALSE(err != 0)) {
+      PLOG(WARNING) << "Failed to close fd " << fd_;
+    }
+  }
+
+ private:
+  int fd_;
+};
+
+Status IOError(const std::string& context, int err_number) {
+  switch (err_number) {
+    case ENOENT:
+      return Status::NotFound(context, ErrnoToString(err_number), err_number);
+    case EEXIST:
+      return Status::AlreadyPresent(context, ErrnoToString(err_number), err_number);
+    case EOPNOTSUPP:
+      return Status::NotSupported(context, ErrnoToString(err_number), err_number);
+    case EIO:
+      if (FLAGS_crash_on_eio) {
+        // TODO(awong): This is very, very coarse-grained. A more comprehensive
+        // approach is described in KUDU-616.
+        LOG(FATAL) << "Fatal I/O error, context: " << context;
+      } else {
+        LOG(ERROR) << "I/O error, context: " << context;
+      }
+  }
+  return Status::IOError(context, ErrnoToString(err_number), err_number);
+}
+
+Status DoSync(int fd, const string& filename) {
+  MAYBE_RETURN_EIO(filename, IOError(Env::kInjectedFailureStatusMsg, EIO));
+
+  ThreadRestrictions::AssertIOAllowed();
+  if (FLAGS_never_fsync) return Status::OK();
+  if (FLAGS_env_use_fsync) {
+    TRACE_COUNTER_SCOPE_LATENCY_US("fsync_us");
+    TRACE_COUNTER_INCREMENT("fsync", 1);
+    if (fsync(fd) < 0) {
+      return IOError(filename, errno);
+    }
+  } else {
+    TRACE_COUNTER_INCREMENT("fdatasync", 1);
+    TRACE_COUNTER_SCOPE_LATENCY_US("fdatasync_us");
+    if (fdatasync(fd) < 0) {
+      return IOError(filename, errno);
+    }
+  }
+  return Status::OK();
+}
+
+Status DoOpen(const string& filename, Env::CreateMode mode, int* fd) {
+  MAYBE_RETURN_EIO(filename, IOError(Env::kInjectedFailureStatusMsg, EIO));
+  ThreadRestrictions::AssertIOAllowed();
+  int flags = O_RDWR;
+  switch (mode) {
+    case Env::CREATE_IF_NON_EXISTING_TRUNCATE:
+      flags |= O_CREAT | O_TRUNC;
+      break;
+    case Env::CREATE_NON_EXISTING:
+      flags |= O_CREAT | O_EXCL;
+      break;
+    case Env::OPEN_EXISTING:
+      break;
+    default:
+      return Status::NotSupported(Substitute("Unknown create mode $0", mode));
+  }
+  int f;
+  RETRY_ON_EINTR(f, open(filename.c_str(), flags, 0666));
+  if (f < 0) {
+    return IOError(filename, errno);
+  }
+  *fd = f;
+  return Status::OK();
+}
+
+Status DoReadV(int fd, const string& filename, uint64_t offset,
+               ArrayView<Slice> results) {
+  MAYBE_RETURN_EIO(filename, IOError(Env::kInjectedFailureStatusMsg, EIO));
+  ThreadRestrictions::AssertIOAllowed();
+
+  // Convert the results into the iovec vector to request
+  // and calculate the total bytes requested
+  size_t bytes_req = 0;
+  size_t iov_size = results.size();
+  struct iovec iov[iov_size];
+  for (size_t i = 0; i < iov_size; i++) {
+    Slice& result = results[i];
+    bytes_req += result.size();
+    iov[i] = { result.mutable_data(), result.size() };
+  }
+
+  uint64_t cur_offset = offset;
+  size_t completed_iov = 0;
+  size_t rem = bytes_req;
+  while (rem > 0) {
+    // Never request more than IOV_MAX in one request
+    size_t iov_count = std::min(iov_size - completed_iov, static_cast<size_t>(IOV_MAX));
+    ssize_t r;
+    RETRY_ON_EINTR(r, preadv(fd, iov + completed_iov, iov_count, cur_offset));
+
+    // Fake a short read for testing
+    if (PREDICT_FALSE(FLAGS_env_inject_short_read_bytes > 0 && rem == bytes_req)) {
+      DCHECK_LT(FLAGS_env_inject_short_read_bytes, r);
+      r -= FLAGS_env_inject_short_read_bytes;
+    }
+
+    if (PREDICT_FALSE(r < 0)) {
+      // An error: return a non-ok status.
+      return IOError(filename, errno);
+    }
+    if (PREDICT_FALSE(r == 0)) {
+      // EOF.
+      return Status::EndOfFile(
+          Substitute("EOF trying to read $0 bytes at offset $1", bytes_req, offset));
+    }
+    if (PREDICT_TRUE(r == rem)) {
+      // All requested bytes were read. This is almost always the case.
+      return Status::OK();
+    }
+    DCHECK_LE(r, rem);
+    // Adjust iovec vector based on bytes read for the next request
+    ssize_t bytes_rem = r;
+    for (size_t i = completed_iov; i < iov_size; i++) {
+      if (bytes_rem >= iov[i].iov_len) {
+        // The full length of this iovec was read
+        completed_iov++;
+        bytes_rem -= iov[i].iov_len;
+      } else {
+        // Partially read this result.
+        // Adjust the iov_len and iov_base to request only the missing data.
+        iov[i].iov_base = static_cast<uint8_t *>(iov[i].iov_base) + bytes_rem;
+        iov[i].iov_len -= bytes_rem;
+        break; // Don't need to adjust remaining iovec's
+      }
+    }
+    cur_offset += r;
+    rem -= r;
+  }
+  DCHECK_EQ(0, rem);
+  return Status::OK();
+}
+
+Status DoWriteV(int fd, const string& filename, uint64_t offset, ArrayView<const Slice> data) {
+  MAYBE_RETURN_EIO(filename, IOError(Env::kInjectedFailureStatusMsg, EIO));
+  ThreadRestrictions::AssertIOAllowed();
+
+  // Convert the results into the iovec vector to request
+  // and calculate the total bytes requested.
+  size_t bytes_req = 0;
+  size_t iov_size = data.size();
+  struct iovec iov[iov_size];
+  for (size_t i = 0; i < iov_size; i++) {
+    const Slice& result = data[i];
+    bytes_req += result.size();
+    iov[i] = { const_cast<uint8_t*>(result.data()), result.size() };
+  }
+
+  uint64_t cur_offset = offset;
+  size_t completed_iov = 0;
+  size_t rem = bytes_req;
+  while (rem > 0) {
+    // Never request more than IOV_MAX in one request.
+    size_t iov_count = std::min(iov_size - completed_iov, static_cast<size_t>(IOV_MAX));
+    ssize_t w;
+    RETRY_ON_EINTR(w, pwritev(fd, iov + completed_iov, iov_count, cur_offset));
+
+    // Fake a short write for testing.
+    if (PREDICT_FALSE(FLAGS_env_inject_short_write_bytes > 0 && rem == bytes_req)) {
+      DCHECK_LT(FLAGS_env_inject_short_write_bytes, w);
+      w -= FLAGS_env_inject_short_read_bytes;
+    }
+
+    if (PREDICT_FALSE(w < 0)) {
+      // An error: return a non-ok status.
+      return IOError(filename, errno);
+    }
+
+    DCHECK_LE(w, rem);
+
+    if (PREDICT_TRUE(w == rem)) {
+      // All requested bytes were read. This is almost always the case.
+      return Status::OK();
+    }
+    // Adjust iovec vector based on bytes read for the next request.
+    ssize_t bytes_rem = w;
+    for (size_t i = completed_iov; i < iov_size; i++) {
+      if (bytes_rem >= iov[i].iov_len) {
+        // The full length of this iovec was written.
+        completed_iov++;
+        bytes_rem -= iov[i].iov_len;
+      } else {
+        // Partially wrote this result.
+        // Adjust the iov_len and iov_base to write only the missing data.
+        iov[i].iov_base = static_cast<uint8_t *>(iov[i].iov_base) + bytes_rem;
+        iov[i].iov_len -= bytes_rem;
+        break; // Don't need to adjust remaining iovec's.
+      }
+    }
+    cur_offset += w;
+    rem -= w;
+  }
+  DCHECK_EQ(0, rem);
+  return Status::OK();
+}
+
+Status DoIsOnXfsFilesystem(const string& path, bool* result) {
+#ifdef __APPLE__
+  *result = false;
+#else
+  struct statfs buf;
+  int ret;
+  RETRY_ON_EINTR(ret, statfs(path.c_str(), &buf));
+  if (ret == -1) {
+    return IOError(Substitute("statfs: $0", path), errno);
+  }
+  // This magic number isn't defined in any header but is the value of the
+  // US-ASCII string 'XFSB' expressed in hexadecimal.
+  *result = (buf.f_type == 0x58465342);
+#endif
+  return Status::OK();
+}
+
+const char* ResourceLimitTypeToString(Env::ResourceLimitType t) {
+  switch (t) {
+    case Env::ResourceLimitType::OPEN_FILES_PER_PROCESS:
+      return "open files per process";
+    case Env::ResourceLimitType::RUNNING_THREADS_PER_EUID:
+      return "running threads per effective uid";
+    default: LOG(FATAL) << "Unknown resource limit type";
+  }
+}
+
+int ResourceLimitTypeToUnixRlimit(Env::ResourceLimitType t) {
+  switch (t) {
+    case Env::ResourceLimitType::OPEN_FILES_PER_PROCESS: return RLIMIT_NOFILE;
+    case Env::ResourceLimitType::RUNNING_THREADS_PER_EUID: return RLIMIT_NPROC;
+    default: LOG(FATAL) << "Unknown resource limit type: " << t;
+  }
+}
+
+#ifdef __APPLE__
+const char* ResourceLimitTypeToMacosRlimit(Env::ResourceLimitType t) {
+  switch (t) {
+    case Env::ResourceLimitType::OPEN_FILES_PER_PROCESS:
+      return "kern.maxfilesperproc";
+    case Env::ResourceLimitType::RUNNING_THREADS_PER_EUID:
+      return "kern.maxprocperuid";
+    default: LOG(FATAL) << "Unknown resource limit type: " << t;
+  }
+}
+#endif
+
+class PosixSequentialFile: public SequentialFile {
+ private:
+  std::string filename_;
+  FILE* file_;
+
+ public:
+  PosixSequentialFile(std::string fname, FILE* f)
+      : filename_(std::move(fname)), file_(f) {}
+  virtual ~PosixSequentialFile() {
+    int err;
+    RETRY_ON_EINTR(err, fclose(file_));
+    if (PREDICT_FALSE(err != 0)) {
+      PLOG(WARNING) << "Failed to close " << filename_;
+    }
+  }
+
+  virtual Status Read(Slice* result) OVERRIDE {
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    size_t r;
+    STREAM_RETRY_ON_EINTR(r, file_, fread_unlocked(result->mutable_data(), 1,
+                                                   result->size(), file_));
+    if (r < result->size()) {
+      if (feof(file_)) {
+        // We leave status as ok if we hit the end of the file.
+        // We need to adjust the slice size.
+        result->truncate(r);
+      } else {
+        // A partial read with an error: return a non-ok status.
+        return IOError(filename_, errno);
+      }
+    }
+    return Status::OK();
+  }
+
+  virtual Status Skip(uint64_t n) OVERRIDE {
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    TRACE_EVENT1("io", "PosixSequentialFile::Skip", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    if (fseek(file_, n, SEEK_CUR)) {
+      return IOError(filename_, errno);
+    }
+    return Status::OK();
+  }
+
+  virtual const string& filename() const OVERRIDE { return filename_; }
+};
+
+// pread() based random-access
+class PosixRandomAccessFile: public RandomAccessFile {
+ private:
+  std::string filename_;
+  int fd_;
+
+ public:
+  PosixRandomAccessFile(std::string fname, int fd)
+      : filename_(std::move(fname)), fd_(fd) {}
+  virtual ~PosixRandomAccessFile() {
+    int err;
+    RETRY_ON_EINTR(err, close(fd_));
+    if (PREDICT_FALSE(err != 0)) {
+      PLOG(WARNING) << "Failed to close " << filename_;
+    }
+  }
+
+  virtual Status Read(uint64_t offset, Slice result) const OVERRIDE {
+    return DoReadV(fd_, filename_, offset, ArrayView<Slice>(&result, 1));
+  }
+
+  virtual Status ReadV(uint64_t offset, ArrayView<Slice> results) const OVERRIDE {
+    return DoReadV(fd_, filename_, offset, results);
+  }
+
+  virtual Status Size(uint64_t *size) const OVERRIDE {
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    TRACE_EVENT1("io", "PosixRandomAccessFile::Size", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    struct stat st;
+    if (fstat(fd_, &st) == -1) {
+      return IOError(filename_, errno);
+    }
+    *size = st.st_size;
+    return Status::OK();
+  }
+
+  virtual const string& filename() const OVERRIDE { return filename_; }
+
+  virtual size_t memory_footprint() const OVERRIDE {
+    return kudu_malloc_usable_size(this) + filename_.capacity();
+  }
+};
+
+// Use non-memory mapped POSIX files to write data to a file.
+//
+// TODO (perf) investigate zeroing a pre-allocated allocated area in
+// order to further improve Sync() performance.
+class PosixWritableFile : public WritableFile {
+ public:
+  PosixWritableFile(std::string fname, int fd, uint64_t file_size,
+                    bool sync_on_close)
+      : filename_(std::move(fname)),
+        fd_(fd),
+        sync_on_close_(sync_on_close),
+        filesize_(file_size),
+        pre_allocated_size_(0),
+        pending_sync_(false) {}
+
+  ~PosixWritableFile() {
+    if (fd_ >= 0) {
+      WARN_NOT_OK(Close(), "Failed to close " + filename_);
+    }
+  }
+
+  virtual Status Append(const Slice& data) OVERRIDE {
+    return AppendV(ArrayView<const Slice>(&data, 1));
+  }
+
+  virtual Status AppendV(ArrayView<const Slice> data) OVERRIDE {
+    ThreadRestrictions::AssertIOAllowed();
+    RETURN_NOT_OK(DoWriteV(fd_, filename_, filesize_, data));
+    // Calculate the amount of data written
+    size_t bytes_written = accumulate(data.begin(), data.end(), static_cast<size_t>(0),
+                                      [&](int sum, const Slice& curr) {
+                                        return sum + curr.size();
+                                      });
+    filesize_ += bytes_written;
+    pending_sync_ = true;
+    return Status::OK();
+  }
+
+  virtual Status PreAllocate(uint64_t size) OVERRIDE {
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+
+    TRACE_EVENT1("io", "PosixWritableFile::PreAllocate", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    uint64_t offset = std::max(filesize_, pre_allocated_size_);
+    int ret;
+    RETRY_ON_EINTR(ret, fallocate(fd_, 0, offset, size));
+    if (ret != 0) {
+      if (errno == EOPNOTSUPP) {
+        KLOG_FIRST_N(WARNING, 1) << "The filesystem does not support fallocate().";
+      } else if (errno == ENOSYS) {
+        KLOG_FIRST_N(WARNING, 1) << "The kernel does not implement fallocate().";
+      } else {
+        return IOError(filename_, errno);
+      }
+    }
+    pre_allocated_size_ = offset + size;
+    return Status::OK();
+  }
+
+  virtual Status Close() OVERRIDE {
+    TRACE_EVENT1("io", "PosixWritableFile::Close", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    Status s;
+
+    // If we've allocated more space than we used, truncate to the
+    // actual size of the file and perform Sync().
+    if (filesize_ < pre_allocated_size_) {
+      int ret;
+      RETRY_ON_EINTR(ret, ftruncate(fd_, filesize_));
+      if (ret != 0) {
+        s = IOError(filename_, errno);
+        pending_sync_ = true;
+      }
+    }
+
+    if (sync_on_close_) {
+      Status sync_status = Sync();
+      if (!sync_status.ok()) {
+        LOG(ERROR) << "Unable to Sync " << filename_ << ": " << sync_status.ToString();
+        if (s.ok()) {
+          s = sync_status;
+        }
+      }
+    }
+
+    int ret;
+    RETRY_ON_EINTR(ret, close(fd_));
+    if (ret < 0) {
+      if (s.ok()) {
+        s = IOError(filename_, errno);
+      }
+    }
+
+    fd_ = -1;
+    return s;
+  }
+
+  virtual Status Flush(FlushMode mode) OVERRIDE {
+    TRACE_EVENT1("io", "PosixWritableFile::Flush", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+#if defined(__linux__)
+    int flags = SYNC_FILE_RANGE_WRITE;
+    if (mode == FLUSH_SYNC) {
+      flags |= SYNC_FILE_RANGE_WAIT_BEFORE;
+      flags |= SYNC_FILE_RANGE_WAIT_AFTER;
+    }
+    if (sync_file_range(fd_, 0, 0, flags) < 0) {
+      return IOError(filename_, errno);
+    }
+#else
+    if (mode == FLUSH_SYNC && fsync(fd_) < 0) {
+      return IOError(filename_, errno);
+    }
+#endif
+    return Status::OK();
+  }
+
+  virtual Status Sync() OVERRIDE {
+    TRACE_EVENT1("io", "PosixWritableFile::Sync", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    LOG_SLOW_EXECUTION(WARNING, 1000, Substitute("sync call for $0", filename_)) {
+      if (pending_sync_) {
+        pending_sync_ = false;
+        RETURN_NOT_OK(DoSync(fd_, filename_));
+      }
+    }
+    return Status::OK();
+  }
+
+  virtual uint64_t Size() const OVERRIDE {
+    return filesize_;
+  }
+
+  virtual const string& filename() const OVERRIDE { return filename_; }
+
+ private:
+  const std::string filename_;
+  int fd_;
+  bool sync_on_close_;
+  uint64_t filesize_;
+  uint64_t pre_allocated_size_;
+
+  bool pending_sync_;
+};
+
+class PosixRWFile : public RWFile {
+ public:
+  PosixRWFile(string fname, int fd, bool sync_on_close)
+      : filename_(std::move(fname)),
+        fd_(fd),
+        sync_on_close_(sync_on_close),
+        is_on_xfs_(false),
+        closed_(false) {}
+
+  ~PosixRWFile() {
+    WARN_NOT_OK(Close(), "Failed to close " + filename_);
+  }
+
+  virtual Status Read(uint64_t offset, Slice result) const OVERRIDE {
+    return DoReadV(fd_, filename_, offset, ArrayView<Slice>(&result, 1));
+  }
+
+  virtual Status ReadV(uint64_t offset, ArrayView<Slice> results) const OVERRIDE {
+    return DoReadV(fd_, filename_, offset, results);
+  }
+
+  virtual Status Write(uint64_t offset, const Slice& data) OVERRIDE {
+    return WriteV(offset, ArrayView<const Slice>(&data, 1));
+  }
+
+  virtual Status WriteV(uint64_t offset, ArrayView<const Slice> data) OVERRIDE {
+    return DoWriteV(fd_, filename_, offset, data);
+  }
+
+  virtual Status PreAllocate(uint64_t offset,
+                             size_t length,
+                             PreAllocateMode mode) OVERRIDE {
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+
+    TRACE_EVENT1("io", "PosixRWFile::PreAllocate", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    int falloc_mode = 0;
+    if (mode == DONT_CHANGE_FILE_SIZE) {
+      falloc_mode = FALLOC_FL_KEEP_SIZE;
+    }
+    int ret;
+    RETRY_ON_EINTR(ret, fallocate(fd_, falloc_mode, offset, length));
+    if (ret != 0) {
+      if (errno == EOPNOTSUPP) {
+        KLOG_FIRST_N(WARNING, 1) << "The filesystem does not support fallocate().";
+      } else if (errno == ENOSYS) {
+        KLOG_FIRST_N(WARNING, 1) << "The kernel does not implement fallocate().";
+      } else {
+        return IOError(filename_, errno);
+      }
+    }
+    return Status::OK();
+  }
+
+  virtual Status Truncate(uint64_t length) OVERRIDE {
+    TRACE_EVENT2("io", "PosixRWFile::Truncate", "path", filename_, "length", length);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    int ret;
+    RETRY_ON_EINTR(ret, ftruncate(fd_, length));
+    if (ret != 0) {
+      int err = errno;
+      return Status::IOError(Substitute("Unable to truncate file $0", filename_),
+                             Substitute("ftruncate() failed: $0", ErrnoToString(err)),
+                             err);
+    }
+    return Status::OK();
+  }
+
+  virtual Status PunchHole(uint64_t offset, size_t length) OVERRIDE {
+#if defined(__linux__)
+    TRACE_EVENT1("io", "PosixRWFile::PunchHole", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+
+    // KUDU-2052: xfs in el6 systems induces an fsync in the kernel whenever it
+    // performs a hole punch through the fallocate() syscall, even if the file
+    // range was already punched out. The older xfs-specific hole punching
+    // ioctl doesn't do this, despite eventually executing the same xfs code.
+    // To keep the code simple, we'll use this ioctl on any xfs system (not
+    // just on el6) and fallocate() on all other filesystems.
+    //
+    // Note: the cast to void* here (and back to PosixRWFile*, in InitIsOnXFS)
+    // is needed to avoid an undefined behavior warning from UBSAN.
+    once_.Init(&InitIsOnXFS, reinterpret_cast<void*>(this));
+    if (is_on_xfs_ && FLAGS_env_use_ioctl_hole_punch_on_xfs) {
+      xfs_flock64_t cmd;
+      memset(&cmd, 0, sizeof(cmd));
+      cmd.l_start = offset;
+      cmd.l_len = length;
+      if (ioctl(fd_, XFS_IOC_UNRESVSP64, &cmd) < 0) {
+        return IOError(filename_, errno);
+      }
+    } else {
+      int ret;
+      RETRY_ON_EINTR(ret, fallocate(
+          fd_, FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE, offset, length));
+      if (ret != 0) {
+        return IOError(filename_, errno);
+      }
+    }
+    return Status::OK();
+#else
+    return Status::NotSupported("Hole punching not supported on this platform");
+#endif
+  }
+
+  virtual Status Flush(FlushMode mode, uint64_t offset, size_t length) OVERRIDE {
+    TRACE_EVENT1("io", "PosixRWFile::Flush", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+#if defined(__linux__)
+    int flags = SYNC_FILE_RANGE_WRITE;
+    if (mode == FLUSH_SYNC) {
+      flags |= SYNC_FILE_RANGE_WAIT_AFTER;
+    }
+    if (sync_file_range(fd_, offset, length, flags) < 0) {
+      return IOError(filename_, errno);
+    }
+#else
+    if (mode == FLUSH_SYNC && fsync(fd_) < 0) {
+      return IOError(filename_, errno);
+    }
+#endif
+    return Status::OK();
+  }
+
+  virtual Status Sync() OVERRIDE {
+    TRACE_EVENT1("io", "PosixRWFile::Sync", "path", filename_);
+    ThreadRestrictions::AssertIOAllowed();
+    LOG_SLOW_EXECUTION(WARNING, 1000, Substitute("sync call for $0", filename())) {
+      RETURN_NOT_OK(DoSync(fd_, filename_));
+    }
+    return Status::OK();
+  }
+
+  virtual Status Close() OVERRIDE {
+    if (closed_) {
+      return Status::OK();
+    }
+    TRACE_EVENT1("io", "PosixRWFile::Close", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status s;
+
+    if (sync_on_close_) {
+      s = Sync();
+      if (!s.ok()) {
+        LOG(ERROR) << "Unable to Sync " << filename_ << ": " << s.ToString();
+      }
+    }
+
+    int ret;
+    RETRY_ON_EINTR(ret, close(fd_));
+    if (ret < 0) {
+      if (s.ok()) {
+        s = IOError(filename_, errno);
+      }
+    }
+
+    closed_ = true;
+    return s;
+  }
+
+  virtual Status Size(uint64_t* size) const OVERRIDE {
+    TRACE_EVENT1("io", "PosixRWFile::Size", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    struct stat st;
+    if (fstat(fd_, &st) == -1) {
+      return IOError(filename_, errno);
+    }
+    *size = st.st_size;
+    return Status::OK();
+  }
+
+  virtual Status GetExtentMap(ExtentMap* out) const OVERRIDE {
+#if !defined(__linux__)
+    return Status::NotSupported("GetExtentMap not supported on this platform");
+#else
+    TRACE_EVENT1("io", "PosixRWFile::GetExtentMap", "path", filename_);
+    MAYBE_RETURN_EIO(filename_, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+
+    // This allocation size is arbitrary.
+    static const int kBufSize = 4096;
+    uint8_t buf[kBufSize] = { 0 };
+
+    struct fiemap* fm = reinterpret_cast<struct fiemap*>(buf);
+    struct fiemap_extent* fme = &fm->fm_extents[0];
+    int avail_extents_in_buffer = (kBufSize - sizeof(*fm)) / sizeof(*fme);
+    bool saw_last_extent = false;
+    ExtentMap extents;
+    do {
+      // Fetch another block of extents.
+      fm->fm_length = FIEMAP_MAX_OFFSET;
+      fm->fm_extent_count = avail_extents_in_buffer;
+      if (ioctl(fd_, FS_IOC_FIEMAP, fm) == -1) {
+        return IOError(filename_, errno);
+      }
+
+      // No extents returned, this file must have no extents.
+      if (fm->fm_mapped_extents == 0) {
+        break;
+      }
+
+      // Parse the extent block.
+      uint64_t last_extent_end_offset;
+      for (int i = 0; i < fm->fm_mapped_extents; i++) {
+        if (fme[i].fe_flags & FIEMAP_EXTENT_LAST) {
+          // This should really be the last extent.
+          CHECK_EQ(fm->fm_mapped_extents - 1, i);
+
+          saw_last_extent = true;
+        }
+        InsertOrDie(&extents, fme[i].fe_logical, fme[i].fe_length);
+        VLOG(3) << Substitute("File $0 extent $1: o $2, l $3 $4",
+                              filename_, i,
+                              fme[i].fe_logical, fme[i].fe_length,
+                              saw_last_extent ? "(final)" : "");
+        last_extent_end_offset = fme[i].fe_logical + fme[i].fe_length;
+        if (saw_last_extent) {
+          break;
+        }
+      }
+
+      fm->fm_start = last_extent_end_offset;
+    } while (!saw_last_extent);
+
+    out->swap(extents);
+    return Status::OK();
+#endif
+  }
+
+  virtual const string& filename() const OVERRIDE {
+    return filename_;
+  }
+
+ private:
+  static void InitIsOnXFS(void* arg) {
+    PosixRWFile* rwf = reinterpret_cast<PosixRWFile*>(arg);
+    bool result;
+    Status s = DoIsOnXfsFilesystem(rwf->filename_, &result);
+    if (s.ok()) {
+      rwf->is_on_xfs_ = result;
+    } else {
+      KLOG_EVERY_N_SECS(WARNING, 1) <<
+          Substitute("Could not determine whether file is on xfs, assuming not: $0",
+                     s.ToString());
+    }
+  }
+
+  const std::string filename_;
+  const int fd_;
+  const bool sync_on_close_;
+
+  GoogleOnceDynamic once_;
+  bool is_on_xfs_;
+  bool closed_;
+};
+
+int LockOrUnlock(int fd, bool lock) {
+  ThreadRestrictions::AssertIOAllowed();
+  errno = 0;
+  struct flock f;
+  memset(&f, 0, sizeof(f));
+  f.l_type = (lock ? F_WRLCK : F_UNLCK);
+  f.l_whence = SEEK_SET;
+  f.l_start = 0;
+  f.l_len = 0;        // Lock/unlock entire file
+  int ret;
+  RETRY_ON_EINTR(ret, fcntl(fd, F_SETLK, &f));
+  return ret;
+}
+
+class PosixFileLock : public FileLock {
+ public:
+  int fd_;
+};
+
+class PosixEnv : public Env {
+ public:
+  PosixEnv();
+  virtual ~PosixEnv() {
+    fprintf(stderr, "Destroying Env::Default()\n");
+    exit(1);
+  }
+
+  virtual Status NewSequentialFile(const std::string& fname,
+                                   unique_ptr<SequentialFile>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewSequentialFile", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    FILE* f;
+    POINTER_RETRY_ON_EINTR(f, fopen(fname.c_str(), "r"));
+    if (f == nullptr) {
+      return IOError(fname, errno);
+    }
+    result->reset(new PosixSequentialFile(fname, f));
+    return Status::OK();
+  }
+
+  virtual Status NewRandomAccessFile(const std::string& fname,
+                                     unique_ptr<RandomAccessFile>* result) OVERRIDE {
+    return NewRandomAccessFile(RandomAccessFileOptions(), fname, result);
+  }
+
+  virtual Status NewRandomAccessFile(const RandomAccessFileOptions& opts,
+                                     const std::string& fname,
+                                     unique_ptr<RandomAccessFile>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewRandomAccessFile", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    int fd;
+    RETRY_ON_EINTR(fd, open(fname.c_str(), O_RDONLY));
+    if (fd < 0) {
+      return IOError(fname, errno);
+    }
+
+    result->reset(new PosixRandomAccessFile(fname, fd));
+    return Status::OK();
+  }
+
+  virtual Status NewWritableFile(const std::string& fname,
+                                 unique_ptr<WritableFile>* result) OVERRIDE {
+    return NewWritableFile(WritableFileOptions(), fname, result);
+  }
+
+  virtual Status NewWritableFile(const WritableFileOptions& opts,
+                                 const std::string& fname,
+                                 unique_ptr<WritableFile>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewWritableFile", "path", fname);
+    int fd;
+    RETURN_NOT_OK(DoOpen(fname, opts.mode, &fd));
+    return InstantiateNewWritableFile(fname, fd, opts, result);
+  }
+
+  virtual Status NewTempWritableFile(const WritableFileOptions& opts,
+                                     const std::string& name_template,
+                                     std::string* created_filename,
+                                     unique_ptr<WritableFile>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewTempWritableFile", "template", name_template);
+    int fd;
+    string tmp_filename;
+    RETURN_NOT_OK(MkTmpFile(name_template, &fd, &tmp_filename));
+    RETURN_NOT_OK(InstantiateNewWritableFile(tmp_filename, fd, opts, result));
+    created_filename->swap(tmp_filename);
+    return Status::OK();
+  }
+
+  virtual Status NewRWFile(const string& fname,
+                           unique_ptr<RWFile>* result) OVERRIDE {
+    return NewRWFile(RWFileOptions(), fname, result);
+  }
+
+  virtual Status NewRWFile(const RWFileOptions& opts,
+                           const string& fname,
+                           unique_ptr<RWFile>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewRWFile", "path", fname);
+    int fd;
+    RETURN_NOT_OK(DoOpen(fname, opts.mode, &fd));
+    result->reset(new PosixRWFile(fname, fd, opts.sync_on_close));
+    return Status::OK();
+  }
+
+  virtual Status NewTempRWFile(const RWFileOptions& opts, const std::string& name_template,
+                               std::string* created_filename, unique_ptr<RWFile>* res) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::NewTempRWFile", "template", name_template);
+    int fd;
+    RETURN_NOT_OK(MkTmpFile(name_template, &fd, created_filename));
+    res->reset(new PosixRWFile(*created_filename, fd, opts.sync_on_close));
+    return Status::OK();
+  }
+
+  virtual bool FileExists(const std::string& fname) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::FileExists", "path", fname);
+    ThreadRestrictions::AssertIOAllowed();
+    return access(fname.c_str(), F_OK) == 0;
+  }
+
+  virtual Status GetChildren(const std::string& dir,
+                             std::vector<std::string>* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetChildren", "path", dir);
+    MAYBE_RETURN_EIO(dir, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    result->clear();
+    DIR* d = opendir(dir.c_str());
+    if (d == nullptr) {
+      return IOError(dir, errno);
+    }
+    struct dirent* entry;
+    // TODO: lint: Consider using readdir_r(...) instead of readdir(...) for improved thread safety.
+    while ((entry = readdir(d)) != nullptr) {
+      result->push_back(entry->d_name);
+    }
+    closedir(d);
+    return Status::OK();
+  }
+
+  virtual Status DeleteFile(const std::string& fname) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::DeleteFile", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status result;
+    if (unlink(fname.c_str()) != 0) {
+      result = IOError(fname, errno);
+    }
+    return result;
+  };
+
+  virtual Status CreateDir(const std::string& name) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::CreateDir", "path", name);
+    MAYBE_RETURN_EIO(name, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status result;
+    if (mkdir(name.c_str(), 0777) != 0) {
+      result = IOError(name, errno);
+    }
+    return result;
+  };
+
+  virtual Status DeleteDir(const std::string& name) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::DeleteDir", "path", name);
+    MAYBE_RETURN_EIO(name, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status result;
+    if (rmdir(name.c_str()) != 0) {
+      result = IOError(name, errno);
+    }
+    return result;
+  };
+
+  Status GetCurrentWorkingDir(string* cwd) const override {
+    TRACE_EVENT0("io", "PosixEnv::GetCurrentWorkingDir");
+    ThreadRestrictions::AssertIOAllowed();
+    unique_ptr<char, FreeDeleter> wd(getcwd(NULL, 0));
+    if (!wd) {
+      return IOError("getcwd()", errno);
+    }
+    cwd->assign(wd.get());
+
+    MAYBE_RETURN_EIO(*cwd, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    return Status::OK();
+  }
+
+  Status ChangeDir(const string& dest) override {
+    TRACE_EVENT1("io", "PosixEnv::ChangeDir", "dest", dest);
+    MAYBE_RETURN_EIO(dest, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status result;
+    if (chdir(dest.c_str()) != 0) {
+      result = IOError(dest, errno);
+    }
+    return result;
+  }
+
+  virtual Status SyncDir(const std::string& dirname) OVERRIDE {
+    TRACE_EVENT1("io", "SyncDir", "path", dirname);
+    MAYBE_RETURN_EIO(dirname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    if (FLAGS_never_fsync) return Status::OK();
+    int dir_fd;
+    RETRY_ON_EINTR(dir_fd, open(dirname.c_str(), O_DIRECTORY|O_RDONLY));
+    if (dir_fd < 0) {
+      return IOError(dirname, errno);
+    }
+    ScopedFdCloser fd_closer(dir_fd);
+    if (fsync(dir_fd) != 0) {
+      return IOError(dirname, errno);
+    }
+    return Status::OK();
+  }
+
+  virtual Status DeleteRecursively(const std::string &name) OVERRIDE {
+    return Walk(name, POST_ORDER, Bind(&PosixEnv::DeleteRecursivelyCb,
+                                       Unretained(this)));
+  }
+
+  virtual Status GetFileSize(const std::string& fname, uint64_t* size) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetFileSize", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status s;
+    struct stat sbuf;
+    if (stat(fname.c_str(), &sbuf) != 0) {
+      s = IOError(fname, errno);
+    } else {
+      *size = sbuf.st_size;
+    }
+    return s;
+  }
+
+  virtual Status GetFileSizeOnDisk(const std::string& fname, uint64_t* size) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetFileSizeOnDisk", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status s;
+    struct stat sbuf;
+    if (stat(fname.c_str(), &sbuf) != 0) {
+      s = IOError(fname, errno);
+    } else {
+      // From stat(2):
+      //
+      //   The st_blocks field indicates the number of blocks allocated to
+      //   the file, 512-byte units. (This may be smaller than st_size/512
+      //   when the file has holes.)
+      *size = sbuf.st_blocks * 512;
+    }
+    return s;
+  }
+
+  virtual Status GetFileSizeOnDiskRecursively(const string& root,
+                                              uint64_t* bytes_used) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetFileSizeOnDiskRecursively", "path", root);
+    uint64_t total = 0;
+    RETURN_NOT_OK(Walk(root, Env::PRE_ORDER,
+                       Bind(&PosixEnv::GetFileSizeOnDiskRecursivelyCb,
+                            Unretained(this), &total)));
+    *bytes_used = total;
+    return Status::OK();
+  }
+
+  virtual Status GetBlockSize(const string& fname, uint64_t* block_size) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetBlockSize", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status s;
+    struct stat sbuf;
+    if (stat(fname.c_str(), &sbuf) != 0) {
+      s = IOError(fname, errno);
+    } else {
+      *block_size = sbuf.st_blksize;
+    }
+    return s;
+  }
+
+  virtual Status GetFileModifiedTime(const string& fname, int64_t* timestamp) override {
+    TRACE_EVENT1("io", "PosixEnv::GetFileModifiedTime", "fname", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+
+    struct stat s;
+    if (stat(fname.c_str(), &s) != 0) {
+      return IOError(fname, errno);
+    }
+#ifdef __APPLE__
+    *timestamp = s.st_mtimespec.tv_sec * 1000000 + s.st_mtimespec.tv_nsec / 1000;
+#else
+    *timestamp = s.st_mtim.tv_sec * 1000000 + s.st_mtim.tv_nsec / 1000;
+#endif
+    return Status::OK();
+  }
+
+  // Local convenience function for safely running statvfs().
+  static Status StatVfs(const string& path, struct statvfs* buf) {
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    int ret;
+    RETRY_ON_EINTR(ret, statvfs(path.c_str(), buf));
+    if (ret == -1) {
+      return IOError(Substitute("statvfs: $0", path), errno);
+    }
+    return Status::OK();
+  }
+
+  virtual Status GetSpaceInfo(const string& path, SpaceInfo* space_info) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetSpaceInfo", "path", path);
+    struct statvfs buf;
+    RETURN_NOT_OK(StatVfs(path, &buf));
+    space_info->capacity_bytes = buf.f_frsize * buf.f_blocks;
+    space_info->free_bytes = buf.f_frsize * buf.f_bavail;
+    return Status::OK();
+  }
+
+  virtual Status RenameFile(const std::string& src, const std::string& target) OVERRIDE {
+    TRACE_EVENT2("io", "PosixEnv::RenameFile", "src", src, "dst", target);
+    MAYBE_RETURN_EIO(src, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    MAYBE_RETURN_EIO(target, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status result;
+    if (rename(src.c_str(), target.c_str()) != 0) {
+      result = IOError(src, errno);
+    }
+    return result;
+  }
+
+  virtual Status LockFile(const std::string& fname, FileLock** lock) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::LockFile", "path", fname);
+    MAYBE_RETURN_EIO(fname, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    if (ShouldInject(fname, FLAGS_env_inject_lock_failure_globs)) {
+      return IOError("lock " + fname, EAGAIN);
+    }
+    ThreadRestrictions::AssertIOAllowed();
+    *lock = nullptr;
+    Status result;
+    int fd;
+    RETRY_ON_EINTR(fd, open(fname.c_str(), O_RDWR | O_CREAT, 0666));
+    if (fd < 0) {
+      result = IOError(fname, errno);
+    } else if (LockOrUnlock(fd, true) == -1) {
+      result = IOError("lock " + fname, errno);
+      int err;
+      RETRY_ON_EINTR(err, close(fd));
+      if (PREDICT_FALSE(err != 0)) {
+        PLOG(WARNING) << "Failed to close fd " << fd;
+      }
+    } else {
+      auto my_lock = new PosixFileLock;
+      my_lock->fd_ = fd;
+      *lock = my_lock;
+    }
+    return result;
+  }
+
+  virtual Status UnlockFile(FileLock* lock) OVERRIDE {
+    TRACE_EVENT0("io", "PosixEnv::UnlockFile");
+    ThreadRestrictions::AssertIOAllowed();
+    unique_ptr<PosixFileLock> my_lock(reinterpret_cast<PosixFileLock*>(lock));
+    Status result;
+    if (LockOrUnlock(my_lock->fd_, false) == -1) {
+      result = IOError("unlock", errno);
+    }
+    int err;
+    RETRY_ON_EINTR(err, close(my_lock->fd_));
+    if (PREDICT_FALSE(err != 0)) {
+      PLOG(WARNING) << "Failed to close fd " << my_lock->fd_;
+    }
+    return result;
+  }
+
+  virtual Status GetTestDirectory(std::string* result) OVERRIDE {
+    string dir;
+    const char* env = getenv("TEST_TMPDIR");
+    if (env && env[0] != '\0') {
+      dir = env;
+    } else {
+      char buf[100];
+      snprintf(buf, sizeof(buf), "/tmp/kudutest-%d", static_cast<int>(geteuid()));
+      dir = buf;
+    }
+    // Directory may already exist
+    ignore_result(CreateDir(dir));
+    // /tmp may be a symlink, so canonicalize the path.
+    return Canonicalize(dir, result);
+  }
+
+  virtual uint64_t gettid() OVERRIDE {
+    // Platform-independent thread ID.  We can't use pthread_self here,
+    // because that function returns a totally opaque ID, which can't be
+    // compared via normal means.
+    if (thread_local_id == 0) {
+      thread_local_id = Barrier_AtomicIncrement(&cur_thread_local_id_, 1);
+    }
+    return thread_local_id;
+  }
+
+  virtual uint64_t NowMicros() OVERRIDE {
+    struct timeval tv;
+    gettimeofday(&tv, nullptr);
+    return static_cast<uint64_t>(tv.tv_sec) * 1000000 + tv.tv_usec;
+  }
+
+  virtual void SleepForMicroseconds(int micros) OVERRIDE {
+    ThreadRestrictions::AssertWaitAllowed();
+    SleepFor(MonoDelta::FromMicroseconds(micros));
+  }
+
+  virtual Status GetExecutablePath(string* path) OVERRIDE {
+    MAYBE_RETURN_EIO("/proc/self/exe", IOError(Env::kInjectedFailureStatusMsg, EIO));
+    uint32_t size = 64;
+    uint32_t len = 0;
+    while (true) {
+      unique_ptr<char[]> buf(new char[size]);
+#if defined(__linux__)
+      int rc = readlink("/proc/self/exe", buf.get(), size);
+      if (rc == -1) {
+        return IOError("Unable to determine own executable path", errno);
+      } else if (rc >= size) {
+        // The buffer wasn't large enough
+        size *= 2;
+        continue;
+      }
+      len = rc;
+#elif defined(__APPLE__)
+      if (_NSGetExecutablePath(buf.get(), &size) != 0) {
+        // The buffer wasn't large enough; 'size' has been updated.
+        continue;
+      }
+      len = strlen(buf.get());
+#else
+#error Unsupported platform
+#endif
+
+      path->assign(buf.get(), len);
+      break;
+    }
+    return Status::OK();
+  }
+
+  virtual Status IsDirectory(const string& path, bool* is_dir) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::IsDirectory", "path", path);
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    Status s;
+    struct stat sbuf;
+    if (stat(path.c_str(), &sbuf) != 0) {
+      s = IOError(path, errno);
+    } else {
+      *is_dir = S_ISDIR(sbuf.st_mode);
+    }
+    return s;
+  }
+
+  virtual Status Walk(const string& root, DirectoryOrder order, const WalkCallback& cb) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::Walk", "path", root);
+    MAYBE_RETURN_EIO(root, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    // Some sanity checks
+    CHECK_NE(root, "/");
+    CHECK_NE(root, "./");
+    CHECK_NE(root, ".");
+    CHECK_NE(root, "");
+
+    // FTS requires a non-const copy of the name. strdup it and free() when
+    // we leave scope.
+    unique_ptr<char, FreeDeleter> name_dup(strdup(root.c_str()));
+    char *(paths[]) = { name_dup.get(), nullptr };
+
+    // FTS_NOCHDIR is important here to make this thread-safe.
+    FTS* ret;
+    POINTER_RETRY_ON_EINTR(ret, fts_open(
+        paths, FTS_PHYSICAL | FTS_XDEV | FTS_NOCHDIR, nullptr));
+    if (ret == nullptr) {
+      return IOError(root, errno);
+    }
+    unique_ptr<FTS, FtsCloser> tree(ret);
+
+    FTSENT *ent = nullptr;
+    bool had_errors = false;
+    while ((ent = fts_read(tree.get())) != nullptr) {
+      bool doCb = false;
+      FileType type = DIRECTORY_TYPE;
+      switch (ent->fts_info) {
+        case FTS_D:         // Directory in pre-order
+          if (order == PRE_ORDER) {
+            doCb = true;
+          }
+          break;
+        case FTS_DP:        // Directory in post-order
+          if (order == POST_ORDER) {
+            doCb = true;
+          }
+          break;
+        case FTS_F:         // A regular file
+        case FTS_SL:        // A symbolic link
+        case FTS_SLNONE:    // A broken symbolic link
+        case FTS_DEFAULT:   // Unknown type of file
+          doCb = true;
+          type = FILE_TYPE;
+          break;
+
+        case FTS_DNR:
+        case FTS_ERR:
+        case FTS_NS:
+          LOG(WARNING) << "Unable to access file " << ent->fts_path
+                       << " during walk: " << strerror(ent->fts_errno);
+          had_errors = true;
+          break;
+
+        default:
+          LOG(WARNING) << "Unable to access file " << ent->fts_path
+                       << " during walk (code " << ent->fts_info << ")";
+          break;
+      }
+      if (doCb) {
+        if (!cb.Run(type, DirName(ent->fts_path), ent->fts_name).ok()) {
+          had_errors = true;
+        }
+      }
+    }
+
+    if (had_errors) {
+      return Status::IOError(root, "One or more errors occurred");
+    }
+    return Status::OK();
+  }
+
+  Status Glob(const string& path_pattern, vector<string>* paths) override {
+    TRACE_EVENT1("io", "PosixEnv::Glob", "path_pattern", path_pattern);
+    ThreadRestrictions::AssertIOAllowed();
+
+    glob_t result;
+    auto cleanup = MakeScopedCleanup([&] { globfree(&result); });
+
+    errno = 0;
+    int ret = glob(path_pattern.c_str(), GLOB_TILDE | GLOB_ERR , NULL, &result);
+    switch (ret) {
+      case 0: break;
+      case GLOB_NOMATCH: return Status::OK();
+      case GLOB_NOSPACE: return Status::RuntimeError("glob out of memory");
+      default: {
+        string err = (errno != 0) ? ErrnoToString(errno) : "unknown error";
+        return Status::IOError(Substitute("glob failed for $0: $1",
+                                          path_pattern,
+                                          err));
+      }
+    }
+
+    for (size_t i = 0; i < result.gl_pathc; ++i) {
+      paths->emplace_back(result.gl_pathv[i]);
+    }
+    return Status::OK();
+  }
+
+  virtual Status Canonicalize(const string& path, string* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::Canonicalize", "path", path);
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    unique_ptr<char[], FreeDeleter> r(realpath(path.c_str(), nullptr));
+    if (!r) {
+      return IOError(Substitute("Unable to canonicalize $0", path), errno);
+    }
+    *result = string(r.get());
+    return Status::OK();
+  }
+
+  virtual Status GetTotalRAMBytes(int64_t* ram) OVERRIDE {
+#if defined(__APPLE__)
+    int mib[2];
+    size_t length = sizeof(*ram);
+
+    // Get the Physical memory size
+    mib[0] = CTL_HW;
+    mib[1] = HW_MEMSIZE;
+    CHECK_ERR(sysctl(mib, 2, ram, &length, nullptr, 0)) << "sysctl CTL_HW HW_MEMSIZE failed";
+#else
+    struct sysinfo info;
+    if (sysinfo(&info) < 0) {
+      return IOError("sysinfo() failed", errno);
+    }
+    *ram = info.totalram;
+#endif
+    return Status::OK();
+  }
+
+  virtual uint64_t GetResourceLimit(ResourceLimitType t) OVERRIDE {
+    static_assert(std::is_unsigned<rlim_t>::value, "rlim_t must be unsigned");
+    static_assert(RLIM_INFINITY > 0, "RLIM_INFINITY must be positive");
+
+    // There's no reason for this to ever fail.
+    struct rlimit l;
+    PCHECK(getrlimit(ResourceLimitTypeToUnixRlimit(t), &l) == 0);
+    return l.rlim_cur;
+  }
+
+  virtual void IncreaseResourceLimit(ResourceLimitType t) OVERRIDE {
+    // There's no reason for this to ever fail; any process should have
+    // sufficient privilege to increase its soft limit up to the hard limit.
+    //
+    // This change is logged because it is process-wide.
+
+    int rlimit_type = ResourceLimitTypeToUnixRlimit(t);
+    struct rlimit l;
+    PCHECK(getrlimit(rlimit_type, &l) == 0);
+#if defined(__APPLE__)
+    // OS X 10.11 can return RLIM_INFINITY from getrlimit, but allows rlim_cur and
+    // rlim_max to be raised only as high as the value of the maxfilesperproc
+    // kernel variable. Empirically, this value is 10240 across all tested macOS
+    // versions. Testing on OS X 10.10 and macOS 10.12 revealed that getrlimit
+    // returns the true limits (not RLIM_INFINITY), rlim_max can *not* be raised
+    // (when running as non-root), and rlim_cur can only be raised as high as
+    // rlim_max (this is consistent with Linux).
+    // TLDR; OS X 10.11 is whack.
+    if (l.rlim_max == RLIM_INFINITY) {
+      uint32_t limit;
+      size_t len = sizeof(limit);
+      PCHECK(sysctlbyname(ResourceLimitTypeToMacosRlimit(t), &limit, &len,
+                          nullptr, 0) == 0);
+      // Make sure no uninitialized bits are present in the result.
+      DCHECK_EQ(sizeof(limit), len);
+      l.rlim_max = limit;
+    }
+#endif
+    const char* rlimit_str = ResourceLimitTypeToString(t);
+    if (l.rlim_cur < l.rlim_max) {
+      LOG(INFO) << Substitute("Raising this process' $0 limit from $1 to $2",
+                              rlimit_str, l.rlim_cur, l.rlim_max);
+      l.rlim_cur = l.rlim_max;
+      PCHECK(setrlimit(rlimit_type, &l) == 0);
+    } else {
+      LOG(INFO) << Substitute("Not raising this process' $0 limit of $1; it "
+          "is already as high as it can go", rlimit_str, l.rlim_cur);
+    }
+  }
+
+  virtual Status IsOnExtFilesystem(const string& path, bool* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::IsOnExtFilesystem", "path", path);
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+
+#ifdef __APPLE__
+    *result = false;
+#else
+    struct statfs buf;
+    int ret;
+    RETRY_ON_EINTR(ret, statfs(path.c_str(), &buf));
+    if (ret == -1) {
+      return IOError(Substitute("statfs: $0", path), errno);
+    }
+    *result = (buf.f_type == EXT4_SUPER_MAGIC);
+#endif
+    return Status::OK();
+  }
+
+  virtual Status IsOnXfsFilesystem(const string& path, bool* result) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::IsOnXfsFilesystem", "path", path);
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    ThreadRestrictions::AssertIOAllowed();
+    return DoIsOnXfsFilesystem(path, result);
+  }
+
+  virtual string GetKernelRelease() OVERRIDE {
+    // There's no reason for this to ever fail.
+    struct utsname u;
+    PCHECK(uname(&u) == 0);
+    return string(u.release);
+  }
+
+  Status EnsureFileModeAdheresToUmask(const string& path) override {
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    struct stat s;
+    if (stat(path.c_str(), &s) != 0) {
+      return IOError("stat", errno);
+    }
+    CHECK_NE(g_parsed_umask, -1);
+    if (s.st_mode & g_parsed_umask) {
+      uint32_t old_perms = s.st_mode & ACCESSPERMS;
+      uint32_t new_perms = old_perms & ~g_parsed_umask;
+      LOG(WARNING) << "Path " << path << " has permissions "
+                   << StringPrintf("%03o", old_perms)
+                   << " which are less restrictive than current umask value "
+                   << StringPrintf("%03o", g_parsed_umask)
+                   << ": resetting permissions to "
+                   << StringPrintf("%03o", new_perms);
+      if (chmod(path.c_str(), new_perms) != 0) {
+        return IOError("chmod", errno);
+      }
+    }
+    return Status::OK();
+  }
+
+  Status IsFileWorldReadable(const string& path, bool* result) override {
+    ThreadRestrictions::AssertIOAllowed();
+    TRACE_EVENT1("io", "PosixEnv::IsFileWorldReadable", "path", path);
+    MAYBE_RETURN_EIO(path, IOError(Env::kInjectedFailureStatusMsg, EIO));
+    struct stat s;
+    if (stat(path.c_str(), &s) != 0) {
+      return IOError("stat", errno);
+    }
+    *result = (s.st_mode & S_IROTH) != 0;
+    return Status::OK();
+  }
+
+ private:
+  // unique_ptr Deleter implementation for fts_close
+  struct FtsCloser {
+    void operator()(FTS *fts) const {
+      if (fts) {
+        int err;
+        RETRY_ON_EINTR(err, fts_close(fts));
+        if (PREDICT_FALSE(err != 0)) {
+          PLOG(WARNING) << "Failed to close fts";
+        }
+      }
+    }
+  };
+
+  Status MkTmpFile(const string& name_template, int* fd, string* created_filename) {
+    ThreadRestrictions::AssertIOAllowed();
+    unique_ptr<char[]> fname(new char[name_template.size() + 1]);
+    ::snprintf(fname.get(), name_template.size() + 1, "%s", name_template.c_str());
+    MAYBE_RETURN_EIO(fname.get(), IOError(Env::kInjectedFailureStatusMsg, EIO));
+    int created_fd = mkstemp(fname.get());
+    if (created_fd < 0) {
+      return IOError(Substitute("Call to mkstemp() failed on name template $0", name_template),
+                     errno);
+    }
+    // mkstemp defaults to making files with permissions 0600. But, if the
+    // user configured a more permissive umask, then we ensure that the
+    // resulting file gets the desired (wider) permissions.
+    uint32_t new_perms = 0666 & ~g_parsed_umask;
+    if (new_perms != 0600) {
+      CHECK_ERR(fchmod(created_fd, new_perms));
+    }
+    *fd = created_fd;
+    *created_filename = fname.get();
+    return Status::OK();
+  }
+
+  Status InstantiateNewWritableFile(const std::string& fname,
+                                    int fd,
+                                    const WritableFileOptions& opts,
+                                    unique_ptr<WritableFile>* result) {
+    uint64_t file_size = 0;
+    if (opts.mode == OPEN_EXISTING) {
+      RETURN_NOT_OK(GetFileSize(fname, &file_size));
+    }
+    result->reset(new PosixWritableFile(fname, fd, file_size, opts.sync_on_close));
+    return Status::OK();
+  }
+
+  Status DeleteRecursivelyCb(FileType type, const string& dirname, const string& basename) {
+    string full_path = JoinPathSegments(dirname, basename);
+    Status s;
+    switch (type) {
+      case FILE_TYPE:
+        s = DeleteFile(full_path);
+        WARN_NOT_OK(s, "Could not delete file");
+        return s;
+      case DIRECTORY_TYPE:
+        s = DeleteDir(full_path);
+        WARN_NOT_OK(s, "Could not delete directory");
+        return s;
+      default:
+        LOG(FATAL) << "Unknown file type: " << type;
+        return Status::OK();
+    }
+  }
+
+  Status GetFileSizeOnDiskRecursivelyCb(uint64_t* bytes_used,
+                                        Env::FileType type,
+                                        const string& dirname,
+                                        const string& basename) {
+    uint64_t file_bytes_used = 0;
+    switch (type) {
+      case Env::FILE_TYPE:
+        RETURN_NOT_OK(GetFileSizeOnDisk(
+            JoinPathSegments(dirname, basename), &file_bytes_used));
+        *bytes_used += file_bytes_used;
+        break;
+      case Env::DIRECTORY_TYPE:
+        // Ignore directory space consumption as it varies from filesystem to
+        // filesystem.
+        break;
+      default:
+        LOG(FATAL) << "Unknown file type: " << type;
+    }
+    return Status::OK();
+  }
+};
+
+PosixEnv::PosixEnv() {}
+
+}  // namespace
+
+static pthread_once_t once = PTHREAD_ONCE_INIT;
+static Env* default_env;
+static void InitDefaultEnv() { default_env = new PosixEnv; }
+
+Env* Env::Default() {
+  pthread_once(&once, InitDefaultEnv);
+  return default_env;
+}
+
+std::ostream& operator<<(std::ostream& o, Env::ResourceLimitType t) {
+  return o << ResourceLimitTypeToString(t);
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env_util-test.cc b/be/src/kudu/util/env_util-test.cc
new file mode 100644
index 0000000..9c8266f
--- /dev/null
+++ b/be/src/kudu/util/env_util-test.cc
@@ -0,0 +1,192 @@
+// 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 <sys/time.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <cerrno>
+#include <memory>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <glog/stl_logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_int64(disk_reserved_bytes_free_for_testing);
+
+using std::string;
+using std::unique_ptr;
+using std::unordered_set;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace env_util {
+
+class EnvUtilTest: public KuduTest {
+};
+
+// Assert that Status 's' indicates there is not enough space left on the
+// device for the request.
+static void AssertNoSpace(const Status& s) {
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_EQ(ENOSPC, s.posix_code());
+  ASSERT_STR_CONTAINS(s.ToString(), "Insufficient disk space");
+}
+
+TEST_F(EnvUtilTest, TestDiskSpaceCheck) {
+  const int64_t kZeroRequestedBytes = 0;
+  const int64_t kRequestOnePercentReservation = -1;
+  int64_t reserved_bytes = 0;
+  ASSERT_OK(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes, reserved_bytes));
+
+  // Check 1% reservation logic. We loop this in case there are other FS
+  // operations happening concurrent with this test.
+  ASSERT_EVENTUALLY([&] {
+    SpaceInfo space_info;
+    ASSERT_OK(env_->GetSpaceInfo(test_dir_, &space_info));
+    // Try for 1 less byte than 1% free. This request should be rejected.
+    int64_t target_free_bytes = (space_info.capacity_bytes / 100) - 1;
+    int64_t bytes_to_request = std::max<int64_t>(0, space_info.free_bytes - target_free_bytes);
+    NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, bytes_to_request,
+                                                      kRequestOnePercentReservation)));
+  });
+
+  // Make it seem as if the disk is full and specify that we should have
+  // reserved 200 bytes. Even asking for 0 bytes should return an error
+  // indicating we are out of space.
+  FLAGS_disk_reserved_bytes_free_for_testing = 0;
+  reserved_bytes = 200;
+  NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes,
+                                                    reserved_bytes)));
+}
+
+// Ensure that we can recursively create directories using both absolute and
+// relative paths.
+TEST_F(EnvUtilTest, TestCreateDirsRecursively) {
+  // Absolute path.
+  string path = JoinPathSegments(test_dir_, "a/b/c");
+  ASSERT_OK(CreateDirsRecursively(env_, path));
+  bool is_dir;
+  ASSERT_OK(env_->IsDirectory(path, &is_dir));
+  ASSERT_TRUE(is_dir);
+
+  // Repeating the previous command should also succeed (it should be a no-op).
+  ASSERT_OK(CreateDirsRecursively(env_, path));
+  ASSERT_OK(env_->IsDirectory(path, &is_dir));
+  ASSERT_TRUE(is_dir);
+
+  // Relative path.
+  ASSERT_OK(env_->ChangeDir(test_dir_)); // Change to test dir to keep CWD clean.
+  string rel_base = Substitute("$0-$1", CURRENT_TEST_CASE_NAME(), CURRENT_TEST_NAME());
+  ASSERT_FALSE(env_->FileExists(rel_base));
+  path = JoinPathSegments(rel_base, "x/y/z");
+  ASSERT_OK(CreateDirsRecursively(env_, path));
+  ASSERT_OK(env_->IsDirectory(path, &is_dir));
+  ASSERT_TRUE(is_dir);
+
+  // Directory creation should fail if a file is a part of the path.
+  path = JoinPathSegments(test_dir_, "x/y/z");
+  string file_path = JoinPathSegments(test_dir_, "x"); // Conflicts with 'path'.
+  ASSERT_FALSE(env_->FileExists(path));
+  ASSERT_FALSE(env_->FileExists(file_path));
+  // Create an empty file in the path.
+  unique_ptr<WritableFile> out;
+  ASSERT_OK(env_->NewWritableFile(file_path, &out));
+  ASSERT_OK(out->Close());
+  ASSERT_TRUE(env_->FileExists(file_path));
+  // Fail.
+  Status s = CreateDirsRecursively(env_, path);
+  ASSERT_TRUE(s.IsAlreadyPresent()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "File exists");
+
+  // We should be able to create a directory tree even when a symlink exists as
+  // part of the path.
+  path = JoinPathSegments(test_dir_, "link/a/b");
+  string link_path = JoinPathSegments(test_dir_, "link");
+  string real_dir = JoinPathSegments(test_dir_, "real_dir");
+  ASSERT_OK(env_->CreateDir(real_dir));
+  PCHECK(symlink(real_dir.c_str(), link_path.c_str()) == 0);
+  ASSERT_OK(CreateDirsRecursively(env_, path));
+  ASSERT_OK(env_->IsDirectory(path, &is_dir));
+  ASSERT_TRUE(is_dir);
+}
+
+// Ensure that DeleteExcessFilesByPattern() works.
+// We ensure that the number of files remaining after running it is the number
+// expected, and we manually set the modification times on the relevant files
+// to allow us to test that files are deleted oldest-first.
+TEST_F(EnvUtilTest, TestDeleteExcessFilesByPattern) {
+  string dir = JoinPathSegments(test_dir_, "excess");
+  ASSERT_OK(env_->CreateDir(dir));
+  vector<string> filenames = {"a", "b", "c", "d"};
+  int now_sec = GetCurrentTimeMicros() / 1000;
+  for (int i = 0; i < filenames.size(); i++) {
+    const string& filename = filenames[i];
+    string path = JoinPathSegments(dir, filename);
+    unique_ptr<WritableFile> file;
+    ASSERT_OK(env_->NewWritableFile(path, &file));
+    ASSERT_OK(file->Close());
+
+    // Set the last-modified time of the file.
+    struct timeval target_time { .tv_sec = now_sec + (i * 2), .tv_usec = 0 };
+    struct timeval times[2] = { target_time, target_time };
+    ASSERT_EQ(0, utimes(path.c_str(), times)) << errno;
+  }
+  vector<string> children;
+  ASSERT_OK(env_->GetChildren(dir, &children));
+  ASSERT_EQ(6, children.size()); // 4 files plus "." and "..".
+  ASSERT_OK(DeleteExcessFilesByPattern(env_, dir + "/*", 2));
+  ASSERT_OK(env_->GetChildren(dir, &children));
+  ASSERT_EQ(4, children.size()); // 2 files plus "." and "..".
+  unordered_set<string> children_set(children.begin(), children.end());
+  unordered_set<string> expected_set({".", "..", "c", "d"});
+  ASSERT_EQ(expected_set, children_set) << children;
+}
+
+TEST_F(EnvUtilTest, TestIsDirectoryEmpty) {
+  const string kDir = JoinPathSegments(test_dir_, "foo");
+  const string kFile = JoinPathSegments(kDir, "bar");
+
+  bool is_empty;
+  ASSERT_TRUE(env_util::IsDirectoryEmpty(env_, kDir, &is_empty).IsNotFound());
+  ASSERT_OK(env_->CreateDir(kDir));
+  ASSERT_OK(env_util::IsDirectoryEmpty(env_, kDir, &is_empty));
+  ASSERT_TRUE(is_empty);
+
+  unique_ptr<WritableFile> file;
+  ASSERT_OK(env_->NewWritableFile(WritableFileOptions(), kFile, &file));
+  ASSERT_OK(env_util::IsDirectoryEmpty(env_, kDir, &is_empty));
+  ASSERT_FALSE(is_empty);
+}
+
+} // namespace env_util
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env_util.cc b/be/src/kudu/util/env_util.cc
new file mode 100644
index 0000000..dde4138
--- /dev/null
+++ b/be/src/kudu/util/env_util.cc
@@ -0,0 +1,320 @@
+// 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 "kudu/util/env_util.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <cerrno>
+#include <ctime>
+#include <memory>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+
+DEFINE_int64(disk_reserved_bytes_free_for_testing, -1,
+             "For testing only! Set to number of bytes free on each filesystem. "
+             "Set to -1 to disable this test-specific override");
+TAG_FLAG(disk_reserved_bytes_free_for_testing, runtime);
+TAG_FLAG(disk_reserved_bytes_free_for_testing, unsafe);
+
+// We define some flags for testing purposes: Two prefixes and their associated
+// "bytes free" overrides.
+DEFINE_string(disk_reserved_override_prefix_1_path_for_testing, "",
+              "For testing only! Specifies a prefix to override the visible 'bytes free' on. "
+              "Use --disk_reserved_override_prefix_1_bytes_free_for_testing to set the number of "
+              "bytes free for this path prefix. Set to empty string to disable.");
+DEFINE_int64(disk_reserved_override_prefix_1_bytes_free_for_testing, -1,
+             "For testing only! Set number of bytes free on the path prefix specified by "
+             "--disk_reserved_override_prefix_1_path_for_testing. Set to -1 to disable.");
+DEFINE_string(disk_reserved_override_prefix_2_path_for_testing, "",
+              "For testing only! Specifies a prefix to override the visible 'bytes free' on. "
+              "Use --disk_reserved_override_prefix_2_bytes_free_for_testing to set the number of "
+              "bytes free for this path prefix. Set to empty string to disable.");
+DEFINE_int64(disk_reserved_override_prefix_2_bytes_free_for_testing, -1,
+             "For testing only! Set number of bytes free on the path prefix specified by "
+             "--disk_reserved_override_prefix_2_path_for_testing. Set to -1 to disable.");
+TAG_FLAG(disk_reserved_override_prefix_1_path_for_testing, unsafe);
+TAG_FLAG(disk_reserved_override_prefix_2_path_for_testing, unsafe);
+TAG_FLAG(disk_reserved_override_prefix_1_bytes_free_for_testing, unsafe);
+TAG_FLAG(disk_reserved_override_prefix_2_bytes_free_for_testing, unsafe);
+TAG_FLAG(disk_reserved_override_prefix_1_bytes_free_for_testing, runtime);
+TAG_FLAG(disk_reserved_override_prefix_2_bytes_free_for_testing, runtime);
+
+using std::pair;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::unordered_set;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace env_util {
+
+Status OpenFileForWrite(Env* env, const string& path,
+                        shared_ptr<WritableFile>* file) {
+  return OpenFileForWrite(WritableFileOptions(), env, path, file);
+}
+
+Status OpenFileForWrite(const WritableFileOptions& opts,
+                        Env *env, const string &path,
+                        shared_ptr<WritableFile> *file) {
+  unique_ptr<WritableFile> w;
+  RETURN_NOT_OK(env->NewWritableFile(opts, path, &w));
+  file->reset(w.release());
+  return Status::OK();
+}
+
+Status OpenFileForRandom(Env *env, const string &path,
+                         shared_ptr<RandomAccessFile> *file) {
+  unique_ptr<RandomAccessFile> r;
+  RETURN_NOT_OK(env->NewRandomAccessFile(path, &r));
+  file->reset(r.release());
+  return Status::OK();
+}
+
+Status OpenFileForSequential(Env *env, const string &path,
+                             shared_ptr<SequentialFile> *file) {
+  unique_ptr<SequentialFile> r;
+  RETURN_NOT_OK(env->NewSequentialFile(path, &r));
+  file->reset(r.release());
+  return Status::OK();
+}
+
+// If any of the override gflags specifies an override for the given path, then
+// override the free bytes to match what is specified in the flag. See the
+// definitions of these test-only flags for more information.
+static void OverrideBytesFreeWithTestingFlags(const string& path, int64_t* bytes_free) {
+  const string* prefixes[] = { &FLAGS_disk_reserved_override_prefix_1_path_for_testing,
+                               &FLAGS_disk_reserved_override_prefix_2_path_for_testing };
+  const int64_t* overrides[] = { &FLAGS_disk_reserved_override_prefix_1_bytes_free_for_testing,
+                                 &FLAGS_disk_reserved_override_prefix_2_bytes_free_for_testing };
+  for (int i = 0; i < arraysize(prefixes); i++) {
+    if (*overrides[i] != -1 && !prefixes[i]->empty() && HasPrefixString(path, *prefixes[i])) {
+      *bytes_free = *overrides[i];
+      return;
+    }
+  }
+}
+
+Status VerifySufficientDiskSpace(Env *env, const std::string& path,
+                                 int64_t requested_bytes, int64_t reserved_bytes) {
+  const int64_t kOnePercentReservation = -1;
+  DCHECK_GE(requested_bytes, 0);
+
+  SpaceInfo space_info;
+  RETURN_NOT_OK(env->GetSpaceInfo(path, &space_info));
+  int64_t available_bytes = space_info.free_bytes;
+
+  // Allow overriding these values by tests.
+  if (PREDICT_FALSE(FLAGS_disk_reserved_bytes_free_for_testing > -1)) {
+    available_bytes = FLAGS_disk_reserved_bytes_free_for_testing;
+  }
+  if (PREDICT_FALSE(FLAGS_disk_reserved_override_prefix_1_bytes_free_for_testing != -1 ||
+                    FLAGS_disk_reserved_override_prefix_2_bytes_free_for_testing != -1)) {
+    OverrideBytesFreeWithTestingFlags(path, &available_bytes);
+  }
+
+  // If they requested a one percent reservation, calculate what that is in bytes.
+  if (reserved_bytes == kOnePercentReservation) {
+    reserved_bytes = space_info.capacity_bytes / 100;
+  }
+
+  if (available_bytes - requested_bytes < reserved_bytes) {
+    return Status::IOError(Substitute("Insufficient disk space to allocate $0 bytes under path $1 "
+                                      "($2 bytes available vs $3 bytes reserved)",
+                                      requested_bytes, path, available_bytes, reserved_bytes),
+                           "", ENOSPC);
+  }
+  return Status::OK();
+}
+
+Status CreateDirIfMissing(Env* env, const string& path, bool* created) {
+  Status s = env->CreateDir(path);
+  if (created != nullptr) {
+    *created = s.ok();
+  }
+  return s.IsAlreadyPresent() ? Status::OK() : s;
+}
+
+Status CreateDirsRecursively(Env* env, const string& path) {
+  vector<string> segments = SplitPath(path);
+  string partial_path;
+  for (const string& segment : segments) {
+    partial_path = partial_path.empty() ? segment : JoinPathSegments(partial_path, segment);
+    bool is_dir;
+    Status s = env->IsDirectory(partial_path, &is_dir);
+    if (s.ok()) {
+      // We didn't get a NotFound error, so something is there.
+      if (is_dir) continue; // It's a normal directory.
+      // Maybe a file or a symlink. Let's try to follow the symlink.
+      string real_partial_path;
+      RETURN_NOT_OK(env->Canonicalize(partial_path, &real_partial_path));
+      s = env->IsDirectory(real_partial_path, &is_dir);
+      if (s.ok() && is_dir) continue; // It's a symlink to a directory.
+    }
+    RETURN_NOT_OK_PREPEND(env->CreateDir(partial_path), "Unable to create directory");
+  }
+  return Status::OK();
+}
+
+Status CopyFile(Env* env, const string& source_path, const string& dest_path,
+                WritableFileOptions opts) {
+  unique_ptr<SequentialFile> source;
+  RETURN_NOT_OK(env->NewSequentialFile(source_path, &source));
+  uint64_t size;
+  RETURN_NOT_OK(env->GetFileSize(source_path, &size));
+
+  unique_ptr<WritableFile> dest;
+  RETURN_NOT_OK(env->NewWritableFile(opts, dest_path, &dest));
+  RETURN_NOT_OK(dest->PreAllocate(size));
+
+  const int32_t kBufferSize = 1024 * 1024;
+  unique_ptr<uint8_t[]> scratch(new uint8_t[kBufferSize]);
+
+  uint64_t bytes_read = 0;
+  while (bytes_read < size) {
+    uint64_t max_bytes_to_read = std::min<uint64_t>(size - bytes_read, kBufferSize);
+    Slice data(scratch.get(), max_bytes_to_read);
+    RETURN_NOT_OK(source->Read(&data));
+    RETURN_NOT_OK(dest->Append(data));
+    bytes_read += data.size();
+  }
+  return Status::OK();
+}
+
+Status DeleteExcessFilesByPattern(Env* env, const string& pattern, int max_matches) {
+  // Negative numbers don't make sense for our interface.
+  DCHECK_GE(max_matches, 0);
+
+  vector<string> matching_files;
+  RETURN_NOT_OK(env->Glob(pattern, &matching_files));
+
+  if (matching_files.size() <= max_matches) {
+    return Status::OK();
+  }
+
+  vector<pair<time_t, string>> matching_file_mtimes;
+  for (string& matching_file_path : matching_files) {
+    int64_t mtime;
+    RETURN_NOT_OK(env->GetFileModifiedTime(matching_file_path, &mtime));
+    matching_file_mtimes.emplace_back(mtime, std::move(matching_file_path));
+  }
+
+  // Use mtime to determine which matching files to delete. This could
+  // potentially be ambiguous, depending on the resolution of last-modified
+  // timestamp in the filesystem, but that is part of the contract.
+  std::sort(matching_file_mtimes.begin(), matching_file_mtimes.end());
+  matching_file_mtimes.resize(matching_file_mtimes.size() - max_matches);
+
+  for (const auto& matching_file : matching_file_mtimes) {
+    RETURN_NOT_OK(env->DeleteFile(matching_file.second));
+  }
+
+  return Status::OK();
+}
+
+// Callback for DeleteTmpFilesRecursively().
+//
+// Tests 'basename' for the Kudu-specific tmp file infix, and if found,
+// deletes the file.
+static Status DeleteTmpFilesRecursivelyCb(Env* env,
+                                          Env::FileType file_type,
+                                          const string& dirname,
+                                          const string& basename) {
+  if (file_type != Env::FILE_TYPE) {
+    // Skip directories.
+    return Status::OK();
+  }
+
+  if (basename.find(kTmpInfix) != string::npos) {
+    string filename = JoinPathSegments(dirname, basename);
+    WARN_NOT_OK(env->DeleteFile(filename),
+                Substitute("Failed to remove temporary file $0", filename));
+  }
+  return Status::OK();
+}
+
+Status DeleteTmpFilesRecursively(Env* env, const string& path) {
+  return env->Walk(path, Env::PRE_ORDER, Bind(&DeleteTmpFilesRecursivelyCb, env));
+}
+
+Status IsDirectoryEmpty(Env* env, const string& path, bool* is_empty) {
+  vector<string> children;
+  RETURN_NOT_OK(env->GetChildren(path, &children));
+  for (const auto& c : children) {
+    if (c == "." || c == "..") {
+      continue;
+    }
+    *is_empty = false;
+    return Status::OK();
+  }
+  *is_empty = true;
+  return Status::OK();
+}
+
+Status SyncAllParentDirs(Env* env,
+                         const vector<string>& dirs,
+                         const vector<string>& files) {
+  // An unordered_set is used to deduplicate the set of directories.
+  unordered_set<string> to_sync;
+  for (const auto& d : dirs) {
+    to_sync.insert(DirName(d));
+  }
+  for (const auto& f : files) {
+    to_sync.insert(DirName(f));
+  }
+  for (const auto& d : to_sync) {
+    RETURN_NOT_OK_PREPEND(env->SyncDir(d),
+                          Substitute("unable to synchronize directory $0", d));
+  }
+  return Status::OK();
+}
+
+Status ListFilesInDir(Env* env,
+                      const string& path,
+                      vector<string>* entries) {
+  RETURN_NOT_OK(env->GetChildren(path, entries));
+  auto iter = entries->begin();
+  while (iter != entries->end()) {
+    if (*iter == "." || *iter == ".." || iter->find(kTmpInfix) != string::npos) {
+      iter = entries->erase(iter);
+      continue;
+    }
+    ++iter;
+  }
+  return Status::OK();
+}
+
+} // namespace env_util
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/env_util.h b/be/src/kudu/util/env_util.h
new file mode 100644
index 0000000..de1cc3c
--- /dev/null
+++ b/be/src/kudu/util/env_util.h
@@ -0,0 +1,112 @@
+// 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.
+#ifndef KUDU_UTIL_ENV_UTIL_H
+#define KUDU_UTIL_ENV_UTIL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Env;
+class RandomAccessFile;
+class SequentialFile;
+class WritableFile;
+struct WritableFileOptions;
+
+namespace env_util {
+
+Status OpenFileForWrite(Env *env, const std::string &path,
+                        std::shared_ptr<WritableFile> *file);
+
+Status OpenFileForWrite(const WritableFileOptions& opts,
+                        Env *env, const std::string &path,
+                        std::shared_ptr<WritableFile> *file);
+
+Status OpenFileForRandom(Env *env, const std::string &path,
+                         std::shared_ptr<RandomAccessFile> *file);
+
+Status OpenFileForSequential(Env *env, const std::string &path,
+                             std::shared_ptr<SequentialFile> *file);
+
+// Returns Status::IOError with POSIX code ENOSPC if there is not sufficient
+// disk space to write 'bytes' bytes to the file system represented by 'path'.
+// Otherwise returns OK.
+// If 'reserved_bytes' equals -1, it is interpreted as a 1% reservation. No
+// other values less than 0 are supported at this time.
+Status VerifySufficientDiskSpace(Env *env, const std::string& path,
+                                 int64_t requested_bytes, int64_t reserved_bytes);
+
+// Creates the directory given by 'path', unless it already exists.
+//
+// If 'created' is not NULL, sets it to true if the directory was
+// created, false otherwise.
+Status CreateDirIfMissing(Env* env, const std::string& path,
+                          bool* created = NULL);
+
+// Recursively create directories, if they do not exist, along the given path.
+// Returns OK if successful or if the given path already existed.
+// Upon failure, it is possible that some part of the directory structure may
+// have been successfully created. Emulates the behavior of `mkdir -p`.
+Status CreateDirsRecursively(Env* env, const std::string& path);
+
+// Copy the contents of file source_path to file dest_path.
+// This is not atomic, and if there is an error while reading or writing,
+// a partial copy may be left in 'dest_path'. Does not fsync the parent
+// directory of dest_path -- if you need durability then do that yourself.
+Status CopyFile(Env* env, const std::string& source_path, const std::string& dest_path,
+                WritableFileOptions opts);
+
+// Deletes files matching 'pattern' in excess of 'max_matches' files.
+// 'max_matches' must be greater than or equal to 0.
+// The oldest files are deleted first, as determined by last modified time.
+// In the case that multiple files have the same last modified time, it is not
+// defined which file will be deleted first.
+Status DeleteExcessFilesByPattern(Env* env, const std::string& pattern, int max_matches);
+
+// Traverses 'path' recursively and deletes all files matching the special Kudu
+// tmp file infix. Does not follow symlinks.
+//
+// Deletion errors generate warnings but do not halt the traversal.
+Status DeleteTmpFilesRecursively(Env* env, const std::string& path);
+
+// Checks if 'path' is an empty directory.
+//
+// Returns an error if it's not a directory. Otherwise, sets 'is_empty'
+// accordingly.
+Status IsDirectoryEmpty(Env* env, const std::string& path, bool* is_empty);
+
+// Synchronize all of the parent directories belonging to 'dirs' and 'files'
+// to disk.
+Status SyncAllParentDirs(Env* env,
+                         const std::vector<std::string>& dirs,
+                         const std::vector<std::string>& files);
+
+// Return a list of files within the given 'path'.
+Status ListFilesInDir(Env* env,
+                      const std::string& path,
+                      std::vector<std::string>* entries);
+
+} // namespace env_util
+} // namespace kudu
+
+#endif


[48/51] [abbrv] impala git commit: KUDU-2305: Limit sidecars to INT_MAX and fortify socket code

Posted by ta...@apache.org.
KUDU-2305: Limit sidecars to INT_MAX and fortify socket code

NOTE: This commit is part of a set of changes for IMPALA-7006. It
contains pieces of a previous commit that need to be cherry picked
again after rebasing the code in be/src/kudu/{util,security,rpc}.

The original commit message is below:

Inspection of the code revealed some other local variables
that could overflow with large messages. This patch takes
two approaches to eliminate the issues.

First, it limits the total size of the messages by limiting
the total size of the sidecars to INT_MAX. The total size
of the protobuf and header components of the message
should be considerably smaller, so limiting the sidecars
to INT_MAX eliminates messages that are larger than UINT_MAX.
This also means that the sidecar offsets, which are unsigned
32-bit integers, are also safe. Given that
FLAGS_rpc_max_message_size is limited to INT_MAX at startup,
the receiver would reject any message this large anyway.
This also helps with the networking codepath, as any given
sidecar will have a size less than INT_MAX, so every Slice
that interacts with Writev() is shorter than INT_MAX.

Second, even with sidecars limited to INT_MAX, the headers
and protobuf parts of the messages mean that certain messages
could still exceed INT_MAX. This patch changes some of the sockets
codepath to tolerate iovec's that reference more than INT_MAX
bytes total. Specifically, it changes Writev()'s nwritten bytes
to an int64_t for both TlsSocket and Socket. TlsSocket works
because it is sending each Slice individually. The first change
limited any given Slice to INT_MAX, so each individual Write()
should not be impacted. For Socket, Writev() uses sendmsg(). It
should do partial network sends to handle this case. Any Write()
call specifies its size with a 32-bit integer, and that will
not be impacted by this patch.

Testing:
 - Modified TestRpcSidecarLimits() to verify that sidecars are
   limited to INT_MAX bytes.
 - Added a test mode to TestRpcSidecarLimits() where it
   overrides rpc_max_message_size and sends the maximal
   message. This verifies that the client send codepath
   can handle the maximal message.

Reviewed-on: http://gerrit.cloudera.org:8080/9601
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>

Changes from Kudu version:
 - Updated declaration of FLAGS_rpc_max_message_size
   in rpc-mgr.cc and added a warning not to set it
   larger than INT_MAX.

Change-Id: Id23e518995f2bf2f6bf6b49d5f413f3eaa4e79d1
Reviewed-on: http://gerrit.cloudera.org:8080/9748
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/10765
Reviewed-by: Lars Volker <lv...@cloudera.com>
Tested-by: Lars Volker <lv...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/1ac9a3f3
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/1ac9a3f3
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/1ac9a3f3

Branch: refs/heads/master
Commit: 1ac9a3f329e3a792c1f3e26272149d05b14732b5
Parents: bee0182
Author: Joe McDonnell <jo...@cloudera.com>
Authored: Mon Mar 12 16:24:35 2018 -0700
Committer: Lars Volker <lv...@cloudera.com>
Committed: Thu Jul 12 21:35:42 2018 +0000

----------------------------------------------------------------------
 be/src/kudu/rpc/transfer.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/1ac9a3f3/be/src/kudu/rpc/transfer.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/transfer.cc b/be/src/kudu/rpc/transfer.cc
index bdf5191..fefa86e 100644
--- a/be/src/kudu/rpc/transfer.cc
+++ b/be/src/kudu/rpc/transfer.cc
@@ -36,7 +36,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/net/socket.h"
 
-DEFINE_int64(rpc_max_message_size, (50 * 1024 * 1024),
+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);


[35/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/service_queue.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/service_queue.h b/be/src/kudu/rpc/service_queue.h
new file mode 100644
index 0000000..2751a30
--- /dev/null
+++ b/be/src/kudu/rpc/service_queue.h
@@ -0,0 +1,225 @@
+// 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.
+#ifndef KUDU_UTIL_SERVICE_QUEUE_H
+#define KUDU_UTIL_SERVICE_QUEUE_H
+
+#include <memory>
+#include <string>
+#include <set>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+
+namespace boost {
+template <class T>
+class optional;
+}
+
+namespace kudu {
+namespace rpc {
+
+// Return values for ServiceQueue::Put()
+enum QueueStatus {
+  QUEUE_SUCCESS = 0,
+  QUEUE_SHUTDOWN = 1,
+  QUEUE_FULL = 2
+};
+
+// Blocking queue used for passing inbound RPC calls to the service handler pool.
+// Calls are dequeued in 'earliest-deadline first' order. The queue also maintains a
+// bounded number of calls. If the queue overflows, then calls with deadlines farthest
+// in the future are evicted.
+//
+// When calls do not provide deadlines, the RPC layer considers their deadline to
+// be infinitely in the future. This means that any call that does have a deadline
+// can evict any call that does not have a deadline. This incentivizes clients to
+// provide accurate deadlines for their calls.
+//
+// In order to improve concurrent throughput, this class uses a LIFO design:
+// Each consumer thread has its own lock and condition variable. If a
+// consumer arrives and there is no work available in the queue, it will not
+// wait on the queue lock, but rather push its own 'ConsumerState' object
+// to the 'waiting_consumers_' stack. When work arrives, if there are waiting
+// consumers, the top consumer is popped from the stack and woken up.
+//
+// This design has a few advantages over the basic BlockingQueue:
+// - the worker who was most recently busy is the one which will be selected for
+//   new work. This gives an opportunity for the worker to be scheduled again
+//   without going to sleep, and also keeps CPU cache and allocator caches hot.
+// - in the common case that there are enough workers to fully service the incoming
+//   work rate, the queue implementation itself is never used. Thus, we can
+//   have a priority queue without paying extra for it in the common case.
+//
+// NOTE: because of the use of thread-local consumer records, once a consumer
+// thread accesses one LifoServiceQueue, it becomes "bound" to that queue and
+// must never access any other instance.
+class LifoServiceQueue {
+ public:
+  explicit LifoServiceQueue(int max_size);
+
+  ~LifoServiceQueue();
+
+  // Get an element from the queue.  Returns false if we were shut down prior to
+  // getting the element.
+  bool BlockingGet(std::unique_ptr<InboundCall>* out);
+
+  // Add a new call to the queue.
+  // Returns:
+  // - QUEUE_SHUTDOWN if Shutdown() has already been called.
+  // - QUEUE_FULL if the queue is full and 'call' has a later deadline than any
+  //   RPC already in the queue.
+  // - QUEUE_SUCCESS if 'call' was enqueued.
+  //
+  // In the case of a 'QUEUE_SUCCESS' response, the new element may have bumped
+  // another call out of the queue. In that case, *evicted will be set to the
+  // call that was bumped.
+  QueueStatus Put(InboundCall* call, boost::optional<InboundCall*>* evicted);
+
+  // Shut down the queue.
+  // When a blocking queue is shut down, no more elements can be added to it,
+  // and Put() will return QUEUE_SHUTDOWN.
+  // Existing elements will drain out of it, and then BlockingGet will start
+  // returning false.
+  void Shutdown();
+
+  bool empty() const;
+
+  int max_size() const;
+
+  std::string ToString() const;
+
+  // Return an estimate of the current queue length.
+  int estimated_queue_length() const {
+    ANNOTATE_IGNORE_READS_BEGIN();
+    // The C++ standard says that std::multiset::size must be constant time,
+    // so this method won't try to traverse any actual nodes of the underlying
+    // RB tree. Investigation of the libstdcxx implementation confirms that
+    // size() is a simple field access of the _Rb_tree structure.
+    int ret = queue_.size();
+    ANNOTATE_IGNORE_READS_END();
+    return ret;
+  }
+
+  // Return an estimate of the number of idle threads currently awaiting work.
+  int estimated_idle_worker_count() const {
+    ANNOTATE_IGNORE_READS_BEGIN();
+    // Size of a vector is a simple field access so this is safe.
+    int ret = waiting_consumers_.size();
+    ANNOTATE_IGNORE_READS_END();
+    return ret;
+  }
+
+ private:
+  // Comparison function which orders calls by their deadlines.
+  static bool DeadlineLess(const InboundCall* a,
+                           const InboundCall* b) {
+    auto time_a = a->GetClientDeadline();
+    auto time_b = b->GetClientDeadline();
+    if (time_a == time_b) {
+      // If two calls have the same deadline (most likely because neither one specified
+      // one) then we should order them by arrival order.
+      time_a = a->GetTimeReceived();
+      time_b = b->GetTimeReceived();
+    }
+    return time_a < time_b;
+  }
+
+  // Struct functor wrapper for DeadlineLess.
+  struct DeadlineLessStruct {
+    bool operator()(const InboundCall* a, const InboundCall* b) const {
+      return DeadlineLess(a, b);
+    }
+  };
+
+  // The thread-local record corresponding to a single consumer thread.
+  // Threads push this record onto the waiting_consumers_ stack when
+  // they are awaiting work. Producers pop the top waiting consumer and
+  // post work using Post().
+  class ConsumerState {
+   public:
+    explicit ConsumerState(LifoServiceQueue* queue) :
+        cond_(&lock_),
+        call_(nullptr),
+        should_wake_(false),
+        bound_queue_(queue) {
+    }
+
+    void Post(InboundCall* call) {
+      DCHECK(call_ == nullptr);
+      MutexLock l(lock_);
+      call_ = call;
+      should_wake_ = true;
+      cond_.Signal();
+    }
+
+    InboundCall* Wait() {
+      MutexLock l(lock_);
+      while (should_wake_ == false) {
+        cond_.Wait();
+      }
+      should_wake_ = false;
+      InboundCall* ret = call_;
+      call_ = nullptr;
+      return ret;
+    }
+
+    void DCheckBoundInstance(LifoServiceQueue* q) {
+      DCHECK_EQ(q, bound_queue_);
+    }
+
+   private:
+    Mutex lock_;
+    ConditionVariable cond_;
+    InboundCall* call_;
+    bool should_wake_;
+
+    // For the purpose of assertions, tracks the LifoServiceQueue instance that
+    // this consumer is reading from.
+    LifoServiceQueue* bound_queue_;
+  };
+
+  static __thread ConsumerState* tl_consumer_;
+
+  mutable simple_spinlock lock_;
+  bool shutdown_;
+  int max_queue_size_;
+
+  // Stack of consumer threads which are currently waiting for work.
+  std::vector<ConsumerState*> waiting_consumers_;
+
+  // The actual queue. Work is only added to the queue when there were no
+  // consumers available for a "direct hand-off".
+  std::multiset<InboundCall*, DeadlineLessStruct> queue_;
+
+  // The total set of consumers who have ever accessed this queue.
+  std::vector<std::unique_ptr<ConsumerState>> consumers_;
+
+  DISALLOW_COPY_AND_ASSIGN(LifoServiceQueue);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/transfer.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/transfer.cc b/be/src/kudu/rpc/transfer.cc
new file mode 100644
index 0000000..bdf5191
--- /dev/null
+++ b/be/src/kudu/rpc/transfer.cc
@@ -0,0 +1,283 @@
+// 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 "kudu/rpc/transfer.h"
+
+#include <sys/uio.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <iostream>
+#include <limits>
+#include <set>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/net/socket.h"
+
+DEFINE_int64(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);
+TAG_FLAG(rpc_max_message_size, runtime);
+
+static bool ValidateMaxMessageSize(const char* flagname, int64_t value) {
+  if (value < 1 * 1024 * 1024) {
+    LOG(ERROR) << flagname << " must be at least 1MB.";
+    return false;
+  }
+  if (value > std::numeric_limits<int32_t>::max()) {
+    LOG(ERROR) << flagname << " must be less than "
+               << std::numeric_limits<int32_t>::max() << " bytes.";
+  }
+
+  return true;
+}
+static bool dummy = google::RegisterFlagValidator(
+    &FLAGS_rpc_max_message_size, &ValidateMaxMessageSize);
+
+namespace kudu {
+namespace rpc {
+
+using std::ostringstream;
+using std::set;
+using std::string;
+using strings::Substitute;
+
+#define RETURN_ON_ERROR_OR_SOCKET_NOT_READY(status)               \
+  do {                                                            \
+    Status _s = (status);                                         \
+    if (PREDICT_FALSE(!_s.ok())) {                                \
+      if (Socket::IsTemporarySocketError(_s.posix_code())) {      \
+        return Status::OK(); /* EAGAIN, etc. */                   \
+      }                                                           \
+      return _s;                                                  \
+    }                                                             \
+  } while (0)
+
+TransferCallbacks::~TransferCallbacks()
+{}
+
+InboundTransfer::InboundTransfer()
+  : total_length_(kMsgLengthPrefixLength),
+    cur_offset_(0) {
+  buf_.resize(kMsgLengthPrefixLength);
+}
+
+Status InboundTransfer::ReceiveBuffer(Socket &socket) {
+  if (cur_offset_ < kMsgLengthPrefixLength) {
+    // receive uint32 length prefix
+    int32_t rem = kMsgLengthPrefixLength - cur_offset_;
+    int32_t nread;
+    Status status = socket.Recv(&buf_[cur_offset_], rem, &nread);
+    RETURN_ON_ERROR_OR_SOCKET_NOT_READY(status);
+    if (nread == 0) {
+      return Status::OK();
+    }
+    DCHECK_GE(nread, 0);
+    cur_offset_ += nread;
+    if (cur_offset_ < kMsgLengthPrefixLength) {
+      // If we still don't have the full length prefix, we can't continue
+      // reading yet.
+      return Status::OK();
+    }
+    // Since we only read 'rem' bytes above, we should now have exactly
+    // the length prefix in our buffer and no more.
+    DCHECK_EQ(cur_offset_, kMsgLengthPrefixLength);
+
+    // The length prefix doesn't include its own 4 bytes, so we have to
+    // add that back in.
+    total_length_ = NetworkByteOrder::Load32(&buf_[0]) + kMsgLengthPrefixLength;
+    if (total_length_ > FLAGS_rpc_max_message_size) {
+      return Status::NetworkError(Substitute(
+          "RPC frame had a length of $0, but we only support messages up to $1 bytes "
+          "long.", total_length_, FLAGS_rpc_max_message_size));
+    }
+    if (total_length_ <= kMsgLengthPrefixLength) {
+      return Status::NetworkError(Substitute("RPC frame had invalid length of $0",
+                                             total_length_));
+    }
+    buf_.resize(total_length_);
+
+    // Fall through to receive the message body, which is likely to be already
+    // available on the socket.
+  }
+
+  // receive message body
+  int32_t nread;
+
+  // Socket::Recv() handles at most INT_MAX at a time, so cap the remainder at
+  // INT_MAX. The message will be split across multiple Recv() calls.
+  // Note that this is only needed when rpc_max_message_size > INT_MAX, which is
+  // currently only used for unit tests.
+  int32_t rem = std::min(total_length_ - cur_offset_,
+      static_cast<uint32_t>(std::numeric_limits<int32_t>::max()));
+  Status status = socket.Recv(&buf_[cur_offset_], rem, &nread);
+  RETURN_ON_ERROR_OR_SOCKET_NOT_READY(status);
+  cur_offset_ += nread;
+
+  return Status::OK();
+}
+
+bool InboundTransfer::TransferStarted() const {
+  return cur_offset_ != 0;
+}
+
+bool InboundTransfer::TransferFinished() const {
+  return cur_offset_ == total_length_;
+}
+
+string InboundTransfer::StatusAsString() const {
+  return Substitute("$0/$1 bytes received", cur_offset_, total_length_);
+}
+
+OutboundTransfer* OutboundTransfer::CreateForCallRequest(int32_t call_id,
+                                                         const TransferPayload &payload,
+                                                         size_t n_payload_slices,
+                                                         TransferCallbacks *callbacks) {
+  return new OutboundTransfer(call_id, payload, n_payload_slices, callbacks);
+}
+
+OutboundTransfer* OutboundTransfer::CreateForCallResponse(const TransferPayload &payload,
+                                                          size_t n_payload_slices,
+                                                          TransferCallbacks *callbacks) {
+  return new OutboundTransfer(kInvalidCallId, payload, n_payload_slices, callbacks);
+}
+
+OutboundTransfer::OutboundTransfer(int32_t call_id,
+                                   const TransferPayload &payload,
+                                   size_t n_payload_slices,
+                                   TransferCallbacks *callbacks)
+  : cur_slice_idx_(0),
+    cur_offset_in_slice_(0),
+    callbacks_(callbacks),
+    call_id_(call_id),
+    started_(false),
+    aborted_(false) {
+
+  n_payload_slices_ = n_payload_slices;
+  CHECK_LE(n_payload_slices_, payload_slices_.size());
+  for (int i = 0; i < n_payload_slices; i++) {
+    payload_slices_[i] = payload[i];
+  }
+}
+
+OutboundTransfer::~OutboundTransfer() {
+  if (!TransferFinished() && !aborted_) {
+    callbacks_->NotifyTransferAborted(
+      Status::RuntimeError("RPC transfer destroyed before it finished sending"));
+  }
+}
+
+void OutboundTransfer::Abort(const Status &status) {
+  CHECK(!aborted_) << "Already aborted";
+  CHECK(!TransferFinished()) << "Cannot abort a finished transfer";
+  callbacks_->NotifyTransferAborted(status);
+  aborted_ = true;
+}
+
+Status OutboundTransfer::SendBuffer(Socket &socket) {
+  CHECK_LT(cur_slice_idx_, n_payload_slices_);
+
+  started_ = true;
+  int n_iovecs = n_payload_slices_ - cur_slice_idx_;
+  struct iovec iovec[n_iovecs];
+  {
+    int offset_in_slice = cur_offset_in_slice_;
+    for (int i = 0; i < n_iovecs; i++) {
+      Slice &slice = payload_slices_[cur_slice_idx_ + i];
+      iovec[i].iov_base = slice.mutable_data() + offset_in_slice;
+      iovec[i].iov_len = slice.size() - offset_in_slice;
+
+      offset_in_slice = 0;
+    }
+  }
+
+  int64_t written;
+  Status status = socket.Writev(iovec, n_iovecs, &written);
+  RETURN_ON_ERROR_OR_SOCKET_NOT_READY(status);
+
+  // Adjust our accounting of current writer position.
+  for (int i = cur_slice_idx_; i < n_payload_slices_; i++) {
+    Slice &slice = payload_slices_[i];
+    int rem_in_slice = slice.size() - cur_offset_in_slice_;
+    DCHECK_GE(rem_in_slice, 0);
+
+    if (written >= rem_in_slice) {
+      // Used up this entire slice, advance to the next slice.
+      cur_slice_idx_++;
+      cur_offset_in_slice_ = 0;
+      written -= rem_in_slice;
+    } else {
+      // Partially used up this slice, just advance the offset within it.
+      cur_offset_in_slice_ += written;
+      break;
+    }
+  }
+
+  if (cur_slice_idx_ == n_payload_slices_) {
+    callbacks_->NotifyTransferFinished();
+    DCHECK_EQ(0, cur_offset_in_slice_);
+  } else {
+    DCHECK_LT(cur_slice_idx_, n_payload_slices_);
+    DCHECK_LT(cur_offset_in_slice_, payload_slices_[cur_slice_idx_].size());
+  }
+
+  return Status::OK();
+}
+
+bool OutboundTransfer::TransferStarted() const {
+  return started_;
+}
+
+bool OutboundTransfer::TransferFinished() const {
+  if (cur_slice_idx_ == n_payload_slices_) {
+    DCHECK_EQ(0, cur_offset_in_slice_); // sanity check
+    return true;
+  }
+  return false;
+}
+
+string OutboundTransfer::HexDump() const {
+  if (KUDU_SHOULD_REDACT()) {
+    return kRedactionMessage;
+  }
+
+  string ret;
+  for (int i = 0; i < n_payload_slices_; i++) {
+    ret.append(payload_slices_[i].ToDebugString());
+  }
+  return ret;
+}
+
+int32_t OutboundTransfer::TotalLength() const {
+  int32_t ret = 0;
+  for (int i = 0; i < n_payload_slices_; i++) {
+    ret += payload_slices_[i].size();
+  }
+  return ret;
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/transfer.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/transfer.h b/be/src/kudu/rpc/transfer.h
new file mode 100644
index 0000000..b95d43d
--- /dev/null
+++ b/be/src/kudu/rpc/transfer.h
@@ -0,0 +1,212 @@
+// 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.
+
+#ifndef KUDU_RPC_TRANSFER_H
+#define KUDU_RPC_TRANSFER_H
+
+#include <array>
+#include <cstddef>
+#include <cstdint>
+#include <limits.h>
+#include <string>
+
+#include <boost/intrusive/list_hook.hpp>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+DECLARE_int64(rpc_max_message_size);
+
+namespace kudu {
+
+class Socket;
+
+namespace rpc {
+
+struct TransferCallbacks;
+
+class TransferLimits {
+ public:
+  enum {
+    kMaxSidecars = 10,
+    kMaxPayloadSlices = kMaxSidecars + 2, // (header + msg)
+    kMaxTotalSidecarBytes = INT_MAX
+  };
+
+  DISALLOW_IMPLICIT_CONSTRUCTORS(TransferLimits);
+};
+
+typedef std::array<Slice, TransferLimits::kMaxPayloadSlices> TransferPayload;
+
+// This class is used internally by the RPC layer to represent an inbound
+// transfer in progress.
+//
+// Inbound Transfer objects are created by a Connection receiving data. When the
+// message is fully received, it is either parsed as a call, or a call response,
+// and the InboundTransfer object itself is handed off.
+class InboundTransfer {
+ public:
+
+  InboundTransfer();
+
+  // read from the socket into our buffer
+  Status ReceiveBuffer(Socket &socket);
+
+  // Return true if any bytes have yet been sent.
+  bool TransferStarted() const;
+
+  // Return true if the entire transfer has been sent.
+  bool TransferFinished() const;
+
+  Slice data() const {
+    return Slice(buf_);
+  }
+
+  // Return a string indicating the status of this transfer (number of bytes received, etc)
+  // suitable for logging.
+  std::string StatusAsString() const;
+
+ private:
+
+  Status ProcessInboundHeader();
+
+  faststring buf_;
+
+  uint32_t total_length_;
+  uint32_t cur_offset_;
+
+  DISALLOW_COPY_AND_ASSIGN(InboundTransfer);
+};
+
+// When the connection wants to send data, it creates an OutboundTransfer object
+// to encompass it. This sits on a queue within the Connection, so that each time
+// the Connection wakes up with a writable socket, it consumes more bytes off
+// the next pending transfer in the queue.
+//
+// Upon completion of the transfer, a callback is triggered.
+class OutboundTransfer : public boost::intrusive::list_base_hook<> {
+ public:
+  // Factory methods for creating transfers associated with call requests
+  // or responses. The 'payload' slices will be concatenated and
+  // written to the socket. When the transfer completes or errors, the
+  // appropriate method of 'callbacks' is invoked.
+  //
+  // Does not take ownership of the callbacks object or the underlying
+  // memory of the slices. The slices must remain valid until the callback
+  // is triggered.
+  //
+  // NOTE: 'payload' is currently restricted to a maximum of kMaxPayloadSlices
+  // slices.
+  // ------------------------------------------------------------
+
+  // Create an outbound transfer for a call request.
+  static OutboundTransfer* CreateForCallRequest(int32_t call_id,
+                                                const TransferPayload &payload,
+                                                size_t n_payload_slices,
+                                                TransferCallbacks *callbacks);
+
+  // Create an outbound transfer for a call response.
+  // See above for details.
+  static OutboundTransfer* CreateForCallResponse(const TransferPayload &payload,
+                                                 size_t n_payload_slices,
+                                                 TransferCallbacks *callbacks);
+
+  // Destruct the transfer. A transfer object should never be deallocated
+  // before it has either (a) finished transferring, or (b) been Abort()ed.
+  ~OutboundTransfer();
+
+  // Abort the current transfer, with the given status.
+  // This triggers TransferCallbacks::NotifyTransferAborted.
+  void Abort(const Status &status);
+
+  // send from our buffers into the sock
+  Status SendBuffer(Socket &socket);
+
+  // Return true if any bytes have yet been sent.
+  bool TransferStarted() const;
+
+  // Return true if the entire transfer has been sent.
+  bool TransferFinished() const;
+
+  // Return the total number of bytes to be sent (including those already sent)
+  int32_t TotalLength() const;
+
+  std::string HexDump() const;
+
+  bool is_for_outbound_call() const {
+    return call_id_ != kInvalidCallId;
+  }
+
+  // Returns the call ID for a transfer associated with an outbound
+  // call. Must not be called for call responses.
+  int32_t call_id() const {
+    DCHECK_NE(call_id_, kInvalidCallId);
+    return call_id_;
+  }
+
+ private:
+  OutboundTransfer(int32_t call_id,
+                   const TransferPayload& payload,
+                   size_t n_payload_slices,
+                   TransferCallbacks *callbacks);
+
+  // Slices to send. Uses an array here instead of a vector to avoid an expensive
+  // vector construction (improved performance a couple percent).
+  TransferPayload payload_slices_;
+  size_t n_payload_slices_;
+
+  // The current slice that is being sent.
+  int32_t cur_slice_idx_;
+  // The number of bytes in the above slice which has already been sent.
+  int32_t cur_offset_in_slice_;
+
+  TransferCallbacks *callbacks_;
+
+  // In the case of outbound calls, the associated call ID.
+  // In the case of call responses, kInvalidCallId
+  int32_t call_id_;
+
+  // True if SendBuffer() has been called at least once. This can be true even if
+  // no bytes were sent successfully. This is needed as SSL_write() is stateful.
+  // Please see KUDU-2334 for details.
+  bool started_;
+
+  bool aborted_;
+
+  DISALLOW_COPY_AND_ASSIGN(OutboundTransfer);
+};
+
+// Callbacks made after a transfer completes.
+struct TransferCallbacks {
+ public:
+  virtual ~TransferCallbacks();
+
+  // The transfer finished successfully.
+  virtual void NotifyTransferFinished() = 0;
+
+  // The transfer was aborted (e.g because the connection died or an error occurred).
+  virtual void NotifyTransferAborted(const Status &status) = 0;
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/user_credentials.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/user_credentials.cc b/be/src/kudu/rpc/user_credentials.cc
new file mode 100644
index 0000000..7f318fe
--- /dev/null
+++ b/be/src/kudu/rpc/user_credentials.cc
@@ -0,0 +1,64 @@
+// 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 "kudu/rpc/user_credentials.h"
+
+#include <cstddef>
+#include <string>
+#include <utility>
+
+#include <boost/functional/hash/hash.hpp>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+#include "kudu/util/user.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+bool UserCredentials::has_real_user() const {
+  return !real_user_.empty();
+}
+
+void UserCredentials::set_real_user(string real_user) {
+  real_user_ = std::move(real_user);
+}
+
+Status UserCredentials::SetLoggedInRealUser() {
+  return GetLoggedInUser(&real_user_);
+}
+
+string UserCredentials::ToString() const {
+  return strings::Substitute("{real_user=$0}", real_user_);
+}
+
+size_t UserCredentials::HashCode() const {
+  size_t seed = 0;
+  if (has_real_user()) {
+    boost::hash_combine(seed, real_user());
+  }
+  return seed;
+}
+
+bool UserCredentials::Equals(const UserCredentials& other) const {
+  return real_user() == other.real_user();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/user_credentials.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/user_credentials.h b/be/src/kudu/rpc/user_credentials.h
new file mode 100644
index 0000000..5a0434c
--- /dev/null
+++ b/be/src/kudu/rpc/user_credentials.h
@@ -0,0 +1,53 @@
+// 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.
+#pragma once
+
+#include <cstddef>
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace rpc {
+
+// Client-side user credentials. Currently this is more-or-less a simple wrapper
+// around a username string. However, we anticipate moving more credentials such as
+// tokens into a per-Proxy structure rather than Messenger-wide, and this will
+// be the place to store them.
+class UserCredentials {
+ public:
+  // Real user.
+  bool has_real_user() const;
+  void set_real_user(std::string real_user);
+  const std::string& real_user() const { return real_user_; }
+
+  // Sets the real user to the currently logged in user.
+  Status SetLoggedInRealUser();
+
+  // Returns a string representation of the object.
+  std::string ToString() const;
+
+  std::size_t HashCode() const;
+  bool Equals(const UserCredentials& other) const;
+
+ private:
+  // Remember to update HashCode() and Equals() when new fields are added.
+  std::string real_user_;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/CMakeLists.txt b/be/src/kudu/security/CMakeLists.txt
new file mode 100644
index 0000000..b79486e
--- /dev/null
+++ b/be/src/kudu/security/CMakeLists.txt
@@ -0,0 +1,141 @@
+# 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.
+
+# See the comment in krb5_realm_override.cc for details on this library's usage.
+# The top-level CMakeLists sets a ${KRB5_REALM_OVERRIDE} variable which should
+# be linked first into all Kudu binaries.
+
+##############################
+# krb5_realm_override
+##############################
+
+add_library(krb5_realm_override STATIC krb5_realm_override.cc)
+target_link_libraries(krb5_realm_override glog)
+if(NOT APPLE)
+  target_link_libraries(krb5_realm_override dl)
+endif()
+
+##############################
+# token_proto
+##############################
+
+PROTOBUF_GENERATE_CPP(
+  TOKEN_PROTO_SRCS TOKEN_PROTO_HDRS TOKEN_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES token.proto)
+set(TOKEN_PROTO_LIBS protobuf pb_util_proto)
+ADD_EXPORTABLE_LIBRARY(token_proto
+  SRCS ${TOKEN_PROTO_SRCS}
+  DEPS ${TOKEN_PROTO_LIBS}
+  NONLINK_DEPS ${TOKEN_PROTO_TGTS})
+
+
+##############################
+# security
+##############################
+
+# Check for krb5_get_init_creds_opt_set_out_ccache, which is not available in versions
+# of MIT Kerberos older than krb5-1.6, and is also not present in Heimdal kerberos.
+include(CheckLibraryExists)
+check_library_exists("krb5" krb5_get_init_creds_opt_set_out_ccache
+  ${KERBEROS_LIBRARY} HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
+if(HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
+  add_definitions(-DHAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE=1)
+endif()
+
+# Fall back to using the ported functionality if we're using an older version of OpenSSL.
+if (${OPENSSL_VERSION} VERSION_LESS "1.0.2")
+  set(PORTED_X509_CHECK_HOST_CC "x509_check_host.cc")
+endif()
+
+set(SECURITY_SRCS
+  ca/cert_management.cc
+  cert.cc
+  crypto.cc
+  kerberos_util.cc
+  init.cc
+  openssl_util.cc
+  ${PORTED_X509_CHECK_HOST_CC}
+  security_flags.cc
+  simple_acl.cc
+  tls_context.cc
+  tls_handshake.cc
+  tls_socket.cc
+  token_verifier.cc
+  token_signer.cc
+  token_signing_key.cc
+  )
+
+set(SECURITY_LIBS
+  gutil
+  kudu_util
+  token_proto
+
+  krb5
+  openssl_crypto
+  openssl_ssl)
+
+ADD_EXPORTABLE_LIBRARY(security
+  SRCS ${SECURITY_SRCS}
+  DEPS ${SECURITY_LIBS})
+
+
+##############################
+# mini_kdc
+##############################
+
+set(MINI_KDC_SRCS test/mini_kdc.cc)
+
+add_library(mini_kdc ${MINI_KDC_SRCS})
+target_link_libraries(mini_kdc
+  gutil
+  kudu_test_util
+  kudu_util)
+
+##############################
+# security_test_util
+##############################
+
+if (NOT NO_TESTS)
+  set(SECURITY_TEST_SRCS
+    security-test-util.cc
+    test/test_certs.cc
+    test/test_pass.cc)
+
+  add_library(security_test_util ${SECURITY_TEST_SRCS})
+  target_link_libraries(security_test_util
+    gutil
+    kudu_test_util
+    kudu_util
+    security)
+
+  # Tests
+  set(KUDU_TEST_LINK_LIBS
+    mini_kdc
+    security
+    security_test_util
+    ${KUDU_MIN_TEST_LIBS})
+
+  ADD_KUDU_TEST(ca/cert_management-test)
+  ADD_KUDU_TEST(cert-test)
+  ADD_KUDU_TEST(crypto-test)
+  ADD_KUDU_TEST(test/mini_kdc-test)
+  ADD_KUDU_TEST(tls_handshake-test)
+  ADD_KUDU_TEST(tls_socket-test PROCESSORS 2)
+  ADD_KUDU_TEST(token-test)
+endif()

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/ca/cert_management-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/ca/cert_management-test.cc b/be/src/kudu/security/ca/cert_management-test.cc
new file mode 100644
index 0000000..0c8abc8
--- /dev/null
+++ b/be/src/kudu/security/ca/cert_management-test.cc
@@ -0,0 +1,294 @@
+// 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 "kudu/security/ca/cert_management.h"
+
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/security-test-util.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+namespace ca {
+
+class CertManagementTest : public KuduTest {
+ public:
+  void SetUp() override {
+    ASSERT_OK(ca_cert_.FromString(kCaCert, DataFormat::PEM));
+    ASSERT_OK(ca_private_key_.FromString(kCaPrivateKey, DataFormat::PEM));
+    ASSERT_OK(ca_public_key_.FromString(kCaPublicKey, DataFormat::PEM));
+    ASSERT_OK(ca_exp_cert_.FromString(kCaExpiredCert, DataFormat::PEM));
+    ASSERT_OK(ca_exp_private_key_.FromString(kCaExpiredPrivateKey, DataFormat::PEM));
+    // Sanity checks.
+    ASSERT_OK(ca_cert_.CheckKeyMatch(ca_private_key_));
+    ASSERT_OK(ca_exp_cert_.CheckKeyMatch(ca_exp_private_key_));
+  }
+
+ protected:
+  CertRequestGenerator::Config PrepareConfig(
+      const string& hostname = "localhost.localdomain") {
+    return { hostname };
+  }
+
+  CaCertRequestGenerator::Config PrepareCaConfig(const string& cn) {
+    return { cn };
+  }
+
+  // Create a new private key in 'key' and return a CSR associated with that
+  // key.
+  template<class CSRGen = CertRequestGenerator>
+  CertSignRequest PrepareTestCSR(typename CSRGen::Config config,
+                                 PrivateKey* key) {
+    CHECK_OK(GeneratePrivateKey(512, key));
+    CSRGen gen(std::move(config));
+    CHECK_OK(gen.Init());
+    CertSignRequest req;
+    CHECK_OK(gen.GenerateRequest(*key, &req));
+    return req;
+  }
+
+  Cert ca_cert_;
+  PrivateKey ca_private_key_;
+  PublicKey ca_public_key_;
+
+  Cert ca_exp_cert_;
+  PrivateKey ca_exp_private_key_;
+};
+
+// Check for basic constraints while initializing CertRequestGenerator objects.
+TEST_F(CertManagementTest, RequestGeneratorConstraints) {
+  const CertRequestGenerator::Config gen_config = PrepareConfig("");
+  CertRequestGenerator gen(gen_config);
+  const Status s = gen.Init();
+  const string err_msg = s.ToString();
+  ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
+  ASSERT_STR_CONTAINS(err_msg, "hostname must not be empty");
+}
+
+// Check for the basic functionality of the CertRequestGenerator class:
+// check it's able to generate keys of expected number of bits and that it
+// reports an error if trying to generate a key of unsupported number of bits.
+TEST_F(CertManagementTest, RequestGeneratorBasics) {
+  const CertRequestGenerator::Config gen_config = PrepareConfig();
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  CertRequestGenerator gen(gen_config);
+  ASSERT_OK(gen.Init());
+  string key_str;
+  ASSERT_OK(key.ToString(&key_str, DataFormat::PEM));
+  // Check for non-supported number of bits for the key.
+  Status s = GeneratePrivateKey(7, &key);
+  ASSERT_TRUE(s.IsRuntimeError());
+}
+
+// Check that CertSigner behaves in a predictable way if given non-matching
+// CA private key and certificate.
+TEST_F(CertManagementTest, SignerInitWithMismatchedCertAndKey) {
+  PrivateKey key;
+  const auto& csr = PrepareTestCSR(PrepareConfig(), &key);
+  {
+    Cert cert;
+    Status s = CertSigner(&ca_cert_, &ca_exp_private_key_)
+        .Sign(csr, &cert);
+
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsRuntimeError()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "certificate does not match private key");
+  }
+  {
+    Cert cert;
+    Status s = CertSigner(&ca_exp_cert_, &ca_private_key_)
+        .Sign(csr, &cert);
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsRuntimeError()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "certificate does not match private key");
+  }
+}
+
+// Check how CertSigner behaves if given expired CA certificate
+// and corresponding private key.
+TEST_F(CertManagementTest, SignerInitWithExpiredCert) {
+  const CertRequestGenerator::Config gen_config = PrepareConfig();
+  PrivateKey key;
+  CertSignRequest req = PrepareTestCSR(gen_config, &key);
+
+  // Signer works fine even with expired CA certificate.
+  Cert cert;
+  ASSERT_OK(CertSigner(&ca_exp_cert_, &ca_exp_private_key_).Sign(req, &cert));
+  ASSERT_OK(cert.CheckKeyMatch(key));
+}
+
+// Generate X509 CSR and issue corresponding certificate putting the specified
+// hostname into the SAN X509v3 extension field. The fix for KUDU-1981 addresses
+// the issue of enabling Kudu server components on systems with FQDN longer than
+// 64 characters. This test is a regression for KUDU-1981, so let's verify that
+// CSRs and the result X509 cerificates with long hostnames in SAN are handled
+// properly.
+TEST_F(CertManagementTest, SignCertLongHostnameInSan) {
+  for (auto const& hostname :
+      {
+        "foo.bar.com",
+
+        "222222222222222222222222222222222222222222222222222222222222222."
+        "555555555555555555555555555555555555555555555555555555555555555."
+        "555555555555555555555555555555555555555555555555555555555555555."
+        "chaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaars",
+      }) {
+    CertRequestGenerator::Config gen_config;
+    gen_config.hostname = hostname;
+    gen_config.user_id = "test-uid";
+    PrivateKey key;
+    const auto& csr = PrepareTestCSR(gen_config, &key);
+    Cert cert;
+    ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_).Sign(csr, &cert));
+    ASSERT_OK(cert.CheckKeyMatch(key));
+
+    EXPECT_EQ("C = US, ST = CA, O = MyCompany, CN = MyName, emailAddress = my@email.com",
+              cert.IssuerName());
+    EXPECT_EQ("UID = test-uid", cert.SubjectName());
+    vector<string> hostnames = cert.Hostnames();
+    ASSERT_EQ(1, hostnames.size());
+    EXPECT_EQ(hostname, hostnames[0]);
+  }
+}
+
+// Generate X509 CSR and issues corresponding certificate.
+TEST_F(CertManagementTest, SignCert) {
+  CertRequestGenerator::Config gen_config;
+  gen_config.hostname = "foo.bar.com";
+  gen_config.user_id = "test-uid";
+  gen_config.kerberos_principal = "kudu/foo.bar.com@bar.com";
+  PrivateKey key;
+  const auto& csr = PrepareTestCSR(gen_config, &key);
+  Cert cert;
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_).Sign(csr, &cert));
+  ASSERT_OK(cert.CheckKeyMatch(key));
+
+  EXPECT_EQ("C = US, ST = CA, O = MyCompany, CN = MyName, emailAddress = my@email.com",
+            cert.IssuerName());
+  EXPECT_EQ("UID = test-uid", cert.SubjectName());
+  EXPECT_EQ(gen_config.user_id, *cert.UserId());
+  EXPECT_EQ(gen_config.kerberos_principal, *cert.KuduKerberosPrincipal());
+  vector<string> hostnames = cert.Hostnames();
+  ASSERT_EQ(1, hostnames.size());
+  EXPECT_EQ("foo.bar.com", hostnames[0]);
+}
+
+// Generate X509 CA CSR and sign the result certificate.
+TEST_F(CertManagementTest, SignCaCert) {
+  const CaCertRequestGenerator::Config gen_config(PrepareCaConfig("self-ca"));
+  PrivateKey key;
+  const auto& csr = PrepareTestCSR<CaCertRequestGenerator>(gen_config, &key);
+  Cert cert;
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_).Sign(csr, &cert));
+  ASSERT_OK(cert.CheckKeyMatch(key));
+}
+
+// Test the creation and use of a CA which uses a self-signed CA cert
+// generated on the fly.
+TEST_F(CertManagementTest, TestSelfSignedCA) {
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  // Create a key and CSR for the tablet server.
+  const auto& config = PrepareConfig();
+  PrivateKey ts_key;
+  CertSignRequest ts_csr = PrepareTestCSR(config, &ts_key);
+
+  // Sign it using the self-signed CA.
+  Cert ts_cert;
+  ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(ts_csr, &ts_cert));
+  ASSERT_OK(ts_cert.CheckKeyMatch(ts_key));
+}
+
+// Check the transformation chains for X509 CSRs:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, X509CsrFromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  CertRequestGenerator gen(PrepareConfig());
+  ASSERT_OK(gen.Init());
+  CertSignRequest req_ref;
+  ASSERT_OK(gen.GenerateRequest(key, &req_ref));
+
+  for (auto format : kFormats) {
+    SCOPED_TRACE(Substitute("X509 CSR format: $0", DataFormatToString(format)));
+    string str_req_ref;
+    ASSERT_OK(req_ref.ToString(&str_req_ref, format));
+    CertSignRequest req;
+    ASSERT_OK(req.FromString(str_req_ref, format));
+    string str_req;
+    ASSERT_OK(req.ToString(&str_req, format));
+    ASSERT_EQ(str_req_ref, str_req);
+  }
+}
+
+// Check the transformation chains for X509 certs:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, X509FromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  CertRequestGenerator gen(PrepareConfig());
+  ASSERT_OK(gen.Init());
+  CertSignRequest req;
+  ASSERT_OK(gen.GenerateRequest(key, &req));
+
+  Cert cert_ref;
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_)
+            .Sign(req, &cert_ref));
+
+  for (auto format : kFormats) {
+    SCOPED_TRACE(Substitute("X509 format: $0", DataFormatToString(format)));
+    string str_cert_ref;
+    ASSERT_OK(cert_ref.ToString(&str_cert_ref, format));
+    Cert cert;
+    ASSERT_OK(cert.FromString(str_cert_ref, format));
+    string str_cert;
+    ASSERT_OK(cert.ToString(&str_cert, format));
+    ASSERT_EQ(str_cert_ref, str_cert);
+  }
+}
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/ca/cert_management.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/ca/cert_management.cc b/be/src/kudu/security/ca/cert_management.cc
new file mode 100644
index 0000000..7ccc376
--- /dev/null
+++ b/be/src/kudu/security/ca/cert_management.cc
@@ -0,0 +1,423 @@
+// 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 "kudu/security/ca/cert_management.h"
+
+#include <algorithm>
+#include <cstdio>
+#include <memory>
+#include <mutex>
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/conf.h>
+#ifndef OPENSSL_NO_ENGINE
+#include <openssl/engine.h>
+#endif
+#include <openssl/pem.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+
+using std::lock_guard;
+using std::move;
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<ASN1_INTEGER> {
+  static constexpr auto kFreeFunc = &ASN1_INTEGER_free;
+};
+template<> struct SslTypeTraits<BIGNUM> {
+  static constexpr auto kFreeFunc = &BN_free;
+};
+
+namespace ca {
+
+namespace {
+
+Status SetSubjectNameField(X509_NAME* name,
+                           const char* field_code,
+                           const string& field_value) {
+  CHECK(name);
+  CHECK(field_code);
+  OPENSSL_RET_NOT_OK(X509_NAME_add_entry_by_txt(
+      name, field_code, MBSTRING_ASC,
+      reinterpret_cast<const unsigned char*>(field_value.c_str()), -1, -1, 0),
+      Substitute("error setting subject field $0", field_code));
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+CertRequestGenerator::~CertRequestGenerator() {
+  sk_X509_EXTENSION_pop_free(extensions_, X509_EXTENSION_free);
+}
+
+Status CertRequestGeneratorBase::GenerateRequest(const PrivateKey& key,
+                                                 CertSignRequest* ret) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ret);
+  CHECK(Initialized());
+  auto req = ssl_make_unique(X509_REQ_new());
+  OPENSSL_RET_NOT_OK(X509_REQ_set_pubkey(req.get(), key.GetRawData()),
+      "error setting X509 public key");
+
+  // Populate the subject field of the request.
+  RETURN_NOT_OK(SetSubject(req.get()));
+
+  // Set necessary extensions into the request.
+  RETURN_NOT_OK(SetExtensions(req.get()));
+
+  // And finally sign the result.
+  OPENSSL_RET_NOT_OK(X509_REQ_sign(req.get(), key.GetRawData(), EVP_sha256()),
+      "error signing X509 request");
+  ret->AdoptRawData(req.release());
+
+  return Status::OK();
+}
+
+Status CertRequestGeneratorBase::PushExtension(stack_st_X509_EXTENSION* st,
+                                               int32_t nid, StringPiece value) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  auto ex = ssl_make_unique(
+      X509V3_EXT_conf_nid(nullptr, nullptr, nid, const_cast<char*>(value.data())));
+  OPENSSL_RET_IF_NULL(ex, "error configuring extension");
+  OPENSSL_RET_NOT_OK(sk_X509_EXTENSION_push(st, ex.release()),
+      "error pushing extension into the stack");
+  return Status::OK();
+}
+
+CertRequestGenerator::CertRequestGenerator(Config config)
+    : CertRequestGeneratorBase(),
+      config_(std::move(config)) {
+}
+
+Status CertRequestGenerator::Init() {
+  InitializeOpenSSL();
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+
+  CHECK(!is_initialized_);
+
+  // Build the SAN field using the specified hostname. In general, it might be
+  // multiple DNS hostnames in the field, but in our use-cases it's always one.
+  if (config_.hostname.empty()) {
+    return Status::InvalidArgument("hostname must not be empty");
+  }
+  const string san_hosts = Substitute("DNS.0:$0", config_.hostname);
+
+  extensions_ = sk_X509_EXTENSION_new_null();
+
+  // Permitted usages for the generated keys is set via X509 V3
+  // standard/extended key usage attributes.
+  // See https://www.openssl.org/docs/man1.0.1/apps/x509v3_config.html
+  // for details.
+
+  // The generated certificates are for using as TLS certificates for
+  // both client and server.
+  string usage = "critical,digitalSignature,keyEncipherment";
+  if (for_self_signing_) {
+    // If we are generating a CSR for self-signing, then we need to
+    // add this keyUsage attribute. See https://s.apache.org/BFHk
+    usage += ",keyCertSign";
+  }
+
+  RETURN_NOT_OK(PushExtension(extensions_, NID_key_usage, usage));
+  // The generated certificates should be good for authentication
+  // of a server to a client and vice versa: the intended users of the
+  // certificates are tablet servers which are going to talk to master
+  // and other tablet servers via TLS channels.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_ext_key_usage,
+                              "critical,serverAuth,clientAuth"));
+
+  // The generated certificates are not intended to be used as CA certificates
+  // (i.e. they cannot be used to sign/issue certificates).
+  RETURN_NOT_OK(PushExtension(extensions_, NID_basic_constraints,
+                              "critical,CA:FALSE"));
+
+  if (config_.kerberos_principal) {
+    int nid = GetKuduKerberosPrincipalOidNid();
+    RETURN_NOT_OK(PushExtension(extensions_, nid,
+                                Substitute("ASN1:UTF8:$0", *config_.kerberos_principal)));
+  }
+  RETURN_NOT_OK(PushExtension(extensions_, NID_subject_alt_name, san_hosts));
+
+  is_initialized_ = true;
+
+  return Status::OK();
+}
+
+bool CertRequestGenerator::Initialized() const {
+  return is_initialized_;
+}
+
+Status CertRequestGenerator::SetSubject(X509_REQ* req) const {
+  if (config_.user_id) {
+    RETURN_NOT_OK(SetSubjectNameField(X509_REQ_get_subject_name(req),
+                                      "UID", *config_.user_id));
+  }
+  return Status::OK();
+}
+
+Status CertRequestGenerator::SetExtensions(X509_REQ* req) const {
+  OPENSSL_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+      "error setting X509 request extensions");
+  return Status::OK();
+}
+
+CaCertRequestGenerator::CaCertRequestGenerator(Config config)
+    : config_(std::move(config)),
+      extensions_(nullptr),
+      is_initialized_(false) {
+}
+
+CaCertRequestGenerator::~CaCertRequestGenerator() {
+  sk_X509_EXTENSION_pop_free(extensions_, X509_EXTENSION_free);
+}
+
+Status CaCertRequestGenerator::Init() {
+  InitializeOpenSSL();
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+
+  lock_guard<simple_spinlock> guard(lock_);
+  if (is_initialized_) {
+    return Status::OK();
+  }
+  if (config_.cn.empty()) {
+    return Status::InvalidArgument("missing CA service UUID/name");
+  }
+
+  extensions_ = sk_X509_EXTENSION_new_null();
+
+  // Permitted usages for the generated keys is set via X509 V3
+  // standard/extended key usage attributes.
+  // See https://www.openssl.org/docs/man1.0.1/apps/x509v3_config.html
+  // for details.
+
+  // The target ceritifcate is a CA certificate: it's for signing X509 certs.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_key_usage,
+                              "critical,keyCertSign"));
+  // The generated certificates are for the private CA service.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_basic_constraints,
+                              "critical,CA:TRUE"));
+  is_initialized_ = true;
+
+  return Status::OK();
+}
+
+bool CaCertRequestGenerator::Initialized() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  return is_initialized_;
+}
+
+Status CaCertRequestGenerator::SetSubject(X509_REQ* req) const {
+  return SetSubjectNameField(X509_REQ_get_subject_name(req), "CN", config_.cn);
+}
+
+Status CaCertRequestGenerator::SetExtensions(X509_REQ* req) const {
+  OPENSSL_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+      "error setting X509 request extensions");
+  return Status::OK();
+}
+
+Status CertSigner::SelfSignCA(const PrivateKey& key,
+                              CaCertRequestGenerator::Config config,
+                              int64_t cert_expiration_seconds,
+                              Cert* cert) {
+  // Generate a CSR for the CA.
+  CertSignRequest ca_csr;
+  {
+    CaCertRequestGenerator gen(std::move(config));
+    RETURN_NOT_OK(gen.Init());
+    RETURN_NOT_OK(gen.GenerateRequest(key, &ca_csr));
+  }
+
+  // Self-sign the CA's CSR.
+  return CertSigner(nullptr, &key)
+      .set_expiration_interval(MonoDelta::FromSeconds(cert_expiration_seconds))
+      .Sign(ca_csr, cert);
+}
+
+Status CertSigner::SelfSignCert(const PrivateKey& key,
+                                CertRequestGenerator::Config config,
+                                Cert* cert) {
+  // Generate a CSR.
+  CertSignRequest csr;
+  {
+    CertRequestGenerator gen(std::move(config));
+    gen.enable_self_signing();
+    RETURN_NOT_OK(gen.Init());
+    RETURN_NOT_OK(gen.GenerateRequest(key, &csr));
+  }
+
+  // Self-sign the CSR with the key.
+  return CertSigner(nullptr, &key).Sign(csr, cert);
+}
+
+
+CertSigner::CertSigner(const Cert* ca_cert,
+                       const PrivateKey* ca_private_key)
+    : ca_cert_(ca_cert),
+      ca_private_key_(ca_private_key) {
+  // Private key is required.
+  CHECK(ca_private_key_ && ca_private_key_->GetRawData());
+  // The cert is optional, but if we have it, it should be initialized.
+  CHECK(!ca_cert_ || ca_cert_->GetRawData());
+}
+
+Status CertSigner::Sign(const CertSignRequest& req, Cert* ret) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  InitializeOpenSSL();
+  CHECK(ret);
+
+  // If we are not self-signing, then make sure that the provided CA
+  // cert and key match each other. Technically this would be programmer
+  // error since we're always using internally-generated CA certs, but
+  // this isn't a hot path so we'll keep the extra safety.
+  if (ca_cert_) {
+    RETURN_NOT_OK(ca_cert_->CheckKeyMatch(*ca_private_key_));
+  }
+  auto x509 = ssl_make_unique(X509_new());
+  RETURN_NOT_OK(FillCertTemplateFromRequest(req.GetRawData(), x509.get()));
+  RETURN_NOT_OK(DoSign(EVP_sha256(), exp_interval_sec_, x509.get()));
+  ret->AdoptX509(x509.release());
+
+  return Status::OK();
+}
+
+// This is modeled after code in copy_extensions() function from
+// $OPENSSL_ROOT/apps/apps.c with OpenSSL 1.0.2.
+Status CertSigner::CopyExtensions(X509_REQ* req, X509* x) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(req);
+  CHECK(x);
+  STACK_OF(X509_EXTENSION)* exts = X509_REQ_get_extensions(req);
+  SCOPED_CLEANUP({
+    sk_X509_EXTENSION_pop_free(exts, X509_EXTENSION_free);
+  });
+  for (size_t i = 0; i < sk_X509_EXTENSION_num(exts); ++i) {
+    X509_EXTENSION* ext = sk_X509_EXTENSION_value(exts, i);
+    ASN1_OBJECT* obj = X509_EXTENSION_get_object(ext);
+    int32_t idx = X509_get_ext_by_OBJ(x, obj, -1);
+    if (idx != -1) {
+      // If extension exits, delete all extensions of same type.
+      do {
+        auto tmpext = ssl_make_unique(X509_get_ext(x, idx));
+        X509_delete_ext(x, idx);
+        idx = X509_get_ext_by_OBJ(x, obj, -1);
+      } while (idx != -1);
+    }
+    OPENSSL_RET_NOT_OK(X509_add_ext(x, ext, -1), "error adding extension");
+  }
+
+  return Status::OK();
+}
+
+Status CertSigner::FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(req);
+
+  // As of OpenSSL 1.1, req's internals are hidden.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  if (!req->req_info ||
+      !req->req_info->pubkey ||
+      !req->req_info->pubkey->public_key ||
+      !req->req_info->pubkey->public_key->data) {
+    return Status::RuntimeError("corrupted CSR: no public key");
+  }
+#endif
+  auto pub_key = ssl_make_unique(X509_REQ_get_pubkey(req));
+  OPENSSL_RET_IF_NULL(pub_key, "error unpacking public key from CSR");
+  const int rc = X509_REQ_verify(req, pub_key.get());
+  if (rc < 0) {
+    return Status::RuntimeError("CSR signature verification error",
+                                GetOpenSSLErrors());
+  }
+  if (rc == 0) {
+    return Status::RuntimeError("CSR signature mismatch",
+                                GetOpenSSLErrors());
+  }
+  OPENSSL_RET_NOT_OK(X509_set_subject_name(tmpl, X509_REQ_get_subject_name(req)),
+      "error setting cert subject name");
+  RETURN_NOT_OK(CopyExtensions(req, tmpl));
+  OPENSSL_RET_NOT_OK(X509_set_pubkey(tmpl, pub_key.get()),
+      "error setting cert public key");
+  return Status::OK();
+}
+
+Status CertSigner::DigestSign(const EVP_MD* md, EVP_PKEY* pkey, X509* x) {
+  OPENSSL_RET_NOT_OK(X509_sign(x, pkey, md), "error signing certificate");
+  return Status::OK();
+}
+
+Status CertSigner::GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  auto btmp = ssl_make_unique(BN_new());
+  OPENSSL_RET_NOT_OK(BN_pseudo_rand(btmp.get(), 64, 0, 0),
+      "error generating random number");
+  auto serial = ssl_make_unique(ASN1_INTEGER_new());
+  OPENSSL_RET_IF_NULL(BN_to_ASN1_INTEGER(btmp.get(), serial.get()),
+      "error converting number into ASN1 representation");
+  if (ret) {
+    ret->swap(serial);
+  }
+  return Status::OK();
+}
+
+Status CertSigner::DoSign(const EVP_MD* digest, int32_t exp_seconds,
+                          X509* ret) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(ret);
+
+  // Version 3 (v3) of X509 certificates. The integer value is one less
+  // than the version it represents. This is not a typo. :)
+  static const int kX509V3 = 2;
+
+  // If we have a CA cert, then the CA is the issuer.
+  // Otherwise, we are self-signing so the target cert is also the issuer.
+  X509* issuer_cert = ca_cert_ ? ca_cert_->GetTopOfChainX509() : ret;
+  X509_NAME* issuer_name = X509_get_subject_name(issuer_cert);
+  OPENSSL_RET_NOT_OK(X509_set_issuer_name(ret, issuer_name),
+      "error setting issuer name");
+  c_unique_ptr<ASN1_INTEGER> serial;
+  RETURN_NOT_OK(GenerateSerial(&serial));
+  // set version to v3
+  OPENSSL_RET_NOT_OK(X509_set_version(ret, kX509V3),
+      "error setting cert version");
+  OPENSSL_RET_NOT_OK(X509_set_serialNumber(ret, serial.get()),
+      "error setting cert serial");
+  OPENSSL_RET_IF_NULL(X509_gmtime_adj(X509_get_notBefore(ret), 0L),
+      "error setting cert validity time");
+  OPENSSL_RET_IF_NULL(X509_gmtime_adj(X509_get_notAfter(ret), exp_seconds),
+      "error setting cert expiration time");
+  RETURN_NOT_OK(DigestSign(digest, ca_private_key_->GetRawData(), ret));
+
+  return Status::OK();
+}
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/ca/cert_management.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/ca/cert_management.h b/be/src/kudu/security/ca/cert_management.h
new file mode 100644
index 0000000..fb2bd0e
--- /dev/null
+++ b/be/src/kudu/security/ca/cert_management.h
@@ -0,0 +1,226 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+// Forward declarations for the relevant OpenSSL typedefs
+// in addition to openssl_util.h.
+typedef struct asn1_string_st ASN1_INTEGER;
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+typedef struct env_md_st EVP_MD;
+#else
+typedef struct evp_md_st EVP_MD;
+#endif
+typedef struct rsa_st RSA;
+typedef struct x509_st X509;
+typedef struct X509_req_st X509_REQ;
+
+// STACK_OF(X509_EXTENSION)
+struct stack_st_X509_EXTENSION; // IWYU pragma: keep
+
+namespace kudu {
+namespace security {
+
+class Cert;
+class CertSignRequest;
+class PrivateKey;
+
+namespace ca {
+
+// Base utility class for issuing X509 CSRs.
+class CertRequestGeneratorBase {
+ public:
+  CertRequestGeneratorBase() = default;
+  virtual ~CertRequestGeneratorBase() = default;
+
+  virtual Status Init() = 0;
+  virtual bool Initialized() const = 0;
+
+  // Generate X509 CSR using the specified key. To obtain the key,
+  // call the GeneratePrivateKey() function.
+  Status GenerateRequest(const PrivateKey& key, CertSignRequest* ret) const WARN_UNUSED_RESULT;
+
+ protected:
+  // Push the specified extension into the stack provided.
+  static Status PushExtension(stack_st_X509_EXTENSION* st,
+                              int32_t nid,
+                              StringPiece value) WARN_UNUSED_RESULT;
+
+  // Set the certificate-specific subject fields into the specified request.
+  virtual Status SetSubject(X509_REQ* req) const = 0;
+
+  // Set the certificate-specific extensions into the specified request.
+  virtual Status SetExtensions(X509_REQ* req) const = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CertRequestGeneratorBase);
+};
+
+// An utility class that facilitates issuing certificate signing requests
+// (a.k.a. X509 CSRs).
+class CertRequestGenerator : public CertRequestGeneratorBase {
+ public:
+  // Properties for the generated X509 CSR. The 'hostname' is for the name of
+  // the machine the requestor is to use the certificate at. Valid configuration
+  // should contain non-empty 'hostname' field.
+  struct Config {
+    // FQDN name to put into the 'DNS' fields of the subjectAltName extension.
+    std::string hostname;
+    // userId (UID)
+    boost::optional<std::string> user_id;
+    // Our custom extension which stores the full Kerberos principal for IPKI certs.
+    boost::optional<std::string> kerberos_principal;
+  };
+
+  // 'config' contains the properties to fill into the X509 attributes of the
+  // CSR.
+  explicit CertRequestGenerator(Config config);
+  ~CertRequestGenerator();
+
+  Status Init() override WARN_UNUSED_RESULT;
+  bool Initialized() const override;
+
+  CertRequestGenerator& enable_self_signing() {
+    CHECK(!is_initialized_);
+    for_self_signing_ = true;
+    return *this;
+  }
+
+ protected:
+  Status SetSubject(X509_REQ* req) const override WARN_UNUSED_RESULT;
+  Status SetExtensions(X509_REQ* req) const override WARN_UNUSED_RESULT;
+
+ private:
+  const Config config_;
+  stack_st_X509_EXTENSION* extensions_ = nullptr;
+  bool is_initialized_ = false;
+  bool for_self_signing_ = false;
+};
+
+// An utility class that facilitates issuing of root CA self-signed certificate
+// signing requests.
+class CaCertRequestGenerator : public CertRequestGeneratorBase {
+ public:
+  // Properties for the generated X509 CA CSR.
+  struct Config {
+    // Common name (CN); e.g. 'master 239D6D2F-BDD2-4463-8933-78D9559C2124'.
+    // Don't put hostname/FQDN in here: for CA cert it does not make sense and
+    // it might be longer than 64 characters which is the limit specified
+    // by RFC5280. The limit is enforced by the OpenSSL library.
+    std::string cn;
+  };
+
+  explicit CaCertRequestGenerator(Config config);
+  ~CaCertRequestGenerator();
+
+  Status Init() override WARN_UNUSED_RESULT;
+  bool Initialized() const override;
+
+ protected:
+  Status SetSubject(X509_REQ* req) const override WARN_UNUSED_RESULT;
+  Status SetExtensions(X509_REQ* req) const override WARN_UNUSED_RESULT;
+
+ private:
+  const Config config_;
+  stack_st_X509_EXTENSION* extensions_;
+  mutable simple_spinlock lock_;
+  bool is_initialized_; // protected by lock_
+};
+
+// An utility class for issuing and signing certificates.
+//
+// This is used in "fluent" style. For example:
+//
+//    CHECK_OK(CertSigner(&my_ca_cert, &my_ca_key)
+//      .set_expiration_interval(MonoDelta::FromSeconds(3600))
+//      .Sign(csr, &cert));
+//
+// As such, this class is not guaranteed thread-safe.
+class CertSigner {
+ public:
+  // Generate a self-signed certificate authority using the given key
+  // and CSR configuration.
+  static Status SelfSignCA(const PrivateKey& key,
+                           CaCertRequestGenerator::Config config,
+                           int64_t cert_expiration_seconds,
+                           Cert* cert) WARN_UNUSED_RESULT;
+
+  // Generate a self-signed certificate using the given key and CSR
+  // configuration.
+  static Status SelfSignCert(const PrivateKey& key,
+                             CertRequestGenerator::Config config,
+                             Cert* cert) WARN_UNUSED_RESULT;
+
+  // Create a CertSigner.
+  //
+  // The given cert and key must stay valid for the lifetime of the
+  // cert signer. See class documentation above for recommended usage.
+  //
+  // 'ca_cert' may be nullptr in order to perform self-signing (though
+  // the SelfSignCA() static method above is recommended).
+  CertSigner(const Cert* ca_cert, const PrivateKey* ca_private_key);
+  ~CertSigner() = default;
+
+  // Set the expiration interval for certs signed by this signer.
+  // This may be changed at any point.
+  CertSigner& set_expiration_interval(MonoDelta expiration) {
+    exp_interval_sec_ = expiration.ToSeconds();
+    return *this;
+  }
+
+  Status Sign(const CertSignRequest& req, Cert* ret) const WARN_UNUSED_RESULT;
+
+ private:
+
+  static Status CopyExtensions(X509_REQ* req, X509* x) WARN_UNUSED_RESULT;
+  static Status FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl) WARN_UNUSED_RESULT;
+  static Status DigestSign(const EVP_MD* md, EVP_PKEY* pkey, X509* x) WARN_UNUSED_RESULT;
+  static Status GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret) WARN_UNUSED_RESULT;
+
+  Status DoSign(const EVP_MD* digest, int32_t exp_seconds, X509 *ret) const WARN_UNUSED_RESULT;
+
+  // The expiration interval of certs signed by this signer.
+  int32_t exp_interval_sec_ = 24 * 60 * 60;
+
+  // The CA cert. null if this CertSigner is configured for self-signing.
+  const Cert* const ca_cert_;
+
+  // The CA private key. If configured for self-signing, this is the
+  // private key associated with the target cert.
+  const PrivateKey* const ca_private_key_;
+
+  DISALLOW_COPY_AND_ASSIGN(CertSigner);
+};
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/cert-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/cert-test.cc b/be/src/kudu/security/cert-test.cc
new file mode 100644
index 0000000..12205e1
--- /dev/null
+++ b/be/src/kudu/security/cert-test.cc
@@ -0,0 +1,165 @@
+// 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 <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/barrier.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::pair;
+using std::string;
+using std::thread;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+// Test for various certificate-related functionality in the security library.
+// These do not cover CA certificate mananagement part; check
+// cert_management-test.cc for those.
+class CertTest : public KuduTest {
+ public:
+  void SetUp() override {
+    ASSERT_OK(ca_cert_.FromString(kCaCert, DataFormat::PEM));
+    ASSERT_OK(ca_private_key_.FromString(kCaPrivateKey, DataFormat::PEM));
+    ASSERT_OK(ca_public_key_.FromString(kCaPublicKey, DataFormat::PEM));
+    ASSERT_OK(ca_exp_cert_.FromString(kCaExpiredCert, DataFormat::PEM));
+    ASSERT_OK(ca_exp_private_key_.FromString(kCaExpiredPrivateKey,
+                                             DataFormat::PEM));
+    // Sanity checks.
+    ASSERT_OK(ca_cert_.CheckKeyMatch(ca_private_key_));
+    ASSERT_OK(ca_exp_cert_.CheckKeyMatch(ca_exp_private_key_));
+  }
+
+ protected:
+  Cert ca_cert_;
+  PrivateKey ca_private_key_;
+  PublicKey ca_public_key_;
+
+  Cert ca_exp_cert_;
+  PrivateKey ca_exp_private_key_;
+};
+
+// Regression test to make sure that GetKuduKerberosPrincipalOidNid is thread
+// safe. OpenSSL 1.0.0's OBJ_create method is not thread safe.
+TEST_F(CertTest, GetKuduKerberosPrincipalOidNidConcurrent) {
+  int kConcurrency = 16;
+  Barrier barrier(kConcurrency);
+
+  vector<thread> threads;
+  for (int i = 0; i < kConcurrency; i++) {
+    threads.emplace_back([&] () {
+        barrier.Wait();
+        CHECK_NE(NID_undef, GetKuduKerberosPrincipalOidNid());
+    });
+  }
+
+  for (auto& thread : threads) {
+    thread.join();
+  }
+}
+
+// Check input/output of the X509 certificates in PEM format.
+TEST_F(CertTest, CertInputOutputPEM) {
+  const Cert& cert = ca_cert_;
+  string cert_str;
+  ASSERT_OK(cert.ToString(&cert_str, DataFormat::PEM));
+  RemoveExtraWhitespace(&cert_str);
+
+  string ca_input_cert(kCaCert);
+  RemoveExtraWhitespace(&ca_input_cert);
+  EXPECT_EQ(ca_input_cert, cert_str);
+}
+
+// Check that Cert behaves in a predictable way if given invalid PEM data.
+TEST_F(CertTest, CertInvalidInput) {
+  // Providing files which guaranteed to exists, but do not contain valid data.
+  // This is to make sure the init handles that situation correctly and
+  // does not choke on the wrong input data.
+  Cert c;
+  ASSERT_FALSE(c.FromFile("/bin/sh", DataFormat::PEM).ok());
+}
+
+// Check X509 certificate/private key matching: match cases.
+TEST_F(CertTest, CertMatchesRsaPrivateKey) {
+  const pair<const Cert*, const PrivateKey*> cases[] = {
+    { &ca_cert_,      &ca_private_key_      },
+    { &ca_exp_cert_,  &ca_exp_private_key_  },
+  };
+  for (const auto& e : cases) {
+    EXPECT_OK(e.first->CheckKeyMatch(*e.second));
+  }
+}
+
+// Check X509 certificate/private key matching: mismatch cases.
+TEST_F(CertTest, CertMismatchesRsaPrivateKey) {
+  const pair<const Cert*, const PrivateKey*> cases[] = {
+    { &ca_cert_,      &ca_exp_private_key_  },
+    { &ca_exp_cert_,  &ca_private_key_      },
+  };
+  for (const auto& e : cases) {
+    const Status s = e.first->CheckKeyMatch(*e.second);
+    EXPECT_TRUE(s.IsRuntimeError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "certificate does not match private key");
+  }
+}
+
+TEST_F(CertTest, TestGetKuduSpecificFieldsWhenMissing) {
+  EXPECT_EQ(boost::none, ca_cert_.UserId());
+  EXPECT_EQ(boost::none, ca_cert_.KuduKerberosPrincipal());
+}
+
+TEST_F(CertTest, DnsHostnameInSanField) {
+  const string hostname_foo_bar = "foo.bar.com";
+  const string hostname_mega_giga = "mega.giga.io";
+  const string hostname_too_long =
+      "toooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo."
+      "looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+      "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+      "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+      "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+      "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+      "ng.hostname.io";
+
+  Cert cert;
+  ASSERT_OK(cert.FromString(kCertDnsHostnamesInSan, DataFormat::PEM));
+
+  EXPECT_EQ("C = US, ST = CA, O = MyCompany, CN = MyName, emailAddress = my@email.com",
+            cert.IssuerName());
+  vector<string> hostnames = cert.Hostnames();
+  ASSERT_EQ(3, hostnames.size());
+  EXPECT_EQ(hostname_mega_giga, hostnames[0]);
+  EXPECT_EQ(hostname_foo_bar, hostnames[1]);
+  EXPECT_EQ(hostname_too_long, hostnames[2]);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/cert.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/cert.cc b/be/src/kudu/security/cert.cc
new file mode 100644
index 0000000..b81d263
--- /dev/null
+++ b/be/src/kudu/security/cert.cc
@@ -0,0 +1,301 @@
+// 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 "kudu/security/cert.h"
+
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <openssl/evp.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/openssl_util_bio.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<GENERAL_NAMES> {
+  static constexpr auto kFreeFunc = &GENERAL_NAMES_free;
+};
+
+// This OID is generated via the UUID method.
+static const char* kKuduKerberosPrincipalOidStr = "2.25.243346677289068076843480765133256509912";
+
+string X509NameToString(X509_NAME* name) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  CHECK(name);
+  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
+  OPENSSL_CHECK_OK(X509_NAME_print_ex(bio.get(), name, 0, XN_FLAG_ONELINE));
+
+  BUF_MEM* membuf;
+  OPENSSL_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
+  return string(membuf->data, membuf->length);
+}
+
+int GetKuduKerberosPrincipalOidNid() {
+  InitializeOpenSSL();
+  static std::once_flag flag;
+  static int nid;
+  std::call_once(flag, [&] () {
+      nid = OBJ_create(kKuduKerberosPrincipalOidStr, "kuduPrinc", "kuduKerberosPrincipal");
+      CHECK_NE(nid, NID_undef) << "failed to create kuduPrinc oid: " << GetOpenSSLErrors();
+  });
+  return nid;
+}
+
+X509* Cert::GetTopOfChainX509() const {
+  CHECK_GT(chain_len(), 0);
+  return sk_X509_value(data_.get(), 0);
+}
+
+Status Cert::FromString(const std::string& data, DataFormat format) {
+  RETURN_NOT_OK(::kudu::security::FromString(data, format, &data_));
+  if (sk_X509_num(data_.get()) < 1) {
+    return Status::RuntimeError("Certificate chain is empty. Expected at least one certificate.");
+  }
+  return Status::OK();
+}
+
+Status Cert::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString(data, format, data_.get());
+}
+
+Status Cert::FromFile(const std::string& fpath, DataFormat format) {
+  RETURN_NOT_OK(::kudu::security::FromFile(fpath, format, &data_));
+  if (sk_X509_num(data_.get()) < 1) {
+    return Status::RuntimeError("Certificate chain is empty. Expected at least one certificate.");
+  }
+  return Status::OK();
+}
+
+string Cert::SubjectName() const {
+  return X509NameToString(X509_get_subject_name(GetTopOfChainX509()));
+}
+
+string Cert::IssuerName() const {
+  return X509NameToString(X509_get_issuer_name(GetTopOfChainX509()));
+}
+
+boost::optional<string> Cert::UserId() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  X509_NAME* name = X509_get_subject_name(GetTopOfChainX509());
+  char buf[1024];
+  int len = X509_NAME_get_text_by_NID(name, NID_userId, buf, arraysize(buf));
+  if (len < 0) return boost::none;
+  return string(buf, len);
+}
+
+vector<string> Cert::Hostnames() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  vector<string> result;
+  auto gens = ssl_make_unique(reinterpret_cast<GENERAL_NAMES*>(X509_get_ext_d2i(
+      GetTopOfChainX509(), NID_subject_alt_name, nullptr, nullptr)));
+  if (gens) {
+    for (int i = 0; i < sk_GENERAL_NAME_num(gens.get()); ++i) {
+      GENERAL_NAME* gen = sk_GENERAL_NAME_value(gens.get(), i);
+      if (gen->type != GEN_DNS) {
+        continue;
+      }
+      const ASN1_STRING* cstr = gen->d.dNSName;
+      if (cstr->type != V_ASN1_IA5STRING || cstr->data == nullptr) {
+        LOG(DFATAL) << "invalid DNS name in the SAN field";
+        return {};
+      }
+      result.emplace_back(reinterpret_cast<char*>(cstr->data), cstr->length);
+    }
+  }
+  return result;
+}
+
+boost::optional<string> Cert::KuduKerberosPrincipal() const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  int idx = X509_get_ext_by_NID(GetTopOfChainX509(), GetKuduKerberosPrincipalOidNid(), -1);
+  if (idx < 0) return boost::none;
+  X509_EXTENSION* ext = X509_get_ext(GetTopOfChainX509(), idx);
+  ASN1_OCTET_STRING* octet_str = X509_EXTENSION_get_data(ext);
+  const unsigned char* octet_str_data = octet_str->data;
+  long len; // NOLINT
+  int tag, xclass;
+  if (ASN1_get_object(&octet_str_data, &len, &tag, &xclass, octet_str->length) != 0 ||
+      tag != V_ASN1_UTF8STRING) {
+    LOG(DFATAL) << "invalid extension value in cert " << SubjectName();
+    return boost::none;
+  }
+
+  return string(reinterpret_cast<const char*>(octet_str_data), len);
+}
+
+Status Cert::CheckKeyMatch(const PrivateKey& key) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  OPENSSL_RET_NOT_OK(X509_check_private_key(GetTopOfChainX509(), key.GetRawData()),
+                     "certificate does not match private key");
+  return Status::OK();
+}
+
+Status Cert::GetServerEndPointChannelBindings(string* channel_bindings) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  // Find the signature type of the certificate. This corresponds to the digest
+  // (hash) algorithm, and the public key type which signed the cert.
+
+#if OPENSSL_VERSION_NUMBER >= 0x10002000L
+  int signature_nid = X509_get_signature_nid(GetTopOfChainX509());
+#else
+  // Older version of OpenSSL appear not to have a public way to get the
+  // signature digest method from a certificate. Instead, we reach into the
+  // 'private' internals.
+  int signature_nid = OBJ_obj2nid(GetTopOfChainX509()->sig_alg->algorithm);
+#endif
+
+  // Retrieve the digest algorithm type.
+  int digest_nid;
+  int public_key_nid;
+  OBJ_find_sigid_algs(signature_nid, &digest_nid, &public_key_nid);
+
+  // RFC 5929: if the certificate's signatureAlgorithm uses no hash functions or
+  // uses multiple hash functions, then this channel binding type's channel
+  // bindings are undefined at this time (updates to is channel binding type may
+  // occur to address this issue if it ever arises).
+  //
+  // TODO(dan): can the multiple hash function scenario actually happen? What
+  // does OBJ_find_sigid_algs do in that scenario?
+  if (digest_nid == NID_undef) {
+    return Status::NotSupported("server certificate has no signature digest (hash) algorithm");
+  }
+
+  // RFC 5929: if the certificate's signatureAlgorithm uses a single hash
+  // function, and that hash function is either MD5 [RFC1321] or SHA-1
+  // [RFC3174], then use SHA-256 [FIPS-180-3];
+  if (digest_nid == NID_md5 || digest_nid == NID_sha1) {
+    digest_nid = NID_sha256;
+  }
+
+  const EVP_MD* md = EVP_get_digestbynid(digest_nid);
+  OPENSSL_RET_IF_NULL(md, "digest for nid not found");
+
+  // Create a digest BIO. All data written to the BIO will be sent through the
+  // digest (hash) function. The digest BIO requires a null BIO to writethrough to.
+  auto null_bio = ssl_make_unique(BIO_new(BIO_s_null()));
+  auto md_bio = ssl_make_unique(BIO_new(BIO_f_md()));
+  OPENSSL_RET_NOT_OK(BIO_set_md(md_bio.get(), md), "failed to set digest for BIO");
+  BIO_push(md_bio.get(), null_bio.get());
+
+  // Write the cert to the digest BIO.
+  RETURN_NOT_OK(ToBIO(md_bio.get(), DataFormat::DER, data_.get()));
+
+  // Read the digest from the BIO and append it to 'channel_bindings'.
+  char buf[EVP_MAX_MD_SIZE];
+  int digest_len = BIO_gets(md_bio.get(), buf, sizeof(buf));
+  OPENSSL_RET_NOT_OK(digest_len, "failed to get cert digest from BIO");
+  channel_bindings->assign(buf, digest_len);
+  return Status::OK();
+}
+
+void Cert::AdoptAndAddRefRawData(RawDataType* data) {
+  DCHECK_EQ(sk_X509_num(data), 1);
+  X509* cert = sk_X509_value(data, sk_X509_num(data) - 1);
+
+  DCHECK(cert);
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  CHECK_GT(CRYPTO_add(&cert->references, 1, CRYPTO_LOCK_X509), 1) << "X509 use-after-free detected";
+#else
+  OPENSSL_CHECK_OK(X509_up_ref(cert)) << "X509 use-after-free detected: " << GetOpenSSLErrors();
+#endif
+  // We copy the STACK_OF() object, but the copy and the original both internally point to the
+  // same elements.
+  AdoptRawData(sk_X509_dup(data));
+}
+
+void Cert::AdoptX509(X509* cert) {
+  // Free current STACK_OF(X509).
+  sk_X509_pop_free(data_.get(), X509_free);
+  // Allocate new STACK_OF(X509) and populate with 'cert'.
+  STACK_OF(X509)* sk = sk_X509_new_null();
+  DCHECK(sk);
+  sk_X509_push(sk, cert);
+  AdoptRawData(sk);
+}
+
+void Cert::AdoptAndAddRefX509(X509* cert) {
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  CHECK_GT(CRYPTO_add(&cert->references, 1, CRYPTO_LOCK_X509), 1) << "X509 use-after-free detected";
+#else
+  OPENSSL_CHECK_OK(X509_up_ref(cert)) << "X509 use-after-free detected: " << GetOpenSSLErrors();
+#endif
+  AdoptX509(cert);
+}
+
+Status Cert::GetPublicKey(PublicKey* key) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  EVP_PKEY* raw_key = X509_get_pubkey(GetTopOfChainX509());
+  OPENSSL_RET_IF_NULL(raw_key, "unable to get certificate public key");
+  key->AdoptRawData(raw_key);
+  return Status::OK();
+}
+
+Status CertSignRequest::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString(data, format, &data_);
+}
+
+Status CertSignRequest::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString(data, format, data_.get());
+}
+
+Status CertSignRequest::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile(fpath, format, &data_);
+}
+
+CertSignRequest CertSignRequest::Clone() const {
+  X509_REQ* cloned_req;
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  CHECK_GT(CRYPTO_add(&data_->references, 1, CRYPTO_LOCK_X509_REQ), 1)
+    << "X509_REQ use-after-free detected";
+  cloned_req = GetRawData();
+#else
+  // With OpenSSL 1.1, data structure internals are hidden, and there doesn't
+  // seem to be a public method that increments data_'s refcount.
+  cloned_req = X509_REQ_dup(GetRawData());
+  CHECK(cloned_req != nullptr)
+    << "X509 allocation failure detected: " << GetOpenSSLErrors();
+#endif
+
+  CertSignRequest clone;
+  clone.AdoptRawData(cloned_req);
+  return clone;
+}
+
+Status CertSignRequest::GetPublicKey(PublicKey* key) const {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  EVP_PKEY* raw_key = X509_REQ_get_pubkey(data_.get());
+  OPENSSL_RET_IF_NULL(raw_key, "unable to get CSR public key");
+  key->AdoptRawData(raw_key);
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu


[09/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util.h b/be/src/kudu/util/pb_util.h
new file mode 100644
index 0000000..6c132a6
--- /dev/null
+++ b/be/src/kudu/util/pb_util.h
@@ -0,0 +1,513 @@
+// 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.
+//
+// Utilities for dealing with protocol buffers.
+// These are mostly just functions similar to what are found in the protobuf
+// library itself, but using kudu::faststring instances instead of STL strings.
+#ifndef KUDU_UTIL_PB_UTIL_H
+#define KUDU_UTIL_PB_UTIL_H
+
+#include <cstdint>
+#include <iosfwd>
+#include <memory>
+#include <string>
+
+#include <boost/optional/optional.hpp>
+#include <google/protobuf/message.h>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/debug/trace_event_impl.h"
+
+namespace google {
+namespace protobuf {
+class DescriptorPool;
+class FileDescriptor;
+class FileDescriptorSet;
+class MessageLite;
+class SimpleDescriptorDatabase;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Env;
+class RandomAccessFile;
+class SequentialFile;
+class Slice;
+class Status;
+class RWFile;
+class faststring;
+
+namespace pb_util {
+
+enum SyncMode {
+  SYNC,
+  NO_SYNC
+};
+
+enum CreateMode {
+  OVERWRITE,
+  NO_OVERWRITE
+};
+
+enum class FileState {
+  NOT_INITIALIZED,
+  OPEN,
+  CLOSED
+};
+
+// The minimum valid length of a PBC file.
+extern const int kPBContainerMinimumValidLength;
+
+// See MessageLite::AppendToString
+void AppendToString(const google::protobuf::MessageLite &msg, faststring *output);
+
+// See MessageLite::AppendPartialToString
+void AppendPartialToString(const google::protobuf::MessageLite &msg, faststring *output);
+
+// See MessageLite::SerializeToString.
+void SerializeToString(const google::protobuf::MessageLite &msg, faststring *output);
+
+// See MessageLite::ParseFromZeroCopyStream
+Status ParseFromSequentialFile(google::protobuf::MessageLite *msg, SequentialFile *rfile);
+
+// Similar to MessageLite::ParseFromArray, with the difference that it returns
+// Status::Corruption() if the message could not be parsed.
+Status ParseFromArray(google::protobuf::MessageLite* msg, const uint8_t* data, uint32_t length);
+
+// Load a protobuf from the given path.
+Status ReadPBFromPath(Env* env, const std::string& path, google::protobuf::MessageLite* msg);
+
+// Serialize a protobuf to the given path.
+//
+// If SyncMode SYNC is provided, ensures the changes are made durable.
+Status WritePBToPath(Env* env, const std::string& path,
+                     const google::protobuf::MessageLite& msg, SyncMode sync);
+
+// Truncate any 'bytes' or 'string' fields of this message to max_len.
+// The text "<truncated>" is appended to any such truncated fields.
+void TruncateFields(google::protobuf::Message* message, int max_len);
+
+// Redaction-sensitive variant of Message::DebugString.
+//
+// For most protobufs, this has identical output to Message::DebugString. However,
+// a field with string or binary type may be tagged with the 'kudu.REDACT' option,
+// available by importing 'pb_util.proto'. When such a field is encountered by this
+// method, its contents will be redacted using the 'KUDU_REDACT' macro as documented
+// in kudu/util/logging.h.
+std::string SecureDebugString(const google::protobuf::Message& msg);
+
+// Same as SecureDebugString() above, but equivalent to Message::ShortDebugString.
+std::string SecureShortDebugString(const google::protobuf::Message& msg);
+
+// A protobuf "container" has the following format (all integers in
+// little-endian byte order).
+//
+// <file header>
+// <1 or more records>
+//
+// Note: There are two versions (version 1 and version 2) of the record format.
+// Version 2 of the file format differs from version 1 in the following ways:
+//
+//   * Version 2 has a file header checksum.
+//   * Version 2 has separate checksums for the record length and record data
+//     fields.
+//
+// File header format
+// ------------------
+//
+// Each protobuf container file contains a file header identifying the file.
+// This includes:
+//
+// magic number: 8 byte string identifying the file format.
+//
+//    Included so that we have a minimal guarantee that this file is of the
+//    type we expect and that we are not just reading garbage.
+//
+// container_version: 4 byte unsigned integer indicating the "version" of the
+//                    container format. May be set to 1 or 2.
+//
+//    Included so that this file format may be extended at some later date
+//    while maintaining backwards compatibility.
+//
+// file_header_checksum (version 2+ only): 4 byte unsigned integer with a CRC32C
+//                                         of the magic and version fields.
+//
+//    Included so that we can validate the container version number.
+//
+// The remaining container fields are considered part of a "record". There may
+// be 1 or more records in a valid protobuf container file.
+//
+// Record format
+// -------------
+//
+// data length: 4 byte unsigned integer indicating the size of the encoded data.
+//
+//    Included because PB messages aren't self-delimiting, and thus
+//    writing a stream of messages to the same file requires
+//    delimiting each with its size.
+//
+//    See https://developers.google.com/protocol-buffers/docs/techniques?hl=zh-cn#streaming
+//    for more details.
+//
+// length checksum (version 2+ only): 4-byte unsigned integer containing the
+//                                    CRC32C checksum of "data length".
+//
+//    Included so that we may discern the difference between a truncated file
+//    and a corrupted length field.
+//
+// data: "size" bytes of protobuf data encoded according to the schema.
+//
+//    Our payload.
+//
+// data checksum: 4 byte unsigned integer containing the CRC32C checksum of "data".
+//
+//    Included to ensure validity of the data on-disk.
+//    Note: In version 1 of the file format, this is a checksum of both the
+//    "data length" and "data" fields. In version 2+, this is only a checksum
+//    of the "data" field.
+//
+// Supplemental header
+// -------------------
+//
+// A valid container must have at least one record, the first of
+// which is known as the "supplemental header". The supplemental header
+// contains additional container-level information, including the protobuf
+// schema used for the records following it. See pb_util.proto for details. As
+// a containerized PB message, the supplemental header is protected by a CRC32C
+// checksum like any other message.
+//
+// Error detection and tolerance
+// -----------------------------
+//
+// It is worth describing the kinds of errors that can be detected by the
+// protobuf container and the kinds that cannot.
+//
+// The checksums in the container are independent, not rolling. As such,
+// they won't detect the disappearance or reordering of entire protobuf
+// messages, which can happen if a range of the file is collapsed (see
+// man fallocate(2)) or if the file is otherwise manually manipulated.
+//
+// In version 1, the checksums do not protect against corruption in the data
+// length field. However, version 2 of the format resolves that problem. The
+// benefit is that version 2 files can tell the difference between a record
+// with a corrupted length field and a record that was only partially written.
+// See ReadablePBContainerFile::ReadNextPB() for discussion on how this
+// difference is expressed via the API.
+//
+// In version 1 of the format, corruption of the version field in the file
+// header is not detectable. However, version 2 of the format addresses that
+// limitation as well.
+//
+// Corruption of the protobuf data itself is detected in all versions of the
+// file format (subject to CRC32 limitations).
+//
+// The container does not include footers or periodic checkpoints. As such, it
+// will not detect if entire records are truncated.
+//
+// The design and implementation relies on data ordering guarantees provided by
+// the file system to ensure that bytes are written to a file before the file
+// metadata (file size) is updated. A partially-written record (the result of a
+// failed append) is identified by one of the following criteria:
+// 1. Too-few bytes remain in the file to constitute a valid record. For
+//    version 2, that would be fewer than 12 bytes (data len, data len
+//    checksum, and data checksum), or
+// 2. Assuming a record's data length field is valid, then fewer bytes remain
+//    in the file than are specified in the data length field (plus enough for
+//    checksums).
+// In the above scenarios, it is assumed that the system faulted while in the
+// middle of appending a record, and it is considered safe to truncate the file
+// at the beginning of the partial record.
+//
+// If filesystem preallocation is used (at the time of this writing, the
+// implementation does not support preallocation) then even version 2 of the
+// format cannot safely support culling trailing partially-written records.
+// This is because it is not possible to reliably tell the difference between a
+// partially-written record that did not complete fsync (resulting in a bad
+// checksum) vs. a record that successfully was written to disk but then fell
+// victim to bit-level disk corruption. See also KUDU-1414.
+//
+// These tradeoffs in error detection are reasonable given the failure
+// environment that Kudu operates within. We tolerate failures such as
+// "kill -9" of the Kudu process, machine power loss, or fsync/fdatasync
+// failure, but not failures like runaway processes mangling data files
+// in arbitrary ways or attackers crafting malicious data files.
+//
+// In short, no version of the file format will detect truncation of entire
+// protobuf records. Version 2 relies on ordered data flushing semantics for
+// automatic recoverability from partial record writes. Version 1 of the file
+// format cannot support automatic recoverability from partial record writes.
+//
+// For further reading on what files might look like following a normal
+// filesystem failure or disk corruption, and the likelihood of various types
+// of disk errors, see the following papers:
+//
+// https://www.usenix.org/system/files/conference/osdi14/osdi14-paper-pillai.pdf
+// https://www.usenix.org/legacy/event/fast08/tech/full_papers/bairavasundaram/bairavasundaram.pdf
+
+// Protobuf container file opened for writing. Can be built around an existing
+// file or a completely new file.
+//
+// Every function is thread-safe unless indicated otherwise.
+class WritablePBContainerFile {
+ public:
+
+  // Initializes the class instance; writer must be open.
+  explicit WritablePBContainerFile(std::shared_ptr<RWFile> writer);
+
+  // Closes the container if not already closed.
+  ~WritablePBContainerFile();
+
+  // Writes the file header to disk and initializes the write offset to the
+  // byte after the file header. This method should NOT be called when opening
+  // an existing file for append; use OpenExisting() for that.
+  //
+  // 'msg' need not be populated; its type is used to "lock" the container
+  // to a particular protobuf message type in Append().
+  //
+  // Not thread-safe.
+  Status CreateNew(const google::protobuf::Message& msg);
+
+  // Opens an existing protobuf container file for append. The file must
+  // already have a valid file header. To initialize a new blank file for
+  // writing, use CreateNew() instead.
+  //
+  // The file header is read and the version specified there is used as the
+  // format version. The length of the file is also read and is used as the
+  // write offset for subsequent Append() calls. WritablePBContainerFile caches
+  // the write offset instead of constantly calling stat() on the file each
+  // time append is called.
+  //
+  // Not thread-safe.
+  Status OpenExisting();
+
+  // Writes a protobuf message to the container, beginning with its size
+  // and ending with its CRC32 checksum. One of CreateNew() or OpenExisting()
+  // must be called prior to calling Append(), i.e. the file must be open.
+  Status Append(const google::protobuf::Message& msg);
+
+  // Asynchronously flushes all dirty container data to the filesystem.
+  // The file must be open.
+  Status Flush();
+
+  // Synchronizes all dirty container data to the filesystem.
+  // The file must be open.
+  //
+  // Note: the parent directory is _not_ synchronized. Because the
+  // container file was provided during construction, we don't know whether
+  // it was created or reopened, and parent directory synchronization is
+  // only needed in the former case.
+  Status Sync();
+
+  // Closes the container.
+  //
+  // Not thread-safe.
+  Status Close();
+
+  // Returns the path to the container's underlying file handle.
+  const std::string& filename() const;
+
+ private:
+  friend class TestPBUtil;
+  FRIEND_TEST(TestPBUtil, TestPopulateDescriptorSet);
+
+  // Set the file format version. Only used for testing.
+  // Must be called before CreateNew().
+  Status SetVersionForTests(int version);
+
+  // Write the protobuf schemas belonging to 'desc' and all of its
+  // dependencies to 'output'.
+  //
+  // Schemas are written in dependency order (i.e. if A depends on B which
+  // depends on C, the order is C, B, A).
+  static void PopulateDescriptorSet(const google::protobuf::FileDescriptor* desc,
+                                    google::protobuf::FileDescriptorSet* output);
+
+  // Serialize the contents of 'msg' into 'buf' along with additional metadata
+  // to aid in deserialization.
+  Status AppendMsgToBuffer(const google::protobuf::Message& msg, faststring* buf);
+
+  // Append bytes to the file.
+  Status AppendBytes(const Slice& data);
+
+  // State of the file.
+  FileState state_;
+
+  // Protects offset_.
+  Mutex offset_lock_;
+
+  // Current write offset into the file.
+  uint64_t offset_;
+
+  // Protobuf container file version.
+  int version_;
+
+  // File writer.
+  std::shared_ptr<RWFile> writer_;
+};
+
+// Protobuf container file opened for reading.
+//
+// Can be built around a file with existing contents or an empty file (in
+// which case it's safe to interleave with WritablePBContainerFile).
+class ReadablePBContainerFile {
+ public:
+
+  // Initializes the class instance; reader must be open.
+  explicit ReadablePBContainerFile(std::shared_ptr<RandomAccessFile> reader);
+
+  // Closes the file if not already closed.
+  ~ReadablePBContainerFile();
+
+  // Reads the header information from the container and validates it.
+  // Must be called before any of the other methods.
+  Status Open();
+
+  // Reads a protobuf message from the container, validating its size and
+  // data using a CRC32 checksum. File must be open.
+  //
+  // Return values:
+  // * If there are no more records in the file, returns Status::EndOfFile.
+  // * If there is a partial record, but it is not long enough to be a full
+  //   record or the written length of the record is less than the remaining
+  //   bytes in the file, returns Status::Incomplete. If Status::Incomplete
+  //   is returned, calling offset() will return the point in the file where
+  //   the invalid partial record begins. In order to append additional records
+  //   to the file, the file must first be truncated at that offset.
+  //   Note: Version 1 of this file format will never return
+  //   Status::Incomplete() from this method.
+  // * If a corrupt record is encountered, returns Status::Corruption.
+  // * On success, stores the result in '*msg' and returns OK.
+  Status ReadNextPB(google::protobuf::Message* msg);
+
+  // Dumps any unread protobuf messages in the container to 'os'. Each
+  // message's DebugString() method is invoked to produce its textual form.
+  // File must be open.
+  enum class Format {
+    // Print each message on multiple lines, with intervening headers.
+    DEFAULT,
+    // Same as DEFAULT but includes additional metadata information.
+    DEBUG,
+    // Print each message on its own line.
+    ONELINE,
+    // Dump in JSON.
+    JSON
+  };
+  Status Dump(std::ostream* os, Format format);
+
+  // Closes the container.
+  Status Close();
+
+  // Expected PB type and schema for each message to be read.
+  //
+  // Only valid after a successful call to Open().
+  const std::string& pb_type() const { return pb_type_; }
+  const google::protobuf::FileDescriptorSet* protos() const {
+    return protos_.get();
+  }
+
+  // Get the prototype instance for the type of messages stored in this
+  // file. The returned Message is owned by this ReadablePBContainerFile instance.
+  Status GetPrototype(const google::protobuf::Message** prototype);
+
+  // Return the protobuf container file format version.
+  // File must be open.
+  int version() const;
+
+  // Return current read offset.
+  // File must be open.
+  uint64_t offset() const;
+
+ private:
+  FileState state_;
+  int version_;
+  uint64_t offset_;
+
+  // The size of the file we are reading, or 'none' if it hasn't yet been
+  // read.
+  boost::optional<uint64_t> cached_file_size_;
+
+  // The fully-qualified PB type name of the messages in the container.
+  std::string pb_type_;
+
+  // Wrapped in a unique_ptr so that clients need not include PB headers.
+  std::unique_ptr<google::protobuf::FileDescriptorSet> protos_;
+
+  // Protobuf infrastructure which owns the message prototype 'prototype_'.
+  std::unique_ptr<google::protobuf::SimpleDescriptorDatabase> db_;
+  std::unique_ptr<google::protobuf::DescriptorPool> descriptor_pool_;
+  std::unique_ptr<google::protobuf::MessageFactory> message_factory_;
+  const google::protobuf::Message* prototype_ = nullptr;
+
+  std::shared_ptr<RandomAccessFile> reader_;
+};
+
+// Convenience functions for protobuf containers holding just one record.
+
+// Load a "containerized" protobuf from the given path.
+// If the file does not exist, returns Status::NotFound(). Otherwise, may
+// return other Status error codes such as Status::IOError.
+Status ReadPBContainerFromPath(Env* env, const std::string& path,
+                               google::protobuf::Message* msg);
+
+// Serialize a "containerized" protobuf to the given path.
+//
+// If create == NO_OVERWRITE and 'path' already exists, the function will fail.
+// If sync == SYNC, the newly created file will be fsynced before returning.
+Status WritePBContainerToPath(Env* env, const std::string& path,
+                              const google::protobuf::Message& msg,
+                              CreateMode create,
+                              SyncMode sync);
+
+// Wrapper for a protobuf message which lazily converts to JSON when
+// the trace buffer is dumped.
+//
+// When tracing, an instance of this class can be associated with
+// a given trace, instead of a stringified PB, thus avoiding doing
+// stringification inline and moving that work to the tracing process.
+//
+// Example usage:
+//  TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+//                         "response", pb_util::PbTracer::TracePb(*response_pb_),
+//                         ...);
+//
+class PbTracer : public debug::ConvertableToTraceFormat {
+ public:
+  enum {
+    kMaxFieldLengthToTrace = 100
+  };
+
+  // Static helper to be called when adding a stringified PB to a trace.
+  // This does not actually stringify 'msg', that will be done later
+  // when/if AppendAsTraceFormat() is called on the returned object.
+  static scoped_refptr<debug::ConvertableToTraceFormat> TracePb(
+      const google::protobuf::Message& msg);
+
+  explicit PbTracer(const google::protobuf::Message& msg);
+
+  // Actually stringifies the PB and appends the string to 'out'.
+  void AppendAsTraceFormat(std::string* out) const override;
+ private:
+  const std::unique_ptr<google::protobuf::Message> msg_;
+};
+
+} // namespace pb_util
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util.proto b/be/src/kudu/util/pb_util.proto
new file mode 100644
index 0000000..b78c0cf
--- /dev/null
+++ b/be/src/kudu/util/pb_util.proto
@@ -0,0 +1,45 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+option java_package = "org.apache.kudu";
+
+import "google/protobuf/descriptor.proto";
+
+// ============================================================================
+//  Protobuf container metadata
+// ============================================================================
+
+// Supplemental protobuf container header, after the main header (see
+// pb_util.h for details).
+message ContainerSupHeaderPB {
+  // The protobuf schema for the messages expected in this container.
+  //
+  // This schema is complete, that is, it includes all of its dependencies
+  // (i.e. other schemas defined in .proto files imported by this schema's
+  // .proto file).
+  required google.protobuf.FileDescriptorSet protos = 1;
+
+  // The PB message type expected in each data entry in this container. Must
+  // be fully qualified (i.e. kudu.tablet.TabletSuperBlockPB).
+  required string pb_type = 2;
+}
+
+extend google.protobuf.FieldOptions {
+  optional bool REDACT = 50001 [default=false];
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pb_util_test.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pb_util_test.proto b/be/src/kudu/util/pb_util_test.proto
new file mode 100644
index 0000000..bac0be0
--- /dev/null
+++ b/be/src/kudu/util/pb_util_test.proto
@@ -0,0 +1,29 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+import "kudu/util/pb_util.proto";
+
+message TestSecurePrintingPB {
+  optional string insecure1 = 1;
+  optional string secure1 = 2 [(kudu.REDACT) = true];
+  optional string insecure2 = 3;
+  optional string secure2 = 4 [(kudu.REDACT) = true];
+  repeated string repeated_secure = 5 [(kudu.REDACT) = true];
+  optional string insecure3 = 6;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/process_memory-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/process_memory-test.cc b/be/src/kudu/util/process_memory-test.cc
new file mode 100644
index 0000000..36df1a9
--- /dev/null
+++ b/be/src/kudu/util/process_memory-test.cc
@@ -0,0 +1,75 @@
+// 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 <atomic>
+#include <cstdint>
+#include <ostream>
+#include <thread>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/monotime.h"
+#include "kudu/util/process_memory.h"
+
+using std::atomic;
+using std::thread;
+using std::vector;
+
+namespace kudu {
+
+// Microbenchmark for our new/delete hooks which track process-wide
+// memory consumption.
+TEST(ProcessMemory, BenchmarkConsumptionTracking) {
+  const int kNumThreads = 200;
+  vector<thread> threads;
+  atomic<bool> done(false);
+  atomic<int64_t> total_count(0);
+
+  // We start many threads, each of which performs 10:1 ratio of
+  // new/delete pairs to consumption lookups. The high number
+  // of threads highlights when there is contention on central
+  // tcmalloc locks.
+  for (int i = 0; i < kNumThreads; i++) {
+    threads.emplace_back([&]() {
+        int64_t local_count = 0;
+        while (!done) {
+          for (int a = 0; a < 10; a++) {
+            // Mark 'x' volatile so that the compiler does not optimize out the
+            // allocation.
+            char* volatile x = new char[8000];
+            delete[] x;
+          }
+          process_memory::CurrentConsumption();
+          local_count++;
+        }
+        total_count += local_count;
+      });
+  }
+  double secs = 3;
+  SleepFor(MonoDelta::FromSeconds(secs));
+  done = true;
+
+  for (auto& t : threads) {
+    t.join();
+  }
+
+  LOG(INFO) << "Performed " << total_count / secs << " iters/sec";
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/process_memory.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/process_memory.cc b/be/src/kudu/util/process_memory.cc
new file mode 100644
index 0000000..d2f3653
--- /dev/null
+++ b/be/src/kudu/util/process_memory.cc
@@ -0,0 +1,287 @@
+// 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 <cstddef>
+#include <ostream>
+#include <string>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#ifdef TCMALLOC_ENABLED
+#include <gperftools/malloc_extension.h>  // IWYU pragma: keep
+#endif
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/once.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"          // IWYU pragma: keep
+#include "kudu/util/debug/trace_event.h"  // IWYU pragma: keep
+#include "kudu/util/env.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/mem_tracker.h"        // IWYU pragma: keep
+#include "kudu/util/process_memory.h"
+#include "kudu/util/random.h"
+#include "kudu/util/status.h"
+
+DEFINE_int64(memory_limit_hard_bytes, 0,
+             "Maximum amount of memory this daemon should use, in bytes. "
+             "A value of 0 autosizes based on the total system memory. "
+             "A value of -1 disables all memory limiting.");
+TAG_FLAG(memory_limit_hard_bytes, stable);
+
+DEFINE_int32(memory_pressure_percentage, 60,
+             "Percentage of the hard memory limit that this daemon may "
+             "consume before flushing of in-memory data becomes prioritized.");
+TAG_FLAG(memory_pressure_percentage, advanced);
+
+DEFINE_int32(memory_limit_soft_percentage, 80,
+             "Percentage of the hard memory limit that this daemon may "
+             "consume before memory throttling of writes begins. The greater "
+             "the excess, the higher the chance of throttling. In general, a "
+             "lower soft limit leads to smoother write latencies but "
+             "decreased throughput, and vice versa for a higher soft limit.");
+TAG_FLAG(memory_limit_soft_percentage, advanced);
+
+DEFINE_int32(memory_limit_warn_threshold_percentage, 98,
+             "Percentage of the hard memory limit that this daemon may "
+             "consume before WARNING level messages are periodically logged.");
+TAG_FLAG(memory_limit_warn_threshold_percentage, advanced);
+
+#ifdef TCMALLOC_ENABLED
+DEFINE_int32(tcmalloc_max_free_bytes_percentage, 10,
+             "Maximum percentage of the RSS that tcmalloc is allowed to use for "
+             "reserved but unallocated memory.");
+TAG_FLAG(tcmalloc_max_free_bytes_percentage, advanced);
+#endif
+
+using strings::Substitute;
+
+namespace kudu {
+namespace process_memory {
+
+namespace {
+int64_t g_hard_limit;
+int64_t g_soft_limit;
+int64_t g_pressure_threshold;
+
+ThreadSafeRandom* g_rand = nullptr;
+
+#ifdef TCMALLOC_ENABLED
+// Total amount of memory released since the last GC. If this
+// is greater than GC_RELEASE_SIZE, this will trigger a tcmalloc gc.
+Atomic64 g_released_memory_since_gc;
+
+// Size, in bytes, that is considered a large value for Release() (or Consume() with
+// a negative value). If tcmalloc is used, this can trigger it to GC.
+// A higher value will make us call into tcmalloc less often (and therefore more
+// efficient). A lower value will mean our memory overhead is lower.
+// TODO(todd): this is a stopgap.
+const int64_t kGcReleaseSize = 128 * 1024L * 1024L;
+
+#endif // TCMALLOC_ENABLED
+
+} // anonymous namespace
+
+
+// Flag validation
+// ------------------------------------------------------------
+// Validate that various flags are percentages.
+static bool ValidatePercentage(const char* flagname, int value) {
+  if (value >= 0 && value <= 100) {
+    return true;
+  }
+  LOG(ERROR) << Substitute("$0 must be a percentage, value $1 is invalid",
+                           flagname, value);
+  return false;
+}
+
+static bool dummy[] = {
+  google::RegisterFlagValidator(&FLAGS_memory_limit_soft_percentage, &ValidatePercentage),
+  google::RegisterFlagValidator(&FLAGS_memory_limit_warn_threshold_percentage, &ValidatePercentage)
+#ifdef TCMALLOC_ENABLED
+  ,google::RegisterFlagValidator(&FLAGS_tcmalloc_max_free_bytes_percentage, &ValidatePercentage)
+#endif
+};
+
+
+// Wrappers around tcmalloc functionality
+// ------------------------------------------------------------
+#ifdef TCMALLOC_ENABLED
+static int64_t GetTCMallocProperty(const char* prop) {
+  size_t value;
+  if (!MallocExtension::instance()->GetNumericProperty(prop, &value)) {
+    LOG(DFATAL) << "Failed to get tcmalloc property " << prop;
+  }
+  return value;
+}
+
+int64_t GetTCMallocCurrentAllocatedBytes() {
+  return GetTCMallocProperty("generic.current_allocated_bytes");
+}
+
+void GcTcmalloc() {
+  TRACE_EVENT0("process", "GcTcmalloc");
+
+  // Number of bytes in the 'NORMAL' free list (i.e reserved by tcmalloc but
+  // not in use).
+  int64_t bytes_overhead = GetTCMallocProperty("tcmalloc.pageheap_free_bytes");
+  // Bytes allocated by the application.
+  int64_t bytes_used = GetTCMallocCurrentAllocatedBytes();
+
+  int64_t max_overhead = bytes_used * FLAGS_tcmalloc_max_free_bytes_percentage / 100.0;
+  if (bytes_overhead > max_overhead) {
+    int64_t extra = bytes_overhead - max_overhead;
+    while (extra > 0) {
+      // Release 1MB at a time, so that tcmalloc releases its page heap lock
+      // allowing other threads to make progress. This still disrupts the current
+      // thread, but is better than disrupting all.
+      MallocExtension::instance()->ReleaseToSystem(1024 * 1024);
+      extra -= 1024 * 1024;
+    }
+  }
+}
+#endif // TCMALLOC_ENABLED
+
+
+// Consumption and soft memory limit behavior
+// ------------------------------------------------------------
+namespace {
+void DoInitLimits() {
+  int64_t limit = FLAGS_memory_limit_hard_bytes;
+  if (limit == 0) {
+    // If no limit is provided, we'll use 80% of system RAM.
+    int64_t total_ram;
+    CHECK_OK(Env::Default()->GetTotalRAMBytes(&total_ram));
+    limit = total_ram * 4;
+    limit /= 5;
+  }
+  g_hard_limit = limit;
+  g_soft_limit = FLAGS_memory_limit_soft_percentage * g_hard_limit / 100;
+  g_pressure_threshold = FLAGS_memory_pressure_percentage * g_hard_limit / 100;
+
+  g_rand = new ThreadSafeRandom(1);
+}
+
+void InitLimits() {
+  static GoogleOnceType once;
+  GoogleOnceInit(&once, &DoInitLimits);
+}
+
+} // anonymous namespace
+
+int64_t CurrentConsumption() {
+#ifdef TCMALLOC_ENABLED
+  const int64_t kReadIntervalMicros = 50000;
+  static Atomic64 last_read_time = 0;
+  static simple_spinlock read_lock;
+  static Atomic64 consumption = 0;
+  uint64_t time = GetMonoTimeMicros();
+  if (time > last_read_time + kReadIntervalMicros && read_lock.try_lock()) {
+    base::subtle::NoBarrier_Store(&consumption, GetTCMallocCurrentAllocatedBytes());
+    // Re-fetch the time after getting the consumption. This way, in case fetching
+    // consumption is extremely slow for some reason (eg due to lots of contention
+    // in tcmalloc) we at least ensure that we wait at least another full interval
+    // before fetching the information again.
+    time = GetMonoTimeMicros();
+    base::subtle::NoBarrier_Store(&last_read_time, time);
+    read_lock.unlock();
+  }
+
+  return base::subtle::NoBarrier_Load(&consumption);
+#else
+  // Without tcmalloc, we have no reliable way of determining our own heap
+  // size (e.g. mallinfo doesn't work in ASAN builds). So, we'll fall back
+  // to just looking at the sum of our tracked memory.
+  return MemTracker::GetRootTracker()->consumption();
+#endif
+}
+
+int64_t HardLimit() {
+  InitLimits();
+  return g_hard_limit;
+}
+
+int64_t SoftLimit() {
+  InitLimits();
+  return g_soft_limit;
+}
+
+int64_t MemoryPressureThreshold() {
+  InitLimits();
+  return g_pressure_threshold;
+}
+
+bool UnderMemoryPressure(double* current_capacity_pct) {
+  InitLimits();
+  int64_t consumption = CurrentConsumption();
+  if (consumption < g_pressure_threshold) {
+    return false;
+  }
+  if (current_capacity_pct) {
+    *current_capacity_pct = static_cast<double>(consumption) / g_hard_limit * 100;
+  }
+  return true;
+}
+
+bool SoftLimitExceeded(double* current_capacity_pct) {
+  InitLimits();
+  int64_t consumption = CurrentConsumption();
+  // Did we exceed the actual limit?
+  if (consumption > g_hard_limit) {
+    if (current_capacity_pct) {
+      *current_capacity_pct = static_cast<double>(consumption) / g_hard_limit * 100;
+    }
+    return true;
+  }
+
+  // No soft limit defined.
+  if (g_hard_limit == g_soft_limit) {
+    return false;
+  }
+
+  // Are we under the soft limit threshold?
+  if (consumption < g_soft_limit) {
+    return false;
+  }
+
+  // We're over the threshold; were we randomly chosen to be over the soft limit?
+  if (consumption + g_rand->Uniform64(g_hard_limit - g_soft_limit) > g_hard_limit) {
+    if (current_capacity_pct) {
+      *current_capacity_pct = static_cast<double>(consumption) / g_hard_limit * 100;
+    }
+    return true;
+  }
+  return false;
+}
+
+void MaybeGCAfterRelease(int64_t released_bytes) {
+#ifdef TCMALLOC_ENABLED
+  int64_t now_released = base::subtle::NoBarrier_AtomicIncrement(
+      &g_released_memory_since_gc, -released_bytes);
+  if (PREDICT_FALSE(now_released > kGcReleaseSize)) {
+    base::subtle::NoBarrier_Store(&g_released_memory_since_gc, 0);
+    GcTcmalloc();
+  }
+#endif
+}
+
+} // namespace process_memory
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/process_memory.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/process_memory.h b/be/src/kudu/util/process_memory.h
new file mode 100644
index 0000000..cba7046
--- /dev/null
+++ b/be/src/kudu/util/process_memory.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.
+
+#include <cstdint>
+
+namespace kudu {
+namespace process_memory {
+
+// Probabilistically returns true if the process-wide soft memory limit is exceeded.
+// The greater the excess, the higher the chance that it returns true.
+//
+// If the soft limit is exceeded and 'current_capacity_pct' is not NULL, the percentage
+// of the hard limit consumed is written to it.
+bool SoftLimitExceeded(double* current_capacity_pct);
+
+// Return true if we are under memory pressure (i.e if we are nearing the point at which
+// SoftLimitExceeded will begin to return true).
+//
+// If the process is under memory pressure, and 'current_capacity_pct' is not NULL,
+// the percentage of the hard limit consumed is written to it.
+bool UnderMemoryPressure(double* current_capacity_pct);
+
+// Potentially trigger a call to release tcmalloc memory back to the
+// OS, after the given amount of memory was released.
+void MaybeGCAfterRelease(int64_t released_bytes);
+
+// Return the total current memory consumption of the process.
+int64_t CurrentConsumption();
+
+// Return the configured hard limit for the process.
+int64_t HardLimit();
+
+// Return the configured soft limit for the process.
+int64_t SoftLimit();
+
+// Return the configured memory pressure threshold for the process.
+int64_t MemoryPressureThreshold();
+
+#ifdef TCMALLOC_ENABLED
+// Get the current amount of allocated memory, according to tcmalloc.
+//
+// This should be equal to CurrentConsumption(), but is made available so that tests
+// can verify the correctness of CurrentConsumption().
+int64_t GetTCMallocCurrentAllocatedBytes();
+#endif
+
+} // namespace process_memory
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/promise.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/promise.h b/be/src/kudu/util/promise.h
new file mode 100644
index 0000000..17f8cec
--- /dev/null
+++ b/be/src/kudu/util/promise.h
@@ -0,0 +1,79 @@
+// 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.
+#ifndef KUDU_UTIL_PROMISE_H
+#define KUDU_UTIL_PROMISE_H
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/countdown_latch.h"
+
+namespace kudu {
+
+// A promise boxes a value which is to be provided at some time in the future.
+// A single producer calls Set(...), and any number of consumers can call Get()
+// to retrieve the produced value.
+//
+// In Guava terms, this is a SettableFuture<T>.
+template<typename T>
+class Promise {
+ public:
+  Promise() : latch_(1) {}
+  ~Promise() {}
+
+  // Reset the promise to be used again.
+  // For this to be safe, there must be some kind of external synchronization
+  // ensuring that no threads are still accessing the value from the previous
+  // incarnation of the promise.
+  void Reset() {
+    latch_.Reset(1);
+    val_ = T();
+  }
+
+  // Block until a value is available, and return a reference to it.
+  const T& Get() const {
+    latch_.Wait();
+    return val_;
+  }
+
+  // Wait for the promised value to become available with the given timeout.
+  //
+  // Returns NULL if the timeout elapses before a value is available.
+  // Otherwise returns a pointer to the value. This pointer's lifetime is
+  // tied to the lifetime of the Promise object.
+  const T* WaitFor(const MonoDelta& delta) const {
+    if (latch_.WaitFor(delta)) {
+      return &val_;
+    } else {
+      return NULL;
+    }
+  }
+
+  // Set the value of this promise.
+  // This may be called at most once.
+  void Set(const T& val) {
+    DCHECK_EQ(latch_.count(), 1) << "Already set!";
+    val_ = val;
+    latch_.CountDown();
+  }
+
+ private:
+  CountDownLatch latch_;
+  T val_;
+  DISALLOW_COPY_AND_ASSIGN(Promise);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_PROMISE_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/proto_container_test.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/proto_container_test.proto b/be/src/kudu/util/proto_container_test.proto
new file mode 100644
index 0000000..4707c08
--- /dev/null
+++ b/be/src/kudu/util/proto_container_test.proto
@@ -0,0 +1,25 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+// Arbitrary protobuf to test writing a containerized protobuf.
+message ProtoContainerTestPB {
+  required string name = 1;
+  required int32 value = 2;
+  optional string note = 3;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/proto_container_test2.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/proto_container_test2.proto b/be/src/kudu/util/proto_container_test2.proto
new file mode 100644
index 0000000..74a1ea3
--- /dev/null
+++ b/be/src/kudu/util/proto_container_test2.proto
@@ -0,0 +1,29 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+// Dependency chain:
+//
+// this file --> proto_container_test.proto
+
+import "kudu/util/proto_container_test.proto";
+
+// Arbitrary protobuf that has one PB dependency.
+message ProtoContainerTest2PB {
+  required kudu.ProtoContainerTestPB record = 1;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/proto_container_test3.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/proto_container_test3.proto b/be/src/kudu/util/proto_container_test3.proto
new file mode 100644
index 0000000..1ed1c31
--- /dev/null
+++ b/be/src/kudu/util/proto_container_test3.proto
@@ -0,0 +1,33 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+// Dependency chain:
+//
+// this file --> proto_container_test.proto
+//           --> proto_container_test2.proto --> proto_container_test.proto
+
+import "kudu/util/proto_container_test.proto";
+import "kudu/util/proto_container_test2.proto";
+
+// Arbitrary protobuf has two PB dependencies.
+// dependency.
+message ProtoContainerTest3PB {
+  required kudu.ProtoContainerTestPB record_one = 1;
+  required kudu.ProtoContainerTest2PB record_two = 2;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/protobuf-annotations.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/protobuf-annotations.h b/be/src/kudu/util/protobuf-annotations.h
new file mode 100644
index 0000000..7fdc961
--- /dev/null
+++ b/be/src/kudu/util/protobuf-annotations.h
@@ -0,0 +1,33 @@
+// 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.
+//
+// Simple header which is inserted into all of our generated protobuf code.
+// We use this to hook protobuf code up to TSAN annotations.
+#ifndef KUDU_UTIL_PROTOBUF_ANNOTATIONS_H
+#define KUDU_UTIL_PROTOBUF_ANNOTATIONS_H
+
+#include "kudu/gutil/dynamic_annotations.h"
+
+// The protobuf internal headers are included before this, so we have to undefine
+// the empty definitions first.
+#undef GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN
+#undef GOOGLE_SAFE_CONCURRENT_WRITES_END
+
+#define GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN ANNOTATE_IGNORE_WRITES_BEGIN
+#define GOOGLE_SAFE_CONCURRENT_WRITES_END ANNOTATE_IGNORE_WRITES_END
+
+#endif /* KUDU_UTIL_PROTOBUF_ANNOTATIONS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/protobuf_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/protobuf_util.h b/be/src/kudu/util/protobuf_util.h
new file mode 100644
index 0000000..cc88eda
--- /dev/null
+++ b/be/src/kudu/util/protobuf_util.h
@@ -0,0 +1,39 @@
+// 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.
+#ifndef KUDU_UTIL_PROTOBUF_UTIL_H
+#define KUDU_UTIL_PROTOBUF_UTIL_H
+
+#include <google/protobuf/message_lite.h>
+
+namespace kudu {
+
+bool AppendPBToString(const google::protobuf::MessageLite &msg, faststring *output) {
+  int old_size = output->size();
+  int byte_size = msg.ByteSize();
+  output->resize(old_size + byte_size);
+  uint8* start = reinterpret_cast<uint8*>(output->data() + old_size);
+  uint8* end = msg.SerializeWithCachedSizesToArray(start);
+  CHECK(end - start == byte_size)
+    << "Error in serialization. byte_size=" << byte_size
+    << " new ByteSize()=" << msg.ByteSize()
+    << " end-start=" << (end-start);
+  return true;
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/protoc-gen-insertions.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/protoc-gen-insertions.cc b/be/src/kudu/util/protoc-gen-insertions.cc
new file mode 100644
index 0000000..5d1097e
--- /dev/null
+++ b/be/src/kudu/util/protoc-gen-insertions.cc
@@ -0,0 +1,77 @@
+// 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.
+//
+// Simple protoc plugin which inserts some code at the top of each generated protobuf.
+// Currently, this just adds an include of protobuf-annotations.h, a file which hooks up
+// the protobuf concurrency annotations to our TSAN annotations.
+
+#include <string>
+
+#include <google/protobuf/compiler/code_generator.h>
+#include <google/protobuf/compiler/plugin.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/io/printer.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+
+using google::protobuf::io::ZeroCopyOutputStream;
+using google::protobuf::io::Printer;
+using std::string;
+
+namespace kudu {
+
+static const char* const kIncludeToInsert = "#include \"kudu/util/protobuf-annotations.h\"\n";
+static const char* const kProtoExtension = ".proto";
+
+class InsertAnnotations : public ::google::protobuf::compiler::CodeGenerator {
+  virtual bool Generate(const google::protobuf::FileDescriptor *file,
+                        const std::string &/*param*/,
+                        google::protobuf::compiler::GeneratorContext *gen_context,
+                        std::string *error) const OVERRIDE {
+
+    // Determine the file name we will substitute into.
+    string path_no_extension;
+    if (!TryStripSuffixString(file->name(), kProtoExtension, &path_no_extension)) {
+      *error = strings::Substitute("file name $0 did not end in $1", file->name(), kProtoExtension);
+      return false;
+    }
+    string pb_file = path_no_extension + ".pb.cc";
+
+    // Actually insert the new #include
+    gscoped_ptr<ZeroCopyOutputStream> inserter(gen_context->OpenForInsert(pb_file, "includes"));
+    Printer printer(inserter.get(), '$');
+    printer.Print(kIncludeToInsert);
+
+    if (printer.failed()) {
+      *error = "Failed to print to output file";
+      return false;
+    }
+
+    return true;
+  }
+};
+
+} // namespace kudu
+
+int main(int argc, char *argv[]) {
+  kudu::InsertAnnotations generator;
+  return google::protobuf::compiler::PluginMain(argc, argv, &generator);
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pstack_watcher-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pstack_watcher-test.cc b/be/src/kudu/util/pstack_watcher-test.cc
new file mode 100644
index 0000000..a993d66
--- /dev/null
+++ b/be/src/kudu/util/pstack_watcher-test.cc
@@ -0,0 +1,100 @@
+// 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 "kudu/util/pstack_watcher.h"
+
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstdio>
+#include <memory>
+#include <string>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using std::shared_ptr;
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+
+TEST(TestPstackWatcher, TestPstackWatcherCancellation) {
+  PstackWatcher watcher(MonoDelta::FromSeconds(1000000));
+  watcher.Shutdown();
+}
+
+TEST(TestPstackWatcher, TestWait) {
+  PstackWatcher watcher(MonoDelta::FromMilliseconds(10));
+  watcher.Wait();
+}
+
+TEST(TestPstackWatcher, TestDumpStacks) {
+  ASSERT_OK(PstackWatcher::DumpStacks());
+}
+
+static FILE* RedirectStdout(string *temp_path) {
+  string temp_dir;
+  CHECK_OK(Env::Default()->GetTestDirectory(&temp_dir));
+  *temp_path = Substitute("$0/pstack_watcher-dump.$1.txt",
+                      temp_dir, getpid());
+  FILE* reopened;
+  POINTER_RETRY_ON_EINTR(reopened, freopen(temp_path->c_str(), "w", stdout));
+  return reopened;
+}
+
+TEST(TestPstackWatcher, TestPstackWatcherRunning) {
+  string stdout_file;
+  int old_stdout;
+  RETRY_ON_EINTR(old_stdout, dup(STDOUT_FILENO));
+  CHECK_ERR(old_stdout);
+  {
+    FILE* out_fp = RedirectStdout(&stdout_file);
+    PCHECK(out_fp != nullptr);
+    SCOPED_CLEANUP({
+        int err;
+        RETRY_ON_EINTR(err, fclose(out_fp));
+      });
+    PstackWatcher watcher(MonoDelta::FromMilliseconds(500));
+    while (watcher.IsRunning()) {
+      SleepFor(MonoDelta::FromMilliseconds(1));
+    }
+  }
+  int dup2_ret;
+  RETRY_ON_EINTR(dup2_ret, dup2(old_stdout, STDOUT_FILENO));
+  CHECK_ERR(dup2_ret);
+  PCHECK(stdout = fdopen(STDOUT_FILENO, "w"));
+
+  faststring contents;
+  CHECK_OK(ReadFileToString(Env::Default(), stdout_file, &contents));
+  ASSERT_STR_CONTAINS(contents.ToString(), "BEGIN STACKS");
+  CHECK_ERR(unlink(stdout_file.c_str()));
+  ASSERT_GE(fprintf(stdout, "%s\n", contents.ToString().c_str()), 0)
+      << "errno=" << errno << ": " << ErrnoToString(errno);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pstack_watcher.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pstack_watcher.cc b/be/src/kudu/util/pstack_watcher.cc
new file mode 100644
index 0000000..2c4481a
--- /dev/null
+++ b/be/src/kudu/util/pstack_watcher.cc
@@ -0,0 +1,249 @@
+// 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 "kudu/util/pstack_watcher.h"
+
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstdio>
+#include <string>
+#include <vector>
+
+#include <boost/bind.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/thread.h"
+
+namespace kudu {
+
+using std::string;
+using std::vector;
+using strings::SkipEmpty;
+using strings::SkipWhitespace;
+using strings::Split;
+using strings::Substitute;
+
+PstackWatcher::PstackWatcher(MonoDelta timeout)
+    : timeout_(timeout), running_(true), cond_(&lock_) {
+  CHECK_OK(Thread::Create("pstack_watcher", "pstack_watcher",
+                 boost::bind(&PstackWatcher::Run, this), &thread_));
+}
+
+PstackWatcher::~PstackWatcher() {
+  Shutdown();
+}
+
+void PstackWatcher::Shutdown() {
+  {
+    MutexLock guard(lock_);
+    running_ = false;
+    cond_.Broadcast();
+  }
+  if (thread_) {
+    CHECK_OK(ThreadJoiner(thread_.get()).Join());
+    thread_.reset();
+  }
+}
+
+bool PstackWatcher::IsRunning() const {
+  MutexLock guard(lock_);
+  return running_;
+}
+
+void PstackWatcher::Wait() const {
+  MutexLock lock(lock_);
+  while (running_) {
+    cond_.Wait();
+  }
+}
+
+void PstackWatcher::Run() {
+  MutexLock guard(lock_);
+  if (!running_) return;
+  cond_.WaitFor(timeout_);
+  if (!running_) return;
+
+  WARN_NOT_OK(DumpStacks(DUMP_FULL), "Unable to print pstack from watcher");
+  running_ = false;
+  cond_.Broadcast();
+}
+
+Status PstackWatcher::HasProgram(const char* progname) {
+  Subprocess proc({ "which", progname } );
+  proc.DisableStderr();
+  proc.DisableStdout();
+  RETURN_NOT_OK_PREPEND(proc.Start(),
+      Substitute("HasProgram($0): error running 'which'", progname));
+  RETURN_NOT_OK(proc.Wait());
+  int exit_status;
+  string exit_info;
+  RETURN_NOT_OK(proc.GetExitStatus(&exit_status, &exit_info));
+  if (exit_status == 0) {
+    return Status::OK();
+  }
+  return Status::NotFound(Substitute("can't find $0: $1", progname, exit_info));
+}
+
+Status PstackWatcher::HasGoodGdb() {
+  // Check for the existence of gdb.
+  RETURN_NOT_OK(HasProgram("gdb"));
+
+  // gdb exists, run it and parse the output of --version. For example:
+  //
+  // GNU gdb (GDB) Red Hat Enterprise Linux (7.2-75.el6)
+  // ...
+  //
+  // Or:
+  //
+  // GNU gdb (Ubuntu 7.11.1-0ubuntu1~16.5) 7.11.1
+  // ...
+  string stdout;
+  RETURN_NOT_OK(Subprocess::Call({"gdb", "--version"}, "", &stdout));
+  vector<string> lines = Split(stdout, "\n", SkipEmpty());
+  if (lines.empty()) {
+    return Status::Incomplete("gdb version not found");
+  }
+  vector<string> words = Split(lines[0], " ", SkipWhitespace());
+  if (words.empty()) {
+    return Status::Incomplete("could not parse gdb version");
+  }
+  string version = words[words.size() - 1];
+  version = StripPrefixString(version, "(");
+  version = StripSuffixString(version, ")");
+
+  // The variable pretty print routine in older versions of gdb is buggy in
+  // that it reads the values of all local variables, including uninitialized
+  // ones. For some variable types with an embedded length (such as std::string
+  // or std::vector), this can lead to all sorts of incorrect memory accesses,
+  // causing deadlocks or seemingly infinite loops within gdb.
+  //
+  // It's not clear exactly when this behavior was fixed, so we whitelist the
+  // oldest known good version: the one found in Ubuntu 14.04.
+  //
+  // See the following gdb bug reports for more information:
+  // - https://sourceware.org/bugzilla/show_bug.cgi?id=11868
+  // - https://sourceware.org/bugzilla/show_bug.cgi?id=12127
+  // - https://sourceware.org/bugzilla/show_bug.cgi?id=16196
+  // - https://sourceware.org/bugzilla/show_bug.cgi?id=16286
+  autodigit_less lt;
+  if (lt(version, "7.7")) {
+    return Status::NotSupported("gdb version too old", version);
+  }
+
+  return Status::OK();
+}
+
+Status PstackWatcher::DumpStacks(int flags) {
+  return DumpPidStacks(getpid(), flags);
+}
+
+Status PstackWatcher::DumpPidStacks(pid_t pid, int flags) {
+
+  // Prefer GDB if available; it gives us line numbers and thread names.
+  Status s = HasGoodGdb();
+  if (s.ok()) {
+    return RunGdbStackDump(pid, flags);
+  }
+  WARN_NOT_OK(s, "gdb not available");
+
+  // Otherwise, try to use pstack or gstack.
+  for (const auto& p : { "pstack", "gstack" }) {
+    s = HasProgram(p);
+    if (s.ok()) {
+      return RunPstack(p, pid);
+    }
+    WARN_NOT_OK(s, Substitute("$0 not available", p));
+  }
+
+  return Status::ServiceUnavailable("Neither gdb, pstack, nor gstack appear to be installed.");
+}
+
+Status PstackWatcher::RunGdbStackDump(pid_t pid, int flags) {
+  // Command: gdb -quiet -batch -nx -ex cmd1 -ex cmd2 /proc/$PID/exe $PID
+  vector<string> argv;
+  argv.emplace_back("gdb");
+  // Don't print introductory version/copyright messages.
+  argv.emplace_back("-quiet");
+  // Exit after processing all of the commands below.
+  argv.emplace_back("-batch");
+  // Don't run commands from .gdbinit
+  argv.emplace_back("-nx");
+  argv.emplace_back("-ex");
+  argv.emplace_back("set print pretty on");
+  argv.emplace_back("-ex");
+  argv.emplace_back("info threads");
+  argv.emplace_back("-ex");
+  argv.emplace_back("thread apply all bt");
+  if (flags & DUMP_FULL) {
+    argv.emplace_back("-ex");
+    argv.emplace_back("thread apply all bt full");
+  }
+  string executable;
+  Env* env = Env::Default();
+  RETURN_NOT_OK(env->GetExecutablePath(&executable));
+  argv.push_back(executable);
+  argv.push_back(Substitute("$0", pid));
+  return RunStackDump(argv);
+}
+
+Status PstackWatcher::RunPstack(const std::string& progname, pid_t pid) {
+  string pid_string(Substitute("$0", pid));
+  vector<string> argv;
+  argv.push_back(progname);
+  argv.push_back(pid_string);
+  return RunStackDump(argv);
+}
+
+Status PstackWatcher::RunStackDump(const vector<string>& argv) {
+  printf("************************ BEGIN STACKS **************************\n");
+  if (fflush(stdout) == EOF) {
+    return Status::IOError("Unable to flush stdout", ErrnoToString(errno), errno);
+  }
+  Subprocess pstack_proc(argv);
+  RETURN_NOT_OK_PREPEND(pstack_proc.Start(), "RunStackDump proc.Start() failed");
+  int ret;
+  RETRY_ON_EINTR(ret, ::close(pstack_proc.ReleaseChildStdinFd()));
+  if (ret == -1) {
+    return Status::IOError("Unable to close child stdin", ErrnoToString(errno), errno);
+  }
+  RETURN_NOT_OK_PREPEND(pstack_proc.Wait(), "RunStackDump proc.Wait() failed");
+  int exit_code;
+  string exit_info;
+  RETURN_NOT_OK_PREPEND(pstack_proc.GetExitStatus(&exit_code, &exit_info),
+                        "RunStackDump proc.GetExitStatus() failed");
+  if (exit_code != 0) {
+    return Status::RuntimeError("RunStackDump proc.Wait() error", exit_info);
+  }
+  printf("************************* END STACKS ***************************\n");
+  if (fflush(stdout) == EOF) {
+    return Status::IOError("Unable to flush stdout", ErrnoToString(errno), errno);
+  }
+
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/pstack_watcher.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/pstack_watcher.h b/be/src/kudu/util/pstack_watcher.h
new file mode 100644
index 0000000..882e6d2
--- /dev/null
+++ b/be/src/kudu/util/pstack_watcher.h
@@ -0,0 +1,101 @@
+// 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.
+#ifndef KUDU_UTIL_PSTACK_WATCHER_H
+#define KUDU_UTIL_PSTACK_WATCHER_H
+
+#include <sys/types.h>
+
+#include <string>
+#include <vector>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Thread;
+
+// PstackWatcher is an object which will pstack the current process and print
+// the results to stdout.  It does this after a certain timeout has occured.
+class PstackWatcher {
+ public:
+
+  enum Flags {
+    NO_FLAGS = 0,
+
+    // Run 'thread apply all bt full', which is very verbose output
+    DUMP_FULL = 1
+  };
+
+  // Static method to collect and write stack dump output to stdout of the current
+  // process.
+  static Status DumpStacks(int flags = NO_FLAGS);
+
+  // Like the above but for any process, not just the current one.
+  static Status DumpPidStacks(pid_t pid, int flags = NO_FLAGS);
+
+  // Instantiate a watcher that writes a pstack to stdout after the given
+  // timeout expires.
+  explicit PstackWatcher(MonoDelta timeout);
+
+  ~PstackWatcher();
+
+  // Shut down the watcher and do not log a pstack.
+  // This method is not thread-safe.
+  void Shutdown();
+
+  // Test whether the watcher is still running or has shut down.
+  // Thread-safe.
+  bool IsRunning() const;
+
+  // Wait until the timeout expires and the watcher logs a pstack.
+  // Thread-safe.
+  void Wait() const;
+
+ private:
+  // Test for the existence of the given program in the system path.
+  static Status HasProgram(const char* progname);
+
+  // Check whether the system path has 'gdb' and whether it is modern enough
+  // for safe stack dump usage.
+  static Status HasGoodGdb();
+
+  // Get a stack dump using GDB directly.
+  static Status RunGdbStackDump(pid_t pid, int flags);
+
+  // Get a stack dump using the pstack or gstack program.
+  static Status RunPstack(const std::string& progname, pid_t pid);
+
+  // Invoke and wait for the stack dump program.
+  static Status RunStackDump(const std::vector<std::string>& argv);
+
+  // Run the thread that waits for the specified duration before logging a
+  // pstack.
+  void Run();
+
+  const MonoDelta timeout_;
+  bool running_;
+  scoped_refptr<Thread> thread_;
+  mutable Mutex lock_;
+  mutable ConditionVariable cond_;
+};
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/random-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/random-test.cc b/be/src/kudu/util/random-test.cc
new file mode 100644
index 0000000..26c7ab5
--- /dev/null
+++ b/be/src/kudu/util/random-test.cc
@@ -0,0 +1,171 @@
+// 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 <cmath>
+#include <cstdint>
+#include <limits>
+#include <unordered_set>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/random.h"
+#include "kudu/util/test_util.h"
+
+using std::numeric_limits;
+using std::unordered_set;
+using std::vector;
+
+namespace kudu {
+
+class RandomTest : public KuduTest {
+ public:
+  RandomTest()
+      : rng_(SeedRandom()) {
+  }
+
+ protected:
+  Random rng_;
+};
+
+// Tests that after a certain number of invocations of Normal(), the
+// actual mean of all samples is within the specified standard
+// deviation of the target mean.
+TEST_F(RandomTest, TestNormalDist) {
+  const double kMean = 5.0;
+  const double kStdDev = 0.01;
+  const int kNumIters = 100000;
+
+  double sum = 0.0;
+  for (int i = 0; i < kNumIters; ++i) {
+    sum += rng_.Normal(kMean, kStdDev);
+  }
+
+  ASSERT_LE(fabs((sum / static_cast<double>(kNumIters)) - kMean), kStdDev);
+}
+
+// Tests that after a large number of invocations of Next32() and Next64(), we
+// have flipped all the bits we claim we should have.
+//
+// This is a regression test for a bug where we were incorrectly bit-shifting
+// in Next64().
+//
+// Note: Our RNG actually only generates 31 bits of randomness for 32 bit
+// integers. If all bits need to be randomized, callers must use Random::Next64().
+// This test reflects that, and if  we change the RNG algo this test should also change.
+TEST_F(RandomTest, TestUseOfBits) {
+  // For Next32():
+  uint32_t ones32 = numeric_limits<uint32_t>::max();
+  uint32_t zeroes32 = 0;
+  // For Next64():
+  uint64_t ones64 = numeric_limits<uint64_t>::max();
+  uint64_t zeroes64 = 0;
+
+  for (int i = 0; i < 10000000; i++) {
+    uint32_t r32 = rng_.Next32();
+    ones32 &= r32;
+    zeroes32 |= r32;
+
+    uint64_t r64 = rng_.Next64();
+    ones64 &= r64;
+    zeroes64 |= r64;
+  }
+
+  // At the end, we should have flipped 31 and 64 bits, respectively. One
+  // detail of the current RNG impl is that Next32() always returns a number
+  // with MSB set to 0.
+  uint32_t expected_bits_31 = numeric_limits<uint32_t>::max() >> 1;
+  uint64_t expected_bits_64 = numeric_limits<uint64_t>::max();
+
+  ASSERT_EQ(0, ones32);
+  ASSERT_EQ(expected_bits_31, zeroes32);
+  ASSERT_EQ(0, ones64);
+  ASSERT_EQ(expected_bits_64, zeroes64);
+}
+
+TEST_F(RandomTest, TestResetSeed) {
+  rng_.Reset(1);
+  uint64_t first = rng_.Next64();
+  rng_.Reset(1);
+  uint64_t second = rng_.Next64();
+  ASSERT_EQ(first, second);
+}
+
+TEST_F(RandomTest, TestReservoirSample) {
+  // Use a constant seed to avoid flakiness.
+  rng_.Reset(12345);
+
+  vector<int> population;
+  for (int i = 0; i < 100; i++) {
+    population.push_back(i);
+  }
+
+  // Run 1000 trials selecting 5 elements.
+  vector<int> results;
+  vector<int> counts(population.size());
+  unordered_set<int> avoid;
+  for (int trial = 0; trial < 1000; trial++) {
+    rng_.ReservoirSample(population, 5, avoid, &results);
+    for (int result : results) {
+      counts[result]++;
+    }
+  }
+
+  // We expect each element to be selected
+  // 50 times on average, but since it's random, it won't be exact.
+  // However, since we use a constant seed, this test won't be flaky.
+  for (int count : counts) {
+    ASSERT_GE(count, 25);
+    ASSERT_LE(count, 75);
+  }
+
+  // Run again, but avoid some particular entries.
+  avoid.insert(3);
+  avoid.insert(10);
+  avoid.insert(20);
+  counts.assign(100, 0);
+  for (int trial = 0; trial < 1000; trial++) {
+    rng_.ReservoirSample(population, 5, avoid, &results);
+    for (int result : results) {
+      counts[result]++;
+    }
+  }
+
+  // Ensure that we didn't ever pick the avoided elements.
+  ASSERT_EQ(0, counts[3]);
+  ASSERT_EQ(0, counts[10]);
+  ASSERT_EQ(0, counts[20]);
+}
+
+TEST_F(RandomTest, TestReservoirSamplePopulationTooSmall) {
+  vector<int> population;
+  for (int i = 0; i < 10; i++) {
+    population.push_back(i);
+  }
+
+  vector<int> results;
+  unordered_set<int> avoid;
+  rng_.ReservoirSample(population, 20, avoid, &results);
+  ASSERT_EQ(population.size(), results.size());
+  ASSERT_EQ(population, results);
+
+  rng_.ReservoirSample(population, 10, avoid, &results);
+  ASSERT_EQ(population.size(), results.size());
+  ASSERT_EQ(population, results);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/random.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/random.h b/be/src/kudu/util/random.h
new file mode 100644
index 0000000..e31e475
--- /dev/null
+++ b/be/src/kudu/util/random.h
@@ -0,0 +1,252 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#ifndef KUDU_UTIL_RANDOM_H_
+#define KUDU_UTIL_RANDOM_H_
+
+#include <cmath>
+#include <cstdint>
+#include <mutex>
+#include <random>
+#include <vector>
+
+#include "kudu/gutil/casts.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/util/locks.h"
+
+namespace kudu {
+
+namespace random_internal {
+
+static const uint32_t M = 2147483647L;   // 2^31-1
+
+} // namespace random_internal
+
+template<class R>
+class StdUniformRNG;
+
+// A very simple random number generator.  Not especially good at
+// generating truly random bits, but good enough for our needs in this
+// package. This implementation is not thread-safe.
+class Random {
+ private:
+  uint32_t seed_;
+ public:
+  explicit Random(uint32_t s) {
+    Reset(s);
+  }
+
+  // Reset the RNG to the given seed value.
+  void Reset(uint32_t s) {
+    seed_ = s & 0x7fffffffu;
+    // Avoid bad seeds.
+    if (seed_ == 0 || seed_ == random_internal::M) {
+      seed_ = 1;
+    }
+  }
+
+  // Next pseudo-random 32-bit unsigned integer.
+  // FIXME: This currently only generates 31 bits of randomness.
+  // The MSB will always be zero.
+  uint32_t Next() {
+    static const uint64_t A = 16807;  // bits 14, 8, 7, 5, 2, 1, 0
+    // We are computing
+    //       seed_ = (seed_ * A) % M,    where M = 2^31-1
+    //
+    // seed_ must not be zero or M, or else all subsequent computed values
+    // will be zero or M respectively.  For all other values, seed_ will end
+    // up cycling through every number in [1,M-1]
+    uint64_t product = seed_ * A;
+
+    // Compute (product % M) using the fact that ((x << 31) % M) == x.
+    seed_ = static_cast<uint32_t>((product >> 31) + (product & random_internal::M));
+    // The first reduction may overflow by 1 bit, so we may need to
+    // repeat.  mod == M is not possible; using > allows the faster
+    // sign-bit-based test.
+    if (seed_ > random_internal::M) {
+      seed_ -= random_internal::M;
+    }
+    return seed_;
+  }
+
+  // Alias for consistency with Next64
+  uint32_t Next32() { return Next(); }
+
+  // Next pseudo-random 64-bit unsigned integer.
+  uint64_t Next64() {
+    uint64_t large = Next();
+    large <<= 31;
+    large |= Next();
+    // Fill in the highest two MSBs.
+    large |= implicit_cast<uint64_t>(Next32()) << 62;
+    return large;
+  }
+
+  // Returns a uniformly distributed value in the range [0..n-1]
+  // REQUIRES: n > 0
+  uint32_t Uniform(uint32_t n) { return Next() % n; }
+
+  // Alias for consistency with Uniform64
+  uint32_t Uniform32(uint32_t n) { return Uniform(n); }
+
+  // Returns a uniformly distributed 64-bit value in the range [0..n-1]
+  // REQUIRES: n > 0
+  uint64_t Uniform64(uint64_t n) { return Next64() % n; }
+
+  // Randomly returns true ~"1/n" of the time, and false otherwise.
+  // REQUIRES: n > 0
+  bool OneIn(int n) { return (Next() % n) == 0; }
+
+  // Skewed: pick "base" uniformly from range [0,max_log] and then
+  // return "base" random bits.  The effect is to pick a number in the
+  // range [0,2^max_log-1] with exponential bias towards smaller numbers.
+  uint32_t Skewed(int max_log) {
+    return Uniform(1 << Uniform(max_log + 1));
+  }
+
+  // Samples a random number from the given normal distribution.
+  double Normal(double mean, double std_dev);
+
+  // Return a random number between 0.0 and 1.0 inclusive.
+  double NextDoubleFraction() {
+    return Next() / static_cast<double>(random_internal::M + 1.0);
+  }
+
+  // Sample 'k' random elements from the collection 'c' into 'result', taking care not to sample any
+  // elements that are already present in 'avoid'.
+  //
+  // In the case that 'c' has fewer than 'k' elements then all elements in 'c' will be selected.
+  //
+  // 'c' should be an iterable STL collection such as a vector, set, or list.
+  // 'avoid' should be an STL-compatible set.
+  //
+  // The results are not stored in a randomized order: the order of results will
+  // match their order in the input collection.
+  template<class Collection, class Set, class T>
+  void ReservoirSample(const Collection& c, int k, const Set& avoid,
+                       std::vector<T>* result) {
+    result->clear();
+    result->reserve(k);
+    int i = 0;
+    for (const T& elem : c) {
+      if (ContainsKey(avoid, elem)) {
+        continue;
+      }
+      i++;
+      // Fill the reservoir if there is available space.
+      if (result->size() < k) {
+        result->push_back(elem);
+        continue;
+      }
+      // Otherwise replace existing elements with decreasing probability.
+      int j = Uniform(i);
+      if (j < k) {
+        (*result)[j] = elem;
+      }
+    }
+  }
+};
+
+// Thread-safe wrapper around Random.
+class ThreadSafeRandom {
+ public:
+  explicit ThreadSafeRandom(uint32_t s)
+      : random_(s) {
+  }
+
+  void Reset(uint32_t s) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    random_.Reset(s);
+  }
+
+  uint32_t Next() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Next();
+  }
+
+  uint32_t Next32() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Next32();
+  }
+
+  uint64_t Next64() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Next64();
+  }
+
+  uint32_t Uniform(uint32_t n) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Uniform(n);
+  }
+
+  uint32_t Uniform32(uint32_t n) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Uniform32(n);
+  }
+
+  uint64_t Uniform64(uint64_t n) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Uniform64(n);
+  }
+
+  bool OneIn(int n) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.OneIn(n);
+  }
+
+  uint32_t Skewed(int max_log) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Skewed(max_log);
+  }
+
+  double Normal(double mean, double std_dev) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Normal(mean, std_dev);
+  }
+
+  double NextDoubleFraction() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.NextDoubleFraction();
+  }
+
+  template<class Collection, class Set, class T>
+  void ReservoirSample(const Collection& c, int k, const Set& avoid,
+                       std::vector<T>* result) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    random_.ReservoirSample(c, k, avoid, result);
+  }
+
+ private:
+  simple_spinlock lock_;
+  Random random_;
+};
+
+// Wraps either Random or ThreadSafeRandom as a C++ standard library
+// compliant UniformRandomNumberGenerator:
+//   http://en.cppreference.com/w/cpp/concept/UniformRandomNumberGenerator
+template<class R>
+class StdUniformRNG {
+ public:
+  typedef uint32_t result_type;
+
+  explicit StdUniformRNG(R* r) : r_(r) {}
+  uint32_t operator()() {
+    return r_->Next32();
+  }
+  constexpr static uint32_t min() { return 0; }
+  constexpr static uint32_t max() { return (1L << 31) - 1; }
+
+ private:
+  R* r_;
+};
+
+// Defined outside the class to make use of StdUniformRNG above.
+inline double Random::Normal(double mean, double std_dev) {
+  std::normal_distribution<> nd(mean, std_dev);
+  StdUniformRNG<Random> gen(this);
+  return nd(gen);
+}
+
+}  // namespace kudu
+
+#endif  // KUDU_UTIL_RANDOM_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/random_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/random_util-test.cc b/be/src/kudu/util/random_util-test.cc
new file mode 100644
index 0000000..993ef15
--- /dev/null
+++ b/be/src/kudu/util/random_util-test.cc
@@ -0,0 +1,75 @@
+// 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 "kudu/util/random_util.h"
+
+#include <cstring>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/util/random.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+class RandomUtilTest : public KuduTest {
+ protected:
+  RandomUtilTest() : rng_(SeedRandom()) {}
+
+  Random rng_;
+
+  static const int kLenMax = 100;
+  static const int kNumTrials = 100;
+};
+
+namespace {
+
+// Checks string defined at start is set to \0 everywhere but [from, to)
+void CheckEmpty(char* start, int from, int to, int stop) {
+  DCHECK_LE(0, from);
+  DCHECK_LE(from, to);
+  DCHECK_LE(to, stop);
+  for (int j = 0; (j == from ? j = to : j) < stop; ++j) {
+    CHECK_EQ(start[j], '\0') << "Index " << j << " not null after defining"
+                             << "indices [" << from << "," << to << ") of "
+                             << "a nulled string [0," << stop << ").";
+  }
+}
+
+} // anonymous namespace
+
+// Makes sure that RandomString only writes the specified amount
+TEST_F(RandomUtilTest, TestRandomString) {
+  char start[kLenMax];
+
+  for (int i = 0; i < kNumTrials; ++i) {
+    memset(start, '\0', kLenMax);
+    int to = rng_.Uniform(kLenMax + 1);
+    int from = rng_.Uniform(to + 1);
+    RandomString(start + from, to - from, &rng_);
+    CheckEmpty(start, from, to, kLenMax);
+  }
+
+  // Corner case
+  memset(start, '\0', kLenMax);
+  RandomString(start, 0, &rng_);
+  CheckEmpty(start, 0, 0, kLenMax);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/random_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/random_util.cc b/be/src/kudu/util/random_util.cc
new file mode 100644
index 0000000..fa7ef12
--- /dev/null
+++ b/be/src/kudu/util/random_util.cc
@@ -0,0 +1,65 @@
+// 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 "kudu/util/random_util.h"
+
+#include <unistd.h>
+
+#include <cstdlib>
+#include <cstring>
+#include <string>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/random.h"
+
+using std::string;
+
+namespace kudu {
+
+void RandomString(void* dest, size_t n, Random* rng) {
+  size_t i = 0;
+  uint32_t random = rng->Next();
+  char* cdest = static_cast<char*>(dest);
+  static const size_t sz = sizeof(random);
+  if (n >= sz) {
+    for (i = 0; i <= n - sz; i += sz) {
+      memcpy(&cdest[i], &random, sizeof(random));
+      random = rng->Next();
+    }
+  }
+  memcpy(cdest + i, &random, n - i);
+}
+
+string RandomString(size_t n, Random* rng) {
+  faststring s;
+  s.resize(n);
+  RandomString(s.data(), n, rng);
+  return s.ToString();
+}
+
+ATTRIBUTE_NO_SANITIZE_INTEGER
+uint32_t GetRandomSeed32() {
+  uint32_t seed = static_cast<uint32_t>(GetCurrentTimeMicros());
+  seed *= getpid();
+  seed *= Env::Default()->gettid();
+  return seed;
+}
+
+} // namespace kudu


[42/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/outbound_call.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/outbound_call.h b/be/src/kudu/rpc/outbound_call.h
new file mode 100644
index 0000000..8d43891
--- /dev/null
+++ b/be/src/kudu/rpc/outbound_call.h
@@ -0,0 +1,348 @@
+// 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.
+#ifndef KUDU_RPC_CLIENT_CALL_H
+#define KUDU_RPC_CLIENT_CALL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+DECLARE_int32(rpc_inject_cancellation_state);
+
+namespace google {
+namespace protobuf {
+class Message;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+namespace rpc {
+
+class CallResponse;
+class DumpRunningRpcsRequestPB;
+class RpcCallInProgressPB;
+class RpcController;
+class RpcSidecar;
+
+// Tracks the status of a call on the client side.
+//
+// This is an internal-facing class -- clients interact with the
+// RpcController class.
+//
+// This is allocated by the Proxy when a call is first created,
+// then passed to the reactor thread to send on the wire. It's typically
+// kept using a shared_ptr because a call may terminate in any number
+// of different threads, making it tricky to enforce single ownership.
+class OutboundCall {
+ public:
+
+  // Phases of an outbound RPC. Making an outbound RPC might involve establishing
+  // a connection to the remote server first, and the actual call is made only
+  // once the connection to the server is established.
+  enum class Phase {
+    // The phase of connection negotiation between the caller and the callee.
+    CONNECTION_NEGOTIATION,
+
+    // The phase of sending a call over already established connection.
+    REMOTE_CALL,
+  };
+
+  OutboundCall(const ConnectionId& conn_id, const RemoteMethod& remote_method,
+               google::protobuf::Message* response_storage,
+               RpcController* controller, ResponseCallback callback);
+
+  ~OutboundCall();
+
+  // Serialize the given request PB into this call's internal storage, and assume
+  // ownership of any sidecars that should accompany this request.
+  //
+  // Because the request data is fully serialized by this call, 'req' may be subsequently
+  // mutated with no ill effects.
+  void SetRequestPayload(const google::protobuf::Message& req,
+      std::vector<std::unique_ptr<RpcSidecar>>&& sidecars);
+
+  // Assign the call ID for this call. This is called from the reactor
+  // thread once a connection has been assigned. Must only be called once.
+  void set_call_id(int32_t call_id) {
+    DCHECK_EQ(header_.call_id(), kInvalidCallId) << "Already has a call ID";
+    header_.set_call_id(call_id);
+  }
+
+  // Serialize the call for the wire. Requires that SetRequestPayload()
+  // is called first. This is called from the Reactor thread.
+  // Returns the number of slices in the serialized call.
+  size_t SerializeTo(TransferPayload* slices);
+
+  // Mark in the call that cancellation has been requested. If the call hasn't yet
+  // started sending or has finished sending the RPC request but is waiting for a
+  // response, cancel the RPC right away. Otherwise, wait until the RPC has finished
+  // sending before cancelling it. If the call is finished, it's a no-op.
+  // REQUIRES: must be called from the reactor thread.
+  void Cancel();
+
+  // Callback after the call has been put on the outbound connection queue.
+  void SetQueued();
+
+  // Update the call state to show that the request has started being sent
+  // on the socket.
+  void SetSending();
+
+  // Update the call state to show that the request has been sent.
+  void SetSent();
+
+  // Mark the call as failed. This also triggers the callback to notify
+  // the caller. If the call failed due to a remote error, then err_pb
+  // should be set to the error returned by the remote server.
+  void SetFailed(Status status,
+                 Phase phase = Phase::REMOTE_CALL,
+                 std::unique_ptr<ErrorStatusPB> err_pb = nullptr);
+
+  // Mark the call as timed out. This also triggers the callback to notify
+  // the caller.
+  void SetTimedOut(Phase phase);
+  bool IsTimedOut() const;
+
+  bool IsNegotiationError() const;
+
+  bool IsCancelled() const;
+
+  // Is the call finished?
+  bool IsFinished() const;
+
+  // Fill in the call response.
+  void SetResponse(gscoped_ptr<CallResponse> resp);
+
+  const std::set<RpcFeatureFlag>& required_rpc_features() const {
+    return required_rpc_features_;
+  }
+
+  std::string ToString() const;
+
+  void DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp);
+
+  ////////////////////////////////////////////////////////////
+  // Getters
+  ////////////////////////////////////////////////////////////
+
+  const ConnectionId& conn_id() const { return conn_id_; }
+  const RemoteMethod& remote_method() const { return remote_method_; }
+  const ResponseCallback &callback() const { return callback_; }
+  RpcController* controller() { return controller_; }
+  const RpcController* controller() const { return controller_; }
+
+  // Return true if a call ID has been assigned to this call.
+  bool call_id_assigned() const {
+    return header_.call_id() != kInvalidCallId;
+  }
+
+  int32_t call_id() const {
+    DCHECK(call_id_assigned());
+    return header_.call_id();
+  }
+
+  // Returns true if cancellation has been requested. Must be called from
+  // reactor thread.
+  bool cancellation_requested() const {
+    return cancellation_requested_;
+  }
+
+  // Test function which returns true if a cancellation request should be injected
+  // at the current state.
+  bool ShouldInjectCancellation() const {
+    return FLAGS_rpc_inject_cancellation_state != -1 &&
+        FLAGS_rpc_inject_cancellation_state == state();
+  }
+
+ private:
+  friend class RpcController;
+  FRIEND_TEST(TestRpc, TestCancellation);
+
+  // Various states the call propagates through.
+  // NB: if adding another state, be sure to update OutboundCall::IsFinished()
+  // and OutboundCall::StateName(State state) as well.
+  enum State {
+    READY = 0,
+    ON_OUTBOUND_QUEUE,
+    SENDING,
+    SENT,
+    NEGOTIATION_TIMED_OUT,
+    TIMED_OUT,
+    CANCELLED,
+    FINISHED_NEGOTIATION_ERROR,
+    FINISHED_ERROR,
+    FINISHED_SUCCESS
+  };
+
+  static std::string StateName(State state);
+
+  // Mark the call as cancelled. This also invokes the callback to notify the caller.
+  void SetCancelled();
+
+  void set_state(State new_state);
+  State state() const;
+
+  // Same as set_state, but requires that the caller already holds
+  // lock_
+  void set_state_unlocked(State new_state);
+
+  // return current status
+  Status status() const;
+
+  // Time when the call was first initiatied.
+  MonoTime start_time_;
+
+  // Return the error protobuf, if a remote error occurred.
+  // This will only be non-NULL if status().IsRemoteError().
+  const ErrorStatusPB* error_pb() const;
+
+  // Lock for state_ status_, error_pb_ fields, since they
+  // may be mutated by the reactor thread while the client thread
+  // reads them.
+  mutable simple_spinlock lock_;
+  State state_;
+  Status status_;
+  std::unique_ptr<ErrorStatusPB> error_pb_;
+
+  // Call the user-provided callback. Note that entries in 'sidecars_' are cleared
+  // prior to invoking the callback so the client can assume that the call doesn't
+  // hold references to outbound sidecars.
+  void CallCallback();
+
+  // The RPC header.
+  // Parts of this (eg the call ID) are only assigned once this call has been
+  // passed to the reactor thread and assigned a connection.
+  RequestHeader header_;
+
+  // The remote method being called.
+  RemoteMethod remote_method_;
+
+  // RPC-system features required to send this call.
+  std::set<RpcFeatureFlag> required_rpc_features_;
+
+  const ConnectionId conn_id_;
+  ResponseCallback callback_;
+  RpcController* controller_;
+
+  // Pointer for the protobuf where the response should be written.
+  google::protobuf::Message* response_;
+
+  // Buffers for storing segments of the wire-format request.
+  faststring header_buf_;
+  faststring request_buf_;
+
+  // Once a response has been received for this call, contains that response.
+  // Otherwise NULL.
+  gscoped_ptr<CallResponse> call_response_;
+
+  // All sidecars to be sent with this call.
+  std::vector<std::unique_ptr<RpcSidecar>> sidecars_;
+
+  // Total size in bytes of all sidecars in 'sidecars_'. Set in SetRequestPayload().
+  // This cannot exceed TransferLimits::kMaxTotalSidecarBytes.
+  int32_t sidecar_byte_size_ = -1;
+
+  // True if cancellation was requested on this call.
+  bool cancellation_requested_;
+
+  DISALLOW_COPY_AND_ASSIGN(OutboundCall);
+};
+
+// A response to a call, on the client side.
+// Upon receiving a response, this is allocated in the reactor thread and filled
+// into the OutboundCall instance via OutboundCall::SetResponse.
+//
+// This may either be a success or error response.
+//
+// This class takes care of separating out the distinct payload slices sent
+// over.
+class CallResponse {
+ public:
+  CallResponse();
+
+  // Parse the response received from a call. This must be called before any
+  // other methods on this object.
+  Status ParseFrom(gscoped_ptr<InboundTransfer> transfer);
+
+  // Return true if the call succeeded.
+  bool is_success() const {
+    DCHECK(parsed_);
+    return !header_.is_error();
+  }
+
+  // Return the call ID that this response is related to.
+  int32_t call_id() const {
+    DCHECK(parsed_);
+    return header_.call_id();
+  }
+
+  // Return the serialized response data. This is just the response "body" --
+  // either a serialized ErrorStatusPB, or the serialized user response protobuf.
+  const Slice &serialized_response() const {
+    DCHECK(parsed_);
+    return serialized_response_;
+  }
+
+  // See RpcController::GetSidecar()
+  Status GetSidecar(int idx, Slice* sidecar) const;
+
+ private:
+  // True once ParseFrom() is called.
+  bool parsed_;
+
+  // The parsed header.
+  ResponseHeader header_;
+
+  // The slice of data for the encoded protobuf response.
+  // This slice refers to memory allocated by transfer_
+  Slice serialized_response_;
+
+  // Slices of data for rpc sidecars. They point into memory owned by transfer_.
+  Slice sidecar_slices_[TransferLimits::kMaxSidecars];
+
+  // The incoming transfer data - retained because serialized_response_
+  // and sidecar_slices_ refer into its data.
+  gscoped_ptr<InboundTransfer> transfer_;
+
+  DISALLOW_COPY_AND_ASSIGN(CallResponse);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/periodic-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/periodic-test.cc b/be/src/kudu/rpc/periodic-test.cc
new file mode 100644
index 0000000..cd793bd
--- /dev/null
+++ b/be/src/kudu/rpc/periodic-test.cc
@@ -0,0 +1,295 @@
+// 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 <atomic>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/periodic.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::atomic;
+using std::shared_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+class PeriodicTimerTest : public KuduTest {
+ public:
+  PeriodicTimerTest()
+      : period_ms_(200) {}
+
+ protected:
+  const int64_t period_ms_;
+};
+
+class JitteredPeriodicTimerTest : public PeriodicTimerTest,
+                                  public ::testing::WithParamInterface<double> {
+ public:
+  // In TSAN builds it takes a long time to de-schedule a thread. Also,
+  // the actual time that thread spends sleeping in SleepFor() scenarios
+  // might be much longer than requested. Setting the task period to be long
+  // enough allows for more stable behavior of the test, so no flakiness
+  // is observed even under substantial load. Otherwise it would be necessary
+  // to introduce additional logic to verify that the actual timings satisfy
+  // the implicit constraints of the test scenarios below.
+  JitteredPeriodicTimerTest()
+      : counter_(0) {
+  }
+
+  virtual void SetUp() override {
+    PeriodicTimerTest::SetUp();
+
+    MessengerBuilder builder("test");
+    ASSERT_OK(builder.Build(&messenger_));
+
+    timer_ = PeriodicTimer::Create(messenger_,
+                                   [&] { counter_++; },
+                                   MonoDelta::FromMilliseconds(period_ms_),
+                                   GetOptions());
+  }
+
+  virtual void TearDown() override {
+    // Ensure that the reactor threads are fully quiesced (and thus no timer
+    // callbacks are running) by the time 'counter_' is destroyed.
+    messenger_->Shutdown();
+
+    KuduTest::TearDown();
+  }
+
+ protected:
+
+  virtual PeriodicTimer::Options GetOptions() {
+    PeriodicTimer::Options opts;
+    opts.jitter_pct = GetParam();
+    return opts;
+  }
+
+  atomic<int64_t> counter_;
+  shared_ptr<Messenger> messenger_;
+  shared_ptr<PeriodicTimer> timer_;
+};
+
+INSTANTIATE_TEST_CASE_P(AllJitterModes,
+                        JitteredPeriodicTimerTest,
+                        ::testing::Values(0.0, 0.25));
+
+TEST_P(JitteredPeriodicTimerTest, TestStartStop) {
+  // Before the timer starts, the counter's value should not change.
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+  ASSERT_EQ(0, counter_);
+
+  // Once started, it should increase (exactly how much depends on load and the
+  // underlying OS scheduler).
+  timer_->Start();
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+  ASSERT_EVENTUALLY([&]{
+    ASSERT_GT(counter_, 0);
+  });
+
+  // After stopping the timer, the value should either remain the same or
+  // increment once (if Stop() raced with a scheduled task).
+  timer_->Stop();
+  int64_t v = counter_;
+  messenger_->Shutdown();
+  ASSERT_TRUE(counter_ == v ||
+              counter_ == v + 1);
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestReset) {
+  timer_->Start();
+  MonoTime start_time = MonoTime::Now();
+
+  // Loop for a little while, resetting the timer's period over and over. As a
+  // result, the timer should never fire.
+  while (true) {
+    MonoTime now = MonoTime::Now();
+    if (now - start_time > MonoDelta::FromMilliseconds(period_ms_ * 5)) {
+      break;
+    }
+    timer_->Snooze();
+    ASSERT_EQ(0, counter_);
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestResetWithDelta) {
+  timer_->Start();
+  timer_->Snooze(MonoDelta::FromMilliseconds(period_ms_ * 5));
+
+  // One period later, the counter still hasn't incremented...
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_));
+  ASSERT_EQ(0, counter_);
+
+  // ...but it will increment eventually.
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GT(counter_, 0);
+  });
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestStartWithDelta) {
+  timer_->Start(MonoDelta::FromMilliseconds(period_ms_ * 5));
+
+  // One period later, the counter still hasn't incremented...
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_));
+  ASSERT_EQ(0, counter_);
+
+  // ...but it will increment eventually.
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GT(counter_, 0);
+  });
+}
+
+TEST_F(PeriodicTimerTest, TestCallbackRestartsTimer) {
+  const int64_t kPeriods = 10;
+
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test").Build(&messenger));
+
+  // Create a timer that restarts itself from within its functor.
+  PeriodicTimer::Options opts;
+  opts.jitter_pct = 0.0; // don't need jittering
+  shared_ptr<PeriodicTimer> timer = PeriodicTimer::Create(
+      messenger,
+      [&] {
+        timer->Stop();
+        timer->Start();
+      },
+      MonoDelta::FromMilliseconds(period_ms_),
+      std::move(opts));
+
+  // Run the timer for a fixed amount of time.
+  timer->Start();
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * kPeriods));
+  timer->Stop();
+
+  // Although the timer is restarted by its functor, its overall period should
+  // remain more or less the same (since the period expired just as the functor
+  // ran). As such, we should see no more than three callbacks per period:
+  // one to start scheduling the callback loop, one when it fires, and one more
+  // after it has been replaced by a new callback loop.
+  ASSERT_LE(timer->NumCallbacksForTests(), kPeriods * 3);
+}
+
+class JitteredOneShotPeriodicTimerTest : public JitteredPeriodicTimerTest {
+ protected:
+  virtual PeriodicTimer::Options GetOptions() override {
+    PeriodicTimer::Options opts;
+    opts.jitter_pct = GetParam();
+    opts.one_shot = true;
+    return opts;
+  }
+};
+
+INSTANTIATE_TEST_CASE_P(AllJitterModes,
+                        JitteredOneShotPeriodicTimerTest,
+                        ::testing::Values(0.0, 0.25));
+
+TEST_P(JitteredOneShotPeriodicTimerTest, TestBasics) {
+  // Kick off the one-shot timer a few times.
+  for (int i = 0; i < 3; i++) {
+    ASSERT_EQ(i, counter_);
+
+    // Eventually the task will run.
+    timer_->Start();
+    ASSERT_EVENTUALLY([&](){
+      ASSERT_EQ(i + 1, counter_);
+    });
+
+    // Even if we explicitly wait another few periods, the counter value
+    // shouldn't change.
+    SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+    ASSERT_EQ(i + 1, counter_);
+  }
+}
+
+TEST_F(PeriodicTimerTest, TestCallbackRestartsOneShotTimer) {
+  atomic<int64_t> counter(0);
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test")
+            .Build(&messenger));
+
+  // Create a timer that restarts itself from within its functor.
+  PeriodicTimer::Options opts;
+  opts.jitter_pct = 0.0; // don't need jittering
+  opts.one_shot = true;
+  shared_ptr<PeriodicTimer> timer = PeriodicTimer::Create(
+      messenger,
+      [&] {
+        counter++;
+        timer->Start();
+      },
+      MonoDelta::FromMilliseconds(period_ms_),
+      std::move(opts));
+
+  // Because the timer restarts itself every time the functor runs, we
+  // should see the counter value increase with each period.
+  timer->Start();
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GE(counter, 5);
+  });
+
+  // Ensure that the reactor threads are fully quiesced (and thus no timer
+  // callbacks are running) by the time 'counter' is destroyed.
+  messenger->Shutdown();
+}
+
+TEST_F(PeriodicTimerTest, TestPerformance) {
+  const int kNumTimers = 1000;
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test")
+            .set_num_reactors(1)
+            .Build(&messenger));
+  SCOPED_CLEANUP({ messenger->Shutdown(); });
+
+  vector<shared_ptr<PeriodicTimer>> timers;
+  for (int i = 0; i < kNumTimers; i++) {
+    timers.emplace_back(PeriodicTimer::Create(
+        messenger,
+        [&] {}, // No-op.
+        MonoDelta::FromMilliseconds(10)));
+    timers.back()->Start();
+  }
+
+  Stopwatch sw(Stopwatch::ALL_THREADS);
+  sw.start();
+  SleepFor(MonoDelta::FromSeconds(1));
+  sw.stop();
+  LOG(INFO) << "User CPU for running " << kNumTimers << " timers for 1 second: "
+            << sw.elapsed().user_cpu_seconds() << "s";
+
+  for (auto& t : timers) {
+    t->Stop();
+  }
+
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/periodic.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/periodic.cc b/be/src/kudu/rpc/periodic.cc
new file mode 100644
index 0000000..9ab9ce9
--- /dev/null
+++ b/be/src/kudu/rpc/periodic.cc
@@ -0,0 +1,219 @@
+// 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 "kudu/rpc/periodic.h"
+
+#include <algorithm>
+#include <memory>
+#include <mutex>
+
+#include <boost/function.hpp>
+#include <glog/logging.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/status.h"
+
+using std::shared_ptr;
+using std::weak_ptr;
+
+namespace kudu {
+namespace rpc {
+
+PeriodicTimer::Options::Options()
+    : jitter_pct(0.25),
+      one_shot(false) {
+}
+
+shared_ptr<PeriodicTimer> PeriodicTimer::Create(
+    shared_ptr<Messenger> messenger,
+    RunTaskFunctor functor,
+    MonoDelta period,
+    Options options) {
+  return PeriodicTimer::make_shared(
+      std::move(messenger), std::move(functor), period, options);
+}
+
+PeriodicTimer::PeriodicTimer(
+    shared_ptr<Messenger> messenger,
+    RunTaskFunctor functor,
+    MonoDelta period,
+    Options options)
+    : messenger_(std::move(messenger)),
+      functor_(std::move(functor)),
+      period_(period),
+      options_(options),
+      rng_(GetRandomSeed32()),
+      current_callback_generation_(0),
+      num_callbacks_for_tests_(0),
+      started_(false) {
+  DCHECK_GE(options_.jitter_pct, 0);
+  DCHECK_LE(options_.jitter_pct, 1);
+}
+
+PeriodicTimer::~PeriodicTimer() {
+  Stop();
+}
+
+void PeriodicTimer::Start(boost::optional<MonoDelta> next_task_delta) {
+  std::unique_lock<simple_spinlock> l(lock_);
+  if (!started_) {
+    started_ = true;
+    SnoozeUnlocked(std::move(next_task_delta));
+    int new_callback_generation = ++current_callback_generation_;
+
+    // Invoke Callback() with the lock released.
+    l.unlock();
+    Callback(new_callback_generation);
+  }
+}
+
+void PeriodicTimer::Stop() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  StopUnlocked();
+}
+
+void PeriodicTimer::StopUnlocked() {
+  DCHECK(lock_.is_locked());
+  started_ = false;
+}
+
+void PeriodicTimer::Snooze(boost::optional<MonoDelta> next_task_delta) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  SnoozeUnlocked(std::move(next_task_delta));
+}
+
+void PeriodicTimer::SnoozeUnlocked(boost::optional<MonoDelta> next_task_delta) {
+  DCHECK(lock_.is_locked());
+  if (!started_) {
+    return;
+  }
+
+  if (!next_task_delta) {
+    // Given jitter percentage J and period P, this yields a delay somewhere
+    // between (1-J)*P and (1+J)*P.
+    next_task_delta = MonoDelta::FromMilliseconds(
+        GetMinimumPeriod().ToMilliseconds() +
+        rng_.NextDoubleFraction() *
+        options_.jitter_pct *
+        (2 * period_.ToMilliseconds()));
+  }
+  next_task_time_ = MonoTime::Now() + *next_task_delta;
+}
+
+bool PeriodicTimer::started() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return started_;
+}
+
+MonoDelta PeriodicTimer::GetMinimumPeriod() {
+  // Given jitter percentage J and period P, this returns (1-J)*P, which is
+  // the lowest possible jittered value.
+  return MonoDelta::FromMilliseconds((1.0 - options_.jitter_pct) *
+                                     period_.ToMilliseconds());
+}
+
+int64_t PeriodicTimer::NumCallbacksForTests() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return num_callbacks_for_tests_;
+}
+
+void PeriodicTimer::Callback(int64_t my_callback_generation) {
+  // To simplify the implementation, a timer may have only one outstanding
+  // callback scheduled at a time. This means that once the callback is
+  // scheduled, the timer's task cannot run any earlier than whenever the
+  // callback runs. Thus, the delay used when scheduling the callback dictates
+  // the lowest possible value of 'next_task_delta' that Snooze() can honor.
+  //
+  // If the callback's delay is very low, Snooze() can honor a low
+  // 'next_task_delta', but the callback will run often and burn more CPU
+  // cycles. If the delay is very high, the timer will be more efficient but
+  // the granularity for 'next_task_delta' will rise accordingly.
+  //
+  // As a "happy medium" we use GetMinimumPeriod() as the delay. This ensures
+  // that a no-arg Snooze() on a jittered timer will always be honored, and as
+  // long as the caller passes a value of at least GetMinimumPeriod() to
+  // Snooze(), that too will be honored.
+  MonoDelta delay = GetMinimumPeriod();
+  bool run_task = false;
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    num_callbacks_for_tests_++;
+
+    // If the timer was stopped, exit.
+    if (!started_) {
+      return;
+    }
+
+    // If there's a new callback loop in town, exit.
+    //
+    // We could check again just before calling Messenger::ScheduleOnReactor()
+    // (in case someone else restarted the timer while the functor ran, or in
+    // case the functor itself restarted the timer), but there's no real reason
+    // to do so: the very next iteration of this callback loop will wind up here
+    // and exit.
+    if (current_callback_generation_ > my_callback_generation) {
+      return;
+    }
+
+    MonoTime now = MonoTime::Now();
+    if (now < next_task_time_) {
+      // It's not yet time to run the task. Reduce the scheduled delay if
+      // enough time has elapsed, but don't increase it.
+      delay = std::min(delay, next_task_time_ - now);
+    } else {
+      // It's time to run the task. Although the next task time is reset now,
+      // it may be reset again by virtue of running the task itself.
+      run_task = true;
+
+      if (options_.one_shot) {
+        // Stop the timer first, in case the task wants to restart it.
+        StopUnlocked();
+      }
+    }
+  }
+
+  if (run_task) {
+    functor_();
+
+    if (options_.one_shot) {
+      // The task was run; exit the loop. Even if the task restarted the timer,
+      // that will have started a new callback loop, so exiting here is always
+      // the correct thing to do.
+      return;
+    }
+    Snooze();
+  }
+
+  // Capture a weak_ptr reference into the submitted functor so that we can
+  // safely handle the functor outliving its timer.
+  weak_ptr<PeriodicTimer> w = shared_from_this();
+  messenger_->ScheduleOnReactor([w, my_callback_generation](const Status& s) {
+    if (!s.ok()) {
+      // The reactor was shut down.
+      return;
+    }
+    if (auto timer = w.lock()) {
+      timer->Callback(my_callback_generation);
+    }
+  }, delay);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/periodic.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/periodic.h b/be/src/kudu/rpc/periodic.h
new file mode 100644
index 0000000..592cbd5
--- /dev/null
+++ b/be/src/kudu/rpc/periodic.h
@@ -0,0 +1,215 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <functional>
+#include <memory>
+
+#include <boost/optional/optional.hpp>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/make_shared.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+
+namespace kudu {
+namespace rpc {
+
+class Messenger;
+
+// Repeatedly runs a task on a fixed period.
+//
+// PeriodicTimer's periodicity is maintained via tail recursive calls to
+// Messenger::ScheduleOnReactor(). Every time the scheduled callback is
+// invoked, it checks the current time, updates some internal bookkeeping,
+// runs the user's task if the time is right, and makes another call to
+// Messenger::ScheduleOnReactor() to run itself again in the future. This
+// looping behavior is called a "callback loop".
+//
+// Every time Stop() and then Start() (or just Start(), if this is the first
+// such call) are invoked, PeriodicTimer will kick off a new callback loop. If
+// there was an old loop, it remains intact until its scheduled callback runs,
+// at which point it will detect that a new loop was created and exit.
+//
+// The use of Messenger::ScheduleOnReactor() is merely for convenience;
+// PeriodicTimer could also be built on libev, a hashed wheel timer, o
+// something equivalent.
+//
+// PeriodicTimers have shared ownership, but that's largely an implementation
+// detail to support asynchronous stopping. Users can treat them as exclusively
+// owned (though care must be taken when writing the task functor; see Stop()
+// for more details).
+//
+// TODO(adar): eventually we should build directly on libev as it supports
+// timer cancelation, which would allow us to implement synchronous Stop(), use
+// exclusive ownership, and remove the restriction that the delta passed
+// into Snooze() be greater than GetMinimumPeriod().
+class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer>,
+                      public enable_make_shared<PeriodicTimer> {
+ public:
+  typedef std::function<void(void)> RunTaskFunctor;
+
+  struct Options {
+    Options();
+
+    // Defines the percentage of the period that will be jittered up or down
+    // randomly. Together with the period, the periodicity of the timer will
+    // vary between (1-J)*P and (1+J)*P.
+    //
+    // Must be between 0 and 1.
+    //
+    // If not set, defaults to 0.25.
+    double jitter_pct;
+
+    // The timer will automatically stop after running the user's task.
+    //
+    // Just as with a normal timer, Snooze() will postpone the running of the
+    // task, and Stop() will cancel the task outright. Unlike a normal timer,
+    // both operations will no-op if the timer has already fired.
+    //
+    // If not set, defaults to false.
+    bool one_shot;
+  };
+
+  // Creates a new PeriodicTimer.
+  //
+  // A ref is taken on 'messenger', which is used for scheduling callbacks.
+  //
+  // 'functor' defines the user's task and is owned for the lifetime of the
+  // PeriodicTimer. The task will run on the messenger's reactor threads so it
+  // should do very little work (i.e. no I/O).
+  //
+  // 'period' defines the period between tasks.
+  //
+  // 'options' allows additional (optional) customization of the timer.
+  static std::shared_ptr<PeriodicTimer> Create(
+      std::shared_ptr<Messenger> messenger,
+      RunTaskFunctor functor,
+      MonoDelta period,
+      Options options = {});
+
+  ~PeriodicTimer();
+
+  // Starts the timer.
+  //
+  // The timer's task will run in accordance with the period and jitter mode
+  // provided during timer construction.
+  //
+  // If 'next_task_delta' is set, it is used verbatim as the delay for the very
+  // first task, with the configured period and jitter mode only applying to
+  // subsequent tasks.
+  //
+  // Does nothing if the timer was already started.
+  void Start(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Snoozes the timer for one period.
+  //
+  // If 'next_task_delta' is set, it is used verbatim as the delay for the next
+  // task. Subsequent tasks will revert to the timer's regular period. The
+  // value of 'next_task_delta' must be greater than GetMinimumPeriod();
+  // otherwise the task is not guaranteed to run in a timely manner.
+  //
+  // Note: Snooze() is not additive. That is, if called at time X and again at
+  // time X + P/2, the timer is snoozed until X+P/2+P, not X+2P.
+  //
+  // Does nothing if the timer is stopped.
+  void Snooze(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Stops the timer.
+  //
+  // Stopping is asynchronous; that is, it is still possible for the task to
+  // run after Stop() returns. Because of this, the task's functor should be
+  // written to do nothing if objects it depends on have been destroyed.
+  //
+  // Does nothing if the timer is already stopped.
+  void Stop();
+
+  // Returns true iff the timer has been started.
+  bool started() const;
+
+ protected:
+  PeriodicTimer(std::shared_ptr<Messenger> messenger,
+                RunTaskFunctor functor,
+                MonoDelta period,
+                Options options);
+
+ private:
+  FRIEND_TEST(PeriodicTimerTest, TestCallbackRestartsTimer);
+  // Calculate the minimum period for the timer, which varies depending on
+  // 'jitter_pct_' and the output of the PRNG.
+  MonoDelta GetMinimumPeriod();
+
+  // Called by Messenger::ScheduleOnReactor when the timer fires.
+  // 'my_callback_generation' is the callback generation assigned to this loop
+  // when it was constructed.
+  void Callback(int64_t my_callback_generation);
+
+  // Like Stop() but must be called with 'lock_' held.
+  void StopUnlocked();
+
+  // Like Snooze() but must be called with 'lock_' held.
+  void SnoozeUnlocked(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Returns the number of times that Callback() has been called by this timer.
+  //
+  // Should only be used for tests!
+  int64_t NumCallbacksForTests() const;
+
+  // Schedules invocations of Callback() in the future.
+  std::shared_ptr<Messenger> messenger_;
+
+  // User-defined task functor.
+  RunTaskFunctor functor_;
+
+  // User-specified task period.
+  const MonoDelta period_;
+
+  // User-specified options.
+  const Options options_;
+
+  // Protects all mutable state below.
+  mutable simple_spinlock lock_;
+
+  // PRNG used when generating jitter.
+  Random rng_;
+
+  // The next time at which the task's functor should be run.
+  MonoTime next_task_time_;
+
+  // The most recent callback generation.
+  //
+  // When started, a callback loop is assigned a generation, which it remembers
+  // for its entire lifespan. If 'current_callback_generation_' exceeds the
+  // loop's assigned generation, that means another loop has been created and
+  // the (now old) loop should exit.
+  int64_t current_callback_generation_;
+
+  // The number of times that Callback() has been invoked.
+  int64_t num_callbacks_for_tests_;
+
+  // Whether the timer is running or not.
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(PeriodicTimer);
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/protoc-gen-krpc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/protoc-gen-krpc.cc b/be/src/kudu/rpc/protoc-gen-krpc.cc
new file mode 100644
index 0000000..cc25f67
--- /dev/null
+++ b/be/src/kudu/rpc/protoc-gen-krpc.cc
@@ -0,0 +1,691 @@
+// 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.
+
+////////////////////////////////////////////////////////////////////////////////
+// Example usage:
+// protoc --plugin=protoc-gen-krpc --krpc_out . --proto_path . <file>.proto
+////////////////////////////////////////////////////////////////////////////////
+
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <google/protobuf/compiler/code_generator.h>
+#include <google/protobuf/compiler/plugin.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
+#include <google/protobuf/io/printer.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
+
+using boost::optional;
+using google::protobuf::FileDescriptor;
+using google::protobuf::io::Printer;
+using google::protobuf::MethodDescriptor;
+using google::protobuf::ServiceDescriptor;
+using std::map;
+using std::shared_ptr;
+using std::set;
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+namespace {
+
+// Return the name of the authorization method specified for this
+// RPC method, or boost::none if none is specified.
+//
+// This handles fallback to the service-wide default.
+optional<string> GetAuthzMethod(const MethodDescriptor& method) {
+  if (method.options().HasExtension(authz_method)) {
+    return method.options().GetExtension(authz_method);
+  }
+  if (method.service()->options().HasExtension(default_authz_method)) {
+    return method.service()->options().GetExtension(default_authz_method);
+  }
+  return boost::none;
+}
+
+} // anonymous namespace
+
+class Substituter {
+ public:
+  virtual ~Substituter() {}
+  virtual void InitSubstitutionMap(map<string, string> *map) const = 0;
+};
+
+// NameInfo contains information about the output names.
+class FileSubstitutions : public Substituter {
+ public:
+  static const std::string kProtoExtension;
+
+  Status Init(const FileDescriptor *file) {
+    const string& path = file->name();
+    map_["path"] = path;
+
+    // Initialize path_
+    // If path = /foo/bar/baz_stuff.proto, path_ = /foo/bar/baz_stuff
+    if (!TryStripSuffixString(path, kProtoExtension, &path_no_extension_)) {
+      return Status::InvalidArgument("file name " + path +
+                                     " did not end in " + kProtoExtension);
+    }
+    map_["path_no_extension"] = path_no_extension_;
+
+    // If path = /foo/bar/baz_stuff.proto, base_ = baz_stuff
+    string base;
+    GetBaseName(path_no_extension_, &base);
+    map_["base"] = base;
+
+    // If path = /foo/bar/baz_stuff.proto, camel_case_ = BazStuff
+    string camel_case;
+    SnakeToCamelCase(base, &camel_case);
+    map_["camel_case"] = camel_case;
+
+    // If path = /foo/bar/baz_stuff.proto, upper_case_ = BAZ_STUFF
+    string upper_case;
+    ToUpperCase(base, &upper_case);
+    map_["upper_case"] = upper_case;
+
+    map_["open_namespace"] = GenerateOpenNamespace(file->package());
+    map_["close_namespace"] = GenerateCloseNamespace(file->package());
+
+    return Status::OK();
+  }
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+    typedef std::map<string, string>::value_type kv_pair;
+    for (const kv_pair &pair : map_) {
+      (*map)[pair.first] = pair.second;
+    }
+  }
+
+  std::string service_header() const {
+    return path_no_extension_ + ".service.h";
+  }
+
+  std::string service() const {
+    return path_no_extension_ + ".service.cc";
+  }
+
+  std::string proxy_header() const {
+    return path_no_extension_ + ".proxy.h";
+  }
+
+  std::string proxy() const {
+    return path_no_extension_ + ".proxy.cc";
+  }
+
+ private:
+  // Extract the last filename component.
+  static void GetBaseName(const string &path,
+                          string *base) {
+    size_t last_slash = path.find_last_of('/');
+    if (last_slash != string::npos) {
+      *base = path.substr(last_slash + 1);
+    } else {
+      *base = path;
+    }
+  }
+
+  static string GenerateOpenNamespace(const string &str) {
+    vector<string> components = strings::Split(str, ".");
+    string out;
+    for (const string &c : components) {
+      out.append("namespace ").append(c).append(" {\n");
+    }
+    return out;
+  }
+
+  static string GenerateCloseNamespace(const string &str) {
+    vector<string> components = strings::Split(str, ".");
+    string out;
+    for (auto c = components.crbegin(); c != components.crend(); c++) {
+      out.append("} // namespace ").append(*c).append("\n");
+    }
+    return out;
+  }
+
+  std::string path_no_extension_;
+  map<string, string> map_;
+};
+
+const std::string FileSubstitutions::kProtoExtension(".proto");
+
+class MethodSubstitutions : public Substituter {
+ public:
+  explicit MethodSubstitutions(const MethodDescriptor *method)
+    : method_(method) {
+  }
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+
+    (*map)["rpc_name"] = method_->name();
+    (*map)["rpc_full_name"] = method_->full_name();
+    (*map)["rpc_full_name_plainchars"] =
+        StringReplace(method_->full_name(), ".", "_", true);
+    (*map)["request"] =
+        ReplaceNamespaceDelimiters(
+            StripNamespaceIfPossible(method_->service()->full_name(),
+                                     method_->input_type()->full_name()));
+    (*map)["response"] =
+        ReplaceNamespaceDelimiters(
+            StripNamespaceIfPossible(method_->service()->full_name(),
+                                     method_->output_type()->full_name()));
+    (*map)["metric_enum_key"] = strings::Substitute("kMetricIndex$0", method_->name());
+    bool track_result = static_cast<bool>(method_->options().GetExtension(track_rpc_result));
+    (*map)["track_result"] = track_result ? " true" : "false";
+    (*map)["authz_method"] = GetAuthzMethod(*method_).get_value_or("AuthorizeAllowAll");
+  }
+
+  // Strips the package from method arguments if they are in the same package as
+  // the service, otherwise leaves them so that we can have fully qualified
+  // namespaces for method arguments.
+  static std::string StripNamespaceIfPossible(const std::string& service_full_name,
+                                              const std::string& arg_full_name) {
+    StringPiece service_package(service_full_name);
+    if (!service_package.contains(".")) {
+      return arg_full_name;
+    }
+    // remove the service name so that we are left with only the package, including
+    // the last '.' so that we account for different packages with the same prefix.
+    service_package.remove_suffix(service_package.length() -
+                                  service_package.find_last_of(".") - 1);
+
+    StringPiece argfqn(arg_full_name);
+    if (argfqn.starts_with(service_package)) {
+      argfqn.remove_prefix(argfqn.find_last_of(".") + 1);
+    }
+    return argfqn.ToString();
+  }
+
+  static std::string ReplaceNamespaceDelimiters(const std::string& arg_full_name) {
+    return JoinStrings(strings::Split(arg_full_name, "."), "::");
+  }
+
+ private:
+  const MethodDescriptor *method_;
+};
+
+class ServiceSubstitutions : public Substituter {
+ public:
+  explicit ServiceSubstitutions(const ServiceDescriptor *service)
+    : service_(service)
+  {}
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+    (*map)["service_name"] = service_->name();
+    (*map)["full_service_name"] = service_->full_name();
+    (*map)["service_method_count"] = SimpleItoa(service_->method_count());
+
+    // TODO: upgrade to protobuf 2.5.x and attach service comments
+    // to the generated service classes using the SourceLocation API.
+  }
+
+ private:
+  const ServiceDescriptor *service_;
+};
+
+
+class SubstitutionContext {
+ public:
+  // Takes ownership of the substituter
+  void Push(const Substituter *sub) {
+    subs_.push_back(shared_ptr<const Substituter>(sub));
+  }
+
+  void PushMethod(const MethodDescriptor *method) {
+    Push(new MethodSubstitutions(method));
+  }
+
+  void PushService(const ServiceDescriptor *service) {
+    Push(new ServiceSubstitutions(service));
+  }
+
+  void Pop() {
+    CHECK(!subs_.empty());
+    subs_.pop_back();
+  }
+
+  void InitSubstitutionMap(map<string, string> *subs) const {
+    for (const shared_ptr<const Substituter> &sub : subs_) {
+      sub->InitSubstitutionMap(subs);
+    }
+  }
+
+ private:
+  vector<shared_ptr<const Substituter> > subs_;
+};
+
+
+
+class CodeGenerator : public ::google::protobuf::compiler::CodeGenerator {
+ public:
+  CodeGenerator() { }
+
+  ~CodeGenerator() { }
+
+  bool Generate(const google::protobuf::FileDescriptor *file,
+        const std::string &/* parameter */,
+        google::protobuf::compiler::GeneratorContext *gen_context,
+        std::string *error) const OVERRIDE {
+    auto name_info = new FileSubstitutions();
+    Status ret = name_info->Init(file);
+    if (!ret.ok()) {
+      *error = "name_info.Init failed: " + ret.ToString();
+      return false;
+    }
+
+    SubstitutionContext subs;
+    subs.Push(name_info);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> ih_output(
+        gen_context->Open(name_info->service_header()));
+    Printer ih_printer(ih_output.get(), '$');
+    GenerateServiceIfHeader(&ih_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> i_output(
+        gen_context->Open(name_info->service()));
+    Printer i_printer(i_output.get(), '$');
+    GenerateServiceIf(&i_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> ph_output(
+        gen_context->Open(name_info->proxy_header()));
+    Printer ph_printer(ph_output.get(), '$');
+    GenerateProxyHeader(&ph_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> p_output(
+        gen_context->Open(name_info->proxy()));
+    Printer p_printer(p_output.get(), '$');
+    GenerateProxy(&p_printer, &subs, file);
+
+    return true;
+  }
+
+ private:
+  void Print(Printer *printer,
+             const SubstitutionContext &sub,
+             const char *text) const {
+    map<string, string> subs;
+    sub.InitSubstitutionMap(&subs);
+    printer->Print(subs, text);
+  }
+
+  void GenerateServiceIfHeader(Printer *printer,
+                               SubstitutionContext *subs,
+                               const FileDescriptor *file) const {
+    Print(printer, *subs,
+      "// THIS FILE IS AUTOGENERATED FROM $path$\n"
+      "\n"
+      "#ifndef KUDU_RPC_$upper_case$_SERVICE_IF_DOT_H\n"
+      "#define KUDU_RPC_$upper_case$_SERVICE_IF_DOT_H\n"
+      "\n"
+      "#include <string>\n"
+      "\n"
+      "#include \"kudu/gutil/ref_counted.h\"\n"
+      "#include \"kudu/rpc/service_if.h\"\n"
+      "\n"
+      "namespace google {\n"
+      "namespace protobuf {\n"
+      "class Message;\n"
+      "} // namespace protobuf\n"
+      "} // namespace google\n"
+      "\n"
+      "namespace kudu {\n"
+      "class MetricEntity;\n"
+      "namespace rpc {\n"
+      "class ResultTracker;\n"
+      "class RpcContext;\n"
+      "} // namespace rpc\n"
+      "} // namespace kudu\n"
+      "\n"
+      "$open_namespace$"
+      "\n");
+
+    for (int service_idx = 0; service_idx < file->service_count();
+         ++service_idx) {
+      const ServiceDescriptor *service = file->service(service_idx);
+      subs->PushService(service);
+
+      Print(printer, *subs,
+        "class $service_name$If : public ::kudu::rpc::GeneratedServiceIf {\n"
+        " public:\n"
+        "  explicit $service_name$If(const scoped_refptr<::kudu::MetricEntity>& entity,"
+            " const scoped_refptr<::kudu::rpc::ResultTracker>& result_tracker);\n"
+        "  virtual ~$service_name$If();\n"
+        "  std::string service_name() const override;\n"
+        "  static std::string static_service_name();\n"
+        "\n"
+        );
+
+      set<string> authz_methods;
+      for (int method_idx = 0; method_idx < service->method_count();
+           ++method_idx) {
+        const MethodDescriptor *method = service->method(method_idx);
+        subs->PushMethod(method);
+
+        Print(printer, *subs,
+        "  virtual void $rpc_name$(const class $request$ *req,\n"
+        "      class $response$ *resp, ::kudu::rpc::RpcContext *context) = 0;\n"
+        );
+        subs->Pop();
+        if (auto m = GetAuthzMethod(*method)) {
+          authz_methods.insert(m.get());
+        }
+      }
+
+      if (!authz_methods.empty()) {
+        printer->Print(
+        "\n\n"
+        "  // Authorization methods\n"
+        "  // ---------------------\n\n");
+      }
+      for (const string& m : authz_methods) {
+        printer->Print({ {"m", m} },
+        "  virtual bool $m$(const google::protobuf::Message* req,\n"
+        "     google::protobuf::Message* resp, ::kudu::rpc::RpcContext *context) = 0;\n");
+      }
+
+      Print(printer, *subs,
+        "\n"
+        "};\n"
+      );
+
+      subs->Pop(); // Service
+    }
+
+    Print(printer, *subs,
+      "\n"
+      "$close_namespace$\n"
+      "#endif\n");
+  }
+
+  void GenerateServiceIf(Printer *printer,
+                         SubstitutionContext *subs,
+                         const FileDescriptor *file) const {
+    Print(printer, *subs,
+      "// THIS FILE IS AUTOGENERATED FROM $path$\n"
+      "\n"
+      "#include <functional>\n"
+      "#include <memory>\n"
+      "#include <unordered_map>\n"
+      "#include <utility>\n"
+      "\n"
+      "#include <google/protobuf/message.h>\n"
+      "\n"
+      "#include \"$path_no_extension$.pb.h\"\n"
+      "#include \"$path_no_extension$.service.h\"\n"
+      "\n"
+      "#include \"kudu/rpc/result_tracker.h\"\n"
+      "#include \"kudu/rpc/service_if.h\"\n"
+      "#include \"kudu/util/metrics.h\"\n"
+      "\n");
+
+    // Define metric prototypes for each method in the service.
+    for (int service_idx = 0; service_idx < file->service_count();
+        ++service_idx) {
+      const ServiceDescriptor *service = file->service(service_idx);
+      subs->PushService(service);
+
+      for (int method_idx = 0; method_idx < service->method_count();
+          ++method_idx) {
+        const MethodDescriptor *method = service->method(method_idx);
+        subs->PushMethod(method);
+        Print(printer, *subs,
+          "METRIC_DEFINE_histogram(server, handler_latency_$rpc_full_name_plainchars$,\n"
+          "  \"$rpc_full_name$ RPC Time\",\n"
+          "  kudu::MetricUnit::kMicroseconds,\n"
+          "  \"Microseconds spent handling $rpc_full_name$() RPC requests\",\n"
+          "  60000000LU, 2);\n"
+          "\n");
+        subs->Pop();
+      }
+
+      subs->Pop();
+    }
+
+    Print(printer, *subs,
+      "using google::protobuf::Message;\n"
+      "using kudu::MetricEntity;\n"
+      "using kudu::rpc::ResultTracker;\n"
+      "using kudu::rpc::RpcContext;\n"
+      "using kudu::rpc::RpcMethodInfo;\n"
+      "using std::unique_ptr;\n"
+      "\n"
+      "$open_namespace$"
+      "\n");
+
+    for (int service_idx = 0; service_idx < file->service_count();
+         ++service_idx) {
+      const ServiceDescriptor *service = file->service(service_idx);
+      subs->PushService(service);
+
+      Print(printer, *subs,
+        "$service_name$If::$service_name$If(const scoped_refptr<MetricEntity>& entity,"
+            " const scoped_refptr<ResultTracker>& result_tracker) {\n"
+            "result_tracker_ = result_tracker;\n"
+      );
+      for (int method_idx = 0; method_idx < service->method_count();
+           ++method_idx) {
+        const MethodDescriptor *method = service->method(method_idx);
+        subs->PushMethod(method);
+
+        Print(printer, *subs,
+              "  {\n"
+              "    scoped_refptr<RpcMethodInfo> mi(new RpcMethodInfo());\n"
+              "    mi->req_prototype.reset(new $request$());\n"
+              "    mi->resp_prototype.reset(new $response$());\n"
+              "    mi->authz_method = [this](const Message* req, Message* resp,\n"
+              "                              RpcContext* ctx) {\n"
+              "      return this->$authz_method$(static_cast<const $request$*>(req),\n"
+              "                           static_cast<$response$*>(resp),\n"
+              "                           ctx);\n"
+              "    };\n"
+              "    mi->track_result = $track_result$;\n"
+              "    mi->handler_latency_histogram =\n"
+              "        METRIC_handler_latency_$rpc_full_name_plainchars$.Instantiate(entity);\n"
+              "    mi->func = [this](const Message* req, Message* resp, RpcContext* ctx) {\n"
+              "      this->$rpc_name$(static_cast<const $request$*>(req),\n"
+              "                       static_cast<$response$*>(resp),\n"
+              "                       ctx);\n"
+              "    };\n"
+              "    methods_by_name_[\"$rpc_name$\"] = std::move(mi);\n"
+              "  }\n");
+        subs->Pop();
+      }
+
+      Print(printer, *subs,
+        "}\n"
+        "\n"
+        "$service_name$If::~$service_name$If() {\n"
+        "}\n"
+        "\n"
+        "std::string $service_name$If::service_name() const {\n"
+        "  return \"$full_service_name$\";\n"
+        "}\n"
+        "std::string $service_name$If::static_service_name() {\n"
+        "  return \"$full_service_name$\";\n"
+        "}\n"
+        "\n"
+      );
+
+      subs->Pop();
+    }
+
+    Print(printer, *subs,
+      "$close_namespace$"
+      );
+  }
+
+  void GenerateProxyHeader(Printer *printer,
+                           SubstitutionContext *subs,
+                           const FileDescriptor *file) const {
+    Print(printer, *subs,
+      "// THIS FILE IS AUTOGENERATED FROM $path$\n"
+      "\n"
+      "#ifndef KUDU_RPC_$upper_case$_PROXY_DOT_H\n"
+      "#define KUDU_RPC_$upper_case$_PROXY_DOT_H\n"
+      "\n"
+      "#include <memory>\n"
+      "#include <string>\n"
+      "\n"
+      "#include \"kudu/rpc/proxy.h\"\n"
+      "#include \"kudu/rpc/response_callback.h\"\n"
+      "#include \"kudu/util/status.h\"\n"
+      "\n"
+      "namespace kudu { class Sockaddr; }\n"
+      "namespace kudu {\n"
+      "namespace rpc {\n"
+      "class Messenger;\n"
+      "class RpcController;\n"
+      "} // namespace rpc\n"
+      "} // namespace kudu\n"
+      "\n"
+      "$open_namespace$"
+      "\n");
+
+    for (int service_idx = 0; service_idx < file->service_count();
+         ++service_idx) {
+      const ServiceDescriptor *service = file->service(service_idx);
+      subs->PushService(service);
+
+      Print(printer, *subs,
+        "class $service_name$Proxy : public ::kudu::rpc::Proxy {\n"
+        " public:\n"
+        "  $service_name$Proxy(std::shared_ptr<::kudu::rpc::Messenger>\n"
+        "                messenger, const ::kudu::Sockaddr &sockaddr,"
+        "                std::string hostname);\n"
+        "  ~$service_name$Proxy();\n"
+        "\n"
+        );
+
+      for (int method_idx = 0; method_idx < service->method_count();
+           ++method_idx) {
+        const MethodDescriptor *method = service->method(method_idx);
+        subs->PushMethod(method);
+
+        Print(printer, *subs,
+        "\n"
+        "  ::kudu::Status $rpc_name$(const class $request$ &req,\n"
+        "                            class $response$ *resp,\n"
+        "                            ::kudu::rpc::RpcController *controller);\n"
+        "  void $rpc_name$Async(const class $request$ &req,\n"
+        "                       class $response$ *response,\n"
+        "                       ::kudu::rpc::RpcController *controller,\n"
+        "                       const ::kudu::rpc::ResponseCallback &callback);\n"
+        );
+        subs->Pop();
+      }
+      Print(printer, *subs,
+      "};\n");
+      subs->Pop();
+    }
+    Print(printer, *subs,
+      "\n"
+      "$close_namespace$"
+      "\n"
+      "#endif\n"
+      );
+  }
+
+  void GenerateProxy(Printer *printer,
+                     SubstitutionContext *subs,
+                     const FileDescriptor *file) const {
+    Print(printer, *subs,
+      "// THIS FILE IS AUTOGENERATED FROM $path$\n"
+      "\n"
+      "#include <string>\n"
+      "#include <utility>\n"
+      "\n"
+      "#include \"$path_no_extension$.pb.h\"\n"
+      "#include \"$path_no_extension$.proxy.h\"\n"
+      "\n"
+      "namespace kudu {\n"
+      "namespace rpc {\n"
+      "class Messenger;\n"
+      "class RpcController;\n"
+      "} // namespace rpc\n"
+      "} // namespace kudu\n"
+      "\n"
+      "$open_namespace$"
+      "\n");
+
+    for (int service_idx = 0; service_idx < file->service_count();
+         ++service_idx) {
+      const ServiceDescriptor *service = file->service(service_idx);
+      subs->PushService(service);
+      Print(printer, *subs,
+        "$service_name$Proxy::$service_name$Proxy(\n"
+        "   std::shared_ptr< ::kudu::rpc::Messenger> messenger,\n"
+        "   const ::kudu::Sockaddr &remote, std::string hostname)\n"
+        "  : Proxy(std::move(messenger), remote, std::move(hostname), \"$full_service_name$\") {\n"
+        "}\n"
+        "\n"
+        "$service_name$Proxy::~$service_name$Proxy() {\n"
+        "}\n"
+        "\n"
+        "\n");
+      for (int method_idx = 0; method_idx < service->method_count();
+           ++method_idx) {
+        const MethodDescriptor *method = service->method(method_idx);
+        subs->PushMethod(method);
+        Print(printer, *subs,
+        "::kudu::Status $service_name$Proxy::$rpc_name$(const $request$ &req, $response$ *resp,\n"
+        "                                     ::kudu::rpc::RpcController *controller) {\n"
+        "  return SyncRequest(\"$rpc_name$\", req, resp, controller);\n"
+        "}\n"
+        "\n"
+        "void $service_name$Proxy::$rpc_name$Async(const $request$ &req,\n"
+        "                     $response$ *resp, ::kudu::rpc::RpcController *controller,\n"
+        "                     const ::kudu::rpc::ResponseCallback &callback) {\n"
+        "  AsyncRequest(\"$rpc_name$\", req, resp, controller, callback);\n"
+        "}\n"
+        "\n");
+        subs->Pop();
+      }
+
+      subs->Pop();
+    }
+    Print(printer, *subs,
+      "$close_namespace$");
+  }
+};
+} // namespace rpc
+} // namespace kudu
+
+int main(int argc, char *argv[]) {
+  kudu::rpc::CodeGenerator generator;
+  return google::protobuf::compiler::PluginMain(argc, argv, &generator);
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/proxy.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/proxy.cc b/be/src/kudu/rpc/proxy.cc
new file mode 100644
index 0000000..54b8085
--- /dev/null
+++ b/be/src/kudu/rpc/proxy.cc
@@ -0,0 +1,116 @@
+// 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 "kudu/rpc/proxy.h"
+
+#include <iostream>
+#include <memory>
+#include <utility>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/core/ref.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+#include "kudu/util/user.h"
+
+using google::protobuf::Message;
+using std::string;
+using std::shared_ptr;
+
+namespace kudu {
+namespace rpc {
+
+Proxy::Proxy(std::shared_ptr<Messenger> messenger,
+             const Sockaddr& remote,
+             string hostname,
+             string service_name)
+    : service_name_(std::move(service_name)),
+      messenger_(std::move(messenger)),
+      is_started_(false) {
+  CHECK(messenger_ != nullptr);
+  DCHECK(!service_name_.empty()) << "Proxy service name must not be blank";
+
+  // By default, we set the real user to the currently logged-in user.
+  // Effective user and password remain blank.
+  string real_user;
+  Status s = GetLoggedInUser(&real_user);
+  if (!s.ok()) {
+    LOG(WARNING) << "Proxy for " << service_name_ << ": Unable to get logged-in user name: "
+        << s.ToString() << " before connecting to remote: " << remote.ToString();
+  }
+
+  UserCredentials creds;
+  creds.set_real_user(std::move(real_user));
+  conn_id_ = ConnectionId(remote, std::move(hostname), std::move(creds));
+}
+
+Proxy::~Proxy() {
+}
+
+void Proxy::AsyncRequest(const string& method,
+                         const google::protobuf::Message& req,
+                         google::protobuf::Message* response,
+                         RpcController* controller,
+                         const ResponseCallback& callback) const {
+  CHECK(!controller->call_) << "Controller should be reset";
+  base::subtle::NoBarrier_Store(&is_started_, true);
+  RemoteMethod remote_method(service_name_, method);
+  controller->call_.reset(
+      new OutboundCall(conn_id_, remote_method, response, controller, callback));
+  controller->SetRequestParam(req);
+  controller->SetMessenger(messenger_.get());
+
+  // If this fails to queue, the callback will get called immediately
+  // and the controller will be in an ERROR state.
+  messenger_->QueueOutboundCall(controller->call_);
+}
+
+
+Status Proxy::SyncRequest(const string& method,
+                          const google::protobuf::Message& req,
+                          google::protobuf::Message* resp,
+                          RpcController* controller) const {
+  CountDownLatch latch(1);
+  AsyncRequest(method, req, DCHECK_NOTNULL(resp), controller,
+               boost::bind(&CountDownLatch::CountDown, boost::ref(latch)));
+
+  latch.Wait();
+  return controller->status();
+}
+
+void Proxy::set_user_credentials(const UserCredentials& user_credentials) {
+  CHECK(base::subtle::NoBarrier_Load(&is_started_) == false)
+    << "It is illegal to call set_user_credentials() after request processing has started";
+  conn_id_.set_user_credentials(user_credentials);
+}
+
+std::string Proxy::ToString() const {
+  return strings::Substitute("$0@$1", service_name_, conn_id_.ToString());
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/proxy.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/proxy.h b/be/src/kudu/rpc/proxy.h
new file mode 100644
index 0000000..92b3079
--- /dev/null
+++ b/be/src/kudu/rpc/proxy.h
@@ -0,0 +1,126 @@
+// 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.
+
+#ifndef KUDU_RPC_PROXY_H
+#define KUDU_RPC_PROXY_H
+
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/util/status.h"
+
+namespace google {
+namespace protobuf {
+class Message;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Sockaddr;
+
+namespace rpc {
+
+class Messenger;
+class RpcController;
+class UserCredentials;
+
+// Interface to send calls to a remote service.
+//
+// Proxy objects do not map one-to-one with TCP connections.  The underlying TCP
+// connection is not established until the first call, and may be torn down and
+// re-established as necessary by the messenger. Additionally, the messenger is
+// likely to multiplex many Proxy objects on the same connection.
+//
+// Proxy objects are thread-safe after initialization only.
+// Setters on the Proxy are not thread-safe, and calling a setter after any RPC
+// request has started will cause a fatal error.
+//
+// After initialization, multiple threads may make calls using the same proxy object.
+class Proxy {
+ public:
+  Proxy(std::shared_ptr<Messenger> messenger,
+        const Sockaddr& remote,
+        std::string hostname,
+        std::string service_name);
+
+  ~Proxy();
+
+  // Call a remote method asynchronously.
+  //
+  // Typically, users will not call this directly, but rather through
+  // a generated Proxy subclass.
+  //
+  // method: the method name to invoke on the remote server.
+  //
+  // req:  the request protobuf. This will be serialized immediately,
+  //       so the caller may free or otherwise mutate 'req' safely.
+  //
+  // resp: the response protobuf. This protobuf will be mutated upon
+  //       completion of the call. The RPC system does not take ownership
+  //       of this storage.
+  //
+  // NOTE: 'req' and 'resp' should be the appropriate protocol buffer implementation
+  // class corresponding to the parameter and result types of the service method
+  // defined in the service's '.proto' file.
+  //
+  // controller: the RpcController to associate with this call. Each call
+  //             must use a unique controller object. Does not take ownership.
+  //
+  // callback: the callback to invoke upon call completion. This callback may
+  //           be invoked before AsyncRequest() itself returns, or any time
+  //           thereafter. It may be invoked either on the caller's thread
+  //           or by an RPC IO thread, and thus should take care to not
+  //           block or perform any heavy CPU work.
+  void AsyncRequest(const std::string& method,
+                    const google::protobuf::Message& req,
+                    google::protobuf::Message* resp,
+                    RpcController* controller,
+                    const ResponseCallback& callback) const;
+
+  // The same as AsyncRequest(), except that the call blocks until the call
+  // finishes. If the call fails, returns a non-OK result.
+  Status SyncRequest(const std::string& method,
+                     const google::protobuf::Message& req,
+                     google::protobuf::Message* resp,
+                     RpcController* controller) const;
+
+  // Set the user credentials which should be used to log in.
+  void set_user_credentials(const UserCredentials& user_credentials);
+
+  // Get the user credentials which should be used to log in.
+  const UserCredentials& user_credentials() const { return conn_id_.user_credentials(); }
+
+  std::string ToString() const;
+
+ private:
+  const std::string service_name_;
+  std::shared_ptr<Messenger> messenger_;
+  ConnectionId conn_id_;
+  mutable Atomic32 is_started_;
+
+  DISALLOW_COPY_AND_ASSIGN(Proxy);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/reactor-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/reactor-test.cc b/be/src/kudu/rpc/reactor-test.cc
new file mode 100644
index 0000000..2de5f58
--- /dev/null
+++ b/be/src/kudu/rpc/reactor-test.cc
@@ -0,0 +1,112 @@
+// 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 <memory>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/function.hpp>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc-test-base.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/thread.h"
+
+using std::shared_ptr;
+
+namespace kudu {
+namespace rpc {
+
+class ReactorTest : public RpcTestBase {
+ public:
+  ReactorTest()
+    : latch_(1) {
+  }
+
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    ASSERT_OK(CreateMessenger("my_messenger", &messenger_, 4));
+  }
+
+  void ScheduledTask(const Status& status, const Status& expected_status) {
+    CHECK_EQ(expected_status.CodeAsString(), status.CodeAsString());
+    latch_.CountDown();
+  }
+
+  void ScheduledTaskCheckThread(const Status& status, const Thread* thread) {
+    CHECK_OK(status);
+    CHECK_EQ(thread, Thread::current_thread());
+    latch_.CountDown();
+  }
+
+  void ScheduledTaskScheduleAgain(const Status& status) {
+    messenger_->ScheduleOnReactor(
+        boost::bind(&ReactorTest::ScheduledTaskCheckThread, this, _1,
+                    Thread::current_thread()),
+        MonoDelta::FromMilliseconds(0));
+    latch_.CountDown();
+  }
+
+ protected:
+  shared_ptr<Messenger> messenger_;
+  CountDownLatch latch_;
+};
+
+TEST_F(ReactorTest, TestFunctionIsCalled) {
+  messenger_->ScheduleOnReactor(
+      boost::bind(&ReactorTest::ScheduledTask, this, _1, Status::OK()),
+      MonoDelta::FromSeconds(0));
+  latch_.Wait();
+}
+
+TEST_F(ReactorTest, TestFunctionIsCalledAtTheRightTime) {
+  MonoTime before = MonoTime::Now();
+  messenger_->ScheduleOnReactor(
+      boost::bind(&ReactorTest::ScheduledTask, this, _1, Status::OK()),
+      MonoDelta::FromMilliseconds(100));
+  latch_.Wait();
+  MonoTime after = MonoTime::Now();
+  MonoDelta delta = after - before;
+  CHECK_GE(delta.ToMilliseconds(), 100);
+}
+
+TEST_F(ReactorTest, TestFunctionIsCalledIfReactorShutdown) {
+  messenger_->ScheduleOnReactor(
+      boost::bind(&ReactorTest::ScheduledTask, this, _1,
+                  Status::Aborted("doesn't matter")),
+      MonoDelta::FromSeconds(60));
+  messenger_->Shutdown();
+  latch_.Wait();
+}
+
+TEST_F(ReactorTest, TestReschedulesOnSameReactorThread) {
+  // Our scheduled task will schedule yet another task.
+  latch_.Reset(2);
+
+  messenger_->ScheduleOnReactor(
+      boost::bind(&ReactorTest::ScheduledTaskScheduleAgain, this, _1),
+      MonoDelta::FromSeconds(0));
+  latch_.Wait();
+  latch_.Wait();
+}
+
+} // namespace rpc
+} // namespace kudu


[40/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/result_tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/result_tracker.cc b/be/src/kudu/rpc/result_tracker.cc
new file mode 100644
index 0000000..d26ff87
--- /dev/null
+++ b/be/src/kudu/rpc/result_tracker.cc
@@ -0,0 +1,595 @@
+// 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 "kudu/rpc/result_tracker.h"
+
+#include <algorithm>
+#include <mutex>
+#include <ostream>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace.h"
+// IWYU pragma: no_include <deque>
+
+DEFINE_int64(remember_clients_ttl_ms, 3600 * 1000 /* 1 hour */,
+    "Maximum amount of time, in milliseconds, the server \"remembers\" a client for the "
+    "purpose of caching its responses. After this period without hearing from it, the "
+    "client is no longer remembered and the memory occupied by its responses is reclaimed. "
+    "Retries of requests older than 'remember_clients_ttl_ms' are treated as new "
+    "ones.");
+TAG_FLAG(remember_clients_ttl_ms, advanced);
+
+DEFINE_int64(remember_responses_ttl_ms, 600 * 1000 /* 10 mins */,
+    "Maximum amount of time, in milliseconds, the server \"remembers\" a response to a "
+    "specific request for a client. After this period has elapsed, the response may have "
+    "been garbage collected and the client might get a response indicating the request is "
+    "STALE.");
+TAG_FLAG(remember_responses_ttl_ms, advanced);
+
+DEFINE_int64(result_tracker_gc_interval_ms, 1000,
+    "Interval at which the result tracker will look for entries to GC.");
+TAG_FLAG(result_tracker_gc_interval_ms, hidden);
+
+namespace kudu {
+namespace rpc {
+
+using google::protobuf::Message;
+using kudu::MemTracker;
+using kudu::pb_util::SecureDebugString;
+using kudu::pb_util::SecureShortDebugString;
+using rpc::InboundCall;
+using std::make_pair;
+using std::move;
+using std::lock_guard;
+using std::pair;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+using strings::SubstituteAndAppend;
+
+// This tracks the size changes of anything that has a memory_footprint() method.
+// It must be instantiated before the updates, and it makes sure that the MemTracker
+// is updated on scope exit.
+template <class T>
+struct ScopedMemTrackerUpdater {
+  ScopedMemTrackerUpdater(MemTracker* tracker_, const T* tracked_)
+      : tracker(tracker_),
+        tracked(tracked_),
+        memory_before(tracked->memory_footprint()),
+        cancelled(false) {
+  }
+
+  ~ScopedMemTrackerUpdater() {
+    if (cancelled) return;
+    tracker->Release(memory_before - tracked->memory_footprint());
+  }
+
+  void Cancel() {
+    cancelled = true;
+  }
+
+  MemTracker* tracker;
+  const T* tracked;
+  int64_t memory_before;
+  bool cancelled;
+};
+
+ResultTracker::ResultTracker(shared_ptr<MemTracker> mem_tracker)
+    : mem_tracker_(std::move(mem_tracker)),
+      clients_(ClientStateMap::key_compare(),
+               ClientStateMapAllocator(mem_tracker_)),
+      gc_thread_stop_latch_(1) {}
+
+ResultTracker::~ResultTracker() {
+  if (gc_thread_) {
+    gc_thread_stop_latch_.CountDown();
+    gc_thread_->Join();
+  }
+
+  lock_guard<simple_spinlock> l(lock_);
+  // Release all the memory for the stuff we'll delete on destruction.
+  for (auto& client_state : clients_) {
+    client_state.second->GCCompletionRecords(
+        mem_tracker_, [] (SequenceNumber, CompletionRecord*){ return true; });
+    mem_tracker_->Release(client_state.second->memory_footprint());
+  }
+}
+
+ResultTracker::RpcState ResultTracker::TrackRpc(const RequestIdPB& request_id,
+                                                Message* response,
+                                                RpcContext* context) {
+  lock_guard<simple_spinlock> l(lock_);
+  return TrackRpcUnlocked(request_id, response, context);
+}
+
+ResultTracker::RpcState ResultTracker::TrackRpcUnlocked(const RequestIdPB& request_id,
+                                                        Message* response,
+                                                        RpcContext* context) {
+  ClientState* client_state = ComputeIfAbsent(
+      &clients_,
+      request_id.client_id(),
+      [&]{
+        unique_ptr<ClientState> client_state(new ClientState(mem_tracker_));
+        mem_tracker_->Consume(client_state->memory_footprint());
+        client_state->stale_before_seq_no = request_id.first_incomplete_seq_no();
+        return client_state;
+      })->get();
+
+  client_state->last_heard_from = MonoTime::Now();
+
+  // If the arriving request is older than our per-client GC watermark, report its
+  // staleness to the client.
+  if (PREDICT_FALSE(request_id.seq_no() < client_state->stale_before_seq_no)) {
+    if (context) {
+      context->call_->RespondFailure(
+          ErrorStatusPB::ERROR_REQUEST_STALE,
+          Status::Incomplete(Substitute("Request with id { $0 } is stale.",
+                                        SecureShortDebugString(request_id))));
+      delete context;
+    }
+    return RpcState::STALE;
+  }
+
+  // GC records according to the client's first incomplete watermark.
+  client_state->GCCompletionRecords(
+      mem_tracker_,
+      [&] (SequenceNumber seq_no, CompletionRecord* completion_record) {
+        return completion_record->state != RpcState::IN_PROGRESS &&
+            seq_no < request_id.first_incomplete_seq_no();
+      });
+
+  auto result = ComputeIfAbsentReturnAbsense(
+      &client_state->completion_records,
+      request_id.seq_no(),
+      [&]{
+        unique_ptr<CompletionRecord> completion_record(new CompletionRecord(
+            RpcState::IN_PROGRESS, request_id.attempt_no()));
+        mem_tracker_->Consume(completion_record->memory_footprint());
+        return completion_record;
+      });
+
+  CompletionRecord* completion_record = result.first->get();
+  ScopedMemTrackerUpdater<CompletionRecord> cr_updater(mem_tracker_.get(), completion_record);
+
+  if (PREDICT_TRUE(result.second)) {
+    // When a follower is applying an operation it doesn't have a response yet, and it won't
+    // have a context, so only set them if they exist.
+    if (response != nullptr) {
+      completion_record->ongoing_rpcs.push_back({response,
+                                                 DCHECK_NOTNULL(context),
+                                                 request_id.attempt_no()});
+    }
+    return RpcState::NEW;
+  }
+
+  completion_record->last_updated = MonoTime::Now();
+  switch (completion_record->state) {
+    case RpcState::COMPLETED: {
+      // If the RPC is COMPLETED and the request originates from a client (context, response are
+      // non-null) copy the response and reply immediately. If there is no context/response
+      // do nothing.
+      if (context != nullptr) {
+        DCHECK_NOTNULL(response)->CopyFrom(*completion_record->response);
+        context->call_->RespondSuccess(*response);
+        delete context;
+      }
+      return RpcState::COMPLETED;
+    }
+    case RpcState::IN_PROGRESS: {
+      // If the RPC is IN_PROGRESS check if there is a context and, if so, attach it
+      // so that the rpc gets the same response when the original one completes.
+      if (context != nullptr) {
+        completion_record->ongoing_rpcs.push_back({DCHECK_NOTNULL(response),
+                                                   context,
+                                                   NO_HANDLER});
+      }
+      return RpcState::IN_PROGRESS;
+    }
+    default:
+      LOG(FATAL) << "Wrong state: " << completion_record->state;
+      // dummy return to avoid warnings
+      return RpcState::STALE;
+  }
+}
+
+ResultTracker::RpcState ResultTracker::TrackRpcOrChangeDriver(const RequestIdPB& request_id) {
+  lock_guard<simple_spinlock> l(lock_);
+  RpcState state = TrackRpcUnlocked(request_id, nullptr, nullptr);
+
+  if (state != RpcState::IN_PROGRESS) return state;
+
+  CompletionRecord* completion_record = FindCompletionRecordOrDieUnlocked(request_id);
+  ScopedMemTrackerUpdater<CompletionRecord> updater(mem_tracker_.get(), completion_record);
+
+  // ... if we did find a CompletionRecord change the driver and return true.
+  completion_record->driver_attempt_no = request_id.attempt_no();
+  completion_record->ongoing_rpcs.push_back({nullptr,
+                                             nullptr,
+                                             request_id.attempt_no()});
+
+  // Since we changed the driver of the RPC, return NEW, so that the caller knows
+  // to store the result.
+  return RpcState::NEW;
+}
+
+bool ResultTracker::IsCurrentDriver(const RequestIdPB& request_id) {
+  lock_guard<simple_spinlock> l(lock_);
+  CompletionRecord* completion_record = FindCompletionRecordOrNullUnlocked(request_id);
+
+  // If we couldn't find the CompletionRecord, someone might have called FailAndRespond() so
+  // just return false.
+  if (completion_record == nullptr) return false;
+
+  // ... if we did find a CompletionRecord return true if we're the driver or false
+  // otherwise.
+  return completion_record->driver_attempt_no == request_id.attempt_no();
+}
+
+void ResultTracker::LogAndTraceAndRespondSuccess(RpcContext* context,
+                                                 const Message& msg) {
+  InboundCall* call = context->call_;
+  VLOG(1) << this << " " << call->remote_method().service_name() << ": Sending RPC success "
+      "response for " << call->ToString() << ":" << std::endl << SecureDebugString(msg);
+  TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                         "response", pb_util::PbTracer::TracePb(msg),
+                         "trace", context->trace()->DumpToString());
+  call->RespondSuccess(msg);
+  delete context;
+}
+
+void ResultTracker::LogAndTraceFailure(RpcContext* context,
+                                       const Message& msg) {
+  InboundCall* call = context->call_;
+  VLOG(1) << this << " " << call->remote_method().service_name() << ": Sending RPC failure "
+      "response for " << call->ToString() << ": " << SecureDebugString(msg);
+  TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                         "response", pb_util::PbTracer::TracePb(msg),
+                         "trace", context->trace()->DumpToString());
+}
+
+void ResultTracker::LogAndTraceFailure(RpcContext* context,
+                                       ErrorStatusPB_RpcErrorCodePB err,
+                                       const Status& status) {
+  InboundCall* call = context->call_;
+  VLOG(1) << this << " " << call->remote_method().service_name() << ": Sending RPC failure "
+      "response for " << call->ToString() << ": " << status.ToString();
+  TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                         "status", status.ToString(),
+                         "trace", context->trace()->DumpToString());
+}
+
+ResultTracker::CompletionRecord* ResultTracker::FindCompletionRecordOrDieUnlocked(
+    const RequestIdPB& request_id) {
+  ClientState* client_state = DCHECK_NOTNULL(FindPointeeOrNull(clients_, request_id.client_id()));
+  return DCHECK_NOTNULL(FindPointeeOrNull(client_state->completion_records, request_id.seq_no()));
+}
+
+pair<ResultTracker::ClientState*, ResultTracker::CompletionRecord*>
+ResultTracker::FindClientStateAndCompletionRecordOrNullUnlocked(const RequestIdPB& request_id) {
+  ClientState* client_state = FindPointeeOrNull(clients_, request_id.client_id());
+  CompletionRecord* completion_record = nullptr;
+  if (client_state != nullptr) {
+    completion_record = FindPointeeOrNull(client_state->completion_records, request_id.seq_no());
+  }
+  return make_pair(client_state, completion_record);
+}
+
+ResultTracker::CompletionRecord*
+ResultTracker::FindCompletionRecordOrNullUnlocked(const RequestIdPB& request_id) {
+  return FindClientStateAndCompletionRecordOrNullUnlocked(request_id).second;
+}
+
+void ResultTracker::RecordCompletionAndRespond(const RequestIdPB& request_id,
+                                               const Message* response) {
+  vector<OnGoingRpcInfo> to_respond;
+  {
+    lock_guard<simple_spinlock> l(lock_);
+
+    CompletionRecord* completion_record = FindCompletionRecordOrDieUnlocked(request_id);
+    ScopedMemTrackerUpdater<CompletionRecord> updater(mem_tracker_.get(), completion_record);
+
+    CHECK_EQ(completion_record->driver_attempt_no, request_id.attempt_no())
+        << "Called RecordCompletionAndRespond() from an executor identified with an "
+        << "attempt number that was not marked as the driver for the RPC. RequestId: "
+        << SecureShortDebugString(request_id) << "\nTracker state:\n " << ToStringUnlocked();
+    DCHECK_EQ(completion_record->state, RpcState::IN_PROGRESS);
+    completion_record->response.reset(DCHECK_NOTNULL(response)->New());
+    completion_record->response->CopyFrom(*response);
+    completion_record->state = RpcState::COMPLETED;
+    completion_record->last_updated = MonoTime::Now();
+
+    CHECK_EQ(completion_record->driver_attempt_no, request_id.attempt_no());
+
+    int64_t handler_attempt_no = request_id.attempt_no();
+
+    // Go through the ongoing RPCs and reply to each one.
+    for (auto orpc_iter = completion_record->ongoing_rpcs.rbegin();
+         orpc_iter != completion_record->ongoing_rpcs.rend();) {
+
+      const OnGoingRpcInfo& ongoing_rpc = *orpc_iter;
+      if (MustHandleRpc(handler_attempt_no, completion_record, ongoing_rpc)) {
+        if (ongoing_rpc.context != nullptr) {
+          to_respond.push_back(ongoing_rpc);
+        }
+        ++orpc_iter;
+        orpc_iter = std::vector<OnGoingRpcInfo>::reverse_iterator(
+            completion_record->ongoing_rpcs.erase(orpc_iter.base()));
+      } else {
+        ++orpc_iter;
+      }
+    }
+  }
+
+  // Respond outside of holding the lock. This reduces lock contention and also
+  // means that we will have fully updated our memory tracking before responding,
+  // which makes testing easier.
+  for (auto& ongoing_rpc : to_respond) {
+    if (PREDICT_FALSE(ongoing_rpc.response != response)) {
+      ongoing_rpc.response->CopyFrom(*response);
+    }
+    LogAndTraceAndRespondSuccess(ongoing_rpc.context, *ongoing_rpc.response);
+  }
+}
+
+void ResultTracker::FailAndRespondInternal(const RequestIdPB& request_id,
+                                           const HandleOngoingRpcFunc& func) {
+  vector<OnGoingRpcInfo> to_handle;
+  {
+    lock_guard<simple_spinlock> l(lock_);
+    auto state_and_record = FindClientStateAndCompletionRecordOrNullUnlocked(request_id);
+    if (PREDICT_FALSE(state_and_record.first == nullptr)) {
+      LOG(FATAL) << "Couldn't find ClientState for request: " << SecureShortDebugString(request_id)
+                 << ". \nTracker state:\n" << ToStringUnlocked();
+    }
+
+    CompletionRecord* completion_record = state_and_record.second;
+
+    // It is possible for this method to be called for an RPC that was never actually
+    // tracked (though RecordCompletionAndRespond() can't). One such case is when a
+    // follower transaction fails on the TransactionManager, for some reason, before it
+    // was tracked. The CompletionCallback still calls this method. In this case, do
+    // nothing.
+    if (completion_record == nullptr) {
+      return;
+    }
+
+    ScopedMemTrackerUpdater<CompletionRecord> cr_updater(mem_tracker_.get(), completion_record);
+    completion_record->last_updated = MonoTime::Now();
+
+    int64_t seq_no = request_id.seq_no();
+    int64_t handler_attempt_no = request_id.attempt_no();
+
+    // If we're copying from a client originated response we need to take care to reply
+    // to that call last, otherwise we'll lose 'response', before we go through all the
+    // CompletionRecords.
+    for (auto orpc_iter = completion_record->ongoing_rpcs.rbegin();
+         orpc_iter != completion_record->ongoing_rpcs.rend();) {
+
+      const OnGoingRpcInfo& ongoing_rpc = *orpc_iter;
+      if (MustHandleRpc(handler_attempt_no, completion_record, ongoing_rpc)) {
+        to_handle.push_back(ongoing_rpc);
+        ++orpc_iter;
+        orpc_iter = std::vector<OnGoingRpcInfo>::reverse_iterator(
+            completion_record->ongoing_rpcs.erase(orpc_iter.base()));
+      } else {
+        ++orpc_iter;
+      }
+    }
+
+    // If we're the last ones trying this and the state is not completed,
+    // delete the completion record.
+    if (completion_record->ongoing_rpcs.size() == 0
+        && completion_record->state != RpcState::COMPLETED) {
+      cr_updater.Cancel();
+      unique_ptr<CompletionRecord> completion_record =
+          EraseKeyReturnValuePtr(&state_and_record.first->completion_records, seq_no);
+      mem_tracker_->Release(completion_record->memory_footprint());
+    }
+  }
+
+  // Wait until outside the lock to do the heavy-weight work.
+  for (auto& ongoing_rpc : to_handle) {
+    if (ongoing_rpc.context != nullptr) {
+      func(ongoing_rpc);
+      delete ongoing_rpc.context;
+    }
+  }
+}
+
+void ResultTracker::FailAndRespond(const RequestIdPB& request_id, Message* response) {
+  auto func = [&](const OnGoingRpcInfo& ongoing_rpc) {
+    // In the common case RPCs are just executed once so, in that case, avoid an extra
+    // copy of the response.
+    if (PREDICT_FALSE(ongoing_rpc.response != response)) {
+      ongoing_rpc.response->CopyFrom(*response);
+    }
+    LogAndTraceFailure(ongoing_rpc.context, *response);
+    ongoing_rpc.context->call_->RespondSuccess(*response);
+  };
+  FailAndRespondInternal(request_id, func);
+}
+
+void ResultTracker::FailAndRespond(const RequestIdPB& request_id,
+                                   ErrorStatusPB_RpcErrorCodePB err, const Status& status) {
+  auto func = [&](const OnGoingRpcInfo& ongoing_rpc) {
+    LogAndTraceFailure(ongoing_rpc.context, err, status);
+    ongoing_rpc.context->call_->RespondFailure(err, status);
+  };
+  FailAndRespondInternal(request_id, func);
+}
+
+void ResultTracker::FailAndRespond(const RequestIdPB& request_id,
+                                   int error_ext_id, const string& message,
+                                   const Message& app_error_pb) {
+  auto func = [&](const OnGoingRpcInfo& ongoing_rpc) {
+    LogAndTraceFailure(ongoing_rpc.context, app_error_pb);
+    ongoing_rpc.context->call_->RespondApplicationError(error_ext_id, message, app_error_pb);
+  };
+  FailAndRespondInternal(request_id, func);
+}
+
+void ResultTracker::StartGCThread() {
+  CHECK(!gc_thread_);
+  CHECK_OK(Thread::Create("server", "result-tracker", &ResultTracker::RunGCThread,
+                          this, &gc_thread_));
+}
+
+void ResultTracker::RunGCThread() {
+  while (!gc_thread_stop_latch_.WaitFor(MonoDelta::FromMilliseconds(
+             FLAGS_result_tracker_gc_interval_ms))) {
+    GCResults();
+  }
+}
+
+void ResultTracker::GCResults() {
+  lock_guard<simple_spinlock> l(lock_);
+  MonoTime now = MonoTime::Now();
+  // Calculate the instants before which we'll start GCing ClientStates and CompletionRecords.
+  MonoTime time_to_gc_clients_from = now;
+  time_to_gc_clients_from.AddDelta(
+      MonoDelta::FromMilliseconds(-FLAGS_remember_clients_ttl_ms));
+  MonoTime time_to_gc_responses_from = now;
+  time_to_gc_responses_from.AddDelta(
+      MonoDelta::FromMilliseconds(-FLAGS_remember_responses_ttl_ms));
+
+  // Now go through the ClientStates. If we haven't heard from a client in a while
+  // GC it and all its completion records (making sure there isn't actually one in progress first).
+  // If we've heard from a client recently, but some of its responses are old, GC those responses.
+  for (auto iter = clients_.begin(); iter != clients_.end();) {
+    auto& client_state = iter->second;
+    if (client_state->last_heard_from < time_to_gc_clients_from) {
+      // Client should be GCed.
+      bool ongoing_request = false;
+      client_state->GCCompletionRecords(
+          mem_tracker_,
+          [&] (SequenceNumber, CompletionRecord* completion_record) {
+            if (PREDICT_FALSE(completion_record->state == RpcState::IN_PROGRESS)) {
+              ongoing_request = true;
+              return false;
+            }
+            return true;
+          });
+      // Don't delete the client state if there is still a request in execution.
+      if (PREDICT_FALSE(ongoing_request)) {
+        ++iter;
+        continue;
+      }
+      mem_tracker_->Release(client_state->memory_footprint());
+      iter = clients_.erase(iter);
+    } else {
+      // Client can't be GCed, but its calls might be GCable.
+      iter->second->GCCompletionRecords(
+          mem_tracker_,
+          [&] (SequenceNumber, CompletionRecord* completion_record) {
+            return completion_record->state != RpcState::IN_PROGRESS &&
+                completion_record->last_updated < time_to_gc_responses_from;
+          });
+      ++iter;
+    }
+  }
+}
+
+string ResultTracker::ToString() {
+  lock_guard<simple_spinlock> l(lock_);
+  return ToStringUnlocked();
+}
+
+string ResultTracker::ToStringUnlocked() const {
+  string result = Substitute("ResultTracker[this: $0, Num. Client States: $1, Client States:\n",
+                             this, clients_.size());
+  for (auto& cs : clients_) {
+    SubstituteAndAppend(&result, Substitute("\n\tClient: $0, $1", cs.first, cs.second->ToString()));
+  }
+  result.append("]");
+  return result;
+}
+
+template<class MustGcRecordFunc>
+void ResultTracker::ClientState::GCCompletionRecords(
+    const shared_ptr<kudu::MemTracker>& mem_tracker,
+    MustGcRecordFunc must_gc_record_func) {
+  ScopedMemTrackerUpdater<ClientState> updater(mem_tracker.get(), this);
+  for (auto iter = completion_records.begin(); iter != completion_records.end();) {
+    if (must_gc_record_func(iter->first, iter->second.get())) {
+      mem_tracker->Release(iter->second->memory_footprint());
+      SequenceNumber deleted_seq_no = iter->first;
+      iter = completion_records.erase(iter);
+      // Each time we GC a response, update 'stale_before_seq_no'.
+      // This will allow to answer clients that their responses are stale if we get
+      // a request with a sequence number lower than or equal to this one.
+      stale_before_seq_no = std::max(deleted_seq_no + 1, stale_before_seq_no);
+      continue;
+    }
+    // Since we store completion records in order, if we found one that shouldn't be GCed,
+    // don't GC anything after it.
+    return;
+  }
+}
+
+string ResultTracker::ClientState::ToString() const {
+  auto since_last_heard =
+      MonoTime::Now().GetDeltaSince(last_heard_from);
+  string result = Substitute("Client State[Last heard from: $0s ago, "
+                             "$1 CompletionRecords:",
+                             since_last_heard.ToString(),
+                             completion_records.size());
+  for (auto& completion_record : completion_records) {
+    SubstituteAndAppend(&result, Substitute("\n\tCompletion Record: $0, $1",
+                                            completion_record.first,
+                                            completion_record.second->ToString()));
+  }
+  result.append("\t]");
+  return result;
+}
+
+string ResultTracker::CompletionRecord::ToString() const {
+  string result = Substitute("Completion Record[State: $0, Driver: $1, "
+                             "Cached response: $2, $3 OngoingRpcs:",
+                             state,
+                             driver_attempt_no,
+                             response ? SecureShortDebugString(*response) : "None",
+                             ongoing_rpcs.size());
+  for (auto& orpc : ongoing_rpcs) {
+    SubstituteAndAppend(&result, Substitute("\n\t$0", orpc.ToString()));
+  }
+  result.append("\t\t]");
+  return result;
+}
+
+string ResultTracker::OnGoingRpcInfo::ToString() const {
+  return Substitute("OngoingRpc[Handler: $0, Context: $1, Response: $2]",
+                    handler_attempt_no, context,
+                    response ? SecureShortDebugString(*response) : "NULL");
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/result_tracker.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/result_tracker.h b/be/src/kudu/rpc/result_tracker.h
new file mode 100644
index 0000000..890bd74
--- /dev/null
+++ b/be/src/kudu/rpc/result_tracker.h
@@ -0,0 +1,401 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <google/protobuf/message.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/request_tracker.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/malloc.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+
+class Status;
+class Thread;
+
+namespace rpc {
+class RpcContext;
+
+// A ResultTracker for RPC results.
+//
+// The ResultTracker is responsible for tracking the results of RPCs and making sure that
+// client calls with the same client ID and sequence number (first attempt and subsequent retries)
+// are executed exactly once.
+//
+// In most cases, the use of ResultTracker is internal to the RPC system: RPCs are tracked when
+// they first arrive, before service methods are called, and calls to ResultTracker to store
+// responses are performed internally by RpcContext. The exception is when an RPC is replicated
+// across multiple servers, such as with writes, in which case direct interaction with the result
+// tracker is required so as to cache responses on replicas which did not receive the RPC directly
+// from the client.
+//
+// Throughout this header and elsewhere we use the following terms:
+//
+// RPC - The operation that a client or another server wants to execute on this server. The client
+//       might attempt one RPC many times, for instance if failures or timeouts happen.
+// Attempt - Each individual attempt of an RPC on the server.
+// Handler - A thread executing an attempt. Usually there is only one handler that executes the
+//           first attempt of an RPC and, when it completes, replies to its own attempt and to all
+//           other attempts that might have arrived after it started.
+// Driver - Only important in cases where there might be multiple handlers (e.g. in replicated
+//          RPCs). In these cases there might be two handlers executing the same RPC, corresponding
+//          to different attempts. Since the RPC must be executed exactly once, only one of the
+//          handlers must be selected as the "driver" and actually perform the operation.
+//
+// If a client wishes to track the result of a given RPC it must send on the RPC header
+// a RequestId with the following information:
+//
+//       Client ID - Uniquely identifies a single client. All the RPCs originating from the same
+//                   client must have the same ID.
+// Sequence number - Uniquely identifies a single RPC, even across retries to multiple servers, for
+//                   replicated RPCs. All retries of the same RPC must have the same sequence
+//                   number.
+//  Attempt number - Uniquely identifies each retry of the same RPC. All retries of the same RPC
+//                   must have different attempt numbers.
+//
+// When a call first arrives from the client the RPC subsystem will call TrackRpc() which
+// will return the state of the RPC in the form of an RpcState enum.
+//
+// If the ResultTracker returns NEW, this signals that it's the first time the server has heard
+// of the RPC and that the corresponding server function should be executed.
+//
+// If anything other than NEW is returned it means that the call has either previously completed or
+// is in the process of being executed. In this case the caller should _not_ execute the function
+// corresponding to the RPC. The ResultTracker itself will take care of responding to the client
+// appropriately. If the RPC was already completed, the ResultTracker replies to the client
+// immediately. If the RPC is still ongoing, the attempt gets "attached" to the ongoing one and will
+// receive the same response when its handler finishes.
+//
+// If handling of the RPC is successful, RecordCompletionAndRespond() must be called
+// to register successful completion, in which case all pending or future RPCs with the same
+// sequence number, from the same client, will receive the same response.
+//
+// On the other hand, if execution of the server function is not successful then one of
+// the FailAndRespond() methods should be called, causing all _pending_ attempts to receive the same
+// error. However this error is not stored, any future attempt with the same sequence number and
+// same client ID will be given a new chance to execute, as if it it had never been tried before.
+// This gives the client a chance to either retry (if the failure reason is transient) or give up.
+//
+// ============================================================================
+// RPCs with multiple handlers
+// ============================================================================
+//
+// Some RPCs results are tracked by single server, i.e. they correspond to the modification of an
+// unreplicated resource and are unpersisted. For those no additional care needs to be taken, the
+// first attempt will be the only handler, and subsequent attempts will receive the response when
+// that first attempt is done.
+// However some RPCs are replicated across servers, using consensus, and thus can have multiple
+// handlers executing different attempts at the same time, e.g. one handler from a client
+// originating retry, and one from a previous leader originating update.
+//
+// In this case we need to make sure that the following invariants are enforced:
+// - Only one handler can actually record a response, the "driver" handler.
+// - Only one handler must respond to "attached" attempts.
+// - Each handler replies to their own RPCs, to avoid races. That is, a live handler should
+//   not mutate another live handler's response/context.
+//
+// This is achieved by naming one handler the "driver" of the RPC and making sure that only
+// the driver can successfully complete it, i.e. call RecordCompletionAndRespond().
+//
+// In order to make sure there is only one driver, there must be an _external_ serialization
+// point, before the final response is produced, after which only one of the handlers will
+// be marked as the driver. For instance, for writes, this serialization point is in
+// TransactionDriver, in a synchronized block where a logic such as this one happens (here
+// in pseudo-ish code):
+//
+// {
+//   lock_guard<simple_spinlock> l(lock_);
+//   if (follower_transaction) {
+//     result_tracker_->TrackRpcOrChangeDriver(request_id);
+//     continue_with_transaction();
+//   } else if (client_transaction) {
+//     bool is_still_driver = result_tracker_->IsCurrentDriver(request_id);
+//     if (is_still_driver) continue_with_transaction();
+//     else abort_transaction();
+//   }
+// }
+//
+// This class is thread safe.
+class ResultTracker : public RefCountedThreadSafe<ResultTracker> {
+ public:
+  typedef rpc::RequestTracker::SequenceNumber SequenceNumber;
+  static const int NO_HANDLER = -1;
+  // Enum returned by TrackRpc that reflects the state of the RPC.
+  enum RpcState {
+    // The RPC is new.
+    NEW,
+    // The RPC has previously completed and the same response has been sent
+    // to the client.
+    COMPLETED,
+    // The RPC is currently in-progress and, when it completes, the same response
+    // will be sent to the client.
+    IN_PROGRESS,
+    // The RPC's state is stale, meaning it's older than our per-client garbage
+    // collection watermark and we do not recall the original response.
+    STALE
+  };
+
+  explicit ResultTracker(std::shared_ptr<kudu::MemTracker> mem_tracker);
+  ~ResultTracker();
+
+  // Tracks the RPC and returns its current state.
+  //
+  // If the RpcState == NEW the caller is supposed to actually start executing the RPC.
+  // The caller still owns the passed 'response' and 'context'.
+  //
+  // If the RpcState is anything else all remaining actions will be taken care of internally,
+  // i.e. the caller no longer needs to execute the RPC and this takes ownership of the passed
+  // 'response' and 'context'.
+  RpcState TrackRpc(const RequestIdPB& request_id,
+                    google::protobuf::Message* response,
+                    RpcContext* context);
+
+  // Used to track RPC attempts which originate from other replicas, and which may race with
+  // client originated ones.
+  // Tracks the RPC if it is untracked or changes the current driver of this RPC, i.e. sets the
+  // attempt number in 'request_id' as the driver of the RPC, if it is tracked and IN_PROGRESS.
+  RpcState TrackRpcOrChangeDriver(const RequestIdPB& request_id);
+
+  // Checks if the attempt at an RPC identified by 'request_id' is the current driver of the
+  // RPC. That is, if the attempt number in 'request_id' corresponds to the attempt marked
+  // as the driver of this RPC, either by initially getting NEW from TrackRpc() or by
+  // explicit driver change with ChangeDriver().
+  bool IsCurrentDriver(const RequestIdPB& request_id);
+
+  // Records the completion of sucessful operation.
+  // This will respond to all RPCs from the same client with the same sequence_number.
+  // The response will be stored so that any future retries of this RPC get the same response.
+  //
+  // Requires that TrackRpc() was called before with the same 'client_id' and
+  // 'sequence_number'.
+  // Requires that the attempt indentified by 'request_id' is the current driver
+  // of the RPC.
+  void RecordCompletionAndRespond(const RequestIdPB& request_id,
+                                  const google::protobuf::Message* response);
+
+  // Responds to all RPCs identified by 'client_id' and 'sequence_number' with the same response,
+  // but doesn't actually store the response.
+  // This should be called when the RPC failed validation or if some transient error occurred.
+  // Based on the response the client can then decide whether to retry the RPC (which will
+  // be treated as a new one) or to give up.
+  //
+  // Requires that TrackRpc() was called before with the same 'client_id' and
+  // 'sequence_number'.
+  // Requires that the attempt indentified by 'request_id' is the current driver
+  // of the RPC.
+  void FailAndRespond(const RequestIdPB& request_id,
+                      google::protobuf::Message* response);
+
+  // Overload to match other types of RpcContext::Respond*Failure()
+  void FailAndRespond(const RequestIdPB& request_id,
+                      ErrorStatusPB_RpcErrorCodePB err, const Status& status);
+
+  // Overload to match other types of RpcContext::Respond*Failure()
+  void FailAndRespond(const RequestIdPB& request_id,
+                      int error_ext_id, const std::string& message,
+                      const google::protobuf::Message& app_error_pb);
+
+  // Start a background thread which periodically runs GCResults().
+  // This thread is automatically stopped in the destructor.
+  //
+  // Must be called at most once.
+  void StartGCThread();
+
+  // Runs time-based garbage collection on the results this result tracker is caching.
+  // When garbage collection runs, it goes through all ClientStates and:
+  // - If a ClientState is older than the 'remember_clients_ttl_ms' flag and no
+  //   requests are in progress, GCs the ClientState and all its CompletionRecords.
+  // - If a ClientState is newer than the 'remember_clients_ttl_ms' flag, goes
+  //   through all CompletionRecords and:
+  //   - If the CompletionRecord is older than the 'remember_responses_ttl_secs' flag,
+  //     GCs the CompletionRecord and advances the 'stale_before_seq_no' watermark.
+  //
+  // Typically this is invoked from an internal thread started by 'StartGCThread()'.
+  void GCResults();
+
+  std::string ToString();
+
+ private:
+  // Information about client originated ongoing RPCs.
+  // The lifecycle of 'response' and 'context' is managed by the RPC layer.
+  struct OnGoingRpcInfo {
+    google::protobuf::Message* response;
+    RpcContext* context;
+    int64_t handler_attempt_no;
+
+    std::string ToString() const;
+  };
+  // A completion record for an IN_PROGRESS or COMPLETED RPC.
+  struct CompletionRecord {
+    CompletionRecord(RpcState state, int64_t driver_attempt_no)
+        : state(state),
+          driver_attempt_no(driver_attempt_no),
+          last_updated(MonoTime::Now()) {
+    }
+
+    // The current state of the RPC.
+    RpcState state;
+
+    // The attempt number that is/was "driving" this RPC.
+    int64_t driver_attempt_no;
+
+    // The timestamp of the last CompletionRecord update.
+    MonoTime last_updated;
+
+    // The cached response, if this RPC is in COMPLETED state.
+    std::unique_ptr<google::protobuf::Message> response;
+
+    // The set of ongoing RPCs that correspond to this record.
+    std::vector<OnGoingRpcInfo> ongoing_rpcs;
+
+    std::string ToString() const;
+
+    // Calculates the memory footprint of this struct.
+    int64_t memory_footprint() const {
+      return kudu_malloc_usable_size(this)
+          + (ongoing_rpcs.capacity() > 0 ? kudu_malloc_usable_size(ongoing_rpcs.data()) : 0)
+          + (response.get() != nullptr ? response->SpaceUsed() : 0);
+    }
+  };
+
+  // The state corresponding to a single client.
+  struct ClientState {
+    typedef MemTrackerAllocator<
+        std::pair<const SequenceNumber,
+                  std::unique_ptr<CompletionRecord>>> CompletionRecordMapAllocator;
+    typedef std::map<SequenceNumber,
+                     std::unique_ptr<CompletionRecord>,
+                     std::less<SequenceNumber>,
+                     CompletionRecordMapAllocator> CompletionRecordMap;
+
+    explicit ClientState(std::shared_ptr<MemTracker> mem_tracker)
+        : stale_before_seq_no(0),
+          completion_records(CompletionRecordMap::key_compare(),
+                             CompletionRecordMapAllocator(std::move(mem_tracker))) {}
+
+    // The last time we've heard from this client.
+    MonoTime last_heard_from;
+
+    // The sequence number of the first response we remember for this client.
+    // All sequence numbers before this one are considered STALE.
+    SequenceNumber stale_before_seq_no;
+
+    // The (un gc'd) CompletionRecords for this client.
+    CompletionRecordMap completion_records;
+
+    // Garbage collects this client's CompletionRecords for which MustGcRecordFunc returns
+    // true. We use a lambda here so that we can have a single method that GCs and releases
+    // the memory for CompletionRecords based on different policies.
+    //
+    // 'func' should have the following signature:
+    //   bool MyFunction(SequenceNumber seq_no, CompletionRecord* record);
+    //
+    template<class MustGcRecordFunc>
+    void GCCompletionRecords(const std::shared_ptr<kudu::MemTracker>& mem_tracker,
+                             MustGcRecordFunc func);
+
+    std::string ToString() const;
+
+    // Calculates the memory footprint of this struct.
+    // This calculation is shallow and doesn't account for the memory the nested data
+    // structures occupy.
+    int64_t memory_footprint() const {
+      return kudu_malloc_usable_size(this);
+    }
+  };
+
+  RpcState TrackRpcUnlocked(const RequestIdPB& request_id,
+                            google::protobuf::Message* response,
+                            RpcContext* context);
+
+  typedef std::function<void (const OnGoingRpcInfo&)> HandleOngoingRpcFunc;
+
+  // Helper method to handle the multiple overloads of FailAndRespond. Takes a lambda
+  // that knows what to do with OnGoingRpcInfo in each individual case.
+  void FailAndRespondInternal(const rpc::RequestIdPB& request_id,
+                              const HandleOngoingRpcFunc& func);
+
+  CompletionRecord* FindCompletionRecordOrNullUnlocked(const RequestIdPB& request_id);
+  CompletionRecord* FindCompletionRecordOrDieUnlocked(const RequestIdPB& request_id);
+  std::pair<ClientState*, CompletionRecord*> FindClientStateAndCompletionRecordOrNullUnlocked(
+      const RequestIdPB& request_id);
+
+  // A handler must handle an RPC attempt if:
+  // 1 - It's its own attempt. I.e. it has the same attempt number of the handler.
+  // 2 - It's the driver of the RPC and the attempt has no handler (was attached).
+  bool MustHandleRpc(int64_t handler_attempt_no,
+                     CompletionRecord* completion_record,
+                     const OnGoingRpcInfo& ongoing_rpc) {
+    if (PREDICT_TRUE(ongoing_rpc.handler_attempt_no == handler_attempt_no)) {
+      return true;
+    }
+    if (completion_record->driver_attempt_no == handler_attempt_no) {
+      return ongoing_rpc.handler_attempt_no == NO_HANDLER;
+    }
+    return false;
+  }
+
+  void LogAndTraceAndRespondSuccess(RpcContext* context, const google::protobuf::Message& msg);
+  void LogAndTraceFailure(RpcContext* context, const google::protobuf::Message& msg);
+  void LogAndTraceFailure(RpcContext* context, ErrorStatusPB_RpcErrorCodePB err,
+                          const Status& status);
+
+  std::string ToStringUnlocked() const;
+
+  void RunGCThread();
+
+  // The memory tracker that tracks this ResultTracker's memory consumption.
+  std::shared_ptr<kudu::MemTracker> mem_tracker_;
+
+  // Lock that protects access to 'clients_' and to the state contained in each
+  // ClientState.
+  // TODO consider a per-ClientState lock if we find this too coarse grained.
+  simple_spinlock lock_;
+
+  typedef MemTrackerAllocator<std::pair<const std::string,
+                                        std::unique_ptr<ClientState>>> ClientStateMapAllocator;
+  typedef std::map<std::string,
+                   std::unique_ptr<ClientState>,
+                   std::less<std::string>,
+                   ClientStateMapAllocator> ClientStateMap;
+
+  ClientStateMap clients_;
+
+  // The thread which runs GC, and a latch to stop it.
+  scoped_refptr<Thread> gc_thread_;
+  CountDownLatch gc_thread_stop_latch_;
+
+  DISALLOW_COPY_AND_ASSIGN(ResultTracker);
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/retriable_rpc.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/retriable_rpc.h b/be/src/kudu/rpc/retriable_rpc.h
new file mode 100644
index 0000000..ba84689
--- /dev/null
+++ b/be/src/kudu/rpc/retriable_rpc.h
@@ -0,0 +1,296 @@
+// 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.
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/request_tracker.h"
+#include "kudu/rpc/rpc.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+namespace rpc {
+
+namespace internal {
+typedef rpc::RequestTracker::SequenceNumber SequenceNumber;
+}
+
+// A base class for retriable RPCs that handles replica picking and retry logic.
+//
+// The 'Server' template parameter refers to the type of the server that will be looked up
+// and passed to the derived classes on Try(). For instance in the case of WriteRpc it's
+// RemoteTabletServer.
+//
+// TODO(unknown): merge RpcRetrier into this class? Can't be done right now as the retrier is used
+// independently elsewhere, but likely possible when all replicated RPCs have a ReplicaPicker.
+//
+// TODO(unknown): allow to target replicas other than the leader, if needed.
+//
+// TODO(unknown): once we have retry handling on all the RPCs merge this with rpc::Rpc.
+template <class Server, class RequestPB, class ResponsePB>
+class RetriableRpc : public Rpc {
+ public:
+  RetriableRpc(const scoped_refptr<ServerPicker<Server>>& server_picker,
+               const scoped_refptr<RequestTracker>& request_tracker,
+               const MonoTime& deadline,
+               std::shared_ptr<Messenger> messenger)
+      : Rpc(deadline, std::move(messenger)),
+        server_picker_(server_picker),
+        request_tracker_(request_tracker),
+        sequence_number_(RequestTracker::kNoSeqNo),
+        num_attempts_(0) {}
+
+  virtual ~RetriableRpc() {
+    DCHECK_EQ(sequence_number_, RequestTracker::kNoSeqNo);
+  }
+
+  // Performs server lookup/initialization.
+  // If/when the server is looked up and initialized successfully RetriableRpc will call
+  // Try() to actually send the request.
+  void SendRpc() override;
+
+  // The callback to call upon retrieving (of failing to retrieve) a new authn
+  // token. This is the callback that subclasses should call in their custom
+  // implementation of the GetNewAuthnTokenAndRetry() method.
+  void GetNewAuthnTokenAndRetryCb(const Status& status);
+
+ protected:
+  // Subclasses implement this method to actually try the RPC.
+  // The server been looked up and is ready to be used.
+  virtual void Try(Server* replica, const ResponseCallback& callback) = 0;
+
+  // Subclasses implement this method to analyze 'status', the controller status or
+  // the response and return a RetriableRpcStatus which will then be used
+  // to decide how to proceed (retry or give up).
+  virtual RetriableRpcStatus AnalyzeResponse(const Status& status) = 0;
+
+  // Subclasses implement this method to perform cleanup and/or final steps.
+  // After this is called the RPC will be no longer retried.
+  virtual void Finish(const Status& status) = 0;
+
+  // Returns 'true' if the RPC is to scheduled for retry with a new authn token,
+  // 'false' otherwise. For RPCs performed in the context of providing token
+  // for authentication it's necessary to implement this method. The default
+  // implementation returns 'false' meaning the calls returning
+  // INVALID_AUTHENTICATION_TOKEN RPC status are not retried.
+  virtual bool GetNewAuthnTokenAndRetry() {
+    return false;
+  }
+
+  // Request body.
+  RequestPB req_;
+
+  // Response body.
+  ResponsePB resp_;
+
+ private:
+  friend class CalculatorServiceRpc;
+
+  // Decides whether to retry the RPC, based on the result of AnalyzeResponse()
+  // and retries if that is the case.
+  // Returns true if the RPC was retried or false otherwise.
+  bool RetryIfNeeded(const RetriableRpcStatus& result, Server* server);
+
+  // Called when the replica has been looked up.
+  void ReplicaFoundCb(const Status& status, Server* server);
+
+  // Called after the RPC was performed.
+  void SendRpcCb(const Status& status) override;
+
+  // Performs final cleanup, after the RPC is done (independently of success).
+  void FinishInternal();
+
+  scoped_refptr<ServerPicker<Server>> server_picker_;
+  scoped_refptr<RequestTracker> request_tracker_;
+  std::shared_ptr<Messenger> messenger_;
+
+  // The sequence number for this RPC.
+  internal::SequenceNumber sequence_number_;
+
+  // The number of times this RPC has been attempted
+  int32_t num_attempts_;
+
+  // Keeps track of the replica the RPCs were sent to.
+  // TODO Remove this and pass the used replica around. For now we need to keep this as
+  // the retrier calls the SendRpcCb directly and doesn't know the replica that was
+  // being written to.
+  Server* current_;
+};
+
+template <class Server, class RequestPB, class ResponsePB>
+void RetriableRpc<Server, RequestPB, ResponsePB>::SendRpc()  {
+  if (sequence_number_ == RequestTracker::kNoSeqNo) {
+    CHECK_OK(request_tracker_->NewSeqNo(&sequence_number_));
+  }
+  server_picker_->PickLeader(Bind(&RetriableRpc::ReplicaFoundCb,
+                                  Unretained(this)),
+                             retrier().deadline());
+}
+
+template <class Server, class RequestPB, class ResponsePB>
+void RetriableRpc<Server, RequestPB, ResponsePB>::GetNewAuthnTokenAndRetryCb(
+    const Status& status) {
+  if (status.ok()) {
+    // Perform the RPC call with the newly fetched authn token.
+    mutable_retrier()->mutable_controller()->Reset();
+    SendRpc();
+  } else {
+    // Back to the retry sequence, hoping for better conditions after some time.
+    VLOG(1) << "Failed to get new authn token: " << status.ToString();
+    mutable_retrier()->DelayedRetry(this, status);
+  }
+}
+
+template <class Server, class RequestPB, class ResponsePB>
+bool RetriableRpc<Server, RequestPB, ResponsePB>::RetryIfNeeded(
+    const RetriableRpcStatus& result, Server* server) {
+  // Handle the cases where we retry.
+  switch (result.result) {
+    case RetriableRpcStatus::SERVICE_UNAVAILABLE:
+      // For writes, always retry the request on the same server in case of the
+      // SERVICE_UNAVAILABLE error.
+      break;
+
+    case RetriableRpcStatus::SERVER_NOT_ACCESSIBLE:
+      // TODO(KUDU-1745): not checking for null here results in a crash, since in the case
+      // of a failed master lookup we have no tablet server corresponding to the error.
+      //
+      // But, with the null check, we end up with a relatively tight retry loop
+      // in this scenario whereas we should be backing off. Need to improve
+      // test coverage here to understand why the back-off is not taking effect.
+      if (server != nullptr) {
+        VLOG(1) << "Failing " << ToString() << " to a new target: " << result.status.ToString();
+        // Mark the server as failed. As for details on the only existing
+        // implementation of ServerPicker::MarkServerFailed(), see the note on
+        // the MetaCacheServerPicker::MarkServerFailed() method.
+        server_picker_->MarkServerFailed(server, result.status);
+      }
+      break;
+
+    case RetriableRpcStatus::RESOURCE_NOT_FOUND:
+      // The TabletServer was not part of the config serving the tablet.
+      // We mark our tablet cache as stale, forcing a master lookup on the
+      // next attempt.
+      //
+      // TODO(KUDU-1314): Don't backoff the first time we hit this error.
+      server_picker_->MarkResourceNotFound(server);
+      break;
+
+    case RetriableRpcStatus::REPLICA_NOT_LEADER:
+      // The TabletServer was not the leader of the quorum.
+      server_picker_->MarkReplicaNotLeader(server);
+      break;
+
+    case RetriableRpcStatus::INVALID_AUTHENTICATION_TOKEN: {
+      // This is a special case for retry: first it's necessary to get a new
+      // authn token and then retry the operation with the new token.
+      if (GetNewAuthnTokenAndRetry()) {
+        // The RPC will be retried.
+        resp_.Clear();
+        return true;
+      }
+      // Do not retry.
+      return false;
+    }
+
+    case RetriableRpcStatus::NON_RETRIABLE_ERROR:
+      if (server != nullptr && result.status.IsTimedOut()) {
+        // For the NON_RETRIABLE_ERROR result in case of TimedOut status,
+        // mark the server as failed. As for details on the only existing
+        // implementation of ServerPicker::MarkServerFailed(), see the note on
+        // the MetaCacheServerPicker::MarkServerFailed() method.
+        VLOG(1) << "Failing " << ToString() << " to a new target: " << result.status.ToString();
+        server_picker_->MarkServerFailed(server, result.status);
+      }
+      // Do not retry in the case of non-retriable error.
+      return false;
+
+    default:
+      // For the OK case we should not retry.
+      DCHECK(result.result == RetriableRpcStatus::OK);
+      return false;
+  }
+  resp_.Clear();
+  current_ = nullptr;
+  mutable_retrier()->DelayedRetry(this, result.status);
+  return true;
+}
+
+template <class Server, class RequestPB, class ResponsePB>
+void RetriableRpc<Server, RequestPB, ResponsePB>::FinishInternal() {
+  // Mark the RPC as completed and set the sequence number to kNoSeqNo to make
+  // sure we're in the appropriate state before destruction.
+  request_tracker_->RpcCompleted(sequence_number_);
+  sequence_number_ = RequestTracker::kNoSeqNo;
+}
+
+template <class Server, class RequestPB, class ResponsePB>
+void RetriableRpc<Server, RequestPB, ResponsePB>::ReplicaFoundCb(const Status& status,
+                                                                 Server* server) {
+  // NOTE: 'server' here may be nullptr in the case that status is not OK!
+  RetriableRpcStatus result = AnalyzeResponse(status);
+  if (RetryIfNeeded(result, server)) return;
+
+  if (result.result == RetriableRpcStatus::NON_RETRIABLE_ERROR) {
+    FinishInternal();
+    Finish(result.status);
+    return;
+  }
+
+  // We successfully found a replica, so prepare the RequestIdPB before we send out the call.
+  std::unique_ptr<RequestIdPB> request_id(new RequestIdPB());
+  request_id->set_client_id(request_tracker_->client_id());
+  request_id->set_seq_no(sequence_number_);
+  request_id->set_first_incomplete_seq_no(request_tracker_->FirstIncomplete());
+  request_id->set_attempt_no(num_attempts_++);
+
+  mutable_retrier()->mutable_controller()->SetRequestIdPB(std::move(request_id));
+
+  DCHECK_EQ(result.result, RetriableRpcStatus::OK);
+  current_ = server;
+  Try(server, boost::bind(&RetriableRpc::SendRpcCb, this, Status::OK()));
+}
+
+template <class Server, class RequestPB, class ResponsePB>
+void RetriableRpc<Server, RequestPB, ResponsePB>::SendRpcCb(const Status& status) {
+  RetriableRpcStatus result = AnalyzeResponse(status);
+  if (RetryIfNeeded(result, current_)) return;
+
+  FinishInternal();
+
+  // From here on out the RPC has either succeeded of suffered a non-retriable
+  // failure.
+  Status final_status = result.status;
+  if (!final_status.ok()) {
+    std::string error_string;
+    if (current_) {
+      error_string = strings::Substitute("Failed to write to server: $0", current_->ToString());
+    } else {
+      error_string = "Failed to write to server: (no server available)";
+    }
+    final_status = final_status.CloneAndPrepend(error_string);
+  }
+  Finish(final_status);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc-bench.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc-bench.cc b/be/src/kudu/rpc/rpc-bench.cc
new file mode 100644
index 0000000..0331e8b
--- /dev/null
+++ b/be/src/kudu/rpc/rpc-bench.cc
@@ -0,0 +1,298 @@
+// 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 <cstdint>
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc-test-base.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/rtest.proxy.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::bind;
+using std::shared_ptr;
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+DEFINE_int32(client_threads, 16,
+             "Number of client threads. For the synchronous benchmark, each thread has "
+             "a single outstanding synchronous request at a time. For the async "
+             "benchmark, this determines the number of client reactors.");
+
+DEFINE_int32(async_call_concurrency, 60,
+             "Number of concurrent requests that will be outstanding at a time for the "
+             "async benchmark. The requests are multiplexed across the number of "
+             "reactors specified by the 'client_threads' flag.");
+
+DEFINE_int32(worker_threads, 1,
+             "Number of server worker threads");
+
+DEFINE_int32(server_reactors, 4,
+             "Number of server reactor threads");
+
+DEFINE_int32(run_seconds, 1, "Seconds to run the test");
+
+DECLARE_bool(rpc_encrypt_loopback_connections);
+DEFINE_bool(enable_encryption, false, "Whether to enable TLS encryption for rpc-bench");
+
+METRIC_DECLARE_histogram(reactor_load_percent);
+METRIC_DECLARE_histogram(reactor_active_latency_us);
+
+namespace kudu {
+namespace rpc {
+
+class RpcBench : public RpcTestBase {
+ public:
+  RpcBench()
+      : should_run_(true),
+        stop_(0)
+  {}
+
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    OverrideFlagForSlowTests("run_seconds", "10");
+
+    n_worker_threads_ = FLAGS_worker_threads;
+    n_server_reactor_threads_ = FLAGS_server_reactors;
+
+    // Set up server.
+    FLAGS_rpc_encrypt_loopback_connections = FLAGS_enable_encryption;
+    ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr_, FLAGS_enable_encryption));
+  }
+
+  void SummarizePerf(CpuTimes elapsed, int total_reqs, bool sync) {
+    float reqs_per_second = static_cast<float>(total_reqs / elapsed.wall_seconds());
+    float user_cpu_micros_per_req = static_cast<float>(elapsed.user / 1000.0 / total_reqs);
+    float sys_cpu_micros_per_req = static_cast<float>(elapsed.system / 1000.0 / total_reqs);
+    float csw_per_req = static_cast<float>(elapsed.context_switches) / total_reqs;
+
+    HdrHistogram reactor_load(*METRIC_reactor_load_percent.Instantiate(
+        server_messenger_->metric_entity())->histogram());
+    HdrHistogram reactor_latency(*METRIC_reactor_active_latency_us.Instantiate(
+        server_messenger_->metric_entity())->histogram());
+
+    LOG(INFO) << "Mode:            " << (sync ? "Sync" : "Async");
+    if (sync) {
+      LOG(INFO) << "Client threads:   " << FLAGS_client_threads;
+    } else {
+      LOG(INFO) << "Client reactors:  " << FLAGS_client_threads;
+      LOG(INFO) << "Call concurrency: " << FLAGS_async_call_concurrency;
+    }
+
+    LOG(INFO) << "Worker threads:   " << FLAGS_worker_threads;
+    LOG(INFO) << "Server reactors:  " << FLAGS_server_reactors;
+    LOG(INFO) << "Encryption:       " << FLAGS_enable_encryption;
+    LOG(INFO) << "----------------------------------";
+    LOG(INFO) << "Reqs/sec:         " << reqs_per_second;
+    LOG(INFO) << "User CPU per req: " << user_cpu_micros_per_req << "us";
+    LOG(INFO) << "Sys CPU per req:  " << sys_cpu_micros_per_req << "us";
+    LOG(INFO) << "Ctx Sw. per req:  " << csw_per_req;
+    LOG(INFO) << "Server Reactor load (mean):     "
+              << reactor_load.MeanValue() << "%";
+    LOG(INFO) << "Server Reactor load (95p):      "
+              << reactor_load.ValueAtPercentile(95) << "%";
+    LOG(INFO) << "Server Reactor Latency (mean):  "
+              << reactor_latency.MeanValue() << "us";
+    LOG(INFO) << "Server Reactor Latency (95p):   "
+              << reactor_latency.ValueAtPercentile(95) << "us";
+
+  }
+
+ protected:
+  friend class ClientThread;
+  friend class ClientAsyncWorkload;
+
+  Sockaddr server_addr_;
+  Atomic32 should_run_;
+  CountDownLatch stop_;
+};
+
+class ClientThread {
+ public:
+  explicit ClientThread(RpcBench *bench)
+    : bench_(bench),
+      request_count_(0) {
+  }
+
+  void Start() {
+    thread_.reset(new thread(&ClientThread::Run, this));
+  }
+
+  void Join() {
+    thread_->join();
+  }
+
+  void Run() {
+    shared_ptr<Messenger> client_messenger;
+    CHECK_OK(bench_->CreateMessenger("Client", &client_messenger));
+
+    CalculatorServiceProxy p(client_messenger, bench_->server_addr_, "localhost");
+
+    AddRequestPB req;
+    AddResponsePB resp;
+    while (Acquire_Load(&bench_->should_run_)) {
+      req.set_x(request_count_);
+      req.set_y(request_count_);
+      RpcController controller;
+      controller.set_timeout(MonoDelta::FromSeconds(10));
+      CHECK_OK(p.Add(req, &resp, &controller));
+      CHECK_EQ(req.x() + req.y(), resp.result());
+      request_count_++;
+    }
+  }
+
+  unique_ptr<thread> thread_;
+  RpcBench *bench_;
+  int request_count_;
+};
+
+
+// Test making successful RPC calls.
+TEST_F(RpcBench, BenchmarkCalls) {
+  Stopwatch sw(Stopwatch::ALL_THREADS);
+  sw.start();
+
+  vector<unique_ptr<ClientThread>> threads;
+  for (int i = 0; i < FLAGS_client_threads; i++) {
+    threads.emplace_back(new ClientThread(this));
+    threads.back()->Start();
+  }
+
+  SleepFor(MonoDelta::FromSeconds(FLAGS_run_seconds));
+  Release_Store(&should_run_, false);
+
+  int total_reqs = 0;
+
+  for (auto& thr : threads) {
+    thr->Join();
+    total_reqs += thr->request_count_;
+  }
+  sw.stop();
+
+  SummarizePerf(sw.elapsed(), total_reqs, true);
+}
+
+class ClientAsyncWorkload {
+ public:
+  ClientAsyncWorkload(RpcBench *bench, shared_ptr<Messenger> messenger)
+    : bench_(bench),
+      messenger_(std::move(messenger)),
+      request_count_(0) {
+    controller_.set_timeout(MonoDelta::FromSeconds(10));
+    proxy_.reset(new CalculatorServiceProxy(messenger_, bench_->server_addr_, "localhost"));
+  }
+
+  void CallOneRpc() {
+    if (request_count_ > 0) {
+      CHECK_OK(controller_.status());
+      CHECK_EQ(req_.x() + req_.y(), resp_.result());
+    }
+    if (!Acquire_Load(&bench_->should_run_)) {
+      bench_->stop_.CountDown();
+      return;
+    }
+    controller_.Reset();
+    req_.set_x(request_count_);
+    req_.set_y(request_count_);
+    request_count_++;
+    proxy_->AddAsync(req_,
+                     &resp_,
+                     &controller_,
+                     bind(&ClientAsyncWorkload::CallOneRpc, this));
+  }
+
+  void Start() {
+    CallOneRpc();
+  }
+
+  RpcBench *bench_;
+  shared_ptr<Messenger> messenger_;
+  unique_ptr<CalculatorServiceProxy> proxy_;
+  uint32_t request_count_;
+  RpcController controller_;
+  AddRequestPB req_;
+  AddResponsePB resp_;
+};
+
+TEST_F(RpcBench, BenchmarkCallsAsync) {
+  int threads = FLAGS_client_threads;
+  int concurrency = FLAGS_async_call_concurrency;
+
+  vector<shared_ptr<Messenger>> messengers;
+  for (int i = 0; i < threads; i++) {
+    shared_ptr<Messenger> m;
+    ASSERT_OK(CreateMessenger("Client", &m));
+    messengers.emplace_back(std::move(m));
+  }
+
+  vector<unique_ptr<ClientAsyncWorkload>> workloads;
+  for (int i = 0; i < concurrency; i++) {
+    workloads.emplace_back(
+        new ClientAsyncWorkload(this, messengers[i % threads]));
+  }
+
+  stop_.Reset(concurrency);
+
+  Stopwatch sw(Stopwatch::ALL_THREADS);
+  sw.start();
+
+  for (int i = 0; i < concurrency; i++) {
+    workloads[i]->Start();
+  }
+
+  SleepFor(MonoDelta::FromSeconds(FLAGS_run_seconds));
+  Release_Store(&should_run_, false);
+
+  sw.stop();
+
+  stop_.Wait();
+  int total_reqs = 0;
+  for (int i = 0; i < concurrency; i++) {
+    total_reqs += workloads[i]->request_count_;
+  }
+
+  SummarizePerf(sw.elapsed(), total_reqs, false);
+}
+
+} // namespace rpc
+} // namespace kudu
+


[45/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/client_negotiation.h b/be/src/kudu/rpc/client_negotiation.h
new file mode 100644
index 0000000..06fb2b8
--- /dev/null
+++ b/be/src/kudu/rpc/client_negotiation.h
@@ -0,0 +1,263 @@
+// 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.
+
+#pragma once
+
+#include <cstdlib>
+#include <memory>
+#include <set>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/sasl_helper.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Slice;
+class faststring;
+
+namespace security {
+class TlsContext;
+}
+
+namespace rpc {
+
+// Class for doing KRPC negotiation with a remote server over a bidirectional socket.
+// Operations on this class are NOT thread-safe.
+class ClientNegotiation {
+ public:
+  // Creates a new client negotiation instance, taking ownership of the
+  // provided socket. After completing the negotiation process by setting the
+  // desired options and calling Negotiate(), the socket can be retrieved with
+  // 'release_socket'.
+  //
+  // The provided TlsContext must outlive this negotiation instance.
+  ClientNegotiation(std::unique_ptr<Socket> socket,
+                    const security::TlsContext* tls_context,
+                    boost::optional<security::SignedTokenPB> authn_token,
+                    RpcEncryption encryption,
+                    std::string sasl_proto_name);
+
+  // Enable PLAIN authentication.
+  // Must be called before Negotiate().
+  Status EnablePlain(const std::string& user,
+                     const std::string& pass);
+
+  // Enable GSSAPI authentication.
+  // Must be called before Negotiate().
+  Status EnableGSSAPI();
+
+  // Returns mechanism negotiated by this connection.
+  // Must be called after Negotiate().
+  SaslMechanism::Type negotiated_mechanism() const;
+
+  // Returns the negotiated authentication type for the connection.
+  // Must be called after Negotiate().
+  AuthenticationType negotiated_authn() const {
+    DCHECK_NE(negotiated_authn_, AuthenticationType::INVALID);
+    return negotiated_authn_;
+  }
+
+  // Returns true if TLS was negotiated.
+  // Must be called after Negotiate().
+  bool tls_negotiated() const {
+    return tls_negotiated_;
+  }
+
+  // Returns the set of RPC system features supported by the remote server.
+  // Must be called before Negotiate().
+  std::set<RpcFeatureFlag> server_features() const {
+    return server_features_;
+  }
+
+  // Returns the set of RPC system features supported by the remote server.
+  // Must be called after Negotiate().
+  // Subsequent calls to this method or server_features() will return an empty set.
+  std::set<RpcFeatureFlag> take_server_features() {
+    return std::move(server_features_);
+  }
+
+  // Specify the fully-qualified domain name of the remote server.
+  // Must be called before Negotiate(). Required for some mechanisms.
+  void set_server_fqdn(const std::string& domain_name);
+
+  // Set deadline for connection negotiation.
+  void set_deadline(const MonoTime& deadline);
+
+  Socket* socket() { return socket_.get(); }
+
+  // Takes and returns the socket owned by this client negotiation. The caller
+  // will own the socket after this call, and the negotiation instance should no
+  // longer be used. Must be called after Negotiate(). Subsequent calls to this
+  // method or socket() will return a null pointer.
+  std::unique_ptr<Socket> release_socket() { return std::move(socket_); }
+
+  // Negotiate with the remote server. Should only be called once per
+  // ClientNegotiation and socket instance, after all options have been set.
+  //
+  // Returns OK on success, otherwise may return NotAuthorized, NotSupported, or
+  // another non-OK status.
+  Status Negotiate(std::unique_ptr<ErrorStatusPB>* rpc_error = nullptr);
+
+  // SASL callback for plugin options, supported mechanisms, etc.
+  // Returns SASL_FAIL if the option is not handled, which does not fail the handshake.
+  int GetOptionCb(const char* plugin_name, const char* option,
+                  const char** result, unsigned* len);
+
+  // SASL callback for SASL_CB_USER, SASL_CB_AUTHNAME, SASL_CB_LANGUAGE
+  int SimpleCb(int id, const char** result, unsigned* len);
+
+  // SASL callback for SASL_CB_PASS
+  int SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret);
+
+  // Check that GSSAPI/Kerberos credentials are available.
+  static Status CheckGSSAPI() WARN_UNUSED_RESULT;
+
+ private:
+
+  // Encode and send the specified negotiate request message to the server.
+  Status SendNegotiatePB(const NegotiatePB& msg) WARN_UNUSED_RESULT;
+
+  // Receive a negotiate response message from the server, deserializing it into 'msg'.
+  // Validates that the response is not an error.
+  Status RecvNegotiatePB(NegotiatePB* msg,
+                         faststring* buffer,
+                         std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  // Parse error status message from raw bytes of an ErrorStatusPB.
+  Status ParseError(const Slice& err_data,
+                    std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  Status SendConnectionHeader() WARN_UNUSED_RESULT;
+
+  // Initialize the SASL client negotiation instance.
+  Status InitSaslClient() WARN_UNUSED_RESULT;
+
+  // Send a NEGOTIATE step message to the server.
+  Status SendNegotiate() WARN_UNUSED_RESULT;
+
+  // Handle NEGOTIATE step response from the server.
+  Status HandleNegotiate(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Send a TLS_HANDSHAKE request message to the server with the provided token.
+  Status SendTlsHandshake(std::string tls_token) WARN_UNUSED_RESULT;
+
+  // Handle a TLS_HANDSHAKE response message from the server.
+  Status HandleTlsHandshake(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Authenticate to the server using SASL.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateBySasl(faststring* recv_buf,
+                            std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  // Authenticate to the server using a token.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateByToken(faststring* recv_buf,
+                             std::unique_ptr<ErrorStatusPB> *rpc_error) WARN_UNUSED_RESULT;
+
+  // Send an SASL_INITIATE message to the server.
+  // Returns:
+  //  Status::OK if the SASL_SUCCESS message is expected next.
+  //  Status::Incomplete if the SASL_CHALLENGE message is expected next.
+  //  Any other status indicates an error.
+  Status SendSaslInitiate() WARN_UNUSED_RESULT;
+
+  // Send a SASL_RESPONSE message to the server.
+  Status SendSaslResponse(const char* resp_msg, unsigned resp_msg_len) WARN_UNUSED_RESULT;
+
+  // Handle case when server sends SASL_CHALLENGE response.
+  // Returns:
+  //  Status::OK if a SASL_SUCCESS message is expected next.
+  //  Status::Incomplete if another SASL_CHALLENGE message is expected.
+  //  Any other status indicates an error.
+  Status HandleSaslChallenge(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Handle case when server sends SASL_SUCCESS response.
+  Status HandleSaslSuccess(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Perform a client-side step of the SASL negotiation.
+  // Input is what came from the server. Output is what we will send back to the server.
+  // Returns:
+  //   Status::OK if sasl_client_step returns SASL_OK.
+  //   Status::Incomplete if sasl_client_step returns SASL_CONTINUE
+  // otherwise returns an appropriate error status.
+  Status DoSaslStep(const std::string& in, const char** out, unsigned* out_len) WARN_UNUSED_RESULT;
+
+  Status SendConnectionContext() WARN_UNUSED_RESULT;
+
+  // The socket to the remote server.
+  std::unique_ptr<Socket> socket_;
+
+  // SASL state.
+  std::vector<sasl_callback_t> callbacks_;
+  std::unique_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
+  SaslHelper helper_;
+  boost::optional<std::string> nonce_;
+
+  // TLS state.
+  const security::TlsContext* tls_context_;
+  security::TlsHandshake tls_handshake_;
+  const RpcEncryption encryption_;
+  bool tls_negotiated_;
+
+  // TSK state.
+  boost::optional<security::SignedTokenPB> authn_token_;
+
+  // Authentication state.
+  std::string plain_auth_user_;
+  std::string plain_pass_;
+  std::unique_ptr<sasl_secret_t, decltype(std::free)*> psecret_;
+
+  // The set of features advertised by the client. Filled in when we send
+  // the first message. This is not necessarily constant since some features
+  // may be dynamically enabled.
+  std::set<RpcFeatureFlag> client_features_;
+
+  // The set of features supported by the server. Filled in during negotiation.
+  std::set<RpcFeatureFlag> server_features_;
+
+  // The authentication type. Filled in during negotiation.
+  AuthenticationType negotiated_authn_;
+
+  // The SASL mechanism used by the connection. Filled in during negotiation.
+  SaslMechanism::Type negotiated_mech_;
+
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
+  // Negotiation timeout deadline.
+  MonoTime deadline_;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/connection.cc b/be/src/kudu/rpc/connection.cc
new file mode 100644
index 0000000..1632dd3
--- /dev/null
+++ b/be/src/kudu/rpc/connection.cc
@@ -0,0 +1,767 @@
+// 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 "kudu/rpc/connection.h"
+
+#include <algorithm>
+#include <cerrno>
+#include <iostream>
+#include <memory>
+#include <set>
+#include <string>
+#include <type_traits>
+
+#include <boost/intrusive/detail/list_iterator.hpp>
+#include <boost/intrusive/list.hpp>
+#include <ev.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/util/slice.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+using std::includes;
+using std::set;
+using std::shared_ptr;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+typedef OutboundCall::Phase Phase;
+
+///
+/// Connection
+///
+Connection::Connection(ReactorThread *reactor_thread,
+                       Sockaddr remote,
+                       unique_ptr<Socket> socket,
+                       Direction direction,
+                       CredentialsPolicy policy)
+    : reactor_thread_(reactor_thread),
+      remote_(remote),
+      socket_(std::move(socket)),
+      direction_(direction),
+      last_activity_time_(MonoTime::Now()),
+      is_epoll_registered_(false),
+      next_call_id_(1),
+      credentials_policy_(policy),
+      negotiation_complete_(false),
+      is_confidential_(false),
+      scheduled_for_shutdown_(false) {
+}
+
+Status Connection::SetNonBlocking(bool enabled) {
+  return socket_->SetNonBlocking(enabled);
+}
+
+void Connection::EpollRegister(ev::loop_ref& loop) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  DVLOG(4) << "Registering connection for epoll: " << ToString();
+  write_io_.set(loop);
+  write_io_.set(socket_->GetFd(), ev::WRITE);
+  write_io_.set<Connection, &Connection::WriteHandler>(this);
+  if (direction_ == CLIENT && negotiation_complete_) {
+    write_io_.start();
+  }
+  read_io_.set(loop);
+  read_io_.set(socket_->GetFd(), ev::READ);
+  read_io_.set<Connection, &Connection::ReadHandler>(this);
+  read_io_.start();
+  is_epoll_registered_ = true;
+}
+
+Connection::~Connection() {
+  // Must clear the outbound_transfers_ list before deleting.
+  CHECK(outbound_transfers_.begin() == outbound_transfers_.end());
+
+  // It's crucial that the connection is Shutdown first -- otherwise
+  // our destructor will end up calling read_io_.stop() and write_io_.stop()
+  // from a possibly non-reactor thread context. This can then make all
+  // hell break loose with libev.
+  CHECK(!is_epoll_registered_);
+}
+
+bool Connection::Idle() const {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  // check if we're in the middle of receiving something
+  InboundTransfer *transfer = inbound_.get();
+  if (transfer && (transfer->TransferStarted())) {
+    return false;
+  }
+  // check if we still need to send something
+  if (!outbound_transfers_.empty()) {
+    return false;
+  }
+  // can't kill a connection if calls are waiting response
+  if (!awaiting_response_.empty()) {
+    return false;
+  }
+
+  if (!calls_being_handled_.empty()) {
+    return false;
+  }
+
+  // We are not idle if we are in the middle of connection negotiation.
+  if (!negotiation_complete_) {
+    return false;
+  }
+
+  return true;
+}
+
+void Connection::Shutdown(const Status &status,
+                          unique_ptr<ErrorStatusPB> rpc_error) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  shutdown_status_ = status.CloneAndPrepend("RPC connection failed");
+
+  if (inbound_ && inbound_->TransferStarted()) {
+    double secs_since_active =
+        (reactor_thread_->cur_time() - last_activity_time_).ToSeconds();
+    LOG(WARNING) << "Shutting down " << ToString()
+                 << " with pending inbound data ("
+                 << inbound_->StatusAsString() << ", last active "
+                 << HumanReadableElapsedTime::ToShortString(secs_since_active)
+                 << " ago, status=" << status.ToString() << ")";
+  }
+
+  // Clear any calls which have been sent and were awaiting a response.
+  for (const car_map_t::value_type &v : awaiting_response_) {
+    CallAwaitingResponse *c = v.second;
+    if (c->call) {
+      // Make sure every awaiting call receives the error info, if any.
+      unique_ptr<ErrorStatusPB> error;
+      if (rpc_error) {
+        error.reset(new ErrorStatusPB(*rpc_error));
+      }
+      c->call->SetFailed(status,
+                         negotiation_complete_ ? Phase::REMOTE_CALL
+                                               : Phase::CONNECTION_NEGOTIATION,
+                         std::move(error));
+    }
+    // And we must return the CallAwaitingResponse to the pool
+    car_pool_.Destroy(c);
+  }
+  awaiting_response_.clear();
+
+  // Clear any outbound transfers.
+  while (!outbound_transfers_.empty()) {
+    OutboundTransfer *t = &outbound_transfers_.front();
+    outbound_transfers_.pop_front();
+    delete t;
+  }
+
+  read_io_.stop();
+  write_io_.stop();
+  is_epoll_registered_ = false;
+  if (socket_) {
+    WARN_NOT_OK(socket_->Close(), "Error closing socket");
+  }
+}
+
+void Connection::QueueOutbound(gscoped_ptr<OutboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (!shutdown_status_.ok()) {
+    // If we've already shut down, then we just need to abort the
+    // transfer rather than bothering to queue it.
+    transfer->Abort(shutdown_status_);
+    return;
+  }
+
+  DVLOG(3) << "Queueing transfer: " << transfer->HexDump();
+
+  outbound_transfers_.push_back(*transfer.release());
+
+  if (negotiation_complete_ && !write_io_.is_active()) {
+    // If we weren't currently in the middle of sending anything,
+    // then our write_io_ interest is stopped. Need to re-start it.
+    // Only do this after connection negotiation is done doing its work.
+    write_io_.start();
+  }
+}
+
+Connection::CallAwaitingResponse::~CallAwaitingResponse() {
+  DCHECK(conn->reactor_thread_->IsCurrentThread());
+}
+
+void Connection::CallAwaitingResponse::HandleTimeout(ev::timer &watcher, int revents) {
+  if (remaining_timeout > 0) {
+    if (watcher.remaining() < -1.0) {
+      LOG(WARNING) << "RPC call timeout handler was delayed by "
+                   << -watcher.remaining() << "s! This may be due to a process-wide "
+                   << "pause such as swapping, logging-related delays, or allocator lock "
+                   << "contention. Will allow an additional "
+                   << remaining_timeout << "s for a response.";
+    }
+
+    watcher.set(remaining_timeout, 0);
+    watcher.start();
+    remaining_timeout = 0;
+    return;
+  }
+
+  conn->HandleOutboundCallTimeout(this);
+}
+
+void Connection::HandleOutboundCallTimeout(CallAwaitingResponse *car) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  DCHECK(car->call);
+  // The timeout timer is stopped by the car destructor exiting Connection::HandleCallResponse()
+  DCHECK(!car->call->IsFinished());
+
+  // Mark the call object as failed.
+  car->call->SetTimedOut(negotiation_complete_ ? Phase::REMOTE_CALL
+                                               : Phase::CONNECTION_NEGOTIATION);
+
+  // Test cancellation when 'car->call' is in 'TIMED_OUT' state
+  MaybeInjectCancellation(car->call);
+
+  // Drop the reference to the call. If the original caller has moved on after
+  // seeing the timeout, we no longer need to hold onto the allocated memory
+  // from the request.
+  car->call.reset();
+
+  // We still leave the CallAwaitingResponse in the map -- this is because we may still
+  // receive a response from the server, and we don't want a spurious log message
+  // when we do finally receive the response. The fact that CallAwaitingResponse::call
+  // is a NULL pointer indicates to the response processing code that the call
+  // already timed out.
+}
+
+void Connection::CancelOutboundCall(const shared_ptr<OutboundCall> &call) {
+  CallAwaitingResponse* car = FindPtrOrNull(awaiting_response_, call->call_id());
+  if (car != nullptr) {
+    // car->call may be NULL if the call has timed out already.
+    DCHECK(!car->call || car->call.get() == call.get());
+    car->call.reset();
+  }
+}
+
+// Inject a cancellation when 'call' is in state 'FLAGS_rpc_inject_cancellation_state'.
+void inline Connection::MaybeInjectCancellation(const shared_ptr<OutboundCall> &call) {
+  if (PREDICT_FALSE(call->ShouldInjectCancellation())) {
+    reactor_thread_->reactor()->messenger()->QueueCancellation(call);
+  }
+}
+
+// Callbacks after sending a call on the wire.
+// This notifies the OutboundCall object to change its state to SENT once it
+// has been fully transmitted.
+struct CallTransferCallbacks : public TransferCallbacks {
+ public:
+  explicit CallTransferCallbacks(shared_ptr<OutboundCall> call,
+                                 Connection *conn)
+      : call_(std::move(call)), conn_(conn) {}
+
+  virtual void NotifyTransferFinished() OVERRIDE {
+    // TODO: would be better to cancel the transfer while it is still on the queue if we
+    // timed out before the transfer started, but there is still a race in the case of
+    // a partial send that we have to handle here
+    if (call_->IsFinished()) {
+      DCHECK(call_->IsTimedOut() || call_->IsCancelled());
+    } else {
+      call_->SetSent();
+      // Test cancellation when 'call_' is in 'SENT' state.
+      conn_->MaybeInjectCancellation(call_);
+    }
+    delete this;
+  }
+
+  virtual void NotifyTransferAborted(const Status &status) OVERRIDE {
+    VLOG(1) << "Transfer of RPC call " << call_->ToString() << " aborted: "
+            << status.ToString();
+    delete this;
+  }
+
+ private:
+  shared_ptr<OutboundCall> call_;
+  Connection* conn_;
+};
+
+void Connection::QueueOutboundCall(shared_ptr<OutboundCall> call) {
+  DCHECK(call);
+  DCHECK_EQ(direction_, CLIENT);
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (PREDICT_FALSE(!shutdown_status_.ok())) {
+    // Already shutdown
+    call->SetFailed(shutdown_status_,
+                    negotiation_complete_ ? Phase::REMOTE_CALL
+                                          : Phase::CONNECTION_NEGOTIATION);
+    return;
+  }
+
+  // At this point the call has a serialized request, but no call header, since we haven't
+  // yet assigned a call ID.
+  DCHECK(!call->call_id_assigned());
+
+  // We shouldn't reach this point if 'call' was requested to be cancelled.
+  DCHECK(!call->cancellation_requested());
+
+  // Assign the call ID.
+  int32_t call_id = GetNextCallId();
+  call->set_call_id(call_id);
+
+  // Serialize the actual bytes to be put on the wire.
+  TransferPayload tmp_slices;
+  size_t n_slices = call->SerializeTo(&tmp_slices);
+
+  call->SetQueued();
+
+  // Test cancellation when 'call_' is in 'ON_OUTBOUND_QUEUE' state.
+  MaybeInjectCancellation(call);
+
+  scoped_car car(car_pool_.make_scoped_ptr(car_pool_.Construct()));
+  car->conn = this;
+  car->call = call;
+
+  // Set up the timeout timer.
+  const MonoDelta &timeout = call->controller()->timeout();
+  if (timeout.Initialized()) {
+    reactor_thread_->RegisterTimeout(&car->timeout_timer);
+    car->timeout_timer.set<CallAwaitingResponse, // NOLINT(*)
+                           &CallAwaitingResponse::HandleTimeout>(car.get());
+
+    // For calls with a timeout of at least 500ms, we actually run the timeout
+    // handler in two stages. The first timeout fires with a timeout 10% less
+    // than the user-specified one. It then schedules a second timeout for the
+    // remaining amount of time.
+    //
+    // The purpose of this two-stage timeout is to be more robust when the client
+    // has some process-wide pause, such as lock contention in tcmalloc, or a
+    // reactor callback that blocks in glog. Consider the following case:
+    //
+    // T = 0s        user issues an RPC with 5 second timeout
+    // T = 0.5s - 6s   process is blocked
+    // T = 6s        process unblocks, and the timeout fires (1s late)
+    //
+    // Without the two-stage timeout, we would determine that the call had timed out,
+    // even though it's likely that the response is waiting on our TCP socket.
+    // With the two-stage timeout, we'll end up with:
+    //
+    // T = 0s           user issues an RPC with 5 second timeout
+    // T = 0.5s - 6s    process is blocked
+    // T = 6s           process unblocks, and the first-stage timeout fires (1.5s late)
+    // T = 6s - 6.200s  time for the client to read the response which is waiting
+    // T = 6.200s       if the response was not actually available, we'll time out here
+    //
+    // We don't bother with this logic for calls with very short timeouts - assumedly
+    // a user setting such a short RPC timeout is well equipped to handle one.
+    double time = timeout.ToSeconds();
+    if (time >= 0.5) {
+      car->remaining_timeout = time * 0.1;
+      time -= car->remaining_timeout;
+    } else {
+      car->remaining_timeout = 0;
+    }
+
+    car->timeout_timer.set(time, 0);
+    car->timeout_timer.start();
+  }
+
+  TransferCallbacks *cb = new CallTransferCallbacks(std::move(call), this);
+  awaiting_response_[call_id] = car.release();
+  QueueOutbound(gscoped_ptr<OutboundTransfer>(
+      OutboundTransfer::CreateForCallRequest(call_id, tmp_slices, n_slices, cb)));
+}
+
+// Callbacks for sending an RPC call response from the server.
+// This takes ownership of the InboundCall object so that, once it has
+// been responded to, we can free up all of the associated memory.
+struct ResponseTransferCallbacks : public TransferCallbacks {
+ public:
+  ResponseTransferCallbacks(gscoped_ptr<InboundCall> call,
+                            Connection *conn) :
+    call_(std::move(call)),
+    conn_(conn)
+  {}
+
+  ~ResponseTransferCallbacks() {
+    // Remove the call from the map.
+    InboundCall *call_from_map = EraseKeyReturnValuePtr(
+      &conn_->calls_being_handled_, call_->call_id());
+    DCHECK_EQ(call_from_map, call_.get());
+  }
+
+  virtual void NotifyTransferFinished() OVERRIDE {
+    delete this;
+  }
+
+  virtual void NotifyTransferAborted(const Status &status) OVERRIDE {
+    LOG(WARNING) << "Connection torn down before " <<
+      call_->ToString() << " could send its response";
+    delete this;
+  }
+
+ private:
+  gscoped_ptr<InboundCall> call_;
+  Connection *conn_;
+};
+
+// Reactor task which puts a transfer on the outbound transfer queue.
+class QueueTransferTask : public ReactorTask {
+ public:
+  QueueTransferTask(gscoped_ptr<OutboundTransfer> transfer,
+                    Connection *conn)
+    : transfer_(std::move(transfer)),
+      conn_(conn)
+  {}
+
+  virtual void Run(ReactorThread *thr) OVERRIDE {
+    conn_->QueueOutbound(std::move(transfer_));
+    delete this;
+  }
+
+  virtual void Abort(const Status &status) OVERRIDE {
+    transfer_->Abort(status);
+    delete this;
+  }
+
+ private:
+  gscoped_ptr<OutboundTransfer> transfer_;
+  Connection *conn_;
+};
+
+void Connection::QueueResponseForCall(gscoped_ptr<InboundCall> call) {
+  // This is usually called by the IPC worker thread when the response
+  // is set, but in some circumstances may also be called by the
+  // reactor thread (e.g. if the service has shut down)
+
+  DCHECK_EQ(direction_, SERVER);
+
+  // If the connection is torn down, then the QueueOutbound() call that
+  // eventually runs in the reactor thread will take care of calling
+  // ResponseTransferCallbacks::NotifyTransferAborted.
+
+  TransferPayload tmp_slices;
+  size_t n_slices = call->SerializeResponseTo(&tmp_slices);
+
+  TransferCallbacks *cb = new ResponseTransferCallbacks(std::move(call), this);
+  // After the response is sent, can delete the InboundCall object.
+  // We set a dummy call ID and required feature set, since these are not needed
+  // when sending responses.
+  gscoped_ptr<OutboundTransfer> t(
+      OutboundTransfer::CreateForCallResponse(tmp_slices, n_slices, cb));
+
+  QueueTransferTask *task = new QueueTransferTask(std::move(t), this);
+  reactor_thread_->reactor()->ScheduleReactorTask(task);
+}
+
+void Connection::set_confidential(bool is_confidential) {
+  is_confidential_ = is_confidential;
+}
+
+bool Connection::SatisfiesCredentialsPolicy(CredentialsPolicy policy) const {
+  DCHECK_EQ(direction_, CLIENT);
+  return (policy == CredentialsPolicy::ANY_CREDENTIALS) ||
+      (policy == credentials_policy_);
+}
+
+RpczStore* Connection::rpcz_store() {
+  return reactor_thread_->reactor()->messenger()->rpcz_store();
+}
+
+void Connection::ReadHandler(ev::io &watcher, int revents) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  DVLOG(3) << ToString() << " ReadHandler(revents=" << revents << ")";
+  if (revents & EV_ERROR) {
+    reactor_thread_->DestroyConnection(this, Status::NetworkError(ToString() +
+                                     ": ReadHandler encountered an error"));
+    return;
+  }
+  last_activity_time_ = reactor_thread_->cur_time();
+
+  while (true) {
+    if (!inbound_) {
+      inbound_.reset(new InboundTransfer());
+    }
+    Status status = inbound_->ReceiveBuffer(*socket_);
+    if (PREDICT_FALSE(!status.ok())) {
+      if (status.posix_code() == ESHUTDOWN) {
+        VLOG(1) << ToString() << " shut down by remote end.";
+      } else {
+        LOG(WARNING) << ToString() << " recv error: " << status.ToString();
+      }
+      reactor_thread_->DestroyConnection(this, status);
+      return;
+    }
+    if (!inbound_->TransferFinished()) {
+      DVLOG(3) << ToString() << ": read is not yet finished yet.";
+      return;
+    }
+    DVLOG(3) << ToString() << ": finished reading " << inbound_->data().size() << " bytes";
+
+    if (direction_ == CLIENT) {
+      HandleCallResponse(std::move(inbound_));
+    } else if (direction_ == SERVER) {
+      HandleIncomingCall(std::move(inbound_));
+    } else {
+      LOG(FATAL) << "Invalid direction: " << direction_;
+    }
+
+    // TODO: it would seem that it would be good to loop around and see if
+    // there is more data on the socket by trying another recv(), but it turns
+    // out that it really hurts throughput to do so. A better approach
+    // might be for each InboundTransfer to actually try to read an extra byte,
+    // and if it succeeds, then we'd copy that byte into a new InboundTransfer
+    // and loop around, since it's likely the next call also arrived at the
+    // same time.
+    break;
+  }
+}
+
+void Connection::HandleIncomingCall(gscoped_ptr<InboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  gscoped_ptr<InboundCall> call(new InboundCall(this));
+  Status s = call->ParseFrom(std::move(transfer));
+  if (!s.ok()) {
+    LOG(WARNING) << ToString() << ": received bad data: " << s.ToString();
+    // TODO: shutdown? probably, since any future stuff on this socket will be
+    // "unsynchronized"
+    return;
+  }
+
+  if (!InsertIfNotPresent(&calls_being_handled_, call->call_id(), call.get())) {
+    LOG(WARNING) << ToString() << ": received call ID " << call->call_id() <<
+      " but was already processing this ID! Ignoring";
+    reactor_thread_->DestroyConnection(
+      this, Status::RuntimeError("Received duplicate call id",
+                                 Substitute("$0", call->call_id())));
+    return;
+  }
+
+  reactor_thread_->reactor()->messenger()->QueueInboundCall(std::move(call));
+}
+
+void Connection::HandleCallResponse(gscoped_ptr<InboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  gscoped_ptr<CallResponse> resp(new CallResponse);
+  CHECK_OK(resp->ParseFrom(std::move(transfer)));
+
+  CallAwaitingResponse *car_ptr =
+    EraseKeyReturnValuePtr(&awaiting_response_, resp->call_id());
+  if (PREDICT_FALSE(car_ptr == nullptr)) {
+    LOG(WARNING) << ToString() << ": Got a response for call id " << resp->call_id() << " which "
+                 << "was not pending! Ignoring.";
+    return;
+  }
+
+  // The car->timeout_timer ev::timer will be stopped automatically by its destructor.
+  scoped_car car(car_pool_.make_scoped_ptr(car_ptr));
+
+  if (PREDICT_FALSE(!car->call)) {
+    // The call already failed due to a timeout.
+    VLOG(1) << "Got response to call id " << resp->call_id() << " after client "
+            << "already timed out or cancelled";
+    return;
+  }
+
+  car->call->SetResponse(std::move(resp));
+
+  // Test cancellation when 'car->call' is in 'FINISHED_SUCCESS' or 'FINISHED_ERROR' state.
+  MaybeInjectCancellation(car->call);
+}
+
+void Connection::WriteHandler(ev::io &watcher, int revents) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (revents & EV_ERROR) {
+    reactor_thread_->DestroyConnection(this, Status::NetworkError(ToString() +
+          ": writeHandler encountered an error"));
+    return;
+  }
+  DVLOG(3) << ToString() << ": writeHandler: revents = " << revents;
+
+  OutboundTransfer *transfer;
+  if (outbound_transfers_.empty()) {
+    LOG(WARNING) << ToString() << " got a ready-to-write callback, but there is "
+      "nothing to write.";
+    write_io_.stop();
+    return;
+  }
+
+  while (!outbound_transfers_.empty()) {
+    transfer = &(outbound_transfers_.front());
+
+    if (!transfer->TransferStarted()) {
+
+      if (transfer->is_for_outbound_call()) {
+        CallAwaitingResponse* car = FindOrDie(awaiting_response_, transfer->call_id());
+        if (!car->call) {
+          // If the call has already timed out or has already been cancelled, the 'call'
+          // field would be set to NULL. In that case, don't bother sending it.
+          outbound_transfers_.pop_front();
+          transfer->Abort(Status::Aborted("already timed out or cancelled"));
+          delete transfer;
+          continue;
+        }
+
+        // If this is the start of the transfer, then check if the server has the
+        // required RPC flags. We have to wait until just before the transfer in
+        // order to ensure that the negotiation has taken place, so that the flags
+        // are available.
+        const set<RpcFeatureFlag>& required_features = car->call->required_rpc_features();
+        if (!includes(remote_features_.begin(), remote_features_.end(),
+                      required_features.begin(), required_features.end())) {
+          outbound_transfers_.pop_front();
+          Status s = Status::NotSupported("server does not support the required RPC features");
+          transfer->Abort(s);
+          Phase phase = negotiation_complete_ ? Phase::REMOTE_CALL : Phase::CONNECTION_NEGOTIATION;
+          car->call->SetFailed(std::move(s), phase);
+          // Test cancellation when 'call_' is in 'FINISHED_ERROR' state.
+          MaybeInjectCancellation(car->call);
+          car->call.reset();
+          delete transfer;
+          continue;
+        }
+
+        car->call->SetSending();
+
+        // Test cancellation when 'call_' is in 'SENDING' state.
+        MaybeInjectCancellation(car->call);
+      }
+    }
+
+    last_activity_time_ = reactor_thread_->cur_time();
+    Status status = transfer->SendBuffer(*socket_);
+    if (PREDICT_FALSE(!status.ok())) {
+      LOG(WARNING) << ToString() << " send error: " << status.ToString();
+      reactor_thread_->DestroyConnection(this, status);
+      return;
+    }
+
+    if (!transfer->TransferFinished()) {
+      DVLOG(3) << ToString() << ": writeHandler: xfer not finished.";
+      return;
+    }
+
+    outbound_transfers_.pop_front();
+    delete transfer;
+  }
+
+  // If we were able to write all of our outbound transfers,
+  // we don't have any more to write.
+  write_io_.stop();
+}
+
+std::string Connection::ToString() const {
+  // This may be called from other threads, so we cannot
+  // include anything in the output about the current state,
+  // which might concurrently change from another thread.
+  return strings::Substitute(
+    "$0 $1",
+    direction_ == SERVER ? "server connection from" : "client connection to",
+    remote_.ToString());
+}
+
+// Reactor task that transitions this Connection from connection negotiation to
+// regular RPC handling. Destroys Connection on negotiation error.
+class NegotiationCompletedTask : public ReactorTask {
+ public:
+  NegotiationCompletedTask(Connection* conn,
+                           Status negotiation_status,
+                           std::unique_ptr<ErrorStatusPB> rpc_error)
+    : conn_(conn),
+      negotiation_status_(std::move(negotiation_status)),
+      rpc_error_(std::move(rpc_error)) {
+  }
+
+  virtual void Run(ReactorThread *rthread) OVERRIDE {
+    rthread->CompleteConnectionNegotiation(conn_,
+                                           negotiation_status_,
+                                           std::move(rpc_error_));
+    delete this;
+  }
+
+  virtual void Abort(const Status &status) OVERRIDE {
+    DCHECK(conn_->reactor_thread()->reactor()->closing());
+    VLOG(1) << "Failed connection negotiation due to shut down reactor thread: "
+            << status.ToString();
+    delete this;
+  }
+
+ private:
+  scoped_refptr<Connection> conn_;
+  const Status negotiation_status_;
+  std::unique_ptr<ErrorStatusPB> rpc_error_;
+};
+
+void Connection::CompleteNegotiation(Status negotiation_status,
+                                     unique_ptr<ErrorStatusPB> rpc_error) {
+  auto task = new NegotiationCompletedTask(
+      this, std::move(negotiation_status), std::move(rpc_error));
+  reactor_thread_->reactor()->ScheduleReactorTask(task);
+}
+
+void Connection::MarkNegotiationComplete() {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  negotiation_complete_ = true;
+}
+
+Status Connection::DumpPB(const DumpRunningRpcsRequestPB& req,
+                          RpcConnectionPB* resp) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  resp->set_remote_ip(remote_.ToString());
+  if (negotiation_complete_) {
+    resp->set_state(RpcConnectionPB::OPEN);
+  } else {
+    resp->set_state(RpcConnectionPB::NEGOTIATING);
+  }
+
+  if (direction_ == CLIENT) {
+    for (const car_map_t::value_type& entry : awaiting_response_) {
+      CallAwaitingResponse *c = entry.second;
+      if (c->call) {
+        c->call->DumpPB(req, resp->add_calls_in_flight());
+      }
+    }
+
+    resp->set_outbound_queue_size(num_queued_outbound_transfers());
+  } else if (direction_ == SERVER) {
+    if (negotiation_complete_) {
+      // It's racy to dump credentials while negotiating, since the Connection
+      // object is owned by the negotiation thread at that point.
+      resp->set_remote_user_credentials(remote_user_.ToString());
+    }
+    for (const inbound_call_map_t::value_type& entry : calls_being_handled_) {
+      InboundCall* c = entry.second;
+      c->DumpPB(req, resp->add_calls_in_flight());
+    }
+  } else {
+    LOG(FATAL);
+  }
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/connection.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/connection.h b/be/src/kudu/rpc/connection.h
new file mode 100644
index 0000000..362a35b
--- /dev/null
+++ b/be/src/kudu/rpc/connection.h
@@ -0,0 +1,391 @@
+// 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.
+
+#ifndef KUDU_RPC_CONNECTION_H
+#define KUDU_RPC_CONNECTION_H
+
+#include <cstddef>
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include <boost/intrusive/list.hpp>
+#include <boost/optional/optional.hpp>
+#include <ev++.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/object_pool.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+namespace rpc {
+
+class DumpRunningRpcsRequestPB;
+class InboundCall;
+class OutboundCall;
+class RpcConnectionPB;
+class ReactorThread;
+class RpczStore;
+enum class CredentialsPolicy;
+
+//
+// A connection between an endpoint and us.
+//
+// Inbound connections are created by AcceptorPools, which eventually schedule
+// RegisterConnection() to be called from the reactor thread.
+//
+// Outbound connections are created by the Reactor thread in order to service
+// outbound calls.
+//
+// Once a Connection is created, it can be used both for sending messages and
+// receiving them, but any given connection is explicitly a client or server.
+// If a pair of servers are making bidirectional RPCs, they will use two separate
+// TCP connections (and Connection objects).
+//
+// This class is not fully thread-safe.  It is accessed only from the context of a
+// single ReactorThread except where otherwise specified.
+//
+class Connection : public RefCountedThreadSafe<Connection> {
+ public:
+  enum Direction {
+    // This host is sending calls via this connection.
+    CLIENT,
+    // This host is receiving calls via this connection.
+    SERVER
+  };
+
+  // Create a new Connection.
+  // reactor_thread: the reactor that owns us.
+  // remote: the address of the remote end
+  // socket: the socket to take ownership of.
+  // direction: whether we are the client or server side
+  Connection(ReactorThread *reactor_thread,
+             Sockaddr remote,
+             std::unique_ptr<Socket> socket,
+             Direction direction,
+             CredentialsPolicy policy = CredentialsPolicy::ANY_CREDENTIALS);
+
+  // Set underlying socket to non-blocking (or blocking) mode.
+  Status SetNonBlocking(bool enabled);
+
+  // Register our socket with an epoll loop.  We will only ever be registered in
+  // one epoll loop at a time.
+  void EpollRegister(ev::loop_ref& loop);
+
+  ~Connection();
+
+  MonoTime last_activity_time() const {
+    return last_activity_time_;
+  }
+
+  // Returns true if we are not in the process of receiving or sending a
+  // message, and we have no outstanding calls.
+  bool Idle() const;
+
+  // Fail any calls which are currently queued or awaiting response.
+  // Prohibits any future calls (they will be failed immediately with this
+  // same Status).
+  void Shutdown(const Status& status,
+                std::unique_ptr<ErrorStatusPB> rpc_error = {});
+
+  // Queue a new call to be made. If the queueing fails, the call will be
+  // marked failed. The caller is expected to check if 'call' has been cancelled
+  // before making the call.
+  // Takes ownership of the 'call' object regardless of whether it succeeds or fails.
+  void QueueOutboundCall(std::shared_ptr<OutboundCall> call);
+
+  // Queue a call response back to the client on the server side.
+  //
+  // This may be called from a non-reactor thread.
+  void QueueResponseForCall(gscoped_ptr<InboundCall> call);
+
+  // Cancel an outbound call by removing any reference to it by CallAwaitingResponse
+  // in 'awaiting_responses_'.
+  void CancelOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // The address of the remote end of the connection.
+  const Sockaddr &remote() const { return remote_; }
+
+  // Set the user credentials for an outbound connection.
+  void set_outbound_connection_id(ConnectionId conn_id) {
+    DCHECK_EQ(direction_, CLIENT);
+    DCHECK(!outbound_connection_id_);
+    outbound_connection_id_ = std::move(conn_id);
+  }
+
+  // Get the user credentials which will be used to log in.
+  const ConnectionId& outbound_connection_id() const {
+    DCHECK_EQ(direction_, CLIENT);
+    DCHECK(outbound_connection_id_);
+    return *outbound_connection_id_;
+  }
+
+  bool is_confidential() const {
+    return is_confidential_;
+  }
+
+  // Set/unset the 'confidentiality' property for this connection.
+  void set_confidential(bool is_confidential);
+
+  // Credentials policy to start connection negotiation.
+  CredentialsPolicy credentials_policy() const { return credentials_policy_; }
+
+  // Whether the connection satisfies the specified credentials policy.
+  //
+  // NOTE: The policy is set prior to connection negotiation, and the actual
+  //       authentication credentials used for connection negotiation might
+  //       effectively make the connection to satisfy a stronger policy.
+  //       An example: the credentials policy for the connection was set to
+  //       ANY_CREDENTIALS, but since the authn token was not available
+  //       at the time of negotiation, the primary credentials were used, making
+  //       the connection de facto satisfying the PRIMARY_CREDENTIALS policy.
+  bool SatisfiesCredentialsPolicy(CredentialsPolicy policy) const;
+
+  RpczStore* rpcz_store();
+
+  // libev callback when data is available to read.
+  void ReadHandler(ev::io &watcher, int revents);
+
+  // libev callback when we may write to the socket.
+  void WriteHandler(ev::io &watcher, int revents);
+
+  // Safe to be called from other threads.
+  std::string ToString() const;
+
+  Direction direction() const { return direction_; }
+
+  Socket* socket() { return socket_.get(); }
+
+  // Go through the process of transferring control of the underlying socket back to the Reactor.
+  void CompleteNegotiation(Status negotiation_status,
+                           std::unique_ptr<ErrorStatusPB> rpc_error);
+
+  // Indicate that negotiation is complete and that the Reactor is now in control of the socket.
+  void MarkNegotiationComplete();
+
+  Status DumpPB(const DumpRunningRpcsRequestPB& req,
+                RpcConnectionPB* resp);
+
+  ReactorThread* reactor_thread() const { return reactor_thread_; }
+
+  std::unique_ptr<Socket> release_socket() {
+    return std::move(socket_);
+  }
+
+  void adopt_socket(std::unique_ptr<Socket> socket) {
+    socket_ = std::move(socket);
+  }
+
+  void set_remote_features(std::set<RpcFeatureFlag> remote_features) {
+    remote_features_ = std::move(remote_features);
+  }
+
+  void set_remote_user(RemoteUser user) {
+    DCHECK_EQ(direction_, SERVER);
+    remote_user_ = std::move(user);
+  }
+
+  const RemoteUser& remote_user() const {
+    DCHECK_EQ(direction_, SERVER);
+    return remote_user_;
+  }
+
+  // Whether the connection is scheduled for shutdown.
+  bool scheduled_for_shutdown() const {
+    DCHECK_EQ(direction_, CLIENT);
+    return scheduled_for_shutdown_;
+  }
+
+  // Mark the connection as scheduled to be shut down. Reactor does not dispatch
+  // new calls on such a connection.
+  void set_scheduled_for_shutdown() {
+    DCHECK_EQ(direction_, CLIENT);
+    scheduled_for_shutdown_ = true;
+  }
+
+  size_t num_queued_outbound_transfers() const {
+    return outbound_transfers_.size();
+  }
+
+ private:
+  friend struct CallAwaitingResponse;
+  friend class QueueTransferTask;
+  friend struct CallTransferCallbacks;
+  friend struct ResponseTransferCallbacks;
+
+  // A call which has been fully sent to the server, which we're waiting for
+  // the server to process. This is used on the client side only.
+  struct CallAwaitingResponse {
+    ~CallAwaitingResponse();
+
+    // Notification from libev that the call has timed out.
+    void HandleTimeout(ev::timer &watcher, int revents);
+
+    Connection *conn;
+    std::shared_ptr<OutboundCall> call;
+    ev::timer timeout_timer;
+
+    // We time out RPC calls in two stages. This is set to the amount of timeout
+    // remaining after the next timeout fires. See Connection::QueueOutboundCall().
+    double remaining_timeout;
+  };
+
+  typedef std::unordered_map<uint64_t, CallAwaitingResponse*> car_map_t;
+  typedef std::unordered_map<uint64_t, InboundCall*> inbound_call_map_t;
+
+  // Returns the next valid (positive) sequential call ID by incrementing a counter
+  // and ensuring we roll over from INT32_MAX to 0.
+  // Negative numbers are reserved for special purposes.
+  int32_t GetNextCallId() {
+    int32_t call_id = next_call_id_;
+    if (PREDICT_FALSE(next_call_id_ == std::numeric_limits<int32_t>::max())) {
+      next_call_id_ = 0;
+    } else {
+      next_call_id_++;
+    }
+    return call_id;
+  }
+
+  // An incoming packet has completed transferring on the server side.
+  // This parses the call and delivers it into the call queue.
+  void HandleIncomingCall(gscoped_ptr<InboundTransfer> transfer);
+
+  // An incoming packet has completed on the client side. This parses the
+  // call response, looks up the CallAwaitingResponse, and calls the
+  // client callback.
+  void HandleCallResponse(gscoped_ptr<InboundTransfer> transfer);
+
+  // The given CallAwaitingResponse has elapsed its user-defined timeout.
+  // Set it to Failed.
+  void HandleOutboundCallTimeout(CallAwaitingResponse *car);
+
+  // Queue a transfer for sending on this connection.
+  // We will take ownership of the transfer.
+  // This must be called from the reactor thread.
+  void QueueOutbound(gscoped_ptr<OutboundTransfer> transfer);
+
+  // Internal test function for injecting cancellation request when 'call'
+  // reaches state specified in 'FLAGS_rpc_inject_cancellation_state'.
+  void MaybeInjectCancellation(const std::shared_ptr<OutboundCall> &call);
+
+  // The reactor thread that created this connection.
+  ReactorThread* const reactor_thread_;
+
+  // The remote address we're talking to.
+  const Sockaddr remote_;
+
+  // The socket we're communicating on.
+  std::unique_ptr<Socket> socket_;
+
+  // The ConnectionId that serves as a key into the client connection map
+  // within this reactor. Only set in the case of outbound connections.
+  boost::optional<ConnectionId> outbound_connection_id_;
+
+  // The authenticated remote user (if this is an inbound connection on the server).
+  RemoteUser remote_user_;
+
+  // whether we are client or server
+  Direction direction_;
+
+  // The last time we read or wrote from the socket.
+  MonoTime last_activity_time_;
+
+  // the inbound transfer, if any
+  gscoped_ptr<InboundTransfer> inbound_;
+
+  // notifies us when our socket is writable.
+  ev::io write_io_;
+
+  // notifies us when our socket is readable.
+  ev::io read_io_;
+
+  // Set to true when the connection is registered on a loop.
+  // This is used for a sanity check in the destructor that we are properly
+  // un-registered before shutting down.
+  bool is_epoll_registered_;
+
+  // waiting to be sent
+  boost::intrusive::list<OutboundTransfer> outbound_transfers_; // NOLINT(*)
+
+  // Calls which have been sent and are now waiting for a response.
+  car_map_t awaiting_response_;
+
+  // Calls which have been received on the server and are currently
+  // being handled.
+  inbound_call_map_t calls_being_handled_;
+
+  // the next call ID to use
+  int32_t next_call_id_;
+
+  // Starts as Status::OK, gets set to a shutdown status upon Shutdown().
+  Status shutdown_status_;
+
+  // RPC features supported by the remote end of the connection.
+  std::set<RpcFeatureFlag> remote_features_;
+
+  // Pool from which CallAwaitingResponse objects are allocated.
+  // Also a funny name.
+  ObjectPool<CallAwaitingResponse> car_pool_;
+  typedef ObjectPool<CallAwaitingResponse>::scoped_ptr scoped_car;
+
+  // The credentials policy to use for connection negotiation. It defines which
+  // type of user credentials used to negotiate a connection. The actual type of
+  // credentials used for authentication during the negotiation process depends
+  // on the credentials availability, but the result credentials guaranteed to
+  // always satisfy the specified credentials policy. In other words, the actual
+  // type of credentials used for connection negotiation might effectively make
+  // the connection to satisfy a stronger/narrower policy.
+  //
+  // An example:
+  //   The credentials policy for the connection was set to ANY_CREDENTIALS,
+  //   but since no secondary credentials (such authn token) were available
+  //   at the time of negotiation, the primary credentials were used,making the
+  //   connection satisfying the PRIMARY_CREDENTIALS policy de facto.
+  const CredentialsPolicy credentials_policy_;
+
+  // Whether we completed connection negotiation.
+  bool negotiation_complete_;
+
+  // Whether it's OK to pass confidential information over the connection.
+  // For example, an encrypted (but not necessarily authenticated) connection
+  // is considered confidential.
+  bool is_confidential_;
+
+  // Whether the connection is scheduled for shutdown.
+  bool scheduled_for_shutdown_;
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/connection_id.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/connection_id.cc b/be/src/kudu/rpc/connection_id.cc
new file mode 100644
index 0000000..6720807
--- /dev/null
+++ b/be/src/kudu/rpc/connection_id.cc
@@ -0,0 +1,85 @@
+// 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 "kudu/rpc/connection_id.h"
+
+#include <cstddef>
+#include <utility>
+
+#include <boost/functional/hash/hash.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+ConnectionId::ConnectionId() {}
+
+ConnectionId::ConnectionId(const Sockaddr& remote,
+                           std::string hostname,
+                           UserCredentials user_credentials)
+    : remote_(remote),
+      hostname_(std::move(hostname)),
+      user_credentials_(std::move(user_credentials)) {
+  CHECK(!hostname_.empty());
+}
+
+void ConnectionId::set_user_credentials(UserCredentials user_credentials) {
+  DCHECK(user_credentials.has_real_user());
+  user_credentials_ = std::move(user_credentials);
+}
+
+string ConnectionId::ToString() const {
+  string remote;
+  if (hostname_ != remote_.host()) {
+    remote = strings::Substitute("$0 ($1)", remote_.ToString(), hostname_);
+  } else {
+    remote = remote_.ToString();
+  }
+
+  return strings::Substitute("{remote=$0, user_credentials=$1}",
+                             remote,
+                             user_credentials_.ToString());
+}
+
+size_t ConnectionId::HashCode() const {
+  size_t seed = 0;
+  boost::hash_combine(seed, remote_.HashCode());
+  boost::hash_combine(seed, hostname_);
+  boost::hash_combine(seed, user_credentials_.HashCode());
+  return seed;
+}
+
+bool ConnectionId::Equals(const ConnectionId& other) const {
+  return remote() == other.remote() &&
+      hostname_ == other.hostname_ &&
+      user_credentials().Equals(other.user_credentials());
+}
+
+size_t ConnectionIdHash::operator() (const ConnectionId& conn_id) const {
+  return conn_id.HashCode();
+}
+
+bool ConnectionIdEqual::operator() (const ConnectionId& cid1, const ConnectionId& cid2) const {
+  return cid1.Equals(cid2);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/connection_id.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/connection_id.h b/be/src/kudu/rpc/connection_id.h
new file mode 100644
index 0000000..67a4786
--- /dev/null
+++ b/be/src/kudu/rpc/connection_id.h
@@ -0,0 +1,84 @@
+// 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.
+#pragma once
+
+#include <cstddef>
+#include <string>
+
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/util/net/sockaddr.h"
+
+namespace kudu {
+namespace rpc {
+
+// Used to key on Connection information.
+// For use as a key in an unordered STL collection, use ConnectionIdHash and ConnectionIdEqual.
+// This class is copyable for STL compatibility, but not assignable (use CopyFrom() for that).
+class ConnectionId {
+ public:
+  ConnectionId();
+
+  // Copy constructor required for use with STL unordered_map.
+  ConnectionId(const ConnectionId& other) = default;
+
+  // Convenience constructor.
+  ConnectionId(const Sockaddr& remote,
+               std::string hostname,
+               UserCredentials user_credentials);
+
+  // The remote address.
+  const Sockaddr& remote() const { return remote_; }
+
+  const std::string& hostname() const { return hostname_; }
+
+  // The credentials of the user associated with this connection, if any.
+  void set_user_credentials(UserCredentials user_credentials);
+
+  const UserCredentials& user_credentials() const { return user_credentials_; }
+
+  // Copy state from another object to this one.
+  void CopyFrom(const ConnectionId& other);
+
+  // Returns a string representation of the object, not including the password field.
+  std::string ToString() const;
+
+  size_t HashCode() const;
+  bool Equals(const ConnectionId& other) const;
+
+ private:
+  // Remember to update HashCode() and Equals() when new fields are added.
+  Sockaddr remote_;
+
+  // The original host name before it was resolved to 'remote_'.
+  // This must be retained since it is used to compute Kerberos Service Principal Names (SPNs).
+  std::string hostname_;
+
+  UserCredentials user_credentials_;
+};
+
+class ConnectionIdHash {
+ public:
+  std::size_t operator() (const ConnectionId& conn_id) const;
+};
+
+class ConnectionIdEqual {
+ public:
+  bool operator() (const ConnectionId& cid1, const ConnectionId& cid2) const;
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/constants.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/constants.cc b/be/src/kudu/rpc/constants.cc
new file mode 100644
index 0000000..a4e024c
--- /dev/null
+++ b/be/src/kudu/rpc/constants.cc
@@ -0,0 +1,37 @@
+// 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 "kudu/rpc/constants.h"
+
+using std::set;
+
+namespace kudu {
+namespace rpc {
+
+const char* const kMagicNumber = "hrpc";
+const char* const kSaslAppName = "kudu";
+
+// NOTE: the TLS flag is dynamically added based on the local encryption
+// configuration.
+//
+// NOTE: the TLS_AUTHENTICATION_ONLY flag is dynamically added on both
+// sides based on the remote peer's address.
+set<RpcFeatureFlag> kSupportedServerRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS };
+set<RpcFeatureFlag> kSupportedClientRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS };
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/constants.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/constants.h b/be/src/kudu/rpc/constants.h
new file mode 100644
index 0000000..a3c7c67
--- /dev/null
+++ b/be/src/kudu/rpc/constants.h
@@ -0,0 +1,60 @@
+// 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.
+
+#ifndef KUDU_RPC_RPC_CONSTANTS_H
+#define KUDU_RPC_RPC_CONSTANTS_H
+
+#include <cstdint>
+#include <set>
+
+#include "kudu/rpc/rpc_header.pb.h"
+
+namespace kudu {
+namespace rpc {
+
+// Magic number bytes sent at connection setup time.
+extern const char* const kMagicNumber;
+
+// App name for SASL library init
+extern const char* const kSaslAppName;
+
+// Current version of the RPC protocol.
+static const uint32_t kCurrentRpcVersion = 9;
+
+// From Hadoop.
+static const int32_t kInvalidCallId = -2;
+static const int32_t kConnectionContextCallId = -3;
+static const int32_t kNegotiateCallId = -33;
+
+static const uint8_t kMagicNumberLength = 4;
+static const uint8_t kHeaderFlagsLength = 3;
+
+// There is a 4-byte length prefix before any packet.
+static const uint8_t kMsgLengthPrefixLength = 4;
+
+// The set of RPC features that this server build supports.
+// Non-const for testing.
+extern std::set<RpcFeatureFlag> kSupportedServerRpcFeatureFlags;
+
+// The set of RPC features that this client build supports.
+// Non-const for testing.
+extern std::set<RpcFeatureFlag> kSupportedClientRpcFeatureFlags;
+
+} // namespace rpc
+} // namespace kudu
+
+#endif // KUDU_RPC_RPC_CONSTANTS_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/exactly_once_rpc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/exactly_once_rpc-test.cc b/be/src/kudu/rpc/exactly_once_rpc-test.cc
new file mode 100644
index 0000000..c94e89c
--- /dev/null
+++ b/be/src/kudu/rpc/exactly_once_rpc-test.cc
@@ -0,0 +1,629 @@
+// 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 <unistd.h>
+
+#include <atomic>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/request_tracker.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/rpc/result_tracker.h"
+#include "kudu/rpc/retriable_rpc.h"
+#include "kudu/rpc/rpc-test-base.h"
+#include "kudu/rpc/rpc.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/rtest.proxy.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+DECLARE_int64(remember_clients_ttl_ms);
+DECLARE_int64(remember_responses_ttl_ms);
+DECLARE_int64(result_tracker_gc_interval_ms);
+
+using kudu::pb_util::SecureDebugString;
+using kudu::pb_util::SecureShortDebugString;
+using std::atomic_int;
+using std::shared_ptr;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+class Messenger;
+
+namespace {
+
+const char* kClientId = "test-client";
+
+void AddRequestId(RpcController* controller,
+                  const std::string& client_id,
+                  ResultTracker::SequenceNumber sequence_number,
+                  int64_t attempt_no) {
+  unique_ptr<RequestIdPB> request_id(new RequestIdPB());
+  request_id->set_client_id(client_id);
+  request_id->set_seq_no(sequence_number);
+  request_id->set_attempt_no(attempt_no);
+  request_id->set_first_incomplete_seq_no(sequence_number);
+  controller->SetRequestIdPB(std::move(request_id));
+}
+
+class TestServerPicker : public ServerPicker<CalculatorServiceProxy> {
+ public:
+  explicit TestServerPicker(CalculatorServiceProxy* proxy) : proxy_(proxy) {}
+
+  void PickLeader(const ServerPickedCallback& callback, const MonoTime& deadline) override {
+    callback.Run(Status::OK(), proxy_);
+  }
+
+  void MarkServerFailed(CalculatorServiceProxy*, const Status&) override {}
+  void MarkReplicaNotLeader(CalculatorServiceProxy*) override {}
+  void MarkResourceNotFound(CalculatorServiceProxy*) override {}
+
+ private:
+  CalculatorServiceProxy* proxy_;
+};
+
+} // anonymous namespace
+
+class CalculatorServiceRpc : public RetriableRpc<CalculatorServiceProxy,
+                                                 ExactlyOnceRequestPB,
+                                                 ExactlyOnceResponsePB> {
+ public:
+  CalculatorServiceRpc(const scoped_refptr<TestServerPicker>& server_picker,
+                       const scoped_refptr<RequestTracker>& request_tracker,
+                       const MonoTime& deadline,
+                       shared_ptr<Messenger> messenger,
+                       int value,
+                       CountDownLatch* latch,
+                       int server_sleep = 0)
+      : RetriableRpc(server_picker, request_tracker, deadline, std::move(messenger)),
+        latch_(latch) {
+    req_.set_value_to_add(value);
+    req_.set_randomly_fail(true);
+    req_.set_sleep_for_ms(server_sleep);
+  }
+
+  void Try(CalculatorServiceProxy* server, const ResponseCallback& callback) override {
+    server->AddExactlyOnceAsync(req_,
+                                &resp_,
+                                mutable_retrier()->mutable_controller(),
+                                callback);
+  }
+
+  RetriableRpcStatus AnalyzeResponse(const Status& rpc_cb_status) override {
+    // We shouldn't get errors from the server/rpc system since we set a high timeout.
+    CHECK_OK(rpc_cb_status);
+
+    if (!mutable_retrier()->controller().status().ok()) {
+      CHECK(mutable_retrier()->controller().status().IsRemoteError());
+      if (mutable_retrier()->controller().error_response()->code()
+          == ErrorStatusPB::ERROR_REQUEST_STALE) {
+        return { RetriableRpcStatus::NON_RETRIABLE_ERROR,
+              mutable_retrier()->controller().status() };
+      }
+      return { RetriableRpcStatus::SERVICE_UNAVAILABLE,
+               mutable_retrier()->controller().status() };
+    }
+
+    // If the controller is not finished we're in the ReplicaFoundCb() callback.
+    // Return ok to proceed with the call to the server.
+    if (!mutable_retrier()->mutable_controller()->finished()) {
+      return { RetriableRpcStatus::OK, Status::OK() };
+    }
+
+    // If we've received a response in the past, all following responses must
+    // match.
+    if (!successful_response_.IsInitialized()) {
+      successful_response_.CopyFrom(resp_);
+    } else {
+      CHECK_EQ(SecureDebugString(successful_response_),
+               SecureDebugString(resp_));
+    }
+
+    if (sometimes_retry_successful_) {
+      // Still report errors, with some probability. This will cause requests to
+      // be retried. Since the requests were originally successful we should get
+      // the same reply back.
+      int random = rand() % 4;
+      switch (random) {
+        case 0:
+          return { RetriableRpcStatus::SERVICE_UNAVAILABLE,
+                   Status::RemoteError("") };
+        case 1:
+          return { RetriableRpcStatus::RESOURCE_NOT_FOUND,
+                   Status::RemoteError("") };
+        case 2:
+          return { RetriableRpcStatus::SERVER_NOT_ACCESSIBLE,
+                   Status::RemoteError("") };
+        case 3:
+          return { RetriableRpcStatus::OK, Status::OK() };
+        default: LOG(FATAL) << "Unexpected value";
+      }
+    }
+    return { RetriableRpcStatus::OK, Status::OK() };
+  }
+
+  void Finish(const Status& status) override {
+    CHECK_OK(status);
+    latch_->CountDown();
+    delete this;
+  }
+
+  std::string ToString() const override { return "test-rpc"; }
+  CountDownLatch* latch_;
+  ExactlyOnceResponsePB successful_response_;
+  bool sometimes_retry_successful_ = true;
+};
+
+class ExactlyOnceRpcTest : public RpcTestBase {
+ public:
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    SeedRandom();
+  }
+
+  Status StartServer() {
+    // Set up server.
+    RETURN_NOT_OK(StartTestServerWithGeneratedCode(&server_addr_));
+    RETURN_NOT_OK(CreateMessenger("Client", &client_messenger_));
+    proxy_.reset(new CalculatorServiceProxy(
+        client_messenger_, server_addr_, server_addr_.host()));
+    test_picker_.reset(new TestServerPicker(proxy_.get()));
+    request_tracker_.reset(new RequestTracker(kClientId));
+    attempt_nos_ = 0;
+
+    return Status::OK();
+  }
+
+  // An exactly once adder that uses RetriableRpc to perform the requests.
+  struct RetriableRpcExactlyOnceAdder {
+    RetriableRpcExactlyOnceAdder(const scoped_refptr<TestServerPicker>& server_picker,
+                     const scoped_refptr<RequestTracker>& request_tracker,
+                     shared_ptr<Messenger> messenger,
+                     int value,
+                     int server_sleep = 0) : latch_(1) {
+      MonoTime now = MonoTime::Now();
+      now.AddDelta(MonoDelta::FromMilliseconds(10000));
+      rpc_ = new CalculatorServiceRpc(server_picker,
+                                      request_tracker,
+                                      now,
+                                      std::move(messenger),
+                                      value,
+                                      &latch_,
+                                      server_sleep);
+    }
+
+    void Start() {
+      CHECK_OK(kudu::Thread::Create(
+                   "test",
+                   "test",
+                   &RetriableRpcExactlyOnceAdder::SleepAndSend, this, &thread));
+    }
+
+    void SleepAndSend() {
+      rpc_->SendRpc();
+      latch_.Wait();
+    }
+
+    CountDownLatch latch_;
+    scoped_refptr<kudu::Thread> thread;
+    CalculatorServiceRpc* rpc_;
+  };
+
+  // An exactly once adder that sends multiple, simultaneous calls, to the server
+  // and makes sure that only one of the calls was successful.
+  struct SimultaneousExactlyOnceAdder {
+    SimultaneousExactlyOnceAdder(CalculatorServiceProxy* p,
+                     ResultTracker::SequenceNumber sequence_number,
+                     int value,
+                     uint64_t client_sleep,
+                     uint64_t server_sleep,
+                     int64_t attempt_no)
+     : proxy(p),
+       client_sleep_for_ms(client_sleep) {
+      req.set_value_to_add(value);
+      req.set_sleep_for_ms(server_sleep);
+      AddRequestId(&controller, kClientId, sequence_number, attempt_no);
+    }
+
+    void Start() {
+      CHECK_OK(kudu::Thread::Create(
+          "test",
+          "test",
+          &SimultaneousExactlyOnceAdder::SleepAndSend, this, &thread));
+    }
+
+    // Sleeps the preset number of msecs before sending the call.
+    void SleepAndSend() {
+      usleep(client_sleep_for_ms * 1000);
+      controller.set_timeout(MonoDelta::FromSeconds(20));
+      CHECK_OK(proxy->AddExactlyOnce(req, &resp, &controller));
+    }
+
+    CalculatorServiceProxy* const proxy;
+    const uint64_t client_sleep_for_ms;
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    ExactlyOnceResponsePB resp;
+    scoped_refptr<kudu::Thread> thread;
+  };
+
+
+  void CheckValueMatches(int expected_value) {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(0);
+    ExactlyOnceResponsePB resp;
+    RequestTracker::SequenceNumber seq_no;
+    CHECK_OK(request_tracker_->NewSeqNo(&seq_no));
+    AddRequestId(&controller, kClientId, seq_no, 0);
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(resp.current_val(), expected_value);
+    request_tracker_->RpcCompleted(seq_no);
+  }
+
+
+  // This continuously issues calls to the server, that often last longer than
+  // 'remember_responses_ttl_ms', making sure that we don't get errors back.
+  void DoLongWritesThread(MonoDelta run_for) {
+    MonoTime run_until = MonoTime::Now();
+    run_until.AddDelta(run_for);
+    int counter = 0;
+    while (MonoTime::Now() < run_until) {
+      unique_ptr<RetriableRpcExactlyOnceAdder> adder(new RetriableRpcExactlyOnceAdder(
+          test_picker_, request_tracker_, client_messenger_, 1,
+          rand() % (2 * FLAGS_remember_responses_ttl_ms)));
+
+      // This thread is used in the stress test where we're constantly running GC.
+      // So, once we get a "success" response, it's likely that the result will be
+      // GCed on the server side, and thus it's not safe to spuriously retry.
+      adder->rpc_->sometimes_retry_successful_ = false;
+      adder->SleepAndSend();
+      SleepFor(MonoDelta::FromMilliseconds(rand() % 10));
+      counter++;
+    }
+    ExactlyOnceResponsePB response;
+    ResultTracker::SequenceNumber sequence_number;
+    CHECK_OK(request_tracker_->NewSeqNo(&sequence_number));
+    CHECK_OK(MakeAddCall(sequence_number, 0, &response));
+    CHECK_EQ(response.current_val(), counter);
+    request_tracker_->RpcCompleted(sequence_number);
+  }
+
+  // Stubbornly sends the same request to the server, this should observe three states.
+  // The request should be successful at first, then its result should be GCed and the
+  // client should be GCed.
+  void StubbornlyWriteTheSameRequestThread(ResultTracker::SequenceNumber sequence_number,
+                                           MonoDelta run_for) {
+    MonoTime run_until = MonoTime::Now();
+    run_until.AddDelta(run_for);
+    // Make an initial request, so that we get a response to compare to.
+    ExactlyOnceResponsePB original_response;
+    CHECK_OK(MakeAddCall(sequence_number, 0, &original_response));
+
+    // Now repeat the same request. At first we should get the same response, then the result
+    // should be GCed and we should get STALE back. Finally the request should succeed again
+    // but we should get a new response.
+    bool result_gced = false;
+    bool client_gced = false;
+    while (MonoTime::Now() < run_until) {
+      ExactlyOnceResponsePB response;
+      Status s = MakeAddCall(sequence_number, 0, &response);
+      if (s.ok()) {
+        if (!result_gced) {
+          CHECK_EQ(SecureDebugString(response), SecureDebugString(original_response));
+        } else {
+          client_gced = true;
+          CHECK_NE(SecureDebugString(response), SecureDebugString(original_response));
+        }
+        SleepFor(MonoDelta::FromMilliseconds(rand() % 10));
+      } else if (s.IsRemoteError()) {
+        result_gced = true;
+        SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_clients_ttl_ms * 2));
+      }
+    }
+    CHECK(result_gced);
+    CHECK(client_gced);
+  }
+
+  Status MakeAddCall(ResultTracker::SequenceNumber sequence_number,
+                     int value_to_add,
+                     ExactlyOnceResponsePB* response,
+                     int attempt_no = -1) {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(value_to_add);
+    if (attempt_no == -1) attempt_no = attempt_nos_.fetch_add(1);
+    AddRequestId(&controller, kClientId, sequence_number, attempt_no);
+    Status s = proxy_->AddExactlyOnce(req, response, &controller);
+    return s;
+  }
+
+ protected:
+  Sockaddr server_addr_;
+  atomic_int attempt_nos_;
+  shared_ptr<Messenger> client_messenger_;
+  std::unique_ptr<CalculatorServiceProxy> proxy_;
+  scoped_refptr<TestServerPicker> test_picker_;
+  scoped_refptr<RequestTracker> request_tracker_;
+};
+
+// Tests that we get exactly once semantics on RPCs when we send a bunch of requests with the
+// same sequence number as previous requests.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsAfterRpcCompleted) {
+  ASSERT_OK(StartServer());
+  ExactlyOnceResponsePB original_resp;
+  int mem_consumption = mem_tracker_->consumption();
+  {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(1);
+
+    // Assign id 0.
+    AddRequestId(&controller, kClientId, 0, 0);
+
+    // Send the request the first time.
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &original_resp, &controller));
+
+    // The incremental usage of a new client is the size of the response itself
+    // plus some fixed overhead for the client-tracking structure.
+    int expected_incremental_usage = original_resp.SpaceUsed() + 200;
+
+    int mem_consumption_after = mem_tracker_->consumption();
+    ASSERT_GT(mem_consumption_after - mem_consumption, expected_incremental_usage);
+    mem_consumption = mem_consumption_after;
+  }
+
+  // Now repeat the rpc 10 times, using the same sequence number, none of these should be executed
+  // and they should get the same response back.
+  for (int i = 0; i < 10; i++) {
+    RpcController controller;
+    controller.set_timeout(MonoDelta::FromSeconds(20));
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(1);
+    ExactlyOnceResponsePB resp;
+    AddRequestId(&controller, kClientId, 0, i + 1);
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(resp.current_val(), 1);
+    ASSERT_EQ(resp.current_time_micros(), original_resp.current_time_micros());
+    // Sleep to give the MemTracker time to update -- we don't expect any update,
+    // but if we had a bug here, we'd only see it with this sleep.
+    SleepFor(MonoDelta::FromMilliseconds(100));
+    // We shouldn't have consumed any more memory since the responses were cached.
+    ASSERT_EQ(mem_consumption, mem_tracker_->consumption());
+  }
+
+  // Making a new request, from a new client, should double the memory consumption.
+  {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    ExactlyOnceResponsePB resp;
+    req.set_value_to_add(1);
+
+    // Assign id 0.
+    AddRequestId(&controller, "test-client2", 0, 0);
+
+    // Send the first request for this new client.
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(mem_tracker_->consumption(), mem_consumption * 2);
+  }
+}
+
+// Performs a series of requests in which each single request is attempted multiple times, as
+// the server side is instructed to spuriously fail attempts.
+// In CalculatorServiceRpc we sure that the same response is returned by all retries and,
+// after all the rpcs are done, we make sure that final result is the expected one.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsWithReplicatedRpc) {
+  ASSERT_OK(StartServer());
+  int kNumIterations = 10;
+  int kNumRpcs = 10;
+
+  if (AllowSlowTests()) {
+    kNumIterations = 100;
+    kNumRpcs = 100;
+  }
+
+  int count = 0;
+  for (int i = 0; i < kNumIterations; i ++) {
+    vector<unique_ptr<RetriableRpcExactlyOnceAdder>> adders;
+    for (int j = 0; j < kNumRpcs; j++) {
+      unique_ptr<RetriableRpcExactlyOnceAdder> adder(
+          new RetriableRpcExactlyOnceAdder(test_picker_, request_tracker_, client_messenger_, j));
+      adders.push_back(std::move(adder));
+      adders[j]->Start();
+      count += j;
+    }
+    for (int j = 0; j < kNumRpcs; j++) {
+      CHECK_OK(ThreadJoiner(adders[j]->thread.get()).Join());
+    }
+    CheckValueMatches(count);
+  }
+}
+
+// Performs a series of requests in which each single request is attempted by multiple threads.
+// On each iteration, after all the threads complete, we expect that the add operation was
+// executed exactly once.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsWithConcurrentUpdaters) {
+  ASSERT_OK(StartServer());
+  int kNumIterations = 10;
+  int kNumThreads = 10;
+
+  if (AllowSlowTests()) {
+    kNumIterations = 100;
+    kNumThreads = 100;
+  }
+
+  ResultTracker::SequenceNumber sequence_number = 0;
+  int memory_consumption_initial = mem_tracker_->consumption();
+  int single_response_size = 0;
+
+  // Measure memory consumption for a single response from the same client.
+  ExactlyOnceResponsePB resp;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+
+  for (int i = 1; i <= kNumIterations; i ++) {
+    vector<unique_ptr<SimultaneousExactlyOnceAdder>> adders;
+    for (int j = 0; j < kNumThreads; j++) {
+      unique_ptr<SimultaneousExactlyOnceAdder> adder(
+          new SimultaneousExactlyOnceAdder(proxy_.get(),
+                                           i, // sequence number
+                                           1, // value
+                                           rand() % 20, // client_sleep
+                                           rand() % 10, // server_sleep
+                                           attempt_nos_.fetch_add(1))); // attempt number
+      adders.push_back(std::move(adder));
+      adders[j]->Start();
+    }
+    uint64_t time_micros = 0;
+    for (int j = 0; j < kNumThreads; j++) {
+      CHECK_OK(ThreadJoiner(adders[j]->thread.get()).Join());
+      ASSERT_EQ(adders[j]->resp.current_val(), i + 1);
+      if (time_micros == 0) {
+        time_micros = adders[j]->resp.current_time_micros();
+      } else {
+        ASSERT_EQ(adders[j]->resp.current_time_micros(), time_micros);
+      }
+    }
+
+    // After all adders finished we should at least the size of one more response.
+    // The actual size depends of multiple factors, for instance, how many calls were "attached"
+    // (which is timing dependent) so we can't be more precise than this.
+    ASSERT_GT(mem_tracker_->consumption(),
+              memory_consumption_initial + single_response_size * i);
+  }
+}
+
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollection) {
+  FLAGS_remember_clients_ttl_ms = 500;
+  FLAGS_remember_responses_ttl_ms = 100;
+
+  ASSERT_OK(StartServer());
+
+  // Make a request.
+  ExactlyOnceResponsePB original;
+  ResultTracker::SequenceNumber sequence_number = 0;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &original));
+
+  // Making the same request again, should return the same response.
+  ExactlyOnceResponsePB resp;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+  ASSERT_EQ(SecureShortDebugString(original), SecureShortDebugString(resp));
+
+  // Now sleep for 'remember_responses_ttl_ms' and run GC, we should then
+  // get a STALE back.
+  SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_responses_ttl_ms));
+  int64_t memory_consumption = mem_tracker_->consumption();
+  result_tracker_->GCResults();
+  ASSERT_LT(mem_tracker_->consumption(), memory_consumption);
+
+  resp.Clear();
+  Status s = MakeAddCall(sequence_number, 1, &resp);
+  ASSERT_TRUE(s.IsRemoteError());
+  ASSERT_STR_CONTAINS(s.ToString(), "is stale");
+
+  // Sleep again, this time for 'remember_clients_ttl_ms' and run GC again.
+  // The request should be successful, but its response should be a new one.
+  SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_clients_ttl_ms));
+  memory_consumption = mem_tracker_->consumption();
+  result_tracker_->GCResults();
+  ASSERT_LT(mem_tracker_->consumption(), memory_consumption);
+
+  resp.Clear();
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+  ASSERT_NE(SecureShortDebugString(resp), SecureShortDebugString(original));
+}
+
+// This test creates a thread continuously making requests to the server, some lasting longer
+// than the GC period, at the same time it runs GC, making sure that the corresponding
+// CompletionRecords/ClientStates are not deleted from underneath the ongoing requests.
+// This also creates a thread that runs GC very frequently and another thread that sends the
+// same request over and over and observes the possible states: request is ok, request is stale
+// request is ok again (because the client was forgotten).
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollectionStressTest) {
+  FLAGS_remember_clients_ttl_ms = 100;
+  FLAGS_remember_responses_ttl_ms = 10;
+  FLAGS_result_tracker_gc_interval_ms = 10;
+
+  ASSERT_OK(StartServer());
+
+  // The write thread runs for a shorter period to make sure client GC has a
+  // chance to run.
+  MonoDelta writes_run_for = MonoDelta::FromSeconds(2);
+  MonoDelta stubborn_run_for = MonoDelta::FromSeconds(3);
+  if (AllowSlowTests()) {
+    writes_run_for = MonoDelta::FromSeconds(10);
+    stubborn_run_for = MonoDelta::FromSeconds(11);
+  }
+
+  result_tracker_->StartGCThread();
+
+  // Assign the first sequence number (0) to the 'stubborn writes' thread.
+  // This thread will keep making RPCs with this sequence number while
+  // the 'write_thread' will make normal requests with increasing sequence
+  // numbers.
+  ResultTracker::SequenceNumber stubborn_req_seq_num;
+  CHECK_OK(request_tracker_->NewSeqNo(&stubborn_req_seq_num));
+  ASSERT_EQ(stubborn_req_seq_num, 0);
+
+  scoped_refptr<kudu::Thread> stubborn_thread;
+  CHECK_OK(kudu::Thread::Create(
+      "stubborn", "stubborn", &ExactlyOnceRpcTest::StubbornlyWriteTheSameRequestThread,
+      this, stubborn_req_seq_num, stubborn_run_for, &stubborn_thread));
+
+  scoped_refptr<kudu::Thread> write_thread;
+  CHECK_OK(kudu::Thread::Create(
+      "write", "write", &ExactlyOnceRpcTest::DoLongWritesThread,
+      this, writes_run_for, &write_thread));
+
+  write_thread->Join();
+  stubborn_thread->Join();
+
+  // Within a few seconds, the consumption should be back to zero.
+  // Really, this should be within 100ms, but we'll give it a bit of
+  // time to avoid test flakiness.
+  AssertEventually([&]() {
+      ASSERT_EQ(0, mem_tracker_->consumption());
+    }, MonoDelta::FromSeconds(5));
+  NO_PENDING_FATALS();
+}
+
+
+} // namespace rpc
+} // namespace kudu


[43/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/negotiation-test.cc b/be/src/kudu/rpc/negotiation-test.cc
new file mode 100644
index 0000000..976f590
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation-test.cc
@@ -0,0 +1,1346 @@
+// 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 "kudu/rpc/rpc-test-base.h"
+
+#include <sys/stat.h>
+#include <unistd.h>
+
+#include <cstdio>
+#include <cstdlib>
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/client_negotiation.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/security-test-util.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/test/mini_kdc.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/tls_socket.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_signer.h"
+#include "kudu/security/token_signing_key.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/user.h"
+
+// HACK: MIT Kerberos doesn't have any way of determining its version number,
+// but the error messages in krb5-1.10 and earlier are broken due to
+// a bug: http://krbdev.mit.edu/rt/Ticket/Display.html?id=6973
+//
+// Since we don't have any way to explicitly figure out the version, we just
+// look for this random macro which was added in 1.11 (the same version in which
+// the above bug was fixed).
+#ifndef KRB5_RESPONDER_QUESTION_PASSWORD
+#define KRB5_VERSION_LE_1_10
+#endif
+
+DEFINE_bool(is_test_child, false,
+            "Used by tests which require clean processes. "
+            "See TestDisableInit.");
+DECLARE_bool(rpc_encrypt_loopback_connections);
+DECLARE_bool(rpc_trace_negotiation);
+
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+using kudu::security::Cert;
+using kudu::security::PkiConfig;
+using kudu::security::PrivateKey;
+using kudu::security::SignedTokenPB;
+using kudu::security::TlsContext;
+using kudu::security::TokenSigner;
+using kudu::security::TokenSigningPrivateKey;
+using kudu::security::TokenVerifier;
+
+namespace kudu {
+namespace rpc {
+
+// The negotiation configuration for a client or server endpoint.
+struct EndpointConfig {
+  // The PKI configuration.
+  PkiConfig pki;
+  // The supported SASL mechanisms.
+  vector<SaslMechanism::Type> sasl_mechs;
+  // For the client, whether the client has the token.
+  // For the server, whether the server has the TSK.
+  bool token;
+  RpcEncryption encryption;
+};
+std::ostream& operator<<(std::ostream& o, EndpointConfig config) {
+  auto bool_string = [] (bool b) { return b ? "true" : "false"; };
+  o << "{pki: " << config.pki
+    << ", sasl-mechs: [" << JoinMapped(config.sasl_mechs, SaslMechanism::name_of, ", ")
+    << "], token: " << bool_string(config.token)
+    << ", encryption: ";
+
+  switch (config.encryption) {
+    case RpcEncryption::DISABLED: o << "DISABLED"; break;
+    case RpcEncryption::OPTIONAL: o << "OPTIONAL"; break;
+    case RpcEncryption::REQUIRED: o << "REQUIRED"; break;
+  }
+
+  o << "}";
+  return o;
+}
+
+// A description of a negotiation sequence, including client and server
+// configuration, as well as expected results.
+struct NegotiationDescriptor {
+  EndpointConfig client;
+  EndpointConfig server;
+
+  bool use_test_socket;
+
+  bool rpc_encrypt_loopback;
+
+  // The expected client status from negotiating.
+  Status client_status;
+  // The expected server status from negotiating.
+  Status server_status;
+
+  // The expected negotiated authentication type.
+  AuthenticationType negotiated_authn;
+
+  // The expected SASL mechanism, if SASL authentication is negotiated.
+  SaslMechanism::Type negotiated_mech;
+
+  // Whether the negotiation is expected to perform a TLS handshake.
+  bool tls_negotiated;
+};
+std::ostream& operator<<(std::ostream& o, NegotiationDescriptor c) {
+  auto bool_string = [] (bool b) { return b ? "true" : "false"; };
+  o << "{client: " << c.client
+    << ", server: " << c.server
+    << "}, rpc-encrypt-loopback: " << bool_string(c.rpc_encrypt_loopback);
+  return o;
+}
+
+class NegotiationTestSocket : public Socket {
+ public:
+  // Return an arbitrary public IP
+  Status GetPeerAddress(Sockaddr *cur_addr) const override {
+    return cur_addr->ParseString("8.8.8.8:12345", 0);
+  }
+};
+
+class TestNegotiation : public RpcTestBase,
+                        public ::testing::WithParamInterface<NegotiationDescriptor> {
+ public:
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    ASSERT_OK(SaslInit());
+  }
+};
+
+TEST_P(TestNegotiation, TestNegotiation) {
+  NegotiationDescriptor desc = GetParam();
+
+  // Generate a trusted root certificate.
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  // Create and configure a TLS context for each endpoint.
+  TlsContext client_tls_context;
+  TlsContext server_tls_context;
+  ASSERT_OK(client_tls_context.Init());
+  ASSERT_OK(server_tls_context.Init());
+  ASSERT_OK(ConfigureTlsContext(desc.client.pki, ca_cert, ca_key, &client_tls_context));
+  ASSERT_OK(ConfigureTlsContext(desc.server.pki, ca_cert, ca_key, &server_tls_context));
+
+  FLAGS_rpc_encrypt_loopback_connections = desc.rpc_encrypt_loopback;
+
+  // Generate an optional client token and server token verifier.
+  TokenSigner token_signer(60, 20, std::make_shared<TokenVerifier>());
+  {
+    unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(token_signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(token_signer.AddKey(std::move(key)));
+  }
+  TokenVerifier token_verifier;
+  boost::optional<SignedTokenPB> authn_token;
+  if (desc.client.token) {
+    authn_token = SignedTokenPB();
+    security::TokenPB token;
+    token.set_expire_unix_epoch_seconds(WallTime_Now() + 60);
+    token.mutable_authn()->set_username("client-token");
+    ASSERT_TRUE(token.SerializeToString(authn_token->mutable_token_data()));
+    ASSERT_OK(token_signer.SignToken(&*authn_token));
+  }
+  if (desc.server.token) {
+    ASSERT_OK(token_verifier.ImportKeys(token_signer.verifier().ExportKeys()));
+  }
+
+  // Create the listening socket, client socket, and server socket.
+  Socket listening_socket;
+  ASSERT_OK(listening_socket.Init(0));
+  ASSERT_OK(listening_socket.BindAndListen(Sockaddr(), 1));
+  Sockaddr server_addr;
+  ASSERT_OK(listening_socket.GetSocketAddress(&server_addr));
+
+  unique_ptr<Socket> client_socket(new Socket());
+  ASSERT_OK(client_socket->Init(0));
+  client_socket->Connect(server_addr);
+
+  unique_ptr<Socket> server_socket(desc.use_test_socket ?
+                                   new NegotiationTestSocket() :
+                                   new Socket());
+
+  Sockaddr client_addr;
+  CHECK_OK(listening_socket.Accept(server_socket.get(), &client_addr, 0));
+
+  // Create and configure the client and server negotiation instances.
+  ClientNegotiation client_negotiation(std::move(client_socket),
+                                       &client_tls_context,
+                                       authn_token,
+                                       desc.client.encryption,
+                                       "kudu");
+  ServerNegotiation server_negotiation(std::move(server_socket),
+                                       &server_tls_context,
+                                       &token_verifier,
+                                       desc.server.encryption,
+                                       "kudu");
+
+  // Set client and server SASL mechanisms.
+  MiniKdc kdc;
+  bool kdc_started = false;
+  auto start_kdc_once = [&] () {
+    if (!kdc_started) {
+      kdc_started = true;
+      RETURN_NOT_OK(kdc.Start());
+    }
+    return Status::OK();
+  };
+  for (auto mech : desc.client.sasl_mechs) {
+    switch (mech) {
+      case SaslMechanism::INVALID: break;
+      case SaslMechanism::PLAIN:
+        ASSERT_OK(client_negotiation.EnablePlain("client-plain", "client-password"));
+        break;
+      case SaslMechanism::GSSAPI:
+        ASSERT_OK(start_kdc_once());
+        ASSERT_OK(kdc.CreateUserPrincipal("client-gssapi"));
+        ASSERT_OK(kdc.Kinit("client-gssapi"));
+        ASSERT_OK(kdc.SetKrb5Environment());
+        client_negotiation.set_server_fqdn("127.0.0.1");
+        ASSERT_OK(client_negotiation.EnableGSSAPI());
+        break;
+    }
+  }
+  for (auto mech : desc.server.sasl_mechs) {
+    switch (mech) {
+      case SaslMechanism::INVALID: break;
+      case SaslMechanism::PLAIN:
+        ASSERT_OK(server_negotiation.EnablePlain());
+        break;
+      case SaslMechanism::GSSAPI:
+        ASSERT_OK(start_kdc_once());
+        // Create the server principal and keytab.
+        string kt_path;
+        ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+        CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+        server_negotiation.set_server_fqdn("127.0.0.1");
+        ASSERT_OK(server_negotiation.EnableGSSAPI());
+        break;
+    }
+  }
+
+  // Run the client/server negotiation. Because negotiation is blocking, it
+  // has to be done on separate threads.
+  Status client_status;
+  Status server_status;
+  thread client_thread([&] () {
+      scoped_refptr<Trace> t(new Trace());
+      ADOPT_TRACE(t.get());
+      client_status = client_negotiation.Negotiate();
+      // Close the socket so that the server will not block forever on error.
+      client_negotiation.socket()->Close();
+
+      if (FLAGS_rpc_trace_negotiation || !client_status.ok()) {
+        string msg = Trace::CurrentTrace()->DumpToString();
+        if (!client_status.ok()) {
+          LOG(WARNING) << "Failed client RPC negotiation. Client trace:\n" << msg;
+        } else {
+          LOG(INFO) << "RPC negotiation tracing enabled. Client trace:\n" << msg;
+        }
+      }
+  });
+  thread server_thread([&] () {
+      scoped_refptr<Trace> t(new Trace());
+      ADOPT_TRACE(t.get());
+      server_status = server_negotiation.Negotiate();
+      // Close the socket so that the client will not block forever on error.
+      server_negotiation.socket()->Close();
+
+      if (FLAGS_rpc_trace_negotiation || !server_status.ok()) {
+        string msg = Trace::CurrentTrace()->DumpToString();
+        if (!server_status.ok()) {
+          LOG(WARNING) << "Failed server RPC negotiation. Server trace:\n" << msg;
+        } else {
+          LOG(INFO) << "RPC negotiation tracing enabled. Server trace:\n" << msg;
+        }
+      }
+  });
+  client_thread.join();
+  server_thread.join();
+
+  // Check the negotiation outcome against the expected outcome.
+  EXPECT_EQ(desc.client_status.CodeAsString(), client_status.CodeAsString());
+  EXPECT_EQ(desc.server_status.CodeAsString(), server_status.CodeAsString());
+  ASSERT_STR_MATCHES(client_status.ToString(), desc.client_status.ToString());
+  ASSERT_STR_MATCHES(server_status.ToString(), desc.server_status.ToString());
+
+  if (client_status.ok()) {
+    EXPECT_TRUE(server_status.ok());
+
+    // Make sure the negotiations agree with the expected values.
+    EXPECT_EQ(desc.negotiated_authn, client_negotiation.negotiated_authn());
+    EXPECT_EQ(desc.negotiated_mech, client_negotiation.negotiated_mechanism());
+    EXPECT_EQ(desc.negotiated_authn, server_negotiation.negotiated_authn());
+    EXPECT_EQ(desc.negotiated_mech, server_negotiation.negotiated_mechanism());
+    EXPECT_EQ(desc.tls_negotiated, server_negotiation.tls_negotiated());
+    EXPECT_EQ(desc.tls_negotiated, server_negotiation.tls_negotiated());
+
+    bool client_tls_socket = dynamic_cast<security::TlsSocket*>(client_negotiation.socket());
+    bool server_tls_socket = dynamic_cast<security::TlsSocket*>(server_negotiation.socket());
+    EXPECT_EQ(desc.rpc_encrypt_loopback, client_tls_socket);
+    EXPECT_EQ(desc.rpc_encrypt_loopback, server_tls_socket);
+
+    // Check that the expected user subject is authenticated.
+    RemoteUser remote_user = server_negotiation.take_authenticated_user();
+    switch (server_negotiation.negotiated_authn()) {
+      case AuthenticationType::SASL:
+        switch (server_negotiation.negotiated_mechanism()) {
+          case SaslMechanism::PLAIN:
+            EXPECT_EQ("client-plain", remote_user.username());
+            break;
+          case SaslMechanism::GSSAPI:
+            EXPECT_EQ("client-gssapi", remote_user.username());
+            EXPECT_EQ("client-gssapi@KRBTEST.COM", remote_user.principal().value_or(""));
+            break;
+          case SaslMechanism::INVALID: LOG(FATAL) << "invalid mechanism negotiated";
+        }
+        break;
+      case AuthenticationType::CERTIFICATE: {
+        // We expect the cert to be using the local username, because it hasn't
+        // logged in from any Keytab.
+        string expected;
+        CHECK_OK(GetLoggedInUser(&expected));
+        EXPECT_EQ(expected, remote_user.username());
+        EXPECT_FALSE(remote_user.principal());
+        break;
+      }
+      case AuthenticationType::TOKEN:
+        EXPECT_EQ("client-token", remote_user.username());
+        break;
+      case AuthenticationType::INVALID: LOG(FATAL) << "invalid authentication negotiated";
+    }
+  }
+}
+
+INSTANTIATE_TEST_CASE_P(NegotiationCombinations,
+                        TestNegotiation,
+                        ::testing::Values(
+
+        // client: no authn/mechs
+        // server: no authn/mechs
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client is not configured with an authentication type"),
+          Status::NetworkError(""),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: PLAIN
+        // server: no authn/mechs
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".* server mechanism list is empty"),
+          Status::NotAuthorized(".* server mechanism list is empty"),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: PLAIN
+        // server: PLAIN
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          false,
+        },
+
+        // client: GSSAPI
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: GSSAPI, PLAIN
+        // server: GSSAPI, PLAIN
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: GSSAPI, PLAIN
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: PLAIN
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not have Kerberos credentials available"),
+          Status::NetworkError(""),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: GSSAPI,
+        // server: GSSAPI, self-signed cert
+        // loopback encryption
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, signed-cert
+        // server: GSSAPI, self-signed cert
+        // This tests that the server will not advertise CERTIFICATE authentication,
+        // since it doesn't have a trusted cert.
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: PLAIN,
+        // server: PLAIN, self-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: signed-cert
+        // server: signed-cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::CERTIFICATE,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: token, trusted cert
+        // server: token, signed-cert, GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::TRUSTED,
+            { },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::TOKEN,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: PLAIN, token
+        // server: PLAIN, token, signed cert
+        // Test that the client won't negotiate token authn if it doesn't have a
+        // trusted cert. We aren't expecting this to happen in practice (the
+        // token and trusted CA cert should come as a package).
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: PLAIN, GSSAPI, signed-cert, token
+        // server: PLAIN, GSSAPI, signed-cert, token
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::CERTIFICATE,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: PLAIN, TLS disabled
+        // server: PLAIN, TLS required
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: PLAIN, TLS required
+        // server: PLAIN, TLS disabled
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*server does not support required TLS encryption"),
+          Status::NetworkError(""),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: GSSAPI, TLS required, externally-signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS optional, externally-signed cert
+        // server: GSSAPI, TLS required, signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS required
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, PLAIN, TLS required, externally-signed cert
+        // server: PLAIN, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: GSSAPI, TLS disabled, signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS required, signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: PLAIN
+        // server: PLAIN
+        // connection from public routable IP
+        NegotiationDescriptor {
+            EndpointConfig {
+                PkiConfig::NONE,
+                { SaslMechanism::PLAIN },
+                false,
+                RpcEncryption::OPTIONAL
+            },
+            EndpointConfig {
+                PkiConfig::NONE,
+                { SaslMechanism::PLAIN },
+                false,
+                RpcEncryption::OPTIONAL
+            },
+            true,
+            false,
+            Status::NotAuthorized(".*unencrypted connections from publicly routable IPs"),
+            Status::NotAuthorized(".*unencrypted connections from publicly routable IPs"),
+            AuthenticationType::SASL,
+            SaslMechanism::PLAIN,
+            false,
+        },
+
+        // client: GSSAPI, TLS required, externally-signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        // connection from public routable IP
+        NegotiationDescriptor {
+            EndpointConfig {
+                PkiConfig::EXTERNALLY_SIGNED,
+                { SaslMechanism::GSSAPI },
+                false,
+                RpcEncryption::REQUIRED,
+            },
+            EndpointConfig {
+                PkiConfig::EXTERNALLY_SIGNED,
+                { SaslMechanism::GSSAPI },
+                false,
+                RpcEncryption::REQUIRED,
+            },
+            true,
+            // true as no longer a loopback connection.
+            true,
+            Status::OK(),
+            Status::OK(),
+            AuthenticationType::SASL,
+            SaslMechanism::GSSAPI,
+            true,
+        }
+));
+
+// A "Callable" that takes a socket for use with starting a thread.
+// Can be used for ServerNegotiation or ClientNegotiation threads.
+typedef std::function<void(unique_ptr<Socket>)> SocketCallable;
+
+// Call Accept() on the socket, then pass the connection to the server runner
+static void RunAcceptingDelegator(Socket* acceptor,
+                                  const SocketCallable& server_runner) {
+  unique_ptr<Socket> conn(new Socket());
+  Sockaddr remote;
+  CHECK_OK(acceptor->Accept(conn.get(), &remote, 0));
+  server_runner(std::move(conn));
+}
+
+// Set up a socket and run a negotiation sequence.
+static void RunNegotiationTest(const SocketCallable& server_runner,
+                               const SocketCallable& client_runner) {
+  Socket server_sock;
+  CHECK_OK(server_sock.Init(0));
+  ASSERT_OK(server_sock.BindAndListen(Sockaddr(), 1));
+  Sockaddr server_bind_addr;
+  ASSERT_OK(server_sock.GetSocketAddress(&server_bind_addr));
+  thread server(RunAcceptingDelegator, &server_sock, server_runner);
+
+  unique_ptr<Socket> client_sock(new Socket());
+  CHECK_OK(client_sock->Init(0));
+  ASSERT_OK(client_sock->Connect(server_bind_addr));
+  thread client(client_runner, std::move(client_sock));
+
+  LOG(INFO) << "Waiting for test threads to terminate...";
+  client.join();
+  LOG(INFO) << "Client thread terminated.";
+
+  server.join();
+  LOG(INFO) << "Server thread terminated.";
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+#ifndef __APPLE__
+template<class T>
+using CheckerFunction = std::function<void(const Status&, T&)>;
+
+// Run GSSAPI negotiation from the server side. Runs
+// 'post_check' after negotiation to verify the result.
+static void RunGSSAPINegotiationServer(unique_ptr<Socket> socket,
+                                       const CheckerFunction<ServerNegotiation>& post_check) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  server_negotiation.set_server_fqdn("127.0.0.1");
+  CHECK_OK(server_negotiation.EnableGSSAPI());
+  post_check(server_negotiation.Negotiate(), server_negotiation);
+}
+
+// Run GSSAPI negotiation from the client side. Runs
+// 'post_check' after negotiation to verify the result.
+static void RunGSSAPINegotiationClient(unique_ptr<Socket> conn,
+                                       const CheckerFunction<ClientNegotiation>& post_check) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(conn), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  client_negotiation.set_server_fqdn("127.0.0.1");
+  CHECK_OK(client_negotiation.EnableGSSAPI());
+  post_check(client_negotiation.Negotiate(), client_negotiation);
+}
+
+// Test invalid SASL negotiations using the GSSAPI (kerberos) mechanism over a socket.
+// This test is ignored on macOS because the system Kerberos implementation
+// (Heimdal) caches the non-existence of client credentials, which causes futher
+// tests to fail.
+TEST_F(TestNegotiation, TestGSSAPIInvalidNegotiation) {
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+
+  // Try to negotiate with no krb5 credentials on either side. It should fail on both
+  // sides.
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  // The client notices there are no credentials and
+                  // doesn't send any failure message to the server.
+                  // Instead, it just disconnects.
+                  //
+                  // TODO(todd): it might be preferable to have the server
+                  // fail to start if it has no valid keytab.
+                  CHECK(s.IsNetworkError());
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  CHECK_GT(s.ToString().find("No Kerberos credentials available"), 0);
+#endif
+                }));
+
+  // Create the server principal and keytab.
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  // Try to negotiate with no krb5 credentials on the client. It should fail on both
+  // sides.
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  // The client notices there are no credentials and
+                  // doesn't send any failure message to the server.
+                  // Instead, it just disconnects.
+                  CHECK(s.IsNetworkError());
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+                  ASSERT_STR_MATCHES(s.ToString(),
+                                     "Not authorized: server requires authentication, "
+                                     "but client does not have Kerberos credentials available");
+                }));
+
+  // Create and kinit as a client user.
+  ASSERT_OK(kdc.CreateUserPrincipal("testuser"));
+  ASSERT_OK(kdc.Kinit("testuser"));
+  ASSERT_OK(kdc.SetKrb5Environment());
+
+  // Change the server's keytab file so that it has inappropriate
+  // credentials.
+  // Authentication should now fail.
+  ASSERT_OK(kdc.CreateServiceKeytab("otherservice/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  ASSERT_STR_CONTAINS(s.ToString(),
+                                      "No key table entry found matching kudu/127.0.0.1");
+#endif
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  ASSERT_STR_CONTAINS(s.ToString(),
+                                      "No key table entry found matching kudu/127.0.0.1");
+#endif
+                }));
+}
+#endif
+
+#ifndef __APPLE__
+// Test that the pre-flight check for servers requiring Kerberos provides
+// nice error messages for missing or bad keytabs.
+//
+// This is ignored on macOS because the system Kerberos implementation does not
+// fail the preflight check when the keytab is inaccessible, probably because
+// the preflight check passes a 0-length token.
+TEST_F(TestNegotiation, TestPreflight) {
+  // Try pre-flight with no keytab.
+  Status s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "Key table file.*not found");
+#endif
+  // Try with a valid krb5 environment and keytab.
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+  ASSERT_OK(kdc.SetKrb5Environment());
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  ASSERT_OK(ServerNegotiation::PreflightCheckGSSAPI("kudu"));
+
+  // Try with an inaccessible keytab.
+  CHECK_ERR(chmod(kt_path.c_str(), 0000));
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "error accessing keytab: Permission denied");
+#endif
+  CHECK_ERR(unlink(kt_path.c_str()));
+
+  // Try with a keytab that has the wrong credentials.
+  ASSERT_OK(kdc.CreateServiceKeytab("wrong-service/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "No key table entry found matching kudu/.*");
+#endif
+}
+#endif
+
+////////////////////////////////////////////////////////////////////////////////
+
+static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(server_negotiation.EnablePlain());
+  Status s = server_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsNetworkError()) << "Expected client to time out and close the connection. Got: "
+                                  << s.ToString();
+}
+
+static void RunTimeoutNegotiationClient(unique_ptr<Socket> sock) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(sock), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(client_negotiation.EnablePlain("test", "test"));
+  MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
+  client_negotiation.set_deadline(deadline);
+  Status s = client_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsTimedOut()) << "Expected timeout! Got: " << s.ToString();
+  CHECK_OK(client_negotiation.socket()->Shutdown(true, true));
+}
+
+// Ensure that the client times out.
+TEST_F(TestNegotiation, TestClientTimeout) {
+  RunNegotiationTest(RunTimeoutExpectingServer, RunTimeoutNegotiationClient);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(server_negotiation.EnablePlain());
+  MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
+  server_negotiation.set_deadline(deadline);
+  Status s = server_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsTimedOut()) << "Expected timeout! Got: " << s.ToString();
+  CHECK_OK(server_negotiation.socket()->Close());
+}
+
+static void RunTimeoutExpectingClient(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(socket), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(client_negotiation.EnablePlain("test", "test"));
+  Status s = client_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsNetworkError()) << "Expected server to time out and close the connection. Got: "
+      << s.ToString();
+}
+
+// Ensure that the server times out.
+TEST_F(TestNegotiation, TestServerTimeout) {
+  RunNegotiationTest(RunTimeoutNegotiationServer, RunTimeoutExpectingClient);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+// This suite of tests ensure that applications that embed the Kudu client are
+// able to externally handle the initialization of SASL. See KUDU-1749 and
+// IMPALA-4497 for context.
+//
+// The tests are a bit tricky because the initialization of SASL is static state
+// that we can't easily clear/reset between test cases. So, each test invokes
+// itself as a subprocess with the appropriate --gtest_filter line as well as a
+// special flag to indicate that it is the test child running.
+class TestDisableInit : public KuduTest {
+ protected:
+  // Run the lambda 'f' in a newly-started process, capturing its stderr
+  // into 'stderr'.
+  template<class TestFunc>
+  void DoTest(const TestFunc& f, string* stderr = nullptr) {
+    if (FLAGS_is_test_child) {
+      f();
+      return;
+    }
+
+    // Invoke the currently-running test case in a new subprocess.
+    string filter_flag = strings::Substitute("--gtest_filter=$0.$1",
+                                             CURRENT_TEST_CASE_NAME(), CURRENT_TEST_NAME());
+    string executable_path;
+    CHECK_OK(env_->GetExecutablePath(&executable_path));
+    string stdout;
+    Status s = Subprocess::Call({ executable_path, "test", filter_flag, "--is_test_child" },
+                                "" /* stdin */,
+                                &stdout,
+                                stderr);
+    ASSERT_TRUE(s.ok()) << "Test failed: " << stdout;
+  }
+};
+
+// Test disabling SASL but not actually properly initializing it before usage.
+TEST_F(TestDisableInit, TestDisableSasl_NotInitialized) {
+  DoTest([]() {
+      CHECK_OK(DisableSaslInitialization());
+      Status s = SaslInit();
+      ASSERT_STR_CONTAINS(s.ToString(), "was disabled, but SASL was not externally initialized");
+    });
+}
+
+// Test disabling SASL with proper initialization by some other app.
+TEST_F(TestDisableInit, TestDisableSasl_Good) {
+  DoTest([]() {
+      rpc::internal::SaslSetMutex();
+      sasl_client_init(NULL);
+      CHECK_OK(DisableSaslInitialization());
+      ASSERT_OK(SaslInit());
+    });
+}
+
+// Test a client which inits SASL itself but doesn't remember to disable Kudu's
+// SASL initialization.
+TEST_F(TestDisableInit, TestMultipleSaslInit) {
+  string stderr;
+  DoTest([]() {
+      rpc::internal::SaslSetMutex();
+      sasl_client_init(NULL);
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child that it automatically
+  // skipped initialization because it detected that it was already initialized.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "Skipping initialization");
+  }
+}
+
+// We are not able to detect mutexes not being set with the macOS version of libsasl.
+#ifndef __APPLE__
+// Test disabling SASL but not remembering to initialize the SASL mutex support. This
+// should succeed but generate a warning.
+TEST_F(TestDisableInit, TestDisableSasl_NoMutexImpl) {
+  string stderr;
+  DoTest([]() {
+      sasl_client_init(NULL);
+      CHECK_OK(DisableSaslInitialization());
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "not provided with a mutex implementation");
+  }
+}
+
+// Test a client which inits SASL itself but doesn't remember to disable Kudu's
+// SASL initialization.
+TEST_F(TestDisableInit, TestMultipleSaslInit_NoMutexImpl) {
+  string stderr;
+  DoTest([]() {
+      sasl_client_init(NULL);
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child that it automatically
+  // skipped initialization because it detected that it was already initialized.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "Skipping initialization");
+    ASSERT_STR_CONTAINS(stderr, "not provided with a mutex implementation");
+  }
+}
+#endif
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/negotiation.cc b/be/src/kudu/rpc/negotiation.cc
new file mode 100644
index 0000000..31e0b33
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation.cc
@@ -0,0 +1,327 @@
+// 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 "kudu/rpc/negotiation.h"
+
+#include <poll.h>
+#include <sys/socket.h>
+
+#include <cerrno>
+#include <ctime>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/client_negotiation.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/trace.h"
+
+DEFINE_bool(rpc_trace_negotiation, false,
+            "If enabled, dump traces of all RPC negotiations to the log");
+TAG_FLAG(rpc_trace_negotiation, runtime);
+TAG_FLAG(rpc_trace_negotiation, advanced);
+TAG_FLAG(rpc_trace_negotiation, experimental);
+
+DEFINE_int32(rpc_negotiation_inject_delay_ms, 0,
+             "If enabled, injects the given number of milliseconds delay into "
+             "the RPC negotiation process on the server side.");
+TAG_FLAG(rpc_negotiation_inject_delay_ms, unsafe);
+
+DEFINE_bool(rpc_encrypt_loopback_connections, false,
+            "Whether to encrypt data transfer on RPC connections that stay within "
+            "a single host. Encryption here is likely to offer no additional "
+            "security benefit since only a local 'root' user could intercept the "
+            "traffic, and wire encryption does not suitably protect against such "
+            "an attacker.");
+TAG_FLAG(rpc_encrypt_loopback_connections, advanced);
+
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+const char* AuthenticationTypeToString(AuthenticationType t) {
+  switch (t) {
+    case AuthenticationType::INVALID: return "INVALID"; break;
+    case AuthenticationType::SASL: return "SASL"; break;
+    case AuthenticationType::TOKEN: return "TOKEN"; break;
+    case AuthenticationType::CERTIFICATE: return "CERTIFICATE"; break;
+  }
+  return "<cannot reach here>";
+}
+
+std::ostream& operator<<(std::ostream& o, AuthenticationType authentication_type) {
+  return o << AuthenticationTypeToString(authentication_type);
+}
+
+// Wait for the client connection to be established and become ready for writing.
+static Status WaitForClientConnect(Socket* socket, const MonoTime& deadline) {
+  TRACE("Waiting for socket to connect");
+  int fd = socket->GetFd();
+  struct pollfd poll_fd;
+  poll_fd.fd = fd;
+  poll_fd.events = POLLOUT;
+  poll_fd.revents = 0;
+
+  MonoTime now;
+  MonoDelta remaining;
+  while (true) {
+    now = MonoTime::Now();
+    remaining = deadline - now;
+    DVLOG(4) << "Client waiting to connect for negotiation, time remaining until timeout deadline: "
+             << remaining.ToString();
+    if (PREDICT_FALSE(remaining.ToNanoseconds() <= 0)) {
+      return Status::TimedOut("Timeout exceeded waiting to connect");
+    }
+#if defined(__linux__)
+    struct timespec ts;
+    remaining.ToTimeSpec(&ts);
+    int ready = ppoll(&poll_fd, 1, &ts, NULL);
+#else
+    int ready = poll(&poll_fd, 1, remaining.ToMilliseconds());
+#endif
+    if (ready == -1) {
+      int err = errno;
+      if (err == EINTR) {
+        // We were interrupted by a signal, let's go again.
+        continue;
+      } else {
+        return Status::NetworkError("Error from ppoll() while waiting to connect",
+            ErrnoToString(err), err);
+      }
+    } else if (ready == 0) {
+      // Timeout exceeded. Loop back to the top to our impending doom.
+      continue;
+    } else {
+      // Success.
+      break;
+    }
+  }
+
+  // Connect finished, but this doesn't mean that we connected successfully.
+  // Check the socket for an error.
+  int so_error = 0;
+  socklen_t socklen = sizeof(so_error);
+  int rc = getsockopt(fd, SOL_SOCKET, SO_ERROR, &so_error, &socklen);
+  if (rc != 0) {
+    return Status::NetworkError("Unable to check connected socket for errors",
+                                ErrnoToString(errno),
+                                errno);
+  }
+  if (so_error != 0) {
+    return Status::NetworkError("connect", ErrnoToString(so_error), so_error);
+  }
+
+  return Status::OK();
+}
+
+// Disable / reset socket timeouts.
+static Status DisableSocketTimeouts(Socket* socket) {
+  RETURN_NOT_OK(socket->SetSendTimeout(MonoDelta::FromNanoseconds(0L)));
+  RETURN_NOT_OK(socket->SetRecvTimeout(MonoDelta::FromNanoseconds(0L)));
+  return Status::OK();
+}
+
+// Perform client negotiation. We don't LOG() anything, we leave that to our caller.
+static Status DoClientNegotiation(Connection* conn,
+                                  RpcAuthentication authentication,
+                                  RpcEncryption encryption,
+                                  MonoTime deadline,
+                                  unique_ptr<ErrorStatusPB>* rpc_error) {
+  const auto* messenger = conn->reactor_thread()->reactor()->messenger();
+  // Prefer secondary credentials (such as authn token) if permitted by policy.
+  const auto authn_token = (conn->credentials_policy() == CredentialsPolicy::PRIMARY_CREDENTIALS)
+      ? boost::none : messenger->authn_token();
+  ClientNegotiation client_negotiation(conn->release_socket(),
+                                       &messenger->tls_context(),
+                                       authn_token,
+                                       encryption,
+                                       messenger->sasl_proto_name());
+
+  client_negotiation.set_server_fqdn(conn->outbound_connection_id().hostname());
+
+  if (authentication != RpcAuthentication::DISABLED) {
+    Status s = client_negotiation.EnableGSSAPI();
+    if (!s.ok()) {
+      // If we can't enable GSSAPI, it's likely the client is just missing the
+      // appropriate SASL plugin. We don't want to require it to be installed
+      // if the user doesn't care about connecting to servers using Kerberos
+      // authentication. So, we'll just VLOG this here. If we try to connect
+      // to a server which requires Kerberos, we'll get a negotiation error
+      // at that point.
+      if (VLOG_IS_ON(1)) {
+        KLOG_FIRST_N(INFO, 1) << "Couldn't enable GSSAPI (Kerberos) SASL plugin: "
+                              << s.message().ToString()
+                              << ". This process will be unable to connect to "
+                              << "servers requiring Kerberos authentication.";
+      }
+
+      if (authentication == RpcAuthentication::REQUIRED &&
+          !authn_token &&
+          !messenger->tls_context().has_signed_cert()) {
+        return Status::InvalidArgument(
+            "Kerberos, token, or PKI certificate credentials must be provided in order to "
+            "require authentication for a client");
+      }
+    }
+  }
+
+  if (authentication != RpcAuthentication::REQUIRED) {
+    const auto& creds = conn->outbound_connection_id().user_credentials();
+    RETURN_NOT_OK(client_negotiation.EnablePlain(creds.real_user(), ""));
+  }
+
+  client_negotiation.set_deadline(deadline);
+
+  RETURN_NOT_OK(WaitForClientConnect(client_negotiation.socket(), deadline));
+  RETURN_NOT_OK(client_negotiation.socket()->SetNonBlocking(false));
+  RETURN_NOT_OK(client_negotiation.Negotiate(rpc_error));
+  RETURN_NOT_OK(DisableSocketTimeouts(client_negotiation.socket()));
+
+  // Transfer the negotiated socket and state back to the connection.
+  conn->adopt_socket(client_negotiation.release_socket());
+  conn->set_remote_features(client_negotiation.take_server_features());
+  conn->set_confidential(client_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
+
+  // Sanity check: if no authn token was supplied as user credentials,
+  // the negotiated authentication type cannot be AuthenticationType::TOKEN.
+  DCHECK(!(authn_token == boost::none &&
+           client_negotiation.negotiated_authn() == AuthenticationType::TOKEN));
+
+  return Status::OK();
+}
+
+// Perform server negotiation. We don't LOG() anything, we leave that to our caller.
+static Status DoServerNegotiation(Connection* conn,
+                                  RpcAuthentication authentication,
+                                  RpcEncryption encryption,
+                                  const MonoTime& deadline) {
+  const auto* messenger = conn->reactor_thread()->reactor()->messenger();
+  if (authentication == RpcAuthentication::REQUIRED &&
+      messenger->keytab_file().empty() &&
+      !messenger->tls_context().is_external_cert()) {
+    return Status::InvalidArgument("RPC authentication (--rpc_authentication) may not be "
+                                   "required unless Kerberos (--keytab_file) or external PKI "
+                                   "(--rpc_certificate_file et al) are configured");
+  }
+
+  if (FLAGS_rpc_negotiation_inject_delay_ms > 0) {
+    LOG(WARNING) << "Injecting " << FLAGS_rpc_negotiation_inject_delay_ms
+                 << "ms delay in negotiation";
+    SleepFor(MonoDelta::FromMilliseconds(FLAGS_rpc_negotiation_inject_delay_ms));
+  }
+
+  // Create a new ServerNegotiation to handle the synchronous negotiation.
+  ServerNegotiation server_negotiation(conn->release_socket(),
+                                       &messenger->tls_context(),
+                                       &messenger->token_verifier(),
+                                       encryption,
+                                       messenger->sasl_proto_name());
+
+  if (authentication != RpcAuthentication::DISABLED && !messenger->keytab_file().empty()) {
+    RETURN_NOT_OK(server_negotiation.EnableGSSAPI());
+  }
+  if (authentication != RpcAuthentication::REQUIRED) {
+    RETURN_NOT_OK(server_negotiation.EnablePlain());
+  }
+
+  server_negotiation.set_deadline(deadline);
+
+  RETURN_NOT_OK(server_negotiation.socket()->SetNonBlocking(false));
+
+  RETURN_NOT_OK(server_negotiation.Negotiate());
+  RETURN_NOT_OK(DisableSocketTimeouts(server_negotiation.socket()));
+
+  // Transfer the negotiated socket and state back to the connection.
+  conn->adopt_socket(server_negotiation.release_socket());
+  conn->set_remote_features(server_negotiation.take_client_features());
+  conn->set_remote_user(server_negotiation.take_authenticated_user());
+  conn->set_confidential(server_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
+
+  return Status::OK();
+}
+
+void Negotiation::RunNegotiation(const scoped_refptr<Connection>& conn,
+                                 RpcAuthentication authentication,
+                                 RpcEncryption encryption,
+                                 MonoTime deadline) {
+  Status s;
+  unique_ptr<ErrorStatusPB> rpc_error;
+  if (conn->direction() == Connection::SERVER) {
+    s = DoServerNegotiation(conn.get(), authentication, encryption, deadline);
+  } else {
+    s = DoClientNegotiation(conn.get(), authentication, encryption, deadline,
+                            &rpc_error);
+  }
+
+  if (PREDICT_FALSE(!s.ok())) {
+    string msg = Substitute("$0 connection negotiation failed: $1",
+                            conn->direction() == Connection::SERVER ? "Server" : "Client",
+                            conn->ToString());
+    s = s.CloneAndPrepend(msg);
+  }
+  TRACE("Negotiation complete: $0", s.ToString());
+
+  bool is_bad = !s.ok() && !(
+      (s.IsNetworkError() && s.posix_code() == ECONNREFUSED) ||
+      s.IsNotAuthorized());
+
+  if (is_bad || FLAGS_rpc_trace_negotiation) {
+    string msg = Trace::CurrentTrace()->DumpToString();
+    if (is_bad) {
+      LOG(WARNING) << "Failed RPC negotiation. Trace:\n" << msg;
+    } else {
+      LOG(INFO) << "RPC negotiation tracing enabled. Trace:\n" << msg;
+    }
+  }
+
+  if (conn->direction() == Connection::SERVER && s.IsNotAuthorized()) {
+    LOG(WARNING) << "Unauthorized connection attempt: " << s.message().ToString();
+  }
+  conn->CompleteNegotiation(std::move(s), std::move(rpc_error));
+}
+
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/negotiation.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/negotiation.h b/be/src/kudu/rpc/negotiation.h
new file mode 100644
index 0000000..b25ed0e
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation.h
@@ -0,0 +1,58 @@
+// 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.
+#ifndef KUDU_RPC_NEGOTIATION_H
+#define KUDU_RPC_NEGOTIATION_H
+
+#include <iosfwd>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/security/security_flags.h"
+
+namespace kudu {
+
+class MonoTime;
+
+namespace rpc {
+
+class Connection;
+
+enum class AuthenticationType {
+  INVALID,
+  SASL,
+  TOKEN,
+  CERTIFICATE,
+};
+const char* AuthenticationTypeToString(AuthenticationType t);
+
+std::ostream& operator<<(std::ostream& o, AuthenticationType authentication_type);
+
+class Negotiation {
+ public:
+
+  // Perform negotiation for a connection (either server or client)
+  static void RunNegotiation(const scoped_refptr<Connection>& conn,
+                             security::RpcAuthentication authentication,
+                             security::RpcEncryption encryption,
+                             MonoTime deadline);
+ private:
+  DISALLOW_IMPLICIT_CONSTRUCTORS(Negotiation);
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif // KUDU_RPC_NEGOTIATION_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/outbound_call.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/outbound_call.cc b/be/src/kudu/rpc/outbound_call.cc
new file mode 100644
index 0000000..37d02ac
--- /dev/null
+++ b/be/src/kudu/rpc/outbound_call.cc
@@ -0,0 +1,531 @@
+// 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 <cstdint>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <type_traits>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <boost/function.hpp>
+#include <gflags/gflags.h>
+#include <google/protobuf/message.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/kernel_stack_watchdog.h"
+#include "kudu/util/net/sockaddr.h"
+
+// 100M cycles should be about 50ms on a 2Ghz box. This should be high
+// enough that involuntary context switches don't trigger it, but low enough
+// that any serious blocking behavior on the reactor would.
+DEFINE_int64(rpc_callback_max_cycles, 100 * 1000 * 1000,
+             "The maximum number of cycles for which an RPC callback "
+             "should be allowed to run without emitting a warning."
+             " (Advanced debugging option)");
+TAG_FLAG(rpc_callback_max_cycles, advanced);
+TAG_FLAG(rpc_callback_max_cycles, runtime);
+
+// Flag used in debug build for injecting cancellation at different code paths.
+DEFINE_int32(rpc_inject_cancellation_state, -1,
+             "If this flag is not -1, it is the state in which a cancellation request "
+             "will be injected. Should use values in OutboundCall::State only");
+TAG_FLAG(rpc_inject_cancellation_state, unsafe);
+
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+using google::protobuf::Message;
+using strings::Substitute;
+
+static const double kMicrosPerSecond = 1000000.0;
+
+///
+/// OutboundCall
+///
+
+OutboundCall::OutboundCall(const ConnectionId& conn_id,
+                           const RemoteMethod& remote_method,
+                           google::protobuf::Message* response_storage,
+                           RpcController* controller,
+                           ResponseCallback callback)
+    : state_(READY),
+      remote_method_(remote_method),
+      conn_id_(conn_id),
+      callback_(std::move(callback)),
+      controller_(DCHECK_NOTNULL(controller)),
+      response_(DCHECK_NOTNULL(response_storage)),
+      cancellation_requested_(false) {
+  DVLOG(4) << "OutboundCall " << this << " constructed with state_: " << StateName(state_)
+           << " and RPC timeout: "
+           << (controller->timeout().Initialized() ? controller->timeout().ToString() : "none");
+  header_.set_call_id(kInvalidCallId);
+  remote_method.ToPB(header_.mutable_remote_method());
+  start_time_ = MonoTime::Now();
+
+  if (!controller_->required_server_features().empty()) {
+    required_rpc_features_.insert(RpcFeatureFlag::APPLICATION_FEATURE_FLAGS);
+  }
+
+  if (controller_->request_id_) {
+    header_.set_allocated_request_id(controller_->request_id_.release());
+  }
+}
+
+OutboundCall::~OutboundCall() {
+  DCHECK(IsFinished());
+  DVLOG(4) << "OutboundCall " << this << " destroyed with state_: " << StateName(state_);
+}
+
+size_t OutboundCall::SerializeTo(TransferPayload* slices) {
+  DCHECK_LT(0, request_buf_.size())
+      << "Must call SetRequestPayload() before SerializeTo()";
+
+  const MonoDelta &timeout = controller_->timeout();
+  if (timeout.Initialized()) {
+    header_.set_timeout_millis(timeout.ToMilliseconds());
+  }
+
+  for (uint32_t feature : controller_->required_server_features()) {
+    header_.add_required_feature_flags(feature);
+  }
+
+  DCHECK_LE(0, sidecar_byte_size_);
+  serialization::SerializeHeader(
+      header_, sidecar_byte_size_ + request_buf_.size(), &header_buf_);
+
+  size_t n_slices = 2 + sidecars_.size();
+  DCHECK_LE(n_slices, slices->size());
+  auto slice_iter = slices->begin();
+  *slice_iter++ = Slice(header_buf_);
+  *slice_iter++ = Slice(request_buf_);
+  for (auto& sidecar : sidecars_) {
+    *slice_iter++ = sidecar->AsSlice();
+  }
+  DCHECK_EQ(slice_iter - slices->begin(), n_slices);
+  return n_slices;
+}
+
+void OutboundCall::SetRequestPayload(const Message& req,
+    vector<unique_ptr<RpcSidecar>>&& sidecars) {
+  DCHECK_EQ(-1, sidecar_byte_size_);
+
+  sidecars_ = move(sidecars);
+  DCHECK_LE(sidecars_.size(), TransferLimits::kMaxSidecars);
+
+  // Compute total size of sidecar payload so that extra space can be reserved as part of
+  // the request body.
+  uint32_t message_size = req.ByteSize();
+  sidecar_byte_size_ = 0;
+  for (const unique_ptr<RpcSidecar>& car: sidecars_) {
+    header_.add_sidecar_offsets(sidecar_byte_size_ + message_size);
+    int32_t sidecar_bytes = car->AsSlice().size();
+    DCHECK_LE(sidecar_byte_size_, TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes);
+    sidecar_byte_size_ += sidecar_bytes;
+  }
+
+  serialization::SerializeMessage(req, &request_buf_, sidecar_byte_size_, true);
+}
+
+Status OutboundCall::status() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return status_;
+}
+
+const ErrorStatusPB* OutboundCall::error_pb() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return error_pb_.get();
+}
+
+string OutboundCall::StateName(State state) {
+  switch (state) {
+    case READY:
+      return "READY";
+    case ON_OUTBOUND_QUEUE:
+      return "ON_OUTBOUND_QUEUE";
+    case SENDING:
+      return "SENDING";
+    case SENT:
+      return "SENT";
+    case NEGOTIATION_TIMED_OUT:
+      return "NEGOTIATION_TIMED_OUT";
+    case TIMED_OUT:
+      return "TIMED_OUT";
+    case CANCELLED:
+      return "CANCELLED";
+    case FINISHED_NEGOTIATION_ERROR:
+      return "FINISHED_NEGOTIATION_ERROR";
+    case FINISHED_ERROR:
+      return "FINISHED_ERROR";
+    case FINISHED_SUCCESS:
+      return "FINISHED_SUCCESS";
+    default:
+      LOG(DFATAL) << "Unknown state in OutboundCall: " << state;
+      return StringPrintf("UNKNOWN(%d)", state);
+  }
+}
+
+void OutboundCall::set_state(State new_state) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  set_state_unlocked(new_state);
+}
+
+OutboundCall::State OutboundCall::state() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return state_;
+}
+
+void OutboundCall::set_state_unlocked(State new_state) {
+  // Sanity check state transitions.
+  DVLOG(3) << "OutboundCall " << this << " (" << ToString() << ") switching from " <<
+    StateName(state_) << " to " << StateName(new_state);
+  switch (new_state) {
+    case ON_OUTBOUND_QUEUE:
+      DCHECK_EQ(state_, READY);
+      break;
+    case SENDING:
+      // Allow SENDING to be set idempotently so we don't have to specifically check
+      // whether the state is transitioning in the RPC code.
+      DCHECK(state_ == ON_OUTBOUND_QUEUE || state_ == SENDING);
+      break;
+    case SENT:
+      DCHECK_EQ(state_, SENDING);
+      break;
+    case NEGOTIATION_TIMED_OUT:
+      DCHECK(state_ == ON_OUTBOUND_QUEUE);
+      break;
+    case TIMED_OUT:
+      DCHECK(state_ == SENT || state_ == ON_OUTBOUND_QUEUE || state_ == SENDING);
+      break;
+    case CANCELLED:
+      DCHECK(state_ == READY || state_ == ON_OUTBOUND_QUEUE || state_ == SENT);
+      break;
+    case FINISHED_SUCCESS:
+      DCHECK_EQ(state_, SENT);
+      break;
+    default:
+      // No sanity checks for others.
+      break;
+  }
+
+  state_ = new_state;
+}
+
+void OutboundCall::Cancel() {
+  cancellation_requested_ = true;
+  // No lock needed as it's called from reactor thread
+  switch (state_) {
+    case READY:
+    case ON_OUTBOUND_QUEUE:
+    case SENT: {
+      SetCancelled();
+      break;
+    }
+    case SENDING:
+    case NEGOTIATION_TIMED_OUT:
+    case TIMED_OUT:
+    case CANCELLED:
+    case FINISHED_NEGOTIATION_ERROR:
+    case FINISHED_ERROR:
+    case FINISHED_SUCCESS:
+      break;
+  }
+}
+
+void OutboundCall::CallCallback() {
+  // Clear references to outbound sidecars before invoking callback.
+  sidecars_.clear();
+
+  int64_t start_cycles = CycleClock::Now();
+  {
+    SCOPED_WATCH_STACK(100);
+    callback_();
+    // Clear the callback, since it may be holding onto reference counts
+    // via bound parameters. We do this inside the timer because it's possible
+    // the user has naughty destructors that block, and we want to account for that
+    // time here if they happen to run on this thread.
+    callback_ = NULL;
+  }
+  int64_t end_cycles = CycleClock::Now();
+  int64_t wait_cycles = end_cycles - start_cycles;
+  if (PREDICT_FALSE(wait_cycles > FLAGS_rpc_callback_max_cycles)) {
+    double micros = static_cast<double>(wait_cycles) / base::CyclesPerSecond()
+      * kMicrosPerSecond;
+
+    LOG(WARNING) << "RPC callback for " << ToString() << " blocked reactor thread for "
+                 << micros << "us";
+  }
+}
+
+void OutboundCall::SetResponse(gscoped_ptr<CallResponse> resp) {
+  call_response_ = std::move(resp);
+  Slice r(call_response_->serialized_response());
+
+  if (call_response_->is_success()) {
+    // TODO: here we're deserializing the call response within the reactor thread,
+    // which isn't great, since it would block processing of other RPCs in parallel.
+    // Should look into a way to avoid this.
+    if (!response_->ParseFromArray(r.data(), r.size())) {
+      SetFailed(Status::IOError("invalid RPC response, missing fields",
+                                response_->InitializationErrorString()));
+      return;
+    }
+    set_state(FINISHED_SUCCESS);
+    CallCallback();
+  } else {
+    // Error
+    unique_ptr<ErrorStatusPB> err(new ErrorStatusPB());
+    if (!err->ParseFromArray(r.data(), r.size())) {
+      SetFailed(Status::IOError("Was an RPC error but could not parse error response",
+                                err->InitializationErrorString()));
+      return;
+    }
+    Status s = Status::RemoteError(err->message());
+    SetFailed(std::move(s), Phase::REMOTE_CALL, std::move(err));
+  }
+}
+
+void OutboundCall::SetQueued() {
+  set_state(ON_OUTBOUND_QUEUE);
+}
+
+void OutboundCall::SetSending() {
+  set_state(SENDING);
+}
+
+void OutboundCall::SetSent() {
+  set_state(SENT);
+
+  // This method is called in the reactor thread, so free the header buf,
+  // which was also allocated from this thread. tcmalloc's thread caching
+  // behavior is a lot more efficient if memory is freed from the same thread
+  // which allocated it -- this lets it keep to thread-local operations instead
+  // of taking a mutex to put memory back on the global freelist.
+  delete [] header_buf_.release();
+
+  // request_buf_ is also done being used here, but since it was allocated by
+  // the caller thread, we would rather let that thread free it whenever it
+  // deletes the RpcController.
+
+  // If cancellation was requested, it's now a good time to do the actual cancellation.
+  if (cancellation_requested()) {
+    SetCancelled();
+  }
+}
+
+void OutboundCall::SetFailed(Status status,
+                             Phase phase,
+                             unique_ptr<ErrorStatusPB> err_pb) {
+  DCHECK(!status.ok());
+  DCHECK(phase == Phase::CONNECTION_NEGOTIATION || phase == Phase::REMOTE_CALL);
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = std::move(status);
+    error_pb_ = std::move(err_pb);
+    set_state_unlocked(phase == Phase::CONNECTION_NEGOTIATION
+        ? FINISHED_NEGOTIATION_ERROR
+        : FINISHED_ERROR);
+  }
+  CallCallback();
+}
+
+void OutboundCall::SetTimedOut(Phase phase) {
+  static const char* kErrMsgNegotiation =
+      "connection negotiation to $1 for RPC $0 timed out after $2 ($3)";
+  static const char* kErrMsgCall = "$0 RPC to $1 timed out after $2 ($3)";
+  DCHECK(phase == Phase::CONNECTION_NEGOTIATION || phase == Phase::REMOTE_CALL);
+
+  // We have to fetch timeout outside the lock to avoid a lock
+  // order inversion between this class and RpcController.
+  const MonoDelta timeout = controller_->timeout();
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = Status::TimedOut(
+        Substitute((phase == Phase::REMOTE_CALL) ? kErrMsgCall : kErrMsgNegotiation,
+                   remote_method_.method_name(),
+                   conn_id_.remote().ToString(),
+                   timeout.ToString(),
+                   StateName(state_)));
+    set_state_unlocked((phase == Phase::REMOTE_CALL) ? TIMED_OUT : NEGOTIATION_TIMED_OUT);
+  }
+  CallCallback();
+}
+
+void OutboundCall::SetCancelled() {
+  DCHECK(!IsFinished());
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = Status::Aborted(
+        Substitute("$0 RPC to $1 is cancelled in state $2",
+                   remote_method_.method_name(),
+                   conn_id_.remote().ToString(),
+                   StateName(state_)));
+    set_state_unlocked(CANCELLED);
+  }
+  CallCallback();
+}
+
+bool OutboundCall::IsTimedOut() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case NEGOTIATION_TIMED_OUT:       // fall-through
+    case TIMED_OUT:
+      return true;
+    default:
+      return false;
+  }
+}
+
+bool OutboundCall::IsCancelled() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return state_ == CANCELLED;
+}
+
+bool OutboundCall::IsNegotiationError() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case FINISHED_NEGOTIATION_ERROR:  // fall-through
+    case NEGOTIATION_TIMED_OUT:
+      return true;
+    default:
+      return false;
+  }
+}
+
+bool OutboundCall::IsFinished() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case READY:
+    case SENDING:
+    case ON_OUTBOUND_QUEUE:
+    case SENT:
+      return false;
+    case NEGOTIATION_TIMED_OUT:
+    case TIMED_OUT:
+    case CANCELLED:
+    case FINISHED_NEGOTIATION_ERROR:
+    case FINISHED_ERROR:
+    case FINISHED_SUCCESS:
+      return true;
+    default:
+      LOG(FATAL) << "Unknown call state: " << state_;
+      return false;
+  }
+}
+
+string OutboundCall::ToString() const {
+  return Substitute("RPC call $0 -> $1", remote_method_.ToString(), conn_id_.ToString());
+}
+
+void OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
+                          RpcCallInProgressPB* resp) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  resp->mutable_header()->CopyFrom(header_);
+  resp->set_micros_elapsed((MonoTime::Now() - start_time_).ToMicroseconds());
+
+  switch (state_) {
+    case READY:
+      // Don't bother setting a state for "READY" since we don't expose a call
+      // until it's at least on the queue of a connection.
+      break;
+    case ON_OUTBOUND_QUEUE:
+      resp->set_state(RpcCallInProgressPB::ON_OUTBOUND_QUEUE);
+      break;
+    case SENDING:
+      resp->set_state(RpcCallInProgressPB::SENDING);
+      break;
+    case SENT:
+      resp->set_state(RpcCallInProgressPB::SENT);
+      break;
+    case NEGOTIATION_TIMED_OUT:
+      resp->set_state(RpcCallInProgressPB::NEGOTIATION_TIMED_OUT);
+      break;
+    case TIMED_OUT:
+      resp->set_state(RpcCallInProgressPB::TIMED_OUT);
+      break;
+    case CANCELLED:
+      resp->set_state(RpcCallInProgressPB::CANCELLED);
+      break;
+    case FINISHED_NEGOTIATION_ERROR:
+      resp->set_state(RpcCallInProgressPB::FINISHED_NEGOTIATION_ERROR);
+      break;
+    case FINISHED_ERROR:
+      resp->set_state(RpcCallInProgressPB::FINISHED_ERROR);
+      break;
+    case FINISHED_SUCCESS:
+      resp->set_state(RpcCallInProgressPB::FINISHED_SUCCESS);
+      break;
+  }
+}
+
+///
+/// CallResponse
+///
+
+CallResponse::CallResponse()
+ : parsed_(false) {
+}
+
+Status CallResponse::GetSidecar(int idx, Slice* sidecar) const {
+  DCHECK(parsed_);
+  if (idx < 0 || idx >= header_.sidecar_offsets_size()) {
+    return Status::InvalidArgument(strings::Substitute(
+        "Index $0 does not reference a valid sidecar", idx));
+  }
+  *sidecar = sidecar_slices_[idx];
+  return Status::OK();
+}
+
+Status CallResponse::ParseFrom(gscoped_ptr<InboundTransfer> transfer) {
+  CHECK(!parsed_);
+  RETURN_NOT_OK(serialization::ParseMessage(transfer->data(), &header_,
+                                            &serialized_response_));
+
+  // Use information from header to extract the payload slices.
+  RETURN_NOT_OK(RpcSidecar::ParseSidecars(header_.sidecar_offsets(),
+          serialized_response_, sidecar_slices_));
+
+  if (header_.sidecar_offsets_size() > 0) {
+    serialized_response_ =
+        Slice(serialized_response_.data(), header_.sidecar_offsets(0));
+  }
+
+  transfer_.swap(transfer);
+  parsed_ = true;
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu


[30/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/x509_check_host.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/x509_check_host.cc b/be/src/kudu/security/x509_check_host.cc
new file mode 100644
index 0000000..4f54ca1
--- /dev/null
+++ b/be/src/kudu/security/x509_check_host.cc
@@ -0,0 +1,439 @@
+/*
+ * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
+ *
+ * Licensed under the OpenSSL license (the "License").  You may not use
+ * this file except in compliance with the License.  You can obtain a copy
+ * in the file LICENSE in the source distribution or at
+ * https://www.openssl.org/source/license.html
+ */
+
+// The following is ported from the OpenSSL-1.1.0b library. The implementations
+// of the functions are for the most part the same except where mentioned in special
+// comments. Explicit casts were also added to bypass compilation errors.
+
+#include <string.h>
+
+#include <openssl/asn1.h>
+#include <openssl/crypto.h>
+#include <openssl/obj_mac.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/security/x509_check_host.h"
+
+// Ported from include/openssl/crypto.h from OpenSSL-1.1.0b
+// Modifed to use __FILE__ and __LINE__ instead of OPENSSL_FILE and OPENSSL_LINE.
+# define OPENSSL_strndup(str, n) \
+        CRYPTO_strndup(str, n, __FILE__, __LINE__)
+
+// Ported from crypto/o_str.c from OpenSSL-1.1.0b.
+// Modified to use strnlen() instead of OPENSSL_strnlen()
+char *CRYPTO_strndup(const char *str, size_t s, const char* file, int line)
+{
+    size_t maxlen;
+    char *ret;
+
+    if (str == NULL)
+        return NULL;
+
+    maxlen = strnlen(str, s);
+
+    ret = (char*)CRYPTO_malloc(maxlen + 1, file, line);
+    if (ret) {
+        memcpy(ret, str, maxlen);
+        ret[maxlen] = '\0';
+    }
+    return ret;
+}
+
+// The remaining code is ported form crypto/x509v3/v3_utl.c
+
+typedef int (*equal_fn) (const unsigned char *pattern, size_t pattern_len,
+                         const unsigned char *subject, size_t subject_len,
+                         unsigned int flags);
+
+/* Skip pattern prefix to match "wildcard" subject */
+static void skip_prefix(const unsigned char **p, size_t *plen,
+                        size_t subject_len,
+                        unsigned int flags)
+{
+    const unsigned char *pattern = *p;
+    size_t pattern_len = *plen;
+
+    /*
+     * If subject starts with a leading '.' followed by more octets, and
+     * pattern is longer, compare just an equal-length suffix with the
+     * full subject (starting at the '.'), provided the prefix contains
+     * no NULs.
+     */
+    if ((flags & _X509_CHECK_FLAG_DOT_SUBDOMAINS) == 0)
+        return;
+
+    while (pattern_len > subject_len && *pattern) {
+        if ((flags & X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS) &&
+            *pattern == '.')
+            break;
+        ++pattern;
+        --pattern_len;
+    }
+
+    /* Skip if entire prefix acceptable */
+    if (pattern_len == subject_len) {
+        *p = pattern;
+        *plen = pattern_len;
+    }
+}
+
+/* Compare while ASCII ignoring case. */
+static int equal_nocase(const unsigned char *pattern, size_t pattern_len,
+                        const unsigned char *subject, size_t subject_len,
+                        unsigned int flags)
+{
+    skip_prefix(&pattern, &pattern_len, subject_len, flags);
+    if (pattern_len != subject_len)
+        return 0;
+    while (pattern_len) {
+        unsigned char l = *pattern;
+        unsigned char r = *subject;
+        /* The pattern must not contain NUL characters. */
+        if (l == 0)
+            return 0;
+        if (l != r) {
+            if ('A' <= l && l <= 'Z')
+                l = (l - 'A') + 'a';
+            if ('A' <= r && r <= 'Z')
+                r = (r - 'A') + 'a';
+            if (l != r)
+                return 0;
+        }
+        ++pattern;
+        ++subject;
+        --pattern_len;
+    }
+    return 1;
+}
+
+/* Compare using memcmp. */
+static int equal_case(const unsigned char *pattern, size_t pattern_len,
+                      const unsigned char *subject, size_t subject_len,
+                      unsigned int flags)
+{
+    skip_prefix(&pattern, &pattern_len, subject_len, flags);
+    if (pattern_len != subject_len)
+        return 0;
+    return !memcmp(pattern, subject, pattern_len);
+}
+
+/*
+ * RFC 5280, section 7.5, requires that only the domain is compared in a
+ * case-insensitive manner.
+ */
+static int equal_email(const unsigned char *a, size_t a_len,
+                       const unsigned char *b, size_t b_len,
+                       unsigned int unused_flags)
+{
+    size_t i = a_len;
+    if (a_len != b_len)
+        return 0;
+    /*
+     * We search backwards for the '@' character, so that we do not have to
+     * deal with quoted local-parts.  The domain part is compared in a
+     * case-insensitive manner.
+     */
+    while (i > 0) {
+        --i;
+        if (a[i] == '@' || b[i] == '@') {
+            if (!equal_nocase(a + i, a_len - i, b + i, a_len - i, 0))
+                return 0;
+            break;
+        }
+    }
+    if (i == 0)
+        i = a_len;
+    return equal_case(a, i, b, i, 0);
+}
+
+/*
+ * Compare the prefix and suffix with the subject, and check that the
+ * characters in-between are valid.
+ */
+static int wildcard_match(const unsigned char *prefix, size_t prefix_len,
+                          const unsigned char *suffix, size_t suffix_len,
+                          const unsigned char *subject, size_t subject_len,
+                          unsigned int flags)
+{
+    const unsigned char *wildcard_start;
+    const unsigned char *wildcard_end;
+    const unsigned char *p;
+    int allow_multi = 0;
+    int allow_idna = 0;
+
+    if (subject_len < prefix_len + suffix_len)
+        return 0;
+    if (!equal_nocase(prefix, prefix_len, subject, prefix_len, flags))
+        return 0;
+    wildcard_start = subject + prefix_len;
+    wildcard_end = subject + (subject_len - suffix_len);
+    if (!equal_nocase(wildcard_end, suffix_len, suffix, suffix_len, flags))
+        return 0;
+    /*
+     * If the wildcard makes up the entire first label, it must match at
+     * least one character.
+     */
+    if (prefix_len == 0 && *suffix == '.') {
+        if (wildcard_start == wildcard_end)
+            return 0;
+        allow_idna = 1;
+        if (flags & X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS)
+            allow_multi = 1;
+    }
+    /* IDNA labels cannot match partial wildcards */
+    if (!allow_idna &&
+        subject_len >= 4 && strncasecmp((char *)subject, "xn--", 4) == 0)
+        return 0;
+    /* The wildcard may match a literal '*' */
+    if (wildcard_end == wildcard_start + 1 && *wildcard_start == '*')
+        return 1;
+    /*
+     * Check that the part matched by the wildcard contains only
+     * permitted characters and only matches a single label unless
+     * allow_multi is set.
+     */
+    for (p = wildcard_start; p != wildcard_end; ++p)
+        if (!(('0' <= *p && *p <= '9') ||
+              ('A' <= *p && *p <= 'Z') ||
+              ('a' <= *p && *p <= 'z') ||
+              *p == '-' || (allow_multi && *p == '.')))
+            return 0;
+    return 1;
+}
+
+#define LABEL_START     (1 << 0)
+#define LABEL_END       (1 << 1)
+#define LABEL_HYPHEN    (1 << 2)
+#define LABEL_IDNA      (1 << 3)
+
+static const unsigned char *valid_star(const unsigned char *p, size_t len,
+                                       unsigned int flags)
+{
+    const unsigned char *star = 0;
+    size_t i;
+    int state = LABEL_START;
+    int dots = 0;
+    for (i = 0; i < len; ++i) {
+        /*
+         * Locate first and only legal wildcard, either at the start
+         * or end of a non-IDNA first and not final label.
+         */
+        if (p[i] == '*') {
+            int atstart = (state & LABEL_START);
+            int atend = (i == len - 1 || p[i + 1] == '.');
+            /*-
+             * At most one wildcard per pattern.
+             * No wildcards in IDNA labels.
+             * No wildcards after the first label.
+             */
+            if (star != NULL || (state & LABEL_IDNA) != 0 || dots)
+                return NULL;
+            /* Only full-label '*.example.com' wildcards? */
+            if ((flags & X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS)
+                && (!atstart || !atend))
+                return NULL;
+            /* No 'foo*bar' wildcards */
+            if (!atstart && !atend)
+                return NULL;
+            star = &p[i];
+            state &= ~LABEL_START;
+        } else if (('a' <= p[i] && p[i] <= 'z')
+                   || ('A' <= p[i] && p[i] <= 'Z')
+                   || ('0' <= p[i] && p[i] <= '9')) {
+            if ((state & LABEL_START) != 0
+                && len - i >= 4 && strncasecmp((char *)&p[i], "xn--", 4) == 0)
+                state |= LABEL_IDNA;
+            state &= ~(LABEL_HYPHEN | LABEL_START);
+        } else if (p[i] == '.') {
+            if ((state & (LABEL_HYPHEN | LABEL_START)) != 0)
+                return NULL;
+            state = LABEL_START;
+            ++dots;
+        } else if (p[i] == '-') {
+            /* no domain/subdomain starts with '-' */
+            if ((state & LABEL_START) != 0)
+                return NULL;
+            state |= LABEL_HYPHEN;
+        } else
+            return NULL;
+    }
+
+    /*
+     * The final label must not end in a hyphen or ".", and
+     * there must be at least two dots after the star.
+     */
+    if ((state & (LABEL_START | LABEL_HYPHEN)) != 0 || dots < 2)
+        return NULL;
+    return star;
+}
+
+/* Compare using wildcards. */
+static int equal_wildcard(const unsigned char *pattern, size_t pattern_len,
+                          const unsigned char *subject, size_t subject_len,
+                          unsigned int flags)
+{
+    const unsigned char *star = NULL;
+
+    /*
+     * Subject names starting with '.' can only match a wildcard pattern
+     * via a subject sub-domain pattern suffix match.
+     */
+    if (!(subject_len > 1 && subject[0] == '.'))
+        star = valid_star(pattern, pattern_len, flags);
+    if (star == NULL)
+        return equal_nocase(pattern, pattern_len,
+                            subject, subject_len, flags);
+    return wildcard_match(pattern, star - pattern,
+                          star + 1, (pattern + pattern_len) - star - 1,
+                          subject, subject_len, flags);
+}
+
+/*
+ * Compare an ASN1_STRING to a supplied string. If they match return 1. If
+ * cmp_type > 0 only compare if string matches the type, otherwise convert it
+ * to UTF8.
+ */
+
+static int do_check_string(const ASN1_STRING *a, int cmp_type, equal_fn equal,
+                           unsigned int flags, const char *b, size_t blen,
+                           char **peername)
+{
+    int rv = 0;
+
+    if (!a->data || !a->length)
+        return 0;
+    if (cmp_type > 0) {
+        if (cmp_type != a->type)
+            return 0;
+        if (cmp_type == V_ASN1_IA5STRING)
+            rv = equal(a->data, a->length, (unsigned char *)b, blen, flags);
+        else if (a->length == (int)blen && !memcmp(a->data, b, blen))
+            rv = 1;
+        if (rv > 0 && peername)
+            *peername = OPENSSL_strndup((char *)a->data, a->length);
+    } else {
+        int astrlen;
+        unsigned char *astr;
+        astrlen = ASN1_STRING_to_UTF8(&astr, (ASN1_STRING*)a);
+        if (astrlen < 0) {
+            /*
+             * -1 could be an internal malloc failure or a decoding error from
+             * malformed input; we can't distinguish.
+             */
+            return -1;
+        }
+        rv = equal(astr, astrlen, (unsigned char *)b, blen, flags);
+        if (rv > 0 && peername)
+            *peername = OPENSSL_strndup((char *)astr, astrlen);
+            //*peername = strndup((char *)astr, astrlen);
+        OPENSSL_free(astr);
+    }
+    return rv;
+}
+
+static int do_x509_check(X509 *x, const char *chk, size_t chklen,
+                         unsigned int flags, int check_type, char **peername)
+{
+    GENERAL_NAMES *gens = NULL;
+    X509_NAME *name = NULL;
+    int i;
+    int cnid = NID_undef;
+    int alt_type;
+    int san_present = 0;
+    int rv = 0;
+    equal_fn equal;
+
+    /* See below, this flag is internal-only */
+    flags &= ~_X509_CHECK_FLAG_DOT_SUBDOMAINS;
+    if (check_type == GEN_EMAIL) {
+        cnid = NID_pkcs9_emailAddress;
+        alt_type = V_ASN1_IA5STRING;
+        equal = equal_email;
+    } else if (check_type == GEN_DNS) {
+        cnid = NID_commonName;
+        /* Implicit client-side DNS sub-domain pattern */
+        if (chklen > 1 && chk[0] == '.')
+            flags |= _X509_CHECK_FLAG_DOT_SUBDOMAINS;
+        alt_type = V_ASN1_IA5STRING;
+        if (flags & X509_CHECK_FLAG_NO_WILDCARDS)
+            equal = equal_nocase;
+        else
+            equal = equal_wildcard;
+    } else {
+        alt_type = V_ASN1_OCTET_STRING;
+        equal = equal_case;
+    }
+
+    if (chklen == 0)
+        chklen = strlen(chk);
+
+    gens = (GENERAL_NAMES*)X509_get_ext_d2i(x, NID_subject_alt_name, NULL, NULL);
+    if (gens) {
+        for (i = 0; i < sk_GENERAL_NAME_num(gens); i++) {
+            GENERAL_NAME *gen;
+            ASN1_STRING *cstr;
+            gen = sk_GENERAL_NAME_value(gens, i);
+            if (gen->type != check_type)
+                continue;
+            san_present = 1;
+            if (check_type == GEN_EMAIL)
+                cstr = gen->d.rfc822Name;
+            else if (check_type == GEN_DNS)
+                cstr = gen->d.dNSName;
+            else
+                cstr = gen->d.iPAddress;
+            /* Positive on success, negative on error! */
+            if ((rv = do_check_string(cstr, alt_type, equal, flags,
+                                      chk, chklen, peername)) != 0)
+                break;
+        }
+        GENERAL_NAMES_free(gens);
+        if (rv != 0)
+            return rv;
+        if (san_present && !(flags & X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT))
+            return 0;
+    }
+
+    /* We're done if CN-ID is not pertinent */
+    if (cnid == NID_undef || (flags & X509_CHECK_FLAG_NEVER_CHECK_SUBJECT))
+        return 0;
+
+    i = -1;
+    name = X509_get_subject_name(x);
+    while ((i = X509_NAME_get_index_by_NID(name, cnid, i)) >= 0) {
+        const X509_NAME_ENTRY *ne = X509_NAME_get_entry(name, i);
+        const ASN1_STRING *str = X509_NAME_ENTRY_get_data((X509_NAME_ENTRY*)ne);
+
+        /* Positive on success, negative on error! */
+        if ((rv = do_check_string(str, -1, equal, flags,
+                                  chk, chklen, peername)) != 0)
+            return rv;
+    }
+    return 0;
+}
+
+int X509_check_host(X509 *x, const char *chk, size_t chklen,
+                    unsigned int flags, char **peername)
+{
+    if (chk == NULL)
+        return -2;
+    /*
+     * Embedded NULs are disallowed, except as the last character of a
+     * string of length 2 or more (tolerate caller including terminating
+     * NUL in string length).
+     */
+    if (chklen == 0)
+        chklen = strlen(chk);
+    else if (memchr(chk, '\0', chklen > 1 ? chklen - 1 : chklen))
+        return -2;
+    if (chklen > 1 && chk[chklen - 1] == '\0')
+        --chklen;
+    return do_x509_check(x, chk, chklen, flags, GEN_DNS, peername);
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/x509_check_host.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/x509_check_host.h b/be/src/kudu/security/x509_check_host.h
new file mode 100644
index 0000000..d2d5af9
--- /dev/null
+++ b/be/src/kudu/security/x509_check_host.h
@@ -0,0 +1,50 @@
+/*
+ * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
+ *
+ * Licensed under the OpenSSL license (the "License").  You may not use
+ * this file except in compliance with the License.  You can obtain a copy
+ * in the file LICENSE in the source distribution or at
+ * https://www.openssl.org/source/license.html
+ */
+
+// The following is ported from the OpenSSL-1.1.0b library.
+
+#ifndef X509_CHECK_HOST_H
+#define X509_CHECK_HOST_H
+
+#include <stdlib.h>
+// IWYU pragma: no_include <openssl/x509.h>
+// IWYU pragma: no_include "openssl/x509.h"
+
+typedef struct x509_st X509;
+
+/* Flags for X509_check_* functions */
+
+/*
+ * Always check subject name for host match even if subject alt names present
+ */
+# define X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT    0x1
+/* Disable wildcard matching for dnsName fields and common name. */
+# define X509_CHECK_FLAG_NO_WILDCARDS    0x2
+/* Wildcards must not match a partial label. */
+# define X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS 0x4
+/* Allow (non-partial) wildcards to match multiple labels. */
+# define X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS 0x8
+/* Constraint verifier subdomain patterns to match a single labels. */
+# define X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS 0x10
+/* Never check the subject CN */
+# define X509_CHECK_FLAG_NEVER_CHECK_SUBJECT    0x20
+/*
+ * Match reference identifiers starting with "." to any sub-domain.
+ * This is a non-public flag, turned on implicitly when the subject
+ * reference identity is a DNS name.
+ */
+# define _X509_CHECK_FLAG_DOT_SUBDOMAINS 0x8000
+
+// Checks if the certificate Subject Alternative Name (SAN) or Subject CommonName (CN)
+// matches the specified host name, which must be encoded in the preferred name syntax
+// described in section 3.5 of RFC 1034.
+int X509_check_host(X509 *x, const char *chk, size_t chklen,
+                    unsigned int flags, char **peername);
+
+#endif // X509_CHECK_HOST_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/CMakeLists.txt b/be/src/kudu/util/CMakeLists.txt
new file mode 100644
index 0000000..94dd2fa
--- /dev/null
+++ b/be/src/kudu/util/CMakeLists.txt
@@ -0,0 +1,482 @@
+# 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.
+
+#######################################
+# util_compression_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  UTIL_COMPRESSION_PROTO_SRCS UTIL_COMPRESSION_PROTO_HDRS UTIL_COMPRESSION_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES compression/compression.proto)
+ADD_EXPORTABLE_LIBRARY(util_compression_proto
+  SRCS ${UTIL_COMPRESSION_PROTO_SRCS}
+  DEPS protobuf
+  NONLINK_DEPS ${UTIL_COMPRESSION_PROTO_TGTS})
+
+#######################################
+# histogram_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  HISTOGRAM_PROTO_SRCS HISTOGRAM_PROTO_HDRS HISTOGRAM_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES histogram.proto)
+ADD_EXPORTABLE_LIBRARY(histogram_proto
+  SRCS ${HISTOGRAM_PROTO_SRCS}
+  DEPS protobuf
+  NONLINK_DEPS ${HISTOGRAM_PROTO_TGTS})
+
+#######################################
+# maintenance_manager_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+    MAINTENANCE_MANAGER_PROTO_SRCS MAINTENANCE_MANAGER_PROTO_HDRS MAINTENANCE_MANAGER_PROTO_TGTS
+    SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+    BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+    PROTO_FILES maintenance_manager.proto)
+ADD_EXPORTABLE_LIBRARY(maintenance_manager_proto
+    SRCS ${MAINTENANCE_MANAGER_PROTO_SRCS}
+    DEPS protobuf
+    NONLINK_DEPS ${MAINTENANCE_MANAGER_PROTO_TGTS})
+
+#######################################
+# pb_util_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  PB_UTIL_PROTO_SRCS PB_UTIL_PROTO_HDRS PB_UTIL_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES pb_util.proto)
+ADD_EXPORTABLE_LIBRARY(pb_util_proto
+  SRCS ${PB_UTIL_PROTO_SRCS}
+  DEPS protobuf
+  NONLINK_DEPS ${PB_UTIL_PROTO_TGTS})
+
+#######################################
+# version_info_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  VERSION_INFO_PROTO_SRCS VERSION_INFO_PROTO_HDRS VERSION_INFO_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES version_info.proto)
+ADD_EXPORTABLE_LIBRARY(version_info_proto
+  SRCS ${VERSION_INFO_PROTO_SRCS}
+  DEPS protobuf
+  NONLINK_DEPS ${VERSION_INFO_PROTO_TGTS})
+
+############################################################
+# Version stamp
+############################################################
+
+# Unlike CMAKE_CURRENT_BINARY_DIR, CMAKE_BINARY_DIR is always the root of
+# the build directory.
+set(VERSION_STAMP_FILE ${CMAKE_BINARY_DIR}/src/kudu/generated/version_defines.h)
+
+list(APPEND GEN_VERSION_INFO_COMMAND "${BUILD_SUPPORT_DIR}/gen_version_info.py")
+list(APPEND GEN_VERSION_INFO_COMMAND "--version=${KUDU_VERSION_NUMBER}")
+list(APPEND GEN_VERSION_INFO_COMMAND "--build-type=${CMAKE_BUILD_TYPE}")
+if(KUDU_GIT_HASH)
+  message(STATUS "Provided git hash: ${KUDU_GIT_HASH}")
+  list(APPEND GEN_VERSION_INFO_COMMAND "--git-hash=${KUDU_GIT_HASH}")
+endif()
+list(APPEND GEN_VERSION_INFO_COMMAND "${VERSION_STAMP_FILE}")
+add_custom_target(gen_version_info
+    COMMAND ${GEN_VERSION_INFO_COMMAND}
+    BYPRODUCTS "${VERSION_STAMP_FILE}")
+
+#######################################
+# kudu_util
+#######################################
+
+if (APPLE)
+  set(SEMAPHORE_CC "semaphore_macosx.cc")
+else ()
+  set(SEMAPHORE_CC "semaphore.cc")
+endif()
+
+set(UTIL_SRCS
+  async_logger.cc
+  atomic.cc
+  bitmap.cc
+  bloom_filter.cc
+  bitmap.cc
+  cache.cc
+  cache_metrics.cc
+  coding.cc
+  condition_variable.cc
+  cow_object.cc
+  crc.cc
+  debug-util.cc
+  decimal_util.cc
+  debug/trace_event_impl.cc
+  debug/trace_event_impl_constants.cc
+  debug/trace_event_synthetic_delay.cc
+  debug/unwind_safeness.cc
+  easy_json.cc
+  env.cc env_posix.cc env_util.cc
+  errno.cc
+  faststring.cc
+  fault_injection.cc
+  file_cache.cc
+  flags.cc
+  flag_tags.cc
+  flag_validators.cc
+  group_varint.cc
+  pstack_watcher.cc
+  hdr_histogram.cc
+  hexdump.cc
+  init.cc
+  jsonreader.cc
+  jsonwriter.cc
+  kernel_stack_watchdog.cc
+  locks.cc
+  logging.cc
+  maintenance_manager.cc
+  malloc.cc
+  memcmpable_varint.cc
+  memory/arena.cc
+  memory/memory.cc
+  memory/overwrite.cc
+  mem_tracker.cc
+  metrics.cc
+  minidump.cc
+  monotime.cc
+  mutex.cc
+  net/dns_resolver.cc
+  net/net_util.cc
+  net/sockaddr.cc
+  net/socket.cc
+  oid_generator.cc
+  once.cc
+  os-util.cc
+  path_util.cc
+  pb_util.cc
+  pb_util-internal.cc
+  process_memory.cc
+  random_util.cc
+  rolling_log.cc
+  rw_mutex.cc
+  rwc_lock.cc
+  ${SEMAPHORE_CC}
+  signal.cc
+  slice.cc
+  spinlock_profiling.cc
+  status.cc
+  status_callback.cc
+  string_case.cc
+  striped64.cc
+  subprocess.cc
+  test_graph.cc
+  test_util_prod.cc
+  thread.cc
+  threadlocal.cc
+  threadpool.cc
+  thread_restrictions.cc
+  throttler.cc
+  trace.cc
+  trace_metrics.cc
+  user.cc
+  url-coding.cc
+  version_info.cc
+  version_util.cc
+  website_util.cc
+  zlib.cc
+)
+
+# overwrite.cc contains a single function which would be a hot spot in
+# debug builds. It's separated into a separate file so it can be
+# optimized regardless of the default optimization options.
+set_source_files_properties(memory/overwrite.cc PROPERTIES COMPILE_FLAGS "-O3")
+
+if(HAVE_LIB_VMEM)
+  set(UTIL_SRCS
+    ${UTIL_SRCS}
+    nvm_cache.cc)
+endif()
+
+set(UTIL_LIBS
+  crcutil
+  gflags
+  glog
+  gutil
+  histogram_proto
+  libev
+  maintenance_manager_proto
+  pb_util_proto
+  protobuf
+  version_info_proto
+  zlib)
+
+if(NOT APPLE)
+  set(UTIL_LIBS
+    ${UTIL_LIBS}
+    breakpad_client
+    dl
+    rt)
+endif()
+
+if(HAVE_LIB_VMEM)
+  set(UTIL_LIBS
+    ${UTIL_LIBS}
+    vmem)
+endif()
+
+# We use MallocExtension, but not in the exported version of the library.
+set(EXPORTED_UTIL_LIBS ${UTIL_LIBS})
+if(${KUDU_TCMALLOC_AVAILABLE})
+  list(APPEND UTIL_LIBS tcmalloc)
+endif()
+
+ADD_EXPORTABLE_LIBRARY(kudu_util
+  SRCS ${UTIL_SRCS}
+  DEPS ${UTIL_LIBS}
+  NONLINK_DEPS gen_version_info
+  EXPORTED_DEPS ${EXPORTED_UTIL_LIBS})
+
+#######################################
+# kudu_util_compression
+#######################################
+set(UTIL_COMPRESSION_SRCS
+  compression/compression_codec.cc)
+set(UTIL_COMPRESSION_LIBS
+  kudu_util
+  util_compression_proto
+
+  glog
+  gutil
+  lz4
+  snappy
+  zlib)
+ADD_EXPORTABLE_LIBRARY(kudu_util_compression
+  SRCS ${UTIL_COMPRESSION_SRCS}
+  DEPS ${UTIL_COMPRESSION_LIBS})
+
+#######################################
+# kudu_test_util
+#######################################
+
+# Used by mini-cluster, so must be built even when NO_TESTS=0.
+add_library(kudu_test_util
+  test_util.cc)
+target_link_libraries(kudu_test_util
+  gflags
+  glog
+  gmock
+  kudu_util)
+
+if(HAVE_LIB_VMEM)
+  target_link_libraries(kudu_test_util
+    vmem)
+endif()
+
+#######################################
+# kudu_curl_util
+#######################################
+if(NOT NO_TESTS)
+  add_library(kudu_curl_util
+    curl_util.cc)
+  target_link_libraries(kudu_curl_util
+    security
+    ${CURL_LIBRARIES}
+    glog
+    gutil)
+endif()
+
+#######################################
+# kudu_test_main
+#######################################
+if(NOT NO_TESTS)
+  add_library(kudu_test_main
+    test_main.cc)
+  target_link_libraries(kudu_test_main
+    ${KRB5_REALM_OVERRIDE}
+    gflags
+    glog
+    gmock
+    kudu_util
+    kudu_test_util)
+
+  if(NOT APPLE)
+    target_link_libraries(kudu_test_main
+      dl
+      rt)
+  endif()
+endif()
+
+#######################################
+# protoc-gen-insertions
+#######################################
+
+add_executable(protoc-gen-insertions protoc-gen-insertions.cc)
+target_link_libraries(protoc-gen-insertions gutil protobuf protoc ${KUDU_BASE_LIBS})
+
+#######################################
+# Unit tests
+#######################################
+
+set(KUDU_TEST_LINK_LIBS kudu_util gutil ${KUDU_MIN_TEST_LIBS})
+ADD_KUDU_TEST(async_util-test)
+ADD_KUDU_TEST(atomic-test)
+ADD_KUDU_TEST(bit-util-test)
+ADD_KUDU_TEST(bitmap-test)
+ADD_KUDU_TEST(blocking_queue-test)
+ADD_KUDU_TEST(bloom_filter-test)
+ADD_KUDU_TEST(cache-bench RUN_SERIAL true)
+ADD_KUDU_TEST(cache-test)
+ADD_KUDU_TEST(callback_bind-test)
+ADD_KUDU_TEST(countdown_latch-test)
+ADD_KUDU_TEST(crc-test RUN_SERIAL true) # has a benchmark
+ADD_KUDU_TEST(debug-util-test)
+ADD_KUDU_TEST(decimal_util-test)
+ADD_KUDU_TEST(easy_json-test)
+ADD_KUDU_TEST(env-test LABELS no_tsan)
+ADD_KUDU_TEST(env_util-test)
+ADD_KUDU_TEST(errno-test)
+ADD_KUDU_TEST(faststring-test)
+ADD_KUDU_TEST(file_cache-test)
+ADD_KUDU_TEST(file_cache-stress-test RUN_SERIAL true)
+ADD_KUDU_TEST(flag_tags-test)
+ADD_KUDU_TEST(flag_validators-test)
+ADD_KUDU_TEST(flags-test)
+ADD_KUDU_TEST(group_varint-test)
+ADD_KUDU_TEST(hash_util-test)
+ADD_KUDU_TEST(hdr_histogram-test)
+ADD_KUDU_TEST(int128-test)
+ADD_KUDU_TEST(inline_slice-test)
+ADD_KUDU_TEST(interval_tree-test)
+ADD_KUDU_TEST(jsonreader-test)
+ADD_KUDU_TEST(knapsack_solver-test)
+ADD_KUDU_TEST(logging-test)
+ADD_KUDU_TEST(maintenance_manager-test)
+ADD_KUDU_TEST(map-util-test)
+ADD_KUDU_TEST(mem_tracker-test)
+ADD_KUDU_TEST(memcmpable_varint-test LABELS no_tsan)
+ADD_KUDU_TEST(memory/arena-test)
+ADD_KUDU_TEST(metrics-test)
+ADD_KUDU_TEST(monotime-test)
+ADD_KUDU_TEST(mt-hdr_histogram-test RUN_SERIAL true)
+ADD_KUDU_TEST(mt-metrics-test RUN_SERIAL true)
+ADD_KUDU_TEST(mt-threadlocal-test RUN_SERIAL true)
+ADD_KUDU_TEST(net/dns_resolver-test)
+ADD_KUDU_TEST(net/net_util-test)
+ADD_KUDU_TEST(net/socket-test)
+ADD_KUDU_TEST(object_pool-test)
+ADD_KUDU_TEST(oid_generator-test)
+ADD_KUDU_TEST(once-test)
+ADD_KUDU_TEST(os-util-test)
+ADD_KUDU_TEST(path_util-test)
+ADD_KUDU_TEST(process_memory-test RUN_SERIAL true)
+ADD_KUDU_TEST(random-test)
+ADD_KUDU_TEST(random_util-test)
+ADD_KUDU_TEST(rle-test)
+ADD_KUDU_TEST(rolling_log-test)
+ADD_KUDU_TEST(rw_mutex-test RUN_SERIAL true)
+ADD_KUDU_TEST(rw_semaphore-test)
+ADD_KUDU_TEST(rwc_lock-test RUN_SERIAL true)
+ADD_KUDU_TEST(safe_math-test)
+ADD_KUDU_TEST(scoped_cleanup-test)
+ADD_KUDU_TEST(slice-test)
+ADD_KUDU_TEST(sorted_disjoint_interval_list-test)
+ADD_KUDU_TEST(spinlock_profiling-test)
+ADD_KUDU_TEST(stack_watchdog-test PROCESSORS 2)
+ADD_KUDU_TEST(status-test)
+ADD_KUDU_TEST(string_case-test)
+ADD_KUDU_TEST(striped64-test RUN_SERIAL true)
+ADD_KUDU_TEST(subprocess-test)
+ADD_KUDU_TEST(thread-test)
+ADD_KUDU_TEST(threadpool-test)
+ADD_KUDU_TEST(throttler-test)
+ADD_KUDU_TEST(trace-test PROCESSORS 4)
+ADD_KUDU_TEST(url-coding-test)
+ADD_KUDU_TEST(user-test)
+ADD_KUDU_TEST(version_util-test)
+
+if (NOT APPLE)
+  ADD_KUDU_TEST(minidump-test)
+  ADD_KUDU_TEST(pstack_watcher-test)
+endif()
+
+#######################################
+# jsonwriter_test_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  JSONWRITER_TEST_PROTO_SRCS JSONWRITER_TEST_PROTO_HDRS JSONWRITER_TEST_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES jsonwriter_test.proto)
+add_library(jsonwriter_test_proto ${JSONWRITER_TEST_PROTO_SRCS} ${JSONWRITER_TEST_PROTO_HDRS})
+target_link_libraries(jsonwriter_test_proto
+  pb_util_proto
+  protobuf)
+
+#######################################
+# jsonwriter-test
+#######################################
+
+ADD_KUDU_TEST(jsonwriter-test)
+if(NOT NO_TESTS)
+  target_link_libraries(jsonwriter-test
+    jsonwriter_test_proto)
+endif()
+
+#######################################
+# pb_util_test_proto
+#######################################
+
+PROTOBUF_GENERATE_CPP(
+  PROTO_CONTAINER_TEST_PROTO_SRCS PROTO_CONTAINER_TEST_PROTO_HDRS PROTO_CONTAINER_TEST_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES
+    proto_container_test.proto
+    proto_container_test2.proto
+    proto_container_test3.proto
+    pb_util_test.proto)
+add_library(pb_util_test_proto
+  ${PROTO_CONTAINER_TEST_PROTO_SRCS}
+  ${PROTO_CONTAINER_TEST_PROTO_HDRS})
+target_link_libraries(pb_util_test_proto
+  pb_util_proto
+  protobuf)
+
+#######################################
+# pb_util-test
+#######################################
+
+ADD_KUDU_TEST(pb_util-test)
+if(NOT NO_TESTS)
+  target_link_libraries(pb_util-test
+    pb_util_test_proto)
+endif()
+
+#######################################
+# util/compression tests
+#######################################
+ADD_KUDU_TEST(compression/compression-test)
+if(NOT NO_TESTS)
+  target_link_libraries(compression-test
+    cfile
+    kudu_util_compression)
+endif()

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/alignment.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/alignment.h b/be/src/kudu/util/alignment.h
new file mode 100644
index 0000000..8e902d2
--- /dev/null
+++ b/be/src/kudu/util/alignment.h
@@ -0,0 +1,28 @@
+// 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.
+//
+// Macros for dealing with memory alignment.
+#ifndef KUDU_UTIL_ALIGNMENT_H
+#define KUDU_UTIL_ALIGNMENT_H
+
+// Round down 'x' to the nearest 'align' boundary
+#define KUDU_ALIGN_DOWN(x, align) ((x) & (~(align) + 1))
+
+// Round up 'x' to the nearest 'align' boundary
+#define KUDU_ALIGN_UP(x, align) (((x) + ((align) - 1)) & (~(align) + 1))
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/array_view.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/array_view.h b/be/src/kudu/util/array_view.h
new file mode 100644
index 0000000..24ee727
--- /dev/null
+++ b/be/src/kudu/util/array_view.h
@@ -0,0 +1,133 @@
+/*
+ *  Copyright 2015 The WebRTC Project Authors. All rights reserved.
+ *
+ *  Use of this source code is governed by a BSD-style license
+ *  that can be found in the LICENSE.txt file in the root of the source
+ *  tree. An additional intellectual property rights grant can be found
+ *  in the same file.  All contributing project authors may
+ *  be found in the AUTHORS file in the root of the WebRTC source tree.
+ *
+ *  Imported into Kudu from WebRTC and modified to fit in the Kudu namespace
+ *  and avoid referring to anything WebRTC-specific like rtc::Buffer.
+ */
+
+#pragma once
+
+#include <glog/logging.h>
+
+namespace kudu {
+
+// Many functions read from or write to arrays. The obvious way to do this is
+// to use two arguments, a pointer to the first element and an element count:
+//
+//   bool Contains17(const int* arr, size_t size) {
+//     for (size_t i = 0; i < size; ++i) {
+//       if (arr[i] == 17)
+//         return true;
+//     }
+//     return false;
+//   }
+//
+// This is flexible, since it doesn't matter how the array is stored (C array,
+// std::vector, std::array, ...), but it's error-prone because the caller has
+// to correctly specify the array length:
+//
+//   Contains17(arr, arraysize(arr));  // C array
+//   Contains17(&arr[0], arr.size());  // std::vector
+//   Contains17(arr, size);            // pointer + size
+//   ...
+//
+// It's also kind of messy to have two separate arguments for what is
+// conceptually a single thing.
+//
+// Enter kudu::ArrayView<T>. It contains a T pointer (to an array it doesn't
+// own) and a count, and supports the basic things you'd expect, such as
+// indexing and iteration. It allows us to write our function like this:
+//
+//   bool Contains17(kudu::ArrayView<const int> arr) {
+//     for (auto e : arr) {
+//       if (e == 17)
+//         return true;
+//     }
+//     return false;
+//   }
+//
+// And even better, because a bunch of things will implicitly convert to
+// ArrayView, we can call it like this:
+//
+//   Contains17(arr);                             // C array
+//   Contains17(arr);                             // std::vector
+//   Contains17(kudu::ArrayView<int>(arr, size)); // pointer + size
+//   ...
+//
+// One important point is that ArrayView<T> and ArrayView<const T> are
+// different types, which allow and don't allow mutation of the array elements,
+// respectively. The implicit conversions work just like you'd hope, so that
+// e.g. vector<int> will convert to either ArrayView<int> or ArrayView<const
+// int>, but const vector<int> will convert only to ArrayView<const int>.
+// (ArrayView itself can be the source type in such conversions, so
+// ArrayView<int> will convert to ArrayView<const int>.)
+//
+// Note: ArrayView is tiny (just a pointer and a count) and trivially copyable,
+// so it's probably cheaper to pass it by value than by const reference.
+template <typename T>
+class ArrayView final {
+ public:
+  // Construct an empty ArrayView.
+  ArrayView() : ArrayView(static_cast<T*>(nullptr), 0) {}
+
+  // Construct an ArrayView for a (pointer,size) pair.
+  template <typename U>
+  ArrayView(U* data, size_t size)
+      : data_(size == 0 ? nullptr : data), size_(size) {
+    CheckInvariant();
+  }
+
+  // Construct an ArrayView for an array.
+  template <typename U, size_t N>
+  ArrayView(U (&array)[N]) : ArrayView(&array[0], N) {} // NOLINT(runtime/explicit)
+
+  // Construct an ArrayView for any type U that has a size() method whose
+  // return value converts implicitly to size_t, and a data() method whose
+  // return value converts implicitly to T*. In particular, this means we allow
+  // conversion from ArrayView<T> to ArrayView<const T>, but not the other way
+  // around. Other allowed conversions include std::vector<T> to ArrayView<T>
+  // or ArrayView<const T>, const std::vector<T> to ArrayView<const T>, and
+  // kudu::faststring to ArrayView<uint8_t> (with the same const behavior as
+  // std::vector).
+  template <typename U>
+  ArrayView(U& u) : ArrayView(u.data(), u.size()) {} // NOLINT(runtime/explicit)
+
+  // Indexing, size, and iteration. These allow mutation even if the ArrayView
+  // is const, because the ArrayView doesn't own the array. (To prevent
+  // mutation, use ArrayView<const T>.)
+  size_t size() const { return size_; }
+  bool empty() const { return size_ == 0; }
+  T* data() const { return data_; }
+  T& operator[](size_t idx) const {
+    DCHECK_LT(idx, size_);
+    DCHECK(data_);  // Follows from size_ > idx and the class invariant.
+    return data_[idx];
+  }
+  T* begin() const { return data_; }
+  T* end() const { return data_ + size_; }
+  const T* cbegin() const { return data_; }
+  const T* cend() const { return data_ + size_; }
+
+  // Comparing two ArrayViews compares their (pointer,size) pairs; it does
+  // *not* dereference the pointers.
+  friend bool operator==(const ArrayView& a, const ArrayView& b) {
+    return a.data_ == b.data_ && a.size_ == b.size_;
+  }
+  friend bool operator!=(const ArrayView& a, const ArrayView& b) {
+    return !(a == b);
+  }
+
+ private:
+  // Invariant: !data_ iff size_ == 0.
+  void CheckInvariant() const { DCHECK_EQ(!data_, size_ == 0); }
+  T* data_;
+  size_t size_;
+};
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/async_logger.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/async_logger.cc b/be/src/kudu/util/async_logger.cc
new file mode 100644
index 0000000..3214a42
--- /dev/null
+++ b/be/src/kudu/util/async_logger.cc
@@ -0,0 +1,151 @@
+// 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 "kudu/util/async_logger.h"
+
+#include <string>
+#include <thread>
+
+#include "kudu/util/monotime.h"
+
+using std::string;
+
+namespace kudu {
+
+AsyncLogger::AsyncLogger(google::base::Logger* wrapped,
+                         int max_buffer_bytes) :
+    max_buffer_bytes_(max_buffer_bytes),
+    wrapped_(DCHECK_NOTNULL(wrapped)),
+    wake_flusher_cond_(&lock_),
+    free_buffer_cond_(&lock_),
+    flush_complete_cond_(&lock_),
+    active_buf_(new Buffer()),
+    flushing_buf_(new Buffer()) {
+  DCHECK_GT(max_buffer_bytes_, 0);
+}
+
+AsyncLogger::~AsyncLogger() {}
+
+void AsyncLogger::Start() {
+  CHECK_EQ(state_, INITTED);
+  state_ = RUNNING;
+  thread_ = std::thread(&AsyncLogger::RunThread, this);
+}
+
+void AsyncLogger::Stop() {
+  {
+    MutexLock l(lock_);
+    CHECK_EQ(state_, RUNNING);
+    state_ = STOPPED;
+    wake_flusher_cond_.Signal();
+  }
+  thread_.join();
+  CHECK(active_buf_->messages.empty());
+  CHECK(flushing_buf_->messages.empty());
+}
+
+void AsyncLogger::Write(bool force_flush,
+                        time_t timestamp,
+                        const char* message,
+                        int message_len) {
+  {
+    MutexLock l(lock_);
+    DCHECK_EQ(state_, RUNNING);
+    while (BufferFull(*active_buf_)) {
+      app_threads_blocked_count_for_tests_++;
+      free_buffer_cond_.Wait();
+    }
+    active_buf_->add(Msg(timestamp, string(message, message_len)),
+                     force_flush);
+    wake_flusher_cond_.Signal();
+  }
+
+  // In most cases, we take the 'force_flush' argument to mean that we'll let the logger
+  // thread do the flushing for us, but not block the application. However, for the
+  // special case of a FATAL log message, we really want to make sure that our message
+  // hits the log before we continue, or else it's likely that the application will exit
+  // while it's still in our buffer.
+  //
+  // NOTE: even if the application doesn't wrap the FATAL-level logger, log messages at
+  // FATAL are also written to all other log files with lower levels. So, a FATAL message
+  // will force a synchronous flush of all lower-level logs before exiting.
+  //
+  // Unfortunately, the underlying log level isn't passed through to this interface, so we
+  // have to use this hack: messages from FATAL errors start with the character 'F'.
+  if (message_len > 0 && message[0] == 'F') {
+    Flush();
+  }
+}
+
+void AsyncLogger::Flush() {
+  MutexLock l(lock_);
+  DCHECK_EQ(state_, RUNNING);
+
+  // Wake up the writer thread at least twice.
+  // This ensures that it has completely flushed both buffers.
+  uint64_t orig_flush_count = flush_count_;
+  while (flush_count_ < orig_flush_count + 2 &&
+         state_ == RUNNING) {
+    active_buf_->flush = true;
+    wake_flusher_cond_.Signal();
+    flush_complete_cond_.Wait();
+  }
+}
+
+uint32_t AsyncLogger::LogSize() {
+  return wrapped_->LogSize();
+}
+
+void AsyncLogger::RunThread() {
+  MutexLock l(lock_);
+  while (state_ == RUNNING || active_buf_->needs_flush_or_write()) {
+    while (!active_buf_->needs_flush_or_write() && state_ == RUNNING) {
+      if (!wake_flusher_cond_.WaitFor(MonoDelta::FromSeconds(FLAGS_logbufsecs))) {
+        // In case of wait timeout, force it to flush regardless whether there is anything enqueued.
+        active_buf_->flush = true;
+      }
+    }
+
+    active_buf_.swap(flushing_buf_);
+    // If the buffer that we are about to flush was full, then
+    // we may have other threads which were blocked that we now
+    // need to wake up.
+    if (BufferFull(*flushing_buf_)) {
+      free_buffer_cond_.Broadcast();
+    }
+    l.Unlock();
+
+    for (const auto& msg : flushing_buf_->messages) {
+      wrapped_->Write(false, msg.ts, msg.message.data(), msg.message.size());
+    }
+    if (flushing_buf_->flush) {
+      wrapped_->Flush();
+    }
+    flushing_buf_->clear();
+
+    l.Lock();
+    flush_count_++;
+    flush_complete_cond_.Broadcast();
+  }
+}
+
+bool AsyncLogger::BufferFull(const Buffer& buf) const {
+  // We evenly divide our total buffer space between the two buffers.
+  return buf.size > (max_buffer_bytes_ / 2);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/async_logger.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/async_logger.h b/be/src/kudu/util/async_logger.h
new file mode 100644
index 0000000..aedbdde
--- /dev/null
+++ b/be/src/kudu/util/async_logger.h
@@ -0,0 +1,206 @@
+// 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.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+
+#include <cstdint>
+#include <ctime>
+#include <memory>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/mutex.h"
+
+namespace kudu {
+
+// Wrapper for a glog Logger which asynchronously writes log messages.
+// This class starts a new thread responsible for forwarding the messages
+// to the logger, and performs double buffering. Writers append to the
+// current buffer and then wake up the logger thread. The logger swaps in
+// a new buffer and writes any accumulated messages to the wrapped
+// Logger.
+//
+// This double-buffering behavior dramatically improves performance, especially
+// for logging messages which require flushing the underlying file (i.e WARNING
+// and above for default). The flush can take a couple of milliseconds, and in
+// some cases can even block for hundreds of milliseconds or more. With the
+// double-buffered approach, threads can proceed with useful work while the IO
+// thread blocks.
+//
+// The semantics provided by this wrapper are slightly weaker than the default
+// glog semantics. By default, glog will immediately (synchronously) flush WARNING
+// and above to the underlying file, whereas here we are deferring that flush to
+// the separate thread. This means that a crash just after a 'LOG(WARNING)' would
+// may be missing the message in the logs, but the perf benefit is probably
+// worth it. We do take care that a glog FATAL message flushes all buffered log
+// messages before exiting.
+//
+// NOTE: the logger limits the total amount of buffer space, so if the underlying
+// log blocks for too long, eventually the threads generating the log messages
+// will block as well. This prevents runaway memory usage.
+class AsyncLogger : public google::base::Logger {
+ public:
+  AsyncLogger(google::base::Logger* wrapped,
+              int max_buffer_bytes);
+  ~AsyncLogger();
+
+  void Start();
+
+  // Stop the thread. Flush() and Write() must not be called after this.
+  //
+  // NOTE: this is currently only used in tests: in real life, we enable async
+  // logging once when the program starts and then never disable it.
+  //
+  // REQUIRES: Start() must have been called.
+  void Stop();
+
+  // Write a message to the log.
+  //
+  // 'force_flush' is set by the GLog library based on the configured '--logbuflevel'
+  // flag. Any messages logged at the configured level or higher result in 'force_flush'
+  // being set to true, indicating that the message should be immediately written to the
+  // log rather than buffered in memory. See the class-level docs above for more detail
+  // about the implementation provided here.
+  //
+  // REQUIRES: Start() must have been called.
+  void Write(bool force_flush,
+             time_t timestamp,
+             const char* message,
+             int message_len) override;
+
+  // Flush any buffered messages.
+  void Flush() override;
+
+  // Get the current LOG file size.
+  // The returned value is approximate since some
+  // logged data may not have been flushed to disk yet.
+  uint32_t LogSize() override;
+
+  // Return a count of how many times an application thread was
+  // blocked due to the buffers being full and the writer thread
+  // not keeping up.
+  int app_threads_blocked_count_for_tests() const {
+    MutexLock l(lock_);
+    return app_threads_blocked_count_for_tests_;
+  }
+
+ private:
+  // A buffered message.
+  //
+  // TODO(todd): using std::string for buffered messages is convenient but not
+  // as efficient as it could be. Better would be to make the buffers just be
+  // Arenas and allocate both the message data and Msg struct from them, forming
+  // a linked list.
+  struct Msg {
+    time_t ts;
+    std::string message;
+
+    Msg(time_t ts, std::string message)
+        : ts(ts),
+          message(std::move(message)) {
+    }
+  };
+
+  // A buffer of messages waiting to be flushed.
+  struct Buffer {
+    std::vector<Msg> messages;
+
+    // Estimate of the size of 'messages'.
+    int size = 0;
+
+    // Whether this buffer needs an explicit flush of the
+    // underlying logger.
+    bool flush = false;
+
+    Buffer() {}
+
+    void clear() {
+      messages.clear();
+      size = 0;
+      flush = false;
+    }
+
+    void add(Msg msg, bool flush) {
+      size += sizeof(msg) + msg.message.size();
+      messages.emplace_back(std::move(msg));
+      this->flush |= flush;
+    }
+
+    bool needs_flush_or_write() const {
+      return flush || !messages.empty();
+    }
+
+   private:
+    DISALLOW_COPY_AND_ASSIGN(Buffer);
+  };
+
+  bool BufferFull(const Buffer& buf) const;
+  void RunThread();
+
+  // The maximum number of bytes used by the entire class.
+  const int max_buffer_bytes_;
+  google::base::Logger* const wrapped_;
+  std::thread thread_;
+
+  // Count of how many times an application thread was blocked due to
+  // a full buffer.
+  int app_threads_blocked_count_for_tests_ = 0;
+
+  // Count of how many times the writer thread has flushed the buffers.
+  // 64 bits should be enough to never worry about overflow.
+  uint64_t flush_count_ = 0;
+
+  // Protects buffers as well as 'state_'.
+  mutable Mutex lock_;
+
+  // Signaled by app threads to wake up the flusher, either for new
+  // data or because 'state_' changed.
+  ConditionVariable wake_flusher_cond_;
+
+  // Signaled by the flusher thread when the flusher has swapped in
+  // a free buffer to write to.
+  ConditionVariable free_buffer_cond_;
+
+  // Signaled by the flusher thread when it has completed flushing
+  // the current buffer.
+  ConditionVariable flush_complete_cond_;
+
+  // The buffer to which application threads append new log messages.
+  std::unique_ptr<Buffer> active_buf_;
+
+  // The buffer currently being flushed by the logger thread, cleared
+  // after a successful flush.
+  std::unique_ptr<Buffer> flushing_buf_;
+
+  // Trigger for the logger thread to stop.
+  enum State {
+    INITTED,
+    RUNNING,
+    STOPPED
+  };
+  State state_ = INITTED;
+
+  DISALLOW_COPY_AND_ASSIGN(AsyncLogger);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/async_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/async_util-test.cc b/be/src/kudu/util/async_util-test.cc
new file mode 100644
index 0000000..5cb7a63
--- /dev/null
+++ b/be/src/kudu/util/async_util-test.cc
@@ -0,0 +1,129 @@
+// 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 "kudu/util/async_util.h"
+
+#include <unistd.h>
+
+#include <functional>
+#include <thread>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/callback.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::thread;
+using std::vector;
+
+namespace kudu {
+
+class AsyncUtilTest : public KuduTest {
+ public:
+  AsyncUtilTest() {
+    // Set up an alarm to fail the test in case of deadlock.
+    alarm(30);
+  }
+  ~AsyncUtilTest() {
+    // Disable the alarm on test exit.
+    alarm(0);
+  }
+};
+
+// Test completing the synchronizer through each of the APIs it exposes.
+TEST_F(AsyncUtilTest, TestSynchronizerCompletion) {
+  Synchronizer sync;
+
+  {
+    auto waiter = thread([sync] {
+        ignore_result(sync.Wait());
+    });
+    SleepFor(MonoDelta::FromMilliseconds(5));
+    sync.StatusCB(Status::OK());
+    waiter.join();
+  }
+  sync.Reset();
+  {
+    auto cb = sync.AsStatusCallback();
+    auto waiter = thread([sync] {
+        ignore_result(sync.Wait());
+    });
+    SleepFor(MonoDelta::FromMilliseconds(5));
+    cb.Run(Status::OK());
+    waiter.join();
+  }
+  sync.Reset();
+  {
+    auto cb = sync.AsStdStatusCallback();
+    auto waiter = thread([sync] {
+        ignore_result(sync.Wait());
+    });
+    SleepFor(MonoDelta::FromMilliseconds(5));
+    cb(Status::OK());
+    waiter.join();
+  }
+}
+
+TEST_F(AsyncUtilTest, TestSynchronizerMultiWait) {
+  Synchronizer sync;
+  vector<thread> waiters;
+  for (int i = 0; i < 5; i++) {
+    waiters.emplace_back([sync] {
+        ignore_result(sync.Wait());
+    });
+  }
+  SleepFor(MonoDelta::FromMilliseconds(5));
+  sync.StatusCB(Status::OK());
+
+  for (auto& waiter : waiters) {
+    waiter.join();
+  }
+}
+
+TEST_F(AsyncUtilTest, TestSynchronizerTimedWait) {
+  thread waiter;
+  {
+    Synchronizer sync;
+    auto cb = sync.AsStatusCallback();
+    waiter = thread([cb] {
+        SleepFor(MonoDelta::FromMilliseconds(5));
+        cb.Run(Status::OK());
+    });
+    ASSERT_OK(sync.WaitFor(MonoDelta::FromMilliseconds(1000)));
+  }
+  waiter.join();
+
+  {
+    Synchronizer sync;
+    auto cb = sync.AsStatusCallback();
+    waiter = thread([cb] {
+        SleepFor(MonoDelta::FromMilliseconds(1000));
+        cb.Run(Status::OK());
+    });
+    ASSERT_TRUE(sync.WaitFor(MonoDelta::FromMilliseconds(5)).IsTimedOut());
+  }
+
+  // Waiting on the thread gives TSAN to check that no thread safety issues
+  // occurred.
+  waiter.join();
+}
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/async_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/async_util.h b/be/src/kudu/util/async_util.h
new file mode 100644
index 0000000..338c6c2
--- /dev/null
+++ b/be/src/kudu/util/async_util.h
@@ -0,0 +1,99 @@
+// 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.
+//
+// Utility functions which are handy when doing async/callback-based programming.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+#include "kudu/util/status_callback.h"
+
+namespace kudu {
+
+// Simple class which can be used to make async methods synchronous.
+// For example:
+//   Synchronizer s;
+//   SomeAsyncMethod(s.AsStatusCallback());
+//   CHECK_OK(s.Wait());
+//
+// The lifetime of the synchronizer is decoupled from the callback it produces.
+// If the callback outlives the synchronizer then executing it will be a no-op.
+// Callers must be careful not to allow the callback to be destructed without
+// completing it, otherwise the thread waiting on the synchronizer will block
+// indefinitely.
+class Synchronizer {
+ public:
+  Synchronizer()
+    : data_(std::make_shared<Data>()) {
+  }
+
+  void StatusCB(const Status& status) {
+    Data::Callback(std::weak_ptr<Data>(data_), status);
+  }
+
+  StatusCallback AsStatusCallback() {
+    return Bind(Data::Callback, std::weak_ptr<Data>(data_));
+  }
+
+  StdStatusCallback AsStdStatusCallback() {
+    return std::bind(Data::Callback, std::weak_ptr<Data>(data_), std::placeholders::_1);
+  }
+
+  Status Wait() const {
+    data_->latch.Wait();
+    return data_->status;
+  }
+
+  Status WaitFor(const MonoDelta& delta) const {
+    if (PREDICT_FALSE(!data_->latch.WaitFor(delta))) {
+      return Status::TimedOut("timed out while waiting for the callback to be called");
+    }
+    return data_->status;
+  }
+
+  void Reset() {
+    data_->latch.Reset(1);
+  }
+
+ private:
+
+  struct Data {
+    Data() : latch(1) {
+    }
+
+    static void Callback(std::weak_ptr<Data> weak, const Status& status) {
+      auto ptr = weak.lock();
+      if (ptr) {
+        ptr->status = status;
+        ptr->latch.CountDown();
+      }
+    }
+
+    Status status;
+    CountDownLatch latch;
+    DISALLOW_COPY_AND_ASSIGN(Data);
+  };
+
+  std::shared_ptr<Data> data_;
+};
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/atomic-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/atomic-test.cc b/be/src/kudu/util/atomic-test.cc
new file mode 100644
index 0000000..a65d55d
--- /dev/null
+++ b/be/src/kudu/util/atomic-test.cc
@@ -0,0 +1,135 @@
+// 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 "kudu/util/atomic.h"
+
+#include <cstdint>
+#include <limits>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+using std::numeric_limits;
+using std::vector;
+
+// TODO Add some multi-threaded tests; currently AtomicInt is just a
+// wrapper around 'atomicops.h', but should the underlying
+// implemention change, it would help to have tests that make sure
+// invariants are preserved in a multi-threaded environment.
+
+template<typename T>
+class AtomicIntTest : public KuduTest {
+ public:
+
+  AtomicIntTest()
+      : max_(numeric_limits<T>::max()),
+        min_(numeric_limits<T>::min()) {
+    acquire_release_ = { kMemOrderNoBarrier, kMemOrderAcquire, kMemOrderRelease };
+    barrier_ = { kMemOrderNoBarrier, kMemOrderBarrier };
+  }
+
+  vector<MemoryOrder> acquire_release_;
+  vector<MemoryOrder> barrier_;
+
+  T max_;
+  T min_;
+};
+
+typedef ::testing::Types<int32_t, int64_t, uint32_t, uint64_t> IntTypes;
+TYPED_TEST_CASE(AtomicIntTest, IntTypes);
+
+TYPED_TEST(AtomicIntTest, LoadStore) {
+  for (const MemoryOrder mem_order : this->acquire_release_) {
+    AtomicInt<TypeParam> i(0);
+    EXPECT_EQ(0, i.Load(mem_order));
+    i.Store(42, mem_order);
+    EXPECT_EQ(42, i.Load(mem_order));
+    i.Store(this->min_, mem_order);
+    EXPECT_EQ(this->min_, i.Load(mem_order));
+    i.Store(this->max_, mem_order);
+    EXPECT_EQ(this->max_, i.Load(mem_order));
+  }
+}
+
+TYPED_TEST(AtomicIntTest, SetSwapExchange) {
+  for (const MemoryOrder mem_order : this->acquire_release_) {
+    AtomicInt<TypeParam> i(0);
+    EXPECT_TRUE(i.CompareAndSet(0, 5, mem_order));
+    EXPECT_EQ(5, i.Load(mem_order));
+    EXPECT_FALSE(i.CompareAndSet(0, 10, mem_order));
+
+    EXPECT_EQ(5, i.CompareAndSwap(5, this->max_, mem_order));
+    EXPECT_EQ(this->max_, i.CompareAndSwap(42, 42, mem_order));
+    EXPECT_EQ(this->max_, i.CompareAndSwap(this->max_, this->min_, mem_order));
+
+    EXPECT_EQ(this->min_, i.Exchange(this->max_, mem_order));
+    EXPECT_EQ(this->max_, i.Load(mem_order));
+  }
+}
+
+TYPED_TEST(AtomicIntTest, MinMax) {
+  for (const MemoryOrder mem_order : this->acquire_release_) {
+    AtomicInt<TypeParam> i(0);
+
+    i.StoreMax(100, mem_order);
+    EXPECT_EQ(100, i.Load(mem_order));
+    i.StoreMin(50, mem_order);
+    EXPECT_EQ(50, i.Load(mem_order));
+
+    i.StoreMax(25, mem_order);
+    EXPECT_EQ(50, i.Load(mem_order));
+    i.StoreMin(75, mem_order);
+    EXPECT_EQ(50, i.Load(mem_order));
+
+    i.StoreMax(this->max_, mem_order);
+    EXPECT_EQ(this->max_, i.Load(mem_order));
+    i.StoreMin(this->min_, mem_order);
+    EXPECT_EQ(this->min_, i.Load(mem_order));
+  }
+}
+
+TYPED_TEST(AtomicIntTest, Increment) {
+  for (const MemoryOrder mem_order : this->barrier_) {
+    AtomicInt<TypeParam> i(0);
+    EXPECT_EQ(1, i.Increment(mem_order));
+    EXPECT_EQ(3, i.IncrementBy(2, mem_order));
+    EXPECT_EQ(3, i.IncrementBy(0, mem_order));
+  }
+}
+
+TEST(Atomic, AtomicBool) {
+  vector<MemoryOrder> memory_orders = { kMemOrderNoBarrier, kMemOrderRelease, kMemOrderAcquire };
+  for (const MemoryOrder mem_order : memory_orders) {
+    AtomicBool b(false);
+    EXPECT_FALSE(b.Load(mem_order));
+    b.Store(true, mem_order);
+    EXPECT_TRUE(b.Load(mem_order));
+    EXPECT_TRUE(b.CompareAndSet(true, false, mem_order));
+    EXPECT_FALSE(b.Load(mem_order));
+    EXPECT_FALSE(b.CompareAndSet(true, false, mem_order));
+    EXPECT_FALSE(b.CompareAndSwap(false, true, mem_order));
+    EXPECT_TRUE(b.Load(mem_order));
+    EXPECT_TRUE(b.Exchange(false, mem_order));
+    EXPECT_FALSE(b.Load(mem_order));
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/atomic.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/atomic.cc b/be/src/kudu/util/atomic.cc
new file mode 100644
index 0000000..430631f
--- /dev/null
+++ b/be/src/kudu/util/atomic.cc
@@ -0,0 +1,56 @@
+// 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 "kudu/util/atomic.h"
+
+#include <cstdint>
+#include <ostream>
+
+#include <glog/logging.h>
+
+namespace kudu {
+
+template<typename T>
+AtomicInt<T>::AtomicInt(T initial_value) {
+  Store(initial_value, kMemOrderNoBarrier);
+}
+
+template<typename T>
+void AtomicInt<T>::FatalMemOrderNotSupported(const char* caller,
+                                             const char* requested,
+                                             const char* supported) {
+  LOG(FATAL) << caller << " does not support " << requested << ": only "
+             << supported << " are supported.";
+}
+
+template
+class AtomicInt<int32_t>;
+
+template
+class AtomicInt<int64_t>;
+
+template
+class AtomicInt<uint32_t>;
+
+template
+class AtomicInt<uint64_t>;
+
+AtomicBool::AtomicBool(bool value)
+    : underlying_(value) {
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/atomic.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/atomic.h b/be/src/kudu/util/atomic.h
new file mode 100644
index 0000000..3051a41
--- /dev/null
+++ b/be/src/kudu/util/atomic.h
@@ -0,0 +1,322 @@
+// 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.
+
+#ifndef KUDU_UTIL_ATOMIC_H
+#define KUDU_UTIL_ATOMIC_H
+
+#include <algorithm>  // IWYU pragma: keep
+#include <cstdint>
+#include <cstdlib>
+#include <type_traits>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+
+namespace kudu {
+
+// See top-level comments in kudu/gutil/atomicops.h for further
+// explanations of these levels.
+enum MemoryOrder {
+  // Relaxed memory ordering, doesn't use any barriers.
+  kMemOrderNoBarrier = 0,
+
+  // Ensures that no later memory access by the same thread can be
+  // reordered ahead of the operation.
+  kMemOrderAcquire = 1,
+
+  // Ensures that no previous memory access by the same thread can be
+  // reordered after the operation.
+  kMemOrderRelease = 2,
+
+  // Ensures that neither previous NOR later memory access by the same
+  // thread can be reordered after the operation.
+  kMemOrderBarrier = 3,
+};
+
+// Atomic integer class inspired by Impala's AtomicInt and
+// std::atomic<> in C++11.
+//
+// NOTE: All of public operations use an implicit memory order of
+// kMemOrderNoBarrier unless otherwise specified.
+//
+// Unlike std::atomic<>, overflowing an unsigned AtomicInt via Increment or
+// IncrementBy is undefined behavior (it is also undefined for signed types,
+// as always).
+//
+// See also: kudu/gutil/atomicops.h
+template<typename T>
+class AtomicInt {
+ public:
+  // Initialize the underlying value to 'initial_value'. The
+  // initialization performs a Store with 'kMemOrderNoBarrier'.
+  explicit AtomicInt(T initial_value);
+
+  // Returns the underlying value.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  T Load(MemoryOrder mem_order = kMemOrderNoBarrier) const;
+
+  // Sets the underlying value to 'new_value'.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  void Store(T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Iff the underlying value is equal to 'expected_val', sets the
+  // underlying value to 'new_value' and returns true; returns false
+  // otherwise.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  bool CompareAndSet(T expected_val, T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Iff the underlying value is equal to 'expected_val', sets the
+  // underlying value to 'new_value' and returns
+  // 'expected_val'. Otherwise, returns the current underlying
+  // value.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  T CompareAndSwap(T expected_val, T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Sets the underlying value to 'new_value' iff 'new_value' is
+  // greater than the current underlying value.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  void StoreMax(T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Sets the underlying value to 'new_value' iff 'new_value' is less
+  // than the current underlying value.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  void StoreMin(T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Increments the underlying value by 1 and returns the new
+  // underlying value.
+  //
+  // Does not support 'kMemOrderAcquire' or 'kMemOrderRelease'.
+  T Increment(MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Increments the underlying value by 'delta' and returns the new
+  // underlying value.
+
+  // Does not support 'kKemOrderAcquire' or 'kMemOrderRelease'.
+  T IncrementBy(T delta, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+  // Sets the underlying value to 'new_value' and returns the previous
+  // underlying value.
+  //
+  // Does not support 'kMemOrderBarrier'.
+  T Exchange(T new_value, MemoryOrder mem_order = kMemOrderNoBarrier);
+
+ private:
+  // If a method 'caller' doesn't support memory order described as
+  // 'requested', exit by doing perform LOG(FATAL) logging the method
+  // called, the requested memory order, and the supported memory
+  // orders.
+  static void FatalMemOrderNotSupported(const char* caller,
+                                        const char* requested = "kMemOrderBarrier",
+                                        const char* supported =
+                                        "kMemNorderNoBarrier, kMemOrderAcquire, kMemOrderRelease");
+
+  // The gutil/atomicops.h functions only operate on signed types.
+  // So, even if the user specializes on an unsigned type, we use a
+  // signed type internally.
+  typedef typename std::make_signed<T>::type SignedT;
+  SignedT value_;
+
+  DISALLOW_COPY_AND_ASSIGN(AtomicInt);
+};
+
+// Adapts AtomicInt to handle boolean values.
+//
+// NOTE: All of public operations use an implicit memory order of
+// kMemOrderNoBarrier unless otherwise specified.
+//
+// See AtomicInt above for documentation on individual methods.
+class AtomicBool {
+ public:
+  explicit AtomicBool(bool value);
+
+  bool Load(MemoryOrder m = kMemOrderNoBarrier) const {
+    return underlying_.Load(m);
+  }
+  void Store(bool n, MemoryOrder m = kMemOrderNoBarrier) {
+    underlying_.Store(static_cast<int32_t>(n), m);
+  }
+  bool CompareAndSet(bool e, bool n, MemoryOrder m = kMemOrderNoBarrier) {
+    return underlying_.CompareAndSet(static_cast<int32_t>(e), static_cast<int32_t>(n), m);
+  }
+  bool CompareAndSwap(bool e, bool n, MemoryOrder m = kMemOrderNoBarrier) {
+    return underlying_.CompareAndSwap(static_cast<int32_t>(e), static_cast<int32_t>(n), m);
+  }
+  bool Exchange(bool n, MemoryOrder m = kMemOrderNoBarrier) {
+    return underlying_.Exchange(static_cast<int32_t>(n), m);
+  }
+ private:
+  AtomicInt<int32_t> underlying_;
+
+  DISALLOW_COPY_AND_ASSIGN(AtomicBool);
+};
+
+template<typename T>
+inline T AtomicInt<T>::Load(MemoryOrder mem_order) const {
+  switch (mem_order) {
+    case kMemOrderNoBarrier: {
+      return base::subtle::NoBarrier_Load(&value_);
+    }
+    case kMemOrderBarrier: {
+      FatalMemOrderNotSupported("Load");
+      break;
+    }
+    case kMemOrderAcquire: {
+      return base::subtle::Acquire_Load(&value_);
+    }
+    case kMemOrderRelease: {
+      return base::subtle::Release_Load(&value_);
+    }
+  }
+  abort(); // Unnecessary, but avoids gcc complaining.
+}
+
+template<typename T>
+inline void AtomicInt<T>::Store(T new_value, MemoryOrder mem_order) {
+  switch (mem_order) {
+    case kMemOrderNoBarrier: {
+      base::subtle::NoBarrier_Store(&value_, new_value);
+      break;
+    }
+    case kMemOrderBarrier: {
+      FatalMemOrderNotSupported("Store");
+      break;
+    }
+    case kMemOrderAcquire: {
+      base::subtle::Acquire_Store(&value_, new_value);
+      break;
+    }
+    case kMemOrderRelease: {
+      base::subtle::Release_Store(&value_, new_value);
+      break;
+    }
+  }
+}
+
+template<typename T>
+inline bool AtomicInt<T>::CompareAndSet(T expected_val, T new_val, MemoryOrder mem_order) {
+  return CompareAndSwap(expected_val, new_val, mem_order) == expected_val;
+}
+
+template<typename T>
+inline T AtomicInt<T>::CompareAndSwap(T expected_val, T new_val, MemoryOrder mem_order) {
+  switch (mem_order) {
+    case kMemOrderNoBarrier: {
+      return base::subtle::NoBarrier_CompareAndSwap(
+          &value_, expected_val, new_val);
+    }
+    case kMemOrderBarrier: {
+      FatalMemOrderNotSupported("CompareAndSwap/CompareAndSet");
+      break;
+    }
+    case kMemOrderAcquire: {
+      return base::subtle::Acquire_CompareAndSwap(
+          &value_, expected_val, new_val);
+    }
+    case kMemOrderRelease: {
+      return base::subtle::Release_CompareAndSwap(
+          &value_, expected_val, new_val);
+    }
+  }
+  abort();
+}
+
+
+template<typename T>
+inline T AtomicInt<T>::Increment(MemoryOrder mem_order) {
+  return IncrementBy(1, mem_order);
+}
+
+template<typename T>
+inline T AtomicInt<T>::IncrementBy(T delta, MemoryOrder mem_order) {
+  switch (mem_order) {
+    case kMemOrderNoBarrier: {
+      return base::subtle::NoBarrier_AtomicIncrement(&value_, delta);
+    }
+    case kMemOrderBarrier: {
+      return base::subtle::Barrier_AtomicIncrement(&value_, delta);
+    }
+    case kMemOrderAcquire: {
+      FatalMemOrderNotSupported("Increment/IncrementBy",
+                                "kMemOrderAcquire",
+                                "kMemOrderNoBarrier and kMemOrderBarrier");
+      break;
+    }
+    case kMemOrderRelease: {
+      FatalMemOrderNotSupported("Increment/Incrementby",
+                                "kMemOrderAcquire",
+                                "kMemOrderNoBarrier and kMemOrderBarrier");
+      break;
+    }
+  }
+  abort();
+}
+
+template<typename T>
+inline T AtomicInt<T>::Exchange(T new_value, MemoryOrder mem_order) {
+  switch (mem_order) {
+    case kMemOrderNoBarrier: {
+      return base::subtle::NoBarrier_AtomicExchange(&value_, new_value);
+    }
+    case kMemOrderBarrier: {
+      FatalMemOrderNotSupported("Exchange");
+      break;
+    }
+    case kMemOrderAcquire: {
+      return base::subtle::Acquire_AtomicExchange(&value_, new_value);
+    }
+    case kMemOrderRelease: {
+      return base::subtle::Release_AtomicExchange(&value_, new_value);
+    }
+  }
+  abort();
+}
+
+template<typename T>
+inline void AtomicInt<T>::StoreMax(T new_value, MemoryOrder mem_order) {
+  T old_value = Load(mem_order);
+  while (true) {
+    T max_value = std::max(old_value, new_value);
+    T prev_value = CompareAndSwap(old_value, max_value, mem_order);
+    if (PREDICT_TRUE(old_value == prev_value)) {
+      break;
+    }
+    old_value = prev_value;
+  }
+}
+
+template<typename T>
+inline void AtomicInt<T>::StoreMin(T new_value, MemoryOrder mem_order) {
+  T old_value = Load(mem_order);
+  while (true) {
+    T min_value = std::min(old_value, new_value);
+    T prev_value = CompareAndSwap(old_value, min_value, mem_order);
+    if (PREDICT_TRUE(old_value == prev_value)) {
+      break;
+    }
+    old_value = prev_value;
+  }
+}
+
+} // namespace kudu
+#endif /* KUDU_UTIL_ATOMIC_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/auto_release_pool.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/auto_release_pool.h b/be/src/kudu/util/auto_release_pool.h
new file mode 100644
index 0000000..eaed9c2
--- /dev/null
+++ b/be/src/kudu/util/auto_release_pool.h
@@ -0,0 +1,99 @@
+// 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.
+//
+// Simple pool of objects that will be deallocated when the pool is
+// destroyed
+
+#ifndef KUDU_UTIL_AUTO_RELEASE_POOL_H
+#define KUDU_UTIL_AUTO_RELEASE_POOL_H
+
+#include <vector>
+
+#include "kudu/gutil/spinlock.h"
+
+namespace kudu {
+
+// Thread-safe.
+class AutoReleasePool {
+ public:
+  AutoReleasePool(): objects_() { }
+
+  ~AutoReleasePool() {
+    for (auto& object : objects_) {
+      delete object;
+    }
+  }
+
+  template <class T>
+  T *Add(T *t) {
+    base::SpinLockHolder l(&lock_);
+    objects_.push_back(new SpecificElement<T>(t));
+    return t;
+  }
+
+  // Add an array-allocated object to the pool. This is identical to
+  // Add() except that it will be freed with 'delete[]' instead of 'delete'.
+  template<class T>
+  T* AddArray(T *t) {
+    base::SpinLockHolder l(&lock_);
+    objects_.push_back(new SpecificArrayElement<T>(t));
+    return t;
+  }
+
+  // Donate all objects in this pool to another pool.
+  void DonateAllTo(AutoReleasePool* dst) {
+    base::SpinLockHolder l(&lock_);
+    base::SpinLockHolder l_them(&dst->lock_);
+
+    dst->objects_.reserve(dst->objects_.size() + objects_.size());
+    dst->objects_.insert(dst->objects_.end(), objects_.begin(), objects_.end());
+    objects_.clear();
+  }
+
+ private:
+  struct GenericElement {
+    virtual ~GenericElement() {}
+  };
+
+  template <class T>
+  struct SpecificElement : GenericElement {
+    explicit SpecificElement(T *t): t(t) {}
+    ~SpecificElement() {
+      delete t;
+    }
+
+    T *t;
+  };
+
+  template <class T>
+  struct SpecificArrayElement : GenericElement {
+    explicit SpecificArrayElement(T *t): t(t) {}
+    ~SpecificArrayElement() {
+      delete [] t;
+    }
+
+    T *t;
+  };
+
+  typedef std::vector<GenericElement *> ElementVector;
+  ElementVector objects_;
+  base::SpinLock lock_;
+};
+
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/barrier.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/barrier.h b/be/src/kudu/util/barrier.h
new file mode 100644
index 0000000..88e5682
--- /dev/null
+++ b/be/src/kudu/util/barrier.h
@@ -0,0 +1,68 @@
+// 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.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/thread_restrictions.h"
+
+namespace kudu {
+
+// Implementation of pthread-style Barriers.
+class Barrier {
+ public:
+  // Initialize the barrier with the given initial count.
+  explicit Barrier(int count) :
+      cond_(&mutex_),
+      count_(count),
+      initial_count_(count) {
+    DCHECK_GT(count, 0);
+  }
+
+  ~Barrier() {
+  }
+
+  // Wait until all threads have reached the barrier.
+  // Once all threads have reached the barrier, the barrier is reset
+  // to the initial count.
+  void Wait() {
+    ThreadRestrictions::AssertWaitAllowed();
+    MutexLock l(mutex_);
+    if (--count_ == 0) {
+      count_ = initial_count_;
+      cycle_count_++;
+      cond_.Broadcast();
+      return;
+    }
+
+    int initial_cycle = cycle_count_;
+    while (cycle_count_ == initial_cycle) {
+      cond_.Wait();
+    }
+  }
+
+ private:
+  Mutex mutex_;
+  ConditionVariable cond_;
+  int count_;
+  uint32_t cycle_count_ = 0;
+  const int initial_count_;
+  DISALLOW_COPY_AND_ASSIGN(Barrier);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/bit-stream-utils.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/bit-stream-utils.h b/be/src/kudu/util/bit-stream-utils.h
new file mode 100644
index 0000000..c6aeb01
--- /dev/null
+++ b/be/src/kudu/util/bit-stream-utils.h
@@ -0,0 +1,150 @@
+// 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.
+#ifndef IMPALA_UTIL_BIT_STREAM_UTILS_H
+#define IMPALA_UTIL_BIT_STREAM_UTILS_H
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/bit-util.h"
+#include "kudu/util/faststring.h"
+
+namespace kudu {
+
+// Utility class to write bit/byte streams.  This class can write data to either be
+// bit packed or byte aligned (and a single stream that has a mix of both).
+class BitWriter {
+ public:
+  // buffer: buffer to write bits to.
+  explicit BitWriter(faststring *buffer)
+    : buffer_(buffer) {
+    Clear();
+  }
+
+  void Clear() {
+    buffered_values_ = 0;
+    byte_offset_ = 0;
+    bit_offset_ = 0;
+    buffer_->clear();
+  }
+
+  // Returns a pointer to the underlying buffer
+  faststring *buffer() const { return buffer_; }
+
+  // The number of current bytes written, including the current byte (i.e. may include a
+  // fraction of a byte). Includes buffered values.
+  int bytes_written() const { return byte_offset_ + BitUtil::Ceil(bit_offset_, 8); }
+
+  // Writes a value to buffered_values_, flushing to buffer_ if necessary.  This is bit
+  // packed. num_bits must be <= 32. If 'v' is larger than 'num_bits' bits, the higher
+  // bits are ignored.
+  void PutValue(uint64_t v, int num_bits);
+
+  // Writes v to the next aligned byte using num_bits. If T is larger than num_bits, the
+  // extra high-order bits will be ignored.
+  template<typename T>
+  void PutAligned(T v, int num_bits);
+
+  // Write a Vlq encoded int to the buffer. The value is written byte aligned.
+  // For more details on vlq: en.wikipedia.org/wiki/Variable-length_quantity
+  void PutVlqInt(int32_t v);
+
+  // Get the index to the next aligned byte and advance the underlying buffer by num_bytes.
+  size_t GetByteIndexAndAdvance(int num_bytes) {
+    uint8_t* ptr = GetNextBytePtr(num_bytes);
+    return ptr - buffer_->data();
+  }
+
+  // Get a pointer to the next aligned byte and advance the underlying buffer by num_bytes.
+  uint8_t* GetNextBytePtr(int num_bytes);
+
+  // Flushes all buffered values to the buffer. Call this when done writing to the buffer.
+  // If 'align' is true, buffered_values_ is reset and any future writes will be written
+  // to the next byte boundary.
+  void Flush(bool align = false);
+
+ private:
+  // Bit-packed values are initially written to this variable before being memcpy'd to
+  // buffer_. This is faster than writing values byte by byte directly to buffer_.
+  uint64_t buffered_values_;
+
+  faststring *buffer_;
+  int byte_offset_;       // Offset in buffer_
+  int bit_offset_;        // Offset in buffered_values_
+};
+
+// Utility class to read bit/byte stream.  This class can read bits or bytes
+// that are either byte aligned or not.  It also has utilities to read multiple
+// bytes in one read (e.g. encoded int).
+class BitReader {
+ public:
+  // 'buffer' is the buffer to read from.  The buffer's length is 'buffer_len'.
+  BitReader(const uint8_t* buffer, int buffer_len);
+
+  BitReader() : buffer_(NULL), max_bytes_(0) {}
+
+  // Gets the next value from the buffer.  Returns true if 'v' could be read or false if
+  // there are not enough bytes left. num_bits must be <= 32.
+  template<typename T>
+  bool GetValue(int num_bits, T* v);
+
+  // Reads a 'num_bytes'-sized value from the buffer and stores it in 'v'. T needs to be a
+  // little-endian native type and big enough to store 'num_bytes'. The value is assumed
+  // to be byte-aligned so the stream will be advanced to the start of the next byte
+  // before 'v' is read. Returns false if there are not enough bytes left.
+  template<typename T>
+  bool GetAligned(int num_bytes, T* v);
+
+  // Reads a vlq encoded int from the stream.  The encoded int must start at the
+  // beginning of a byte. Return false if there were not enough bytes in the buffer.
+  bool GetVlqInt(int32_t* v);
+
+  // Returns the number of bytes left in the stream, not including the current byte (i.e.,
+  // there may be an additional fraction of a byte).
+  int bytes_left() { return max_bytes_ - (byte_offset_ + BitUtil::Ceil(bit_offset_, 8)); }
+
+  // Current position in the stream, by bit.
+  int position() const { return byte_offset_ * 8 + bit_offset_; }
+
+  // Rewind the stream by 'num_bits' bits
+  void Rewind(int num_bits);
+
+  // Seek to a specific bit in the buffer
+  void SeekToBit(uint stream_position);
+
+  // Maximum byte length of a vlq encoded int
+  static const int MAX_VLQ_BYTE_LEN = 5;
+
+  bool is_initialized() const { return buffer_ != NULL; }
+
+ private:
+  // Used by SeekToBit() and GetValue() to fetch the
+  // the next word into buffer_.
+  void BufferValues();
+
+  const uint8_t* buffer_;
+  int max_bytes_;
+
+  // Bytes are memcpy'd from buffer_ and values are read from this variable. This is
+  // faster than reading values byte by byte directly from buffer_.
+  uint64_t buffered_values_;
+
+  int byte_offset_;       // Offset in buffer_
+  int bit_offset_;        // Offset in buffered_values_
+};
+
+} // namespace kudu
+
+#endif


[37/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_stub-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_stub-test.cc b/be/src/kudu/rpc/rpc_stub-test.cc
new file mode 100644
index 0000000..e626276
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_stub-test.cc
@@ -0,0 +1,726 @@
+// 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 <algorithm>
+#include <atomic>
+#include <csignal>
+#include <cstdio>
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include <boost/bind.hpp>
+#include <boost/core/ref.hpp>
+#include <boost/function.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <glog/stl_logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/proxy.h"
+#include "kudu/rpc/rpc-test-base.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpcz_store.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/rtest.proxy.h"
+#include "kudu/rpc/service_pool.h"
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/random.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread_restrictions.h"
+#include "kudu/util/user.h"
+
+DEFINE_bool(is_panic_test_child, false, "Used by TestRpcPanic");
+DECLARE_bool(socket_inject_short_recvs);
+
+using kudu::pb_util::SecureDebugString;
+using std::shared_ptr;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using base::subtle::NoBarrier_Load;
+
+namespace kudu {
+namespace rpc {
+
+class RpcStubTest : public RpcTestBase {
+ public:
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    // Use a shorter queue length since some tests below need to start enough
+    // threads to saturate the queue.
+    service_queue_length_ = 10;
+    ASSERT_OK(StartTestServerWithGeneratedCode(&server_addr_));
+    ASSERT_OK(CreateMessenger("Client", &client_messenger_));
+  }
+ protected:
+  void SendSimpleCall() {
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+    RpcController controller;
+    AddRequestPB req;
+    req.set_x(10);
+    req.set_y(20);
+    AddResponsePB resp;
+    ASSERT_OK(p.Add(req, &resp, &controller));
+    ASSERT_EQ(30, resp.result());
+  }
+
+  Sockaddr server_addr_;
+  shared_ptr<Messenger> client_messenger_;
+};
+
+TEST_F(RpcStubTest, TestSimpleCall) {
+  SendSimpleCall();
+}
+
+// Regression test for a bug in which we would not properly parse a call
+// response when recv() returned a 'short read'. This injects such short
+// reads and then makes a number of calls.
+TEST_F(RpcStubTest, TestShortRecvs) {
+  FLAGS_socket_inject_short_recvs = true;
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  for (int i = 0; i < 100; i++) {
+    NO_FATALS(SendSimpleCall());
+  }
+}
+
+// Test calls which are rather large.
+// This test sends many of them at once using the async API and then
+// waits for them all to return. This is meant to ensure that the
+// IO threads can deal with read/write calls that don't succeed
+// in sending the entire data in one go.
+TEST_F(RpcStubTest, TestBigCallData) {
+  const int kNumSentAtOnce = 20;
+  const size_t kMessageSize = 5 * 1024 * 1024;
+  string data;
+  data.resize(kMessageSize);
+
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  EchoRequestPB req;
+  req.set_data(data);
+
+  vector<unique_ptr<EchoResponsePB>> resps;
+  vector<unique_ptr<RpcController>> controllers;
+
+  CountDownLatch latch(kNumSentAtOnce);
+  for (int i = 0; i < kNumSentAtOnce; i++) {
+    resps.emplace_back(new EchoResponsePB);
+    controllers.emplace_back(new RpcController);
+
+    p.EchoAsync(req, resps.back().get(), controllers.back().get(),
+                boost::bind(&CountDownLatch::CountDown, boost::ref(latch)));
+  }
+
+  latch.Wait();
+
+  for (const auto& c : controllers) {
+    ASSERT_OK(c->status());
+  }
+}
+
+TEST_F(RpcStubTest, TestRespondDeferred) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController controller;
+  SleepRequestPB req;
+  req.set_sleep_micros(1000);
+  req.set_deferred(true);
+  SleepResponsePB resp;
+  ASSERT_OK(p.Sleep(req, &resp, &controller));
+}
+
+// Test that the default user credentials are propagated to the server.
+TEST_F(RpcStubTest, TestDefaultCredentialsPropagated) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  string expected;
+  ASSERT_OK(GetLoggedInUser(&expected));
+
+  RpcController controller;
+  WhoAmIRequestPB req;
+  WhoAmIResponsePB resp;
+  ASSERT_OK(p.WhoAmI(req, &resp, &controller));
+  ASSERT_EQ(expected, resp.credentials().real_user());
+  ASSERT_FALSE(resp.credentials().has_effective_user());
+}
+
+// Test that the user can specify other credentials.
+TEST_F(RpcStubTest, TestCustomCredentialsPropagated) {
+  const char* const kFakeUserName = "some fake user";
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  UserCredentials creds;
+  creds.set_real_user(kFakeUserName);
+  p.set_user_credentials(creds);
+
+  RpcController controller;
+  WhoAmIRequestPB req;
+  WhoAmIResponsePB resp;
+  ASSERT_OK(p.WhoAmI(req, &resp, &controller));
+  ASSERT_EQ(kFakeUserName, resp.credentials().real_user());
+  ASSERT_FALSE(resp.credentials().has_effective_user());
+}
+
+TEST_F(RpcStubTest, TestAuthorization) {
+  // First test calling WhoAmI() as user "alice", who is disallowed.
+  {
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+    UserCredentials creds;
+    creds.set_real_user("alice");
+    p.set_user_credentials(creds);
+
+    // Alice is disallowed by all RPCs.
+    RpcController controller;
+    WhoAmIRequestPB req;
+    WhoAmIResponsePB resp;
+    Status s = p.WhoAmI(req, &resp, &controller);
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.ToString(),
+              "Remote error: Not authorized: alice is not allowed to call this method");
+  }
+
+  // Try some calls as "bob".
+  {
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+    UserCredentials creds;
+    creds.set_real_user("bob");
+    p.set_user_credentials(creds);
+
+    // "bob" is allowed to call WhoAmI().
+    {
+      RpcController controller;
+      WhoAmIRequestPB req;
+      WhoAmIResponsePB resp;
+      ASSERT_OK(p.WhoAmI(req, &resp, &controller));
+    }
+
+    // "bob" is not allowed to call "Sleep".
+    {
+      RpcController controller;
+      SleepRequestPB req;
+      req.set_sleep_micros(10);
+      SleepResponsePB resp;
+      Status s = p.Sleep(req, &resp, &controller);
+      ASSERT_EQ(s.ToString(),
+                "Remote error: Not authorized: bob is not allowed to call this method");
+    }
+  }
+}
+
+// Test that the user's remote address is accessible to the server.
+TEST_F(RpcStubTest, TestRemoteAddress) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController controller;
+  WhoAmIRequestPB req;
+  WhoAmIResponsePB resp;
+  ASSERT_OK(p.WhoAmI(req, &resp, &controller));
+  ASSERT_STR_CONTAINS(resp.address(), "127.0.0.1:");
+}
+
+////////////////////////////////////////////////////////////
+// Tests for error cases
+////////////////////////////////////////////////////////////
+
+// Test sending a PB parameter with a missing field, where the client
+// thinks it has sent a full PB. (eg due to version mismatch)
+TEST_F(RpcStubTest, TestCallWithInvalidParam) {
+  Proxy p(client_messenger_, server_addr_, server_addr_.host(),
+          CalculatorService::static_service_name());
+
+  rpc_test::AddRequestPartialPB req;
+  req.set_x(rand());
+  // AddRequestPartialPB is missing the 'y' field.
+  AddResponsePB resp;
+  RpcController controller;
+  Status s = p.SyncRequest("Add", req, &resp, &controller);
+  ASSERT_TRUE(s.IsRemoteError()) << "Bad status: " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(),
+                      "Invalid argument: invalid parameter for call "
+                      "kudu.rpc_test.CalculatorService.Add: "
+                      "missing fields: y");
+}
+
+// Wrapper around AtomicIncrement, since AtomicIncrement returns the 'old'
+// value, and our callback needs to be a void function.
+static void DoIncrement(Atomic32* count) {
+  base::subtle::Barrier_AtomicIncrement(count, 1);
+}
+
+// Test sending a PB parameter with a missing field on the client side.
+// This also ensures that the async callback is only called once
+// (regression test for a previously-encountered bug).
+TEST_F(RpcStubTest, TestCallWithMissingPBFieldClientSide) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController controller;
+  AddRequestPB req;
+  req.set_x(10);
+  // Request is missing the 'y' field.
+  AddResponsePB resp;
+  Atomic32 callback_count = 0;
+  p.AddAsync(req, &resp, &controller, boost::bind(&DoIncrement, &callback_count));
+  while (NoBarrier_Load(&callback_count) == 0) {
+    SleepFor(MonoDelta::FromMicroseconds(10));
+  }
+  SleepFor(MonoDelta::FromMicroseconds(100));
+  ASSERT_EQ(1, NoBarrier_Load(&callback_count));
+  ASSERT_STR_CONTAINS(controller.status().ToString(),
+                      "Invalid argument: invalid parameter for call "
+                      "kudu.rpc_test.CalculatorService.Add: missing fields: y");
+}
+
+TEST_F(RpcStubTest, TestResponseWithMissingField) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController rpc;
+  TestInvalidResponseRequestPB req;
+  TestInvalidResponseResponsePB resp;
+  req.set_error_type(rpc_test::TestInvalidResponseRequestPB_ErrorType_MISSING_REQUIRED_FIELD);
+  Status s = p.TestInvalidResponse(req, &resp, &rpc);
+  ASSERT_STR_CONTAINS(s.ToString(),
+                      "invalid RPC response, missing fields: response");
+}
+
+// Test case where the server responds with a message which is larger than the maximum
+// configured RPC message size. The server should send the response, but the client
+// will reject it.
+TEST_F(RpcStubTest, TestResponseLargerThanFrameSize) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController rpc;
+  TestInvalidResponseRequestPB req;
+  TestInvalidResponseResponsePB resp;
+  req.set_error_type(rpc_test::TestInvalidResponseRequestPB_ErrorType_RESPONSE_TOO_LARGE);
+  Status s = p.TestInvalidResponse(req, &resp, &rpc);
+  ASSERT_STR_CONTAINS(s.ToString(), "Network error: RPC frame had a length of");
+}
+
+// Test sending a call which isn't implemented by the server.
+TEST_F(RpcStubTest, TestCallMissingMethod) {
+  Proxy p(client_messenger_, server_addr_, server_addr_.host(),
+          CalculatorService::static_service_name());
+
+  Status s = DoTestSyncCall(p, "DoesNotExist");
+  ASSERT_TRUE(s.IsRemoteError()) << "Bad status: " << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "with an invalid method name: DoesNotExist");
+}
+
+TEST_F(RpcStubTest, TestApplicationError) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  RpcController controller;
+  SleepRequestPB req;
+  SleepResponsePB resp;
+  req.set_sleep_micros(1);
+  req.set_return_app_error(true);
+  Status s = p.Sleep(req, &resp, &controller);
+  ASSERT_TRUE(s.IsRemoteError());
+  EXPECT_EQ("Remote error: Got some error", s.ToString());
+  EXPECT_EQ("message: \"Got some error\"\n"
+            "[kudu.rpc_test.CalculatorError.app_error_ext] {\n"
+            "  extra_error_data: \"some application-specific error data\"\n"
+            "}\n",
+            SecureDebugString(*controller.error_response()));
+}
+
+TEST_F(RpcStubTest, TestRpcPanic) {
+  if (!FLAGS_is_panic_test_child) {
+    // This is a poor man's death test. We call this same
+    // test case, but set the above flag, and verify that
+    // it aborted. gtest death tests don't work here because
+    // there are already threads started up.
+    vector<string> argv;
+    string executable_path;
+    CHECK_OK(env_->GetExecutablePath(&executable_path));
+    argv.push_back(executable_path);
+    argv.emplace_back("--is_panic_test_child");
+    argv.emplace_back("--gtest_filter=RpcStubTest.TestRpcPanic");
+    Subprocess subp(argv);
+    subp.ShareParentStderr(false);
+    CHECK_OK(subp.Start());
+    FILE* in = fdopen(subp.from_child_stderr_fd(), "r");
+    PCHECK(in);
+
+    // Search for string "Test method panicking!" somewhere in stderr
+    char buf[1024];
+    bool found_string = false;
+    while (fgets(buf, sizeof(buf), in)) {
+      if (strstr(buf, "Test method panicking!")) {
+        found_string = true;
+        break;
+      }
+    }
+    CHECK(found_string);
+
+    // Check return status
+    int wait_status = 0;
+    CHECK_OK(subp.Wait(&wait_status));
+    CHECK(!WIFEXITED(wait_status)); // should not have been successful
+    if (WIFSIGNALED(wait_status)) {
+      CHECK_EQ(WTERMSIG(wait_status), SIGABRT);
+    } else {
+      // On some systems, we get exit status 134 from SIGABRT rather than
+      // WIFSIGNALED getting flagged.
+      CHECK_EQ(WEXITSTATUS(wait_status), 134);
+    }
+    return;
+  } else {
+    // Before forcing the panic, explicitly remove the test directory. This
+    // should be safe; this test doesn't generate any data.
+    CHECK_OK(env_->DeleteRecursively(test_dir_));
+
+    // Make an RPC which causes the server to abort.
+    CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+    RpcController controller;
+    PanicRequestPB req;
+    PanicResponsePB resp;
+    p.Panic(req, &resp, &controller);
+  }
+}
+
+struct AsyncSleep {
+  AsyncSleep() : latch(1) {}
+
+  RpcController rpc;
+  SleepRequestPB req;
+  SleepResponsePB resp;
+  CountDownLatch latch;
+};
+
+TEST_F(RpcStubTest, TestDontHandleTimedOutCalls) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+  vector<AsyncSleep*> sleeps;
+  ElementDeleter d(&sleeps);
+
+  // Send enough sleep calls to occupy the worker threads.
+  for (int i = 0; i < n_worker_threads_; i++) {
+    gscoped_ptr<AsyncSleep> sleep(new AsyncSleep);
+    sleep->rpc.set_timeout(MonoDelta::FromSeconds(1));
+    sleep->req.set_sleep_micros(1000*1000); // 1sec
+    p.SleepAsync(sleep->req, &sleep->resp, &sleep->rpc,
+                 boost::bind(&CountDownLatch::CountDown, &sleep->latch));
+    sleeps.push_back(sleep.release());
+  }
+
+  // We asynchronously sent the RPCs above, but the RPCs might still
+  // be in the queue. Because the RPC we send next has a lower timeout,
+  // it would take priority over the long-timeout RPCs. So, we have to
+  // wait until the above RPCs are being processed before we continue
+  // the test.
+  const Histogram* queue_time_metric = service_pool_->IncomingQueueTimeMetricForTests();
+  while (queue_time_metric->TotalCount() < n_worker_threads_) {
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+
+  // Send another call with a short timeout. This shouldn't get processed, because
+  // it'll get stuck in the queue for longer than its timeout.
+  ASSERT_EVENTUALLY([&]() {
+    RpcController rpc;
+    SleepRequestPB req;
+    SleepResponsePB resp;
+    req.set_sleep_micros(1); // unused but required.
+    rpc.set_timeout(MonoDelta::FromMilliseconds(5));
+    Status s = p.Sleep(req, &resp, &rpc);
+    ASSERT_TRUE(s.IsTimedOut()) << s.ToString();
+    // Since our timeout was short, it's possible in rare circumstances
+    // that we time out the RPC on the outbound queue, in which case
+    // we won't trigger the desired behavior here. In that case, the
+    // timeout error status would have the string 'ON_OUTBOUND_QUEUE'
+    // instead of 'SENT', so this assertion would fail and cause the
+    // ASSERT_EVENTUALLY to loop.
+    ASSERT_STR_CONTAINS(s.ToString(), "SENT");
+  });
+
+  for (AsyncSleep* s : sleeps) {
+    s->latch.Wait();
+  }
+
+  // Verify that the timedout call got short circuited before being processed.
+  // We may need to loop a short amount of time as we are racing with the reactor
+  // thread to process the remaining elements of the queue.
+  const Counter* timed_out_in_queue = service_pool_->RpcsTimedOutInQueueMetricForTests();
+  ASSERT_EVENTUALLY([&]{
+    ASSERT_EQ(1, timed_out_in_queue->value());
+  });
+}
+
+// Test which ensures that the RPC queue accepts requests with the earliest
+// deadline first (EDF), and upon overflow rejects requests with the latest deadlines.
+//
+// In particular, this simulates a workload experienced with Impala where the local
+// impalad would spawn more scanner threads than the total number of handlers plus queue
+// slots, guaranteeing that some of those clients would see SERVER_TOO_BUSY rejections on
+// scan requests and be forced to back off and retry.  Without EDF scheduling, we saw that
+// the "unlucky" threads that got rejected would likely continue to get rejected upon
+// retries, and some would be starved continually until they missed their overall deadline
+// and failed the query.
+//
+// With EDF scheduling, the retries take priority over the original requests (because
+// they retain their original deadlines). This prevents starvation of unlucky threads.
+TEST_F(RpcStubTest, TestEarliestDeadlineFirstQueue) {
+  const int num_client_threads = service_queue_length_ + n_worker_threads_ + 5;
+  vector<std::thread> threads;
+  vector<int> successes(num_client_threads);
+  std::atomic<bool> done(false);
+  for (int thread_id = 0; thread_id < num_client_threads; thread_id++) {
+    threads.emplace_back([&, thread_id] {
+        Random rng(thread_id);
+        CalculatorServiceProxy p(
+            client_messenger_, server_addr_, server_addr_.host());
+        while (!done.load()) {
+          // Set a deadline in the future. We'll keep using this same deadline
+          // on each of our retries.
+          MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(8);
+
+          for (int attempt = 1; !done.load(); attempt++) {
+            RpcController controller;
+            SleepRequestPB req;
+            SleepResponsePB resp;
+            controller.set_deadline(deadline);
+            req.set_sleep_micros(100000);
+            Status s = p.Sleep(req, &resp, &controller);
+            if (s.ok()) {
+              successes[thread_id]++;
+              break;
+            }
+            // We expect to get SERVER_TOO_BUSY errors because we have more clients than the
+            // server has handlers and queue slots. No other errors are expected.
+            CHECK(s.IsRemoteError() &&
+                  controller.error_response()->code() == rpc::ErrorStatusPB::ERROR_SERVER_TOO_BUSY)
+                << "Unexpected RPC failure: " << s.ToString();
+            // Randomized exponential backoff (similar to that done by the scanners in the Kudu
+            // client.).
+            int backoff = (0.5 + rng.NextDoubleFraction() * 0.5) * (std::min(1 << attempt, 1000));
+            VLOG(1) << "backoff " << backoff << "ms";
+            SleepFor(MonoDelta::FromMilliseconds(backoff));
+          }
+        }
+      });
+  }
+  // Let the threads run for 5 seconds before stopping them.
+  SleepFor(MonoDelta::FromSeconds(5));
+  done.store(true);
+  for (auto& t : threads) {
+    t.join();
+  }
+
+  // Before switching to earliest-deadline-first scheduling, the results
+  // here would typically look something like:
+  //  1 1 0 1 10 17 6 1 12 12 17 10 8 7 12 9 16 15
+  // With the fix, we see something like:
+  //  9 9 9 8 9 9 9 9 9 9 9 9 9 9 9 9 9
+  LOG(INFO) << "thread RPC success counts: " << successes;
+
+  int sum = 0;
+  int min = std::numeric_limits<int>::max();
+  for (int x : successes) {
+    sum += x;
+    min = std::min(min, x);
+  }
+  int avg = sum / successes.size();
+  ASSERT_GT(min, avg / 2)
+      << "expected the least lucky thread to have at least half as many successes "
+      << "as the average thread: min=" << min << " avg=" << avg;
+}
+
+TEST_F(RpcStubTest, TestDumpCallsInFlight) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+  AsyncSleep sleep;
+  sleep.req.set_sleep_micros(100 * 1000); // 100ms
+  p.SleepAsync(sleep.req, &sleep.resp, &sleep.rpc,
+               boost::bind(&CountDownLatch::CountDown, &sleep.latch));
+
+  // Check the running RPC status on the client messenger.
+  DumpRunningRpcsRequestPB dump_req;
+  DumpRunningRpcsResponsePB dump_resp;
+  dump_req.set_include_traces(true);
+
+  ASSERT_OK(client_messenger_->DumpRunningRpcs(dump_req, &dump_resp));
+  LOG(INFO) << "client messenger: " << SecureDebugString(dump_resp);
+  ASSERT_EQ(1, dump_resp.outbound_connections_size());
+  ASSERT_EQ(1, dump_resp.outbound_connections(0).calls_in_flight_size());
+  ASSERT_EQ("Sleep", dump_resp.outbound_connections(0).calls_in_flight(0).
+                        header().remote_method().method_name());
+  ASSERT_GT(dump_resp.outbound_connections(0).calls_in_flight(0).micros_elapsed(), 0);
+
+  // And the server messenger.
+  // We have to loop this until we find a result since the actual call is sent
+  // asynchronously off of the main thread (ie the server may not be handling it yet)
+  for (int i = 0; i < 100; i++) {
+    dump_resp.Clear();
+    ASSERT_OK(server_messenger_->DumpRunningRpcs(dump_req, &dump_resp));
+    if (dump_resp.inbound_connections_size() > 0 &&
+        dump_resp.inbound_connections(0).calls_in_flight_size() > 0) {
+      break;
+    }
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+
+  LOG(INFO) << "server messenger: " << SecureDebugString(dump_resp);
+  ASSERT_EQ(1, dump_resp.inbound_connections_size());
+  ASSERT_EQ(1, dump_resp.inbound_connections(0).calls_in_flight_size());
+  ASSERT_EQ("Sleep", dump_resp.inbound_connections(0).calls_in_flight(0).
+                        header().remote_method().method_name());
+  ASSERT_GT(dump_resp.inbound_connections(0).calls_in_flight(0).micros_elapsed(), 0);
+  ASSERT_STR_CONTAINS(dump_resp.inbound_connections(0).calls_in_flight(0).trace_buffer(),
+                      "Inserting onto call queue");
+  sleep.latch.Wait();
+}
+
+TEST_F(RpcStubTest, TestDumpSampledCalls) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  // Issue two calls that fall into different latency buckets.
+  AsyncSleep sleeps[2];
+  sleeps[0].req.set_sleep_micros(150 * 1000); // 150ms
+  sleeps[1].req.set_sleep_micros(1500 * 1000); // 1500ms
+
+  for (auto& sleep : sleeps) {
+    p.SleepAsync(sleep.req, &sleep.resp, &sleep.rpc,
+                 boost::bind(&CountDownLatch::CountDown, &sleep.latch));
+  }
+  for (auto& sleep : sleeps) {
+    sleep.latch.Wait();
+  }
+
+  // Dump the sampled RPCs and expect to see the calls
+  // above.
+
+  DumpRpczStoreResponsePB sampled_rpcs;
+  server_messenger_->rpcz_store()->DumpPB(DumpRpczStoreRequestPB(), &sampled_rpcs);
+  EXPECT_EQ(sampled_rpcs.methods_size(), 1);
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
+                      "    metrics {\n"
+                      "      key: \"test_sleep_us\"\n"
+                      "      value: 150000\n"
+                      "    }\n");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
+                      "    metrics {\n"
+                      "      key: \"test_sleep_us\"\n"
+                      "      value: 1500000\n"
+                      "    }\n");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs),
+                      "    metrics {\n"
+                      "      child_path: \"test_child\"\n"
+                      "      key: \"related_trace_metric\"\n"
+                      "      value: 1\n"
+                      "    }");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs), "SleepRequestPB");
+  ASSERT_STR_CONTAINS(SecureDebugString(sampled_rpcs), "duration_ms");
+}
+
+namespace {
+struct RefCountedTest : public RefCountedThreadSafe<RefCountedTest> {
+};
+
+// Test callback which takes a refcounted pointer.
+// We don't use this parameter, but it's used to validate that the bound callback
+// is cleared in TestCallbackClearedAfterRunning.
+void MyTestCallback(CountDownLatch* latch, scoped_refptr<RefCountedTest> my_refptr) {
+  latch->CountDown();
+}
+} // anonymous namespace
+
+// Verify that, after a call has returned, no copy of the call's callback
+// is held. This is important when the callback holds a refcounted ptr,
+// since we expect to be able to release that pointer when the call is done.
+TEST_F(RpcStubTest, TestCallbackClearedAfterRunning) {
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  CountDownLatch latch(1);
+  scoped_refptr<RefCountedTest> my_refptr(new RefCountedTest);
+  RpcController controller;
+  AddRequestPB req;
+  req.set_x(10);
+  req.set_y(20);
+  AddResponsePB resp;
+  p.AddAsync(req, &resp, &controller,
+             boost::bind(MyTestCallback, &latch, my_refptr));
+  latch.Wait();
+
+  // The ref count should go back down to 1. However, we need to loop a little
+  // bit, since the deref is happening on another thread. If the other thread gets
+  // descheduled directly after calling our callback, we'd fail without these sleeps.
+  for (int i = 0; i < 100 && !my_refptr->HasOneRef(); i++) {
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+  ASSERT_TRUE(my_refptr->HasOneRef());
+}
+
+// Regression test for KUDU-1409: if the client reactor thread is blocked (e.g due to a
+// process-wide pause or a slow callback) then we should not cause RPC calls to time out.
+TEST_F(RpcStubTest, DontTimeOutWhenReactorIsBlocked) {
+  CHECK_EQ(client_messenger_->num_reactors(), 1)
+      << "This test requires only a single reactor. Otherwise the injected sleep might "
+      << "be scheduled on a different reactor than the RPC call.";
+
+  CalculatorServiceProxy p(client_messenger_, server_addr_, server_addr_.host());
+
+  // Schedule a 1-second sleep on the reactor thread.
+  //
+  // This will cause us the reactor to be blocked while the call response is received, and
+  // still be blocked when the timeout would normally occur. Despite this, the call should
+  // not time out.
+  //
+  //  0s         0.5s          1.2s     1.5s
+  //  RPC call running
+  //  |---------------------|
+  //              Reactor blocked in sleep
+  //             |----------------------|
+  //                            \_ RPC would normally time out
+
+  client_messenger_->ScheduleOnReactor([](const Status& s) {
+      ThreadRestrictions::ScopedAllowWait allow_wait;
+      SleepFor(MonoDelta::FromSeconds(1));
+    }, MonoDelta::FromSeconds(0.5));
+
+  RpcController controller;
+  SleepRequestPB req;
+  SleepResponsePB resp;
+  req.set_sleep_micros(800 * 1000);
+  controller.set_timeout(MonoDelta::FromMilliseconds(1200));
+  ASSERT_OK(p.Sleep(req, &resp, &controller));
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpcz_store.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpcz_store.cc b/be/src/kudu/rpc/rpcz_store.cc
new file mode 100644
index 0000000..2f0e9c8
--- /dev/null
+++ b/be/src/kudu/rpc/rpcz_store.cc
@@ -0,0 +1,272 @@
+// 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 "kudu/rpc/rpcz_store.h"
+
+#include <algorithm>  // IWYU pragma: keep
+#include <array>
+#include <cstdint>
+#include <mutex> // for unique_lock
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <google/protobuf/message.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/util/atomic.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/trace_metrics.h"
+
+DEFINE_bool(rpc_dump_all_traces, false,
+            "If true, dump all RPC traces at INFO level");
+TAG_FLAG(rpc_dump_all_traces, advanced);
+TAG_FLAG(rpc_dump_all_traces, runtime);
+
+DEFINE_int32(rpc_duration_too_long_ms, 1000,
+             "Threshold (in milliseconds) above which a RPC is considered too long and its "
+             "duration and method name are logged at INFO level. The time measured is between "
+             "when a RPC is accepted and when its call handler completes.");
+TAG_FLAG(rpc_duration_too_long_ms, advanced);
+TAG_FLAG(rpc_duration_too_long_ms, runtime);
+
+using std::pair;
+using std::string;
+using std::vector;
+using std::unique_ptr;
+
+namespace kudu {
+namespace rpc {
+
+// Sample an RPC call once every N milliseconds within each
+// bucket. If the current sample in a latency bucket is older
+// than this threshold, a new sample will be taken.
+static const int kSampleIntervalMs = 1000;
+
+static const int kBucketThresholdsMs[] = {10, 100, 1000};
+static constexpr int kNumBuckets = arraysize(kBucketThresholdsMs) + 1;
+
+// An instance of this class is created For each RPC method implemented
+// on the server. It keeps several recent samples for each RPC, currently
+// based on fixed time buckets.
+class MethodSampler {
+ public:
+  MethodSampler() {}
+  ~MethodSampler() {}
+
+  // Potentially sample a single call.
+  void SampleCall(InboundCall* call);
+
+  // Dump the current samples.
+  void GetSamplePBs(RpczMethodPB* pb);
+
+ private:
+  // Convert the trace metrics from 't' into protobuf entries in 'sample_pb'.
+  // This function recurses through the parent-child relationship graph,
+  // keeping the current tree path in 'child_path' (empty at the root).
+  static void GetTraceMetrics(const Trace& t,
+                              const string& child_path,
+                              RpczSamplePB* sample_pb);
+
+  // An individual recorded sample.
+  struct Sample {
+    RequestHeader header;
+    scoped_refptr<Trace> trace;
+    int duration_ms;
+  };
+
+  // A sample, including the particular time at which it was
+  // sampled, and a lock protecting it.
+  struct SampleBucket {
+    SampleBucket() : last_sample_time(0) {}
+
+    AtomicInt<int64_t> last_sample_time;
+    simple_spinlock sample_lock;
+    Sample sample;
+  };
+  std::array<SampleBucket, kNumBuckets> buckets_;
+
+  DISALLOW_COPY_AND_ASSIGN(MethodSampler);
+};
+
+MethodSampler* RpczStore::SamplerForCall(InboundCall* call) {
+  if (PREDICT_FALSE(!call->method_info())) {
+    return nullptr;
+  }
+
+  // Most likely, we already have a sampler created for the call.
+  {
+    shared_lock<rw_spinlock> l(samplers_lock_.get_lock());
+    auto it = method_samplers_.find(call->method_info());
+    if (PREDICT_TRUE(it != method_samplers_.end())) {
+      return it->second.get();
+    }
+  }
+
+  // If missing, create a new sampler for this method and try to insert it.
+  unique_ptr<MethodSampler> ms(new MethodSampler());
+  std::lock_guard<percpu_rwlock> lock(samplers_lock_);
+  auto it = method_samplers_.find(call->method_info());
+  if (it != method_samplers_.end()) {
+    return it->second.get();
+  }
+  auto* ret = ms.get();
+  method_samplers_[call->method_info()] = std::move(ms);
+  return ret;
+}
+
+void MethodSampler::SampleCall(InboundCall* call) {
+  // First determine which sample bucket to put this in.
+  int duration_ms = call->timing().TotalDuration().ToMilliseconds();
+
+  SampleBucket* bucket = &buckets_[kNumBuckets - 1];
+  for (int i = 0 ; i < kNumBuckets - 1; i++) {
+    if (duration_ms < kBucketThresholdsMs[i]) {
+      bucket = &buckets_[i];
+      break;
+    }
+  }
+
+  MicrosecondsInt64 now = GetMonoTimeMicros();
+  int64_t us_since_trace = now - bucket->last_sample_time.Load();
+  if (us_since_trace > kSampleIntervalMs * 1000) {
+    Sample new_sample = {call->header(), call->trace(), duration_ms};
+    {
+      std::unique_lock<simple_spinlock> lock(bucket->sample_lock, std::try_to_lock);
+      // If another thread is already taking a sample, it's not worth waiting.
+      if (!lock.owns_lock()) {
+        return;
+      }
+      std::swap(bucket->sample, new_sample);
+      bucket->last_sample_time.Store(now);
+    }
+    VLOG(2) << "Sampled call " << call->ToString();
+  }
+}
+
+void MethodSampler::GetTraceMetrics(const Trace& t,
+                                    const string& child_path,
+                                    RpczSamplePB* sample_pb) {
+  auto m = t.metrics().Get();
+  for (const auto& e : m) {
+    auto* pb = sample_pb->add_metrics();
+    pb->set_key(e.first);
+    pb->set_value(e.second);
+    if (!child_path.empty()) {
+      pb->set_child_path(child_path);
+    }
+  }
+
+  for (const auto& child_pair : t.ChildTraces()) {
+    string path = child_path;
+    if (!path.empty()) {
+      path += ".";
+    }
+    path += child_pair.first.ToString();
+    GetTraceMetrics(*child_pair.second.get(), path, sample_pb);
+  }
+}
+
+void MethodSampler::GetSamplePBs(RpczMethodPB* method_pb) {
+  for (auto& bucket : buckets_) {
+    if (bucket.last_sample_time.Load() == 0) continue;
+
+    std::unique_lock<simple_spinlock> lock(bucket.sample_lock);
+    auto* sample_pb = method_pb->add_samples();
+    sample_pb->mutable_header()->CopyFrom(bucket.sample.header);
+    sample_pb->set_trace(bucket.sample.trace->DumpToString(Trace::INCLUDE_TIME_DELTAS));
+
+    GetTraceMetrics(*bucket.sample.trace.get(), "", sample_pb);
+    sample_pb->set_duration_ms(bucket.sample.duration_ms);
+  }
+}
+
+RpczStore::RpczStore() {}
+RpczStore::~RpczStore() {}
+
+void RpczStore::AddCall(InboundCall* call) {
+  LogTrace(call);
+  auto* sampler = SamplerForCall(call);
+  if (PREDICT_FALSE(!sampler)) return;
+
+  sampler->SampleCall(call);
+}
+
+void RpczStore::DumpPB(const DumpRpczStoreRequestPB& req,
+                       DumpRpczStoreResponsePB* resp) {
+  vector<pair<RpcMethodInfo*, MethodSampler*>> samplers;
+  {
+    shared_lock<rw_spinlock> l(samplers_lock_.get_lock());
+    for (const auto& p : method_samplers_) {
+      samplers.emplace_back(p.first, p.second.get());
+    }
+  }
+
+  for (const auto& p : samplers) {
+    auto* sampler = p.second;
+
+    RpczMethodPB* method_pb = resp->add_methods();
+    // TODO: use the actual RPC name instead of the request type name.
+    // Currently this isn't conveniently plumbed here, but the type name
+    // is close enough.
+    method_pb->set_method_name(p.first->req_prototype->GetTypeName());
+    sampler->GetSamplePBs(method_pb);
+  }
+}
+
+void RpczStore::LogTrace(InboundCall* call) {
+  int duration_ms = call->timing().TotalDuration().ToMilliseconds();
+
+  if (call->header_.has_timeout_millis() && call->header_.timeout_millis() > 0) {
+    double log_threshold = call->header_.timeout_millis() * 0.75f;
+    if (duration_ms > log_threshold) {
+      // TODO: consider pushing this onto another thread since it may be slow.
+      // The traces may also be too large to fit in a log message.
+      LOG(WARNING) << call->ToString() << " took " << duration_ms << "ms (client timeout "
+                   << call->header_.timeout_millis() << ").";
+      string s = call->trace()->DumpToString();
+      if (!s.empty()) {
+        LOG(WARNING) << "Trace:\n" << s;
+      }
+      return;
+    }
+  }
+
+  if (PREDICT_FALSE(FLAGS_rpc_dump_all_traces)) {
+    LOG(INFO) << call->ToString() << " took " << duration_ms << "ms. Trace:";
+    call->trace()->Dump(&LOG(INFO), true);
+  } else if (duration_ms > FLAGS_rpc_duration_too_long_ms) {
+    LOG(INFO) << call->ToString() << " took " << duration_ms << "ms. "
+              << "Request Metrics: " << call->trace()->MetricsAsJSON();
+  }
+}
+
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpcz_store.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpcz_store.h b/be/src/kudu/rpc/rpcz_store.h
new file mode 100644
index 0000000..48e4474
--- /dev/null
+++ b/be/src/kudu/rpc/rpcz_store.h
@@ -0,0 +1,74 @@
+// 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.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+
+#include <memory>
+#include <unordered_map>
+
+#include "kudu/util/locks.h"
+
+namespace kudu {
+namespace rpc {
+
+class DumpRpczStoreRequestPB;
+class DumpRpczStoreResponsePB;
+class InboundCall;
+class MethodSampler;
+struct RpcMethodInfo;
+
+// Responsible for storing sampled traces associated with completed calls.
+// Before each call is responded to, it is added to this store.
+class RpczStore {
+ public:
+  RpczStore();
+  ~RpczStore();
+
+  // Process a single call, potentially sampling it for later analysis.
+  //
+  // If the call is sampled, it might be mutated. For example, the request
+  // and response might be taken from the call and stored as part of the
+  // sample. This should be called just before a call response is sent
+  // to the client.
+  void AddCall(InboundCall* c);
+
+  // Dump all of the collected RPC samples in response to a user query.
+  void DumpPB(const DumpRpczStoreRequestPB& req,
+              DumpRpczStoreResponsePB* resp);
+
+ private:
+  // Look up or create the particular MethodSampler instance which should
+  // store samples for this call.
+  MethodSampler* SamplerForCall(InboundCall* call);
+
+  // Log a WARNING message if the RPC response was slow enough that the
+  // client likely timed out. This is based on the client-provided timeout
+  // value.
+  // Also can be configured to log _all_ RPC traces for help debugging.
+  void LogTrace(InboundCall* call);
+
+  percpu_rwlock samplers_lock_;
+
+  // Protected by samplers_lock_.
+  std::unordered_map<RpcMethodInfo*, std::unique_ptr<MethodSampler>> method_samplers_;
+
+  DISALLOW_COPY_AND_ASSIGN(RpczStore);
+};
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rtest.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rtest.proto b/be/src/kudu/rpc/rtest.proto
new file mode 100644
index 0000000..d212cef
--- /dev/null
+++ b/be/src/kudu/rpc/rtest.proto
@@ -0,0 +1,160 @@
+// 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.
+//
+// Test protocol for kudu RPC.
+syntax = "proto2";
+package kudu.rpc_test;
+
+import "kudu/rpc/rpc_header.proto";
+import "kudu/rpc/rtest_diff_package.proto";
+
+message AddRequestPB {
+  required uint32 x = 1;
+  required uint32 y = 2;
+}
+
+// Used by tests to simulate an old client which is missing
+// a newly added required field.
+message AddRequestPartialPB {
+  required uint32 x = 1;
+}
+
+message AddResponsePB {
+  required uint32 result = 1;
+}
+
+message SleepRequestPB {
+  required uint32 sleep_micros = 1;
+
+  // Used in rpc_stub-test: if this is true, it will respond from a different
+  // thread than the one that receives the request.
+  optional bool deferred = 2 [ default = false ];
+
+  // If set, returns a CalculatorError response.
+  optional bool return_app_error = 3 [ default = false ];
+
+  // Used in rpc-test: if this is set to true and no client timeout is set,
+  // the service will respond to the client with an error.
+  optional bool client_timeout_defined = 4 [ default = false ];
+}
+
+message SleepResponsePB {
+}
+
+message SleepWithSidecarRequestPB {
+  required uint32 sleep_micros = 1;
+  required uint32 pattern = 2;
+  required uint32 num_repetitions = 3;
+  required uint32 sidecar_idx = 4;
+}
+
+message SleepWithSidecarResponsePB {
+}
+
+message SendTwoStringsRequestPB {
+  required uint32 random_seed = 1;
+  required uint64 size1 = 2;
+  required uint64 size2 = 3;
+}
+
+message SendTwoStringsResponsePB {
+  required uint32 sidecar1 = 1;
+  required uint32 sidecar2 = 2;
+}
+
+// Push two strings to the server as part of the request, in sidecars.
+message PushTwoStringsRequestPB {
+  required uint32 sidecar1_idx = 1;
+  required uint32 sidecar2_idx = 2;
+}
+
+message PushTwoStringsResponsePB {
+  required uint32 size1 = 1;
+  required string data1 = 2;
+  required uint32 size2 = 3;
+  required string data2 = 4;
+}
+
+message EchoRequestPB {
+  required string data = 1;
+}
+message EchoResponsePB {
+  required string data = 1;
+}
+
+message WhoAmIRequestPB {
+}
+message WhoAmIResponsePB {
+  required kudu.rpc.UserInformationPB credentials = 1;
+  required string address = 2;
+}
+
+message CalculatorError {
+  extend kudu.rpc.ErrorStatusPB {
+    optional CalculatorError app_error_ext = 101;
+  }
+
+  required string extra_error_data = 1;
+}
+
+message PanicRequestPB {}
+message PanicResponsePB {}
+
+message TestInvalidResponseRequestPB {
+  enum ErrorType {
+    MISSING_REQUIRED_FIELD = 1;
+    RESPONSE_TOO_LARGE = 2;
+  }
+  required ErrorType error_type = 1;
+}
+
+message TestInvalidResponseResponsePB {
+  required bytes response = 1;
+}
+
+enum FeatureFlags {
+  UNKNOWN=0;
+  FOO=1;
+}
+
+message ExactlyOnceRequestPB {
+  optional uint32 sleep_for_ms = 1 [default = 0];
+  required uint32 value_to_add = 2;
+  optional bool randomly_fail = 3 [default = false];
+}
+message ExactlyOnceResponsePB {
+  required uint32 current_val = 1;
+  required fixed64 current_time_micros = 2;
+}
+
+service CalculatorService {
+  option (kudu.rpc.default_authz_method) = "AuthorizeDisallowAlice";
+
+  rpc Add(AddRequestPB) returns(AddResponsePB);
+  rpc Sleep(SleepRequestPB) returns(SleepResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeDisallowBob";
+  };
+  rpc Echo(EchoRequestPB) returns(EchoResponsePB);
+  rpc WhoAmI(WhoAmIRequestPB) returns (WhoAmIResponsePB);
+  rpc TestArgumentsInDiffPackage(kudu.rpc_test_diff_package.ReqDiffPackagePB)
+    returns(kudu.rpc_test_diff_package.RespDiffPackagePB);
+  rpc Panic(PanicRequestPB) returns (PanicResponsePB);
+  rpc AddExactlyOnce(ExactlyOnceRequestPB) returns (ExactlyOnceResponsePB) {
+    option (kudu.rpc.track_rpc_result) = true;
+  }
+  rpc TestInvalidResponse(TestInvalidResponseRequestPB) returns (TestInvalidResponseResponsePB);
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rtest_diff_package.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rtest_diff_package.proto b/be/src/kudu/rpc/rtest_diff_package.proto
new file mode 100644
index 0000000..f6f9b60
--- /dev/null
+++ b/be/src/kudu/rpc/rtest_diff_package.proto
@@ -0,0 +1,26 @@
+// 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.
+//
+// Request/Response in different package to test that RPC methods
+// handle arguments with packages different from the service itself.
+syntax = "proto2";
+package kudu.rpc_test_diff_package;
+
+message ReqDiffPackagePB {
+}
+message RespDiffPackagePB {
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/sasl_common.cc b/be/src/kudu/rpc/sasl_common.cc
new file mode 100644
index 0000000..645e854
--- /dev/null
+++ b/be/src/kudu/rpc/sasl_common.cc
@@ -0,0 +1,470 @@
+// 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 "kudu/rpc/sasl_common.h"
+
+#include <cstdio>
+#include <cstring>
+#include <limits>
+#include <mutex>
+#include <ostream>
+#include <string>
+
+#include <boost/algorithm/string/predicate.hpp>
+#include <glog/logging.h>
+#include <regex.h>
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/security/init.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/rw_mutex.h"
+
+using std::set;
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+const char* const kSaslMechPlain = "PLAIN";
+const char* const kSaslMechGSSAPI = "GSSAPI";
+extern const size_t kSaslMaxBufSize = 1024;
+
+// See WrapSaslCall().
+static __thread string* g_auth_failure_capture = nullptr;
+
+// Determine whether initialization was ever called
+static Status sasl_init_status = Status::OK();
+static bool sasl_is_initialized = false;
+
+// If true, then we expect someone else has initialized SASL.
+static bool g_disable_sasl_init = false;
+
+// If true, we expect kerberos to be enabled.
+static bool has_kerberos_keytab = false;
+
+// Output Sasl messages.
+// context: not used.
+// level: logging level.
+// message: message to output;
+static int SaslLogCallback(void* context, int level, const char* message) {
+
+  if (message == nullptr) return SASL_BADPARAM;
+
+  switch (level) {
+    case SASL_LOG_NONE:
+      break;
+
+    case SASL_LOG_ERR:
+      LOG(ERROR) << "SASL: " << message;
+      break;
+
+    case SASL_LOG_WARN:
+      LOG(WARNING) << "SASL: " << message;
+      break;
+
+    case SASL_LOG_NOTE:
+      LOG(INFO) << "SASL: " << message;
+      break;
+
+    case SASL_LOG_FAIL:
+      // Capture authentication failures in a thread-local to be picked up
+      // by WrapSaslCall() below.
+      VLOG(1) << "SASL: " << message;
+      if (g_auth_failure_capture) {
+        *g_auth_failure_capture = message;
+      }
+      break;
+
+    case SASL_LOG_DEBUG:
+      VLOG(1) << "SASL: " << message;
+      break;
+
+    case SASL_LOG_TRACE:
+    case SASL_LOG_PASS:
+      VLOG(3) << "SASL: " << message;
+      break;
+  }
+
+  return SASL_OK;
+}
+
+// Get Sasl option.
+// context: not used
+// plugin_name: name of plugin for which an option is being requested.
+// option: option requested
+// result: set to result which persists until next getopt in same thread,
+//         unchanged if option not found
+// len: length of the result
+// Return SASL_FAIL if the option is not handled, this does not fail the handshake.
+static int SaslGetOption(void* context, const char* plugin_name, const char* option,
+                         const char** result, unsigned* len) {
+  // Handle Sasl Library options
+  if (plugin_name == nullptr) {
+    // Return the logging level that we want the sasl library to use.
+    if (strcmp("log_level", option) == 0) {
+      int level = SASL_LOG_NOTE;
+      if (VLOG_IS_ON(1)) {
+        level = SASL_LOG_DEBUG;
+      } else if (VLOG_IS_ON(3)) {
+        level = SASL_LOG_TRACE;
+      }
+      // The library's contract for this method is that the caller gets to keep
+      // the returned buffer until the next call by the same thread, so we use a
+      // threadlocal for the buffer.
+      static __thread char buf[4];
+      snprintf(buf, arraysize(buf), "%d", level);
+      *result = buf;
+      if (len != nullptr) *len = strlen(buf);
+      return SASL_OK;
+    }
+    // Options can default so don't complain.
+    VLOG(4) << "SaslGetOption: Unknown library option: " << option;
+    return SASL_FAIL;
+  }
+  VLOG(4) << "SaslGetOption: Unknown plugin: " << plugin_name;
+  return SASL_FAIL;
+}
+
+// Array of callbacks for the sasl library.
+static sasl_callback_t callbacks[] = {
+  { SASL_CB_LOG, reinterpret_cast<int (*)()>(&SaslLogCallback), nullptr },
+  { SASL_CB_GETOPT, reinterpret_cast<int (*)()>(&SaslGetOption), nullptr },
+  { SASL_CB_LIST_END, nullptr, nullptr }
+  // TODO(todd): provide a CANON_USER callback? This is necessary if we want
+  // to support some kind of auth-to-local mapping of Kerberos principals
+  // to local usernames. See Impala's implementation for inspiration.
+};
+
+
+// SASL requires mutexes for thread safety, but doesn't implement
+// them itself. So, we have to hook them up to our mutex implementation.
+static void* SaslMutexAlloc() {
+  return static_cast<void*>(new Mutex());
+}
+static void SaslMutexFree(void* m) {
+  delete static_cast<Mutex*>(m);
+}
+static int SaslMutexLock(void* m) {
+  static_cast<Mutex*>(m)->lock();
+  return 0; // indicates success.
+}
+static int SaslMutexUnlock(void* m) {
+  static_cast<Mutex*>(m)->unlock();
+  return 0; // indicates success.
+}
+
+namespace internal {
+void SaslSetMutex() {
+  sasl_set_mutex(&SaslMutexAlloc, &SaslMutexLock, &SaslMutexUnlock, &SaslMutexFree);
+}
+} // namespace internal
+
+// Sasl initialization detection methods. The OS X SASL library doesn't define
+// the sasl_global_utils symbol, so we have to use less robust methods of
+// detection.
+#if defined(__APPLE__)
+static bool SaslIsInitialized() {
+  return sasl_global_listmech() != nullptr;
+}
+static bool SaslMutexImplementationProvided() {
+  return SaslIsInitialized();
+}
+#else
+
+// This symbol is exported by the SASL library but not defined
+// in the headers. It's marked as an API in the library source,
+// so seems safe to rely on.
+extern "C" sasl_utils_t* sasl_global_utils;
+static bool SaslIsInitialized() {
+  return sasl_global_utils != nullptr;
+}
+static bool SaslMutexImplementationProvided() {
+  if (!SaslIsInitialized()) return false;
+  void* m = sasl_global_utils->mutex_alloc();
+  sasl_global_utils->mutex_free(m);
+  // The default implementation of mutex_alloc just returns the constant pointer 0x1.
+  // This is a bit of an ugly heuristic, but seems unlikely that anyone would ever
+  // provide a valid implementation that returns an invalid pointer value.
+  return m != reinterpret_cast<void*>(1);
+}
+#endif
+
+// Actually perform the initialization for the SASL subsystem.
+// Meant to be called via GoogleOnceInit().
+static void DoSaslInit(bool kerberos_keytab_provided) {
+  VLOG(3) << "Initializing SASL library";
+
+  has_kerberos_keytab = kerberos_keytab_provided;
+
+  bool sasl_initialized = SaslIsInitialized();
+  if (sasl_initialized && !g_disable_sasl_init) {
+    LOG(WARNING) << "SASL was initialized prior to Kudu's initialization. Skipping "
+                 << "initialization. Call kudu::client::DisableSaslInitialization() "
+                 << "to suppress this message.";
+    g_disable_sasl_init = true;
+  }
+
+  if (g_disable_sasl_init) {
+    if (!sasl_initialized) {
+      sasl_init_status = Status::RuntimeError(
+          "SASL initialization was disabled, but SASL was not externally initialized.");
+      return;
+    }
+    if (!SaslMutexImplementationProvided()) {
+      LOG(WARNING)
+          << "SASL appears to be initialized by code outside of Kudu "
+          << "but was not provided with a mutex implementation! If "
+          << "manually initializing SASL, use sasl_set_mutex(3).";
+    }
+    return;
+  }
+  internal::SaslSetMutex();
+  int result = sasl_client_init(&callbacks[0]);
+  if (result != SASL_OK) {
+    sasl_init_status = Status::RuntimeError("Could not initialize SASL client",
+                                            sasl_errstring(result, nullptr, nullptr));
+    return;
+  }
+
+  result = sasl_server_init(&callbacks[0], kSaslAppName);
+  if (result != SASL_OK) {
+    sasl_init_status = Status::RuntimeError("Could not initialize SASL server",
+                                            sasl_errstring(result, nullptr, nullptr));
+    return;
+  }
+
+  sasl_is_initialized = true;
+}
+
+Status DisableSaslInitialization() {
+  if (g_disable_sasl_init) return Status::OK();
+  if (sasl_is_initialized) {
+    return Status::IllegalState("SASL already initialized. Initialization can only be disabled "
+                                "before first usage.");
+  }
+  g_disable_sasl_init = true;
+  return Status::OK();
+}
+
+Status SaslInit(bool kerberos_keytab_provided) {
+  // Only execute SASL initialization once
+  static std::once_flag once;
+  std::call_once(once, DoSaslInit, kerberos_keytab_provided);
+  DCHECK_EQ(kerberos_keytab_provided, has_kerberos_keytab);
+
+  return sasl_init_status;
+}
+
+static string CleanSaslError(const string& err) {
+  // In the case of GSS failures, we often get the actual error message
+  // buried inside a bunch of generic cruft. Use a regexp to extract it
+  // out. Note that we avoid std::regex because it appears to be broken
+  // with older libstdcxx.
+  static regex_t re;
+  static std::once_flag once;
+
+#if defined(__APPLE__)
+  static const char* kGssapiPattern = "GSSAPI Error:  Miscellaneous failure \\(see text \\((.+)\\)";
+#else
+  static const char* kGssapiPattern = "Unspecified GSS failure. +"
+                                      "Minor code may provide more information +"
+                                      "\\((.+)\\)";
+#endif
+
+  std::call_once(once, []{ CHECK_EQ(0, regcomp(&re, kGssapiPattern, REG_EXTENDED)); });
+  regmatch_t matches[2];
+  if (regexec(&re, err.c_str(), arraysize(matches), matches, 0) == 0) {
+    return err.substr(matches[1].rm_so, matches[1].rm_eo - matches[1].rm_so);
+  }
+  return err;
+}
+
+static string SaslErrDesc(int status, sasl_conn_t* conn) {
+  string err;
+  if (conn != nullptr) {
+    err = sasl_errdetail(conn);
+  } else {
+    err = sasl_errstring(status, nullptr, nullptr);
+  }
+
+  return CleanSaslError(err);
+}
+
+Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) {
+  // In many cases, the GSSAPI SASL plugin will generate a nice error
+  // message as a message logged at SASL_LOG_FAIL logging level, but then
+  // return a useless one in sasl_errstring(). So, we set a global thread-local
+  // variable to capture any auth failure log message while we make the
+  // call into the library.
+  //
+  // The thread-local thing is a bit of a hack, but the logging callback
+  // is set globally rather than on a per-connection basis.
+  string err;
+  g_auth_failure_capture = &err;
+
+  // Take the 'kerberos_reinit_lock' here to avoid a possible race with ticket renewal.
+  if (has_kerberos_keytab) kudu::security::KerberosReinitLock()->ReadLock();
+  int rc = call();
+  if (has_kerberos_keytab) kudu::security::KerberosReinitLock()->ReadUnlock();
+  g_auth_failure_capture = nullptr;
+
+  switch (rc) {
+    case SASL_OK:
+      return Status::OK();
+    case SASL_CONTINUE:
+      return Status::Incomplete("");
+    case SASL_FAIL:      // Generic failure (encompasses missing krb5 credentials).
+    case SASL_BADAUTH:   // Authentication failure.
+    case SASL_BADMAC:    // Decode failure.
+    case SASL_NOAUTHZ:   // Authorization failure.
+    case SASL_NOUSER:    // User not found.
+    case SASL_WRONGMECH: // Server doesn't support requested mechanism.
+    case SASL_BADSERV: { // Server failed mutual authentication.
+      if (err.empty()) {
+        err = SaslErrDesc(rc, conn);
+      } else {
+        err = CleanSaslError(err);
+      }
+      return Status::NotAuthorized(err);
+    }
+    default:
+      return Status::RuntimeError(SaslErrDesc(rc, conn));
+  }
+}
+
+bool NeedsWrap(sasl_conn_t* sasl_conn) {
+  const unsigned* ssf;
+  int rc = sasl_getprop(sasl_conn, SASL_SSF, reinterpret_cast<const void**>(&ssf));
+  CHECK_EQ(rc, SASL_OK) << "Failed to get SSF property on authenticated SASL connection";
+  return *ssf != 0;
+}
+
+uint32_t GetMaxSendBufferSize(sasl_conn_t* sasl_conn) {
+  const unsigned* max_buf_size;
+  int rc = sasl_getprop(sasl_conn, SASL_MAXOUTBUF, reinterpret_cast<const void**>(&max_buf_size));
+  CHECK_EQ(rc, SASL_OK)
+      << "Failed to get max output buffer property on authenticated SASL connection";
+  return *max_buf_size;
+}
+
+Status SaslEncode(sasl_conn_t* conn, Slice plaintext, Slice* ciphertext) {
+  const char* out;
+  unsigned out_len;
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(conn, [&] {
+      return sasl_encode(conn,
+                         reinterpret_cast<const char*>(plaintext.data()),
+                         plaintext.size(),
+                         &out, &out_len);
+  }), "SASL encode failed");
+  *ciphertext = Slice(out, out_len);
+  return Status::OK();
+}
+
+Status SaslDecode(sasl_conn_t* conn, Slice ciphertext, Slice* plaintext) {
+  const char* out;
+  unsigned out_len;
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(conn, [&] {
+    return sasl_decode(conn,
+                       reinterpret_cast<const char*>(ciphertext.data()),
+                       ciphertext.size(),
+                       &out, &out_len);
+  }), "SASL decode failed");
+  *plaintext = Slice(out, out_len);
+  return Status::OK();
+}
+
+string SaslIpPortString(const Sockaddr& addr) {
+  string addr_str = addr.ToString();
+  size_t colon_pos = addr_str.find(':');
+  if (colon_pos != string::npos) {
+    addr_str[colon_pos] = ';';
+  }
+  return addr_str;
+}
+
+set<SaslMechanism::Type> SaslListAvailableMechs() {
+  set<SaslMechanism::Type> mechs;
+
+  // Array of NULL-terminated strings. Array terminated with NULL.
+  for (const char** mech_strings = sasl_global_listmech();
+       mech_strings != nullptr && *mech_strings != nullptr;
+       mech_strings++) {
+    auto mech = SaslMechanism::value_of(*mech_strings);
+    if (mech != SaslMechanism::INVALID) {
+      mechs.insert(mech);
+    }
+  }
+  return mechs;
+}
+
+sasl_callback_t SaslBuildCallback(int id, int (*proc)(void), void* context) {
+  sasl_callback_t callback;
+  callback.id = id;
+  callback.proc = proc;
+  callback.context = context;
+  return callback;
+}
+
+Status EnableProtection(sasl_conn_t* sasl_conn,
+                        SaslProtection::Type minimum_protection,
+                        size_t max_recv_buf_size) {
+  sasl_security_properties_t sec_props;
+  memset(&sec_props, 0, sizeof(sec_props));
+  sec_props.min_ssf = minimum_protection;
+  sec_props.max_ssf = std::numeric_limits<sasl_ssf_t>::max();
+  sec_props.maxbufsize = max_recv_buf_size;
+
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(sasl_conn, [&] {
+    return sasl_setprop(sasl_conn, SASL_SEC_PROPS, &sec_props);
+  }), "failed to set SASL security properties");
+  return Status::OK();
+}
+
+SaslMechanism::Type SaslMechanism::value_of(const string& mech) {
+  if (boost::iequals(mech, "PLAIN")) {
+    return PLAIN;
+  }
+  if (boost::iequals(mech, "GSSAPI")) {
+    return GSSAPI;
+  }
+  return INVALID;
+}
+
+const char* SaslMechanism::name_of(SaslMechanism::Type val) {
+  switch (val) {
+    case PLAIN: return "PLAIN";
+    case GSSAPI: return "GSSAPI";
+    default:
+      return "INVALID";
+  }
+}
+
+const char* SaslProtection::name_of(SaslProtection::Type val) {
+  switch (val) {
+    case SaslProtection::kAuthentication: return "authentication";
+    case SaslProtection::kIntegrity: return "integrity";
+    case SaslProtection::kPrivacy: return "privacy";
+  }
+  LOG(FATAL) << "unknown SASL protection type: " << val;
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/sasl_common.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/sasl_common.h b/be/src/kudu/rpc/sasl_common.h
new file mode 100644
index 0000000..2454cfd
--- /dev/null
+++ b/be/src/kudu/rpc/sasl_common.h
@@ -0,0 +1,158 @@
+// 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.
+
+#ifndef KUDU_RPC_SASL_COMMON_H
+#define KUDU_RPC_SASL_COMMON_H
+
+#include <cstddef>
+#include <cstdint>
+#include <functional>
+#include <set>
+#include <string>
+
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Sockaddr;
+
+namespace rpc {
+
+// Constants
+extern const char* const kSaslMechPlain;
+extern const char* const kSaslMechGSSAPI;
+extern const size_t kSaslMaxBufSize;
+
+struct SaslMechanism {
+  enum Type {
+    INVALID,
+    PLAIN,
+    GSSAPI
+  };
+  static Type value_of(const std::string& mech);
+  static const char* name_of(Type val);
+};
+
+struct SaslProtection {
+  enum Type {
+    // SASL authentication without integrity or privacy.
+    kAuthentication = 0,
+    // Integrity protection, i.e. messages are HMAC'd.
+    kIntegrity = 1,
+    // Privacy protection, i.e. messages are encrypted.
+    kPrivacy = 2,
+  };
+  static const char* name_of(Type val);
+};
+
+// Initialize the SASL library.
+// appname: Name of the application for logging messages & sasl plugin configuration.
+//          Note that this string must remain allocated for the lifetime of the program.
+// This function must be called before using SASL.
+// If the library initializes without error, calling more than once has no effect.
+//
+// Some SASL plugins take time to initialize random number generators and other things,
+// so the first time this function is invoked it may execute for several seconds.
+// After that, it should be very fast. This function should be invoked as early as possible
+// in the application lifetime to avoid SASL initialization taking place in a
+// performance-critical section.
+//
+// This function is thread safe and uses a static lock.
+// This function should NOT be called during static initialization.
+Status SaslInit(bool kerberos_keytab_provided = false) WARN_UNUSED_RESULT;
+
+// Disable Kudu's initialization of SASL. See equivalent method in client.h.
+Status DisableSaslInitialization() WARN_UNUSED_RESULT;
+
+// Wrap a call into the SASL library. 'call' should be a lambda which
+// returns a SASL error code.
+//
+// The result is translated into a Status as follows:
+//
+//  SASL_OK:       Status::OK()
+//  SASL_CONTINUE: Status::Incomplete()
+//  otherwise:     Status::NotAuthorized()
+//
+// The Status message is beautified to be more user-friendly compared
+// to the underlying sasl_errdetails() call.
+Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) WARN_UNUSED_RESULT;
+
+// Return <ip>;<port> string formatted for SASL library use.
+std::string SaslIpPortString(const Sockaddr& addr);
+
+// Return available plugin mechanisms for the given connection.
+std::set<SaslMechanism::Type> SaslListAvailableMechs();
+
+// Initialize and return a libsasl2 callback data structure based on the passed args.
+// id: A SASL callback identifier (e.g., SASL_CB_GETOPT).
+// proc: A C-style callback with appropriate signature based on the callback id, or NULL.
+// context: An object to pass to the callback as the context pointer, or NULL.
+sasl_callback_t SaslBuildCallback(int id, int (*proc)(void), void* context);
+
+// Enable SASL integrity and privacy protection on the connection. Also allows
+// setting the minimum required protection level, and the maximum receive buffer
+// size.
+Status EnableProtection(sasl_conn_t* sasl_conn,
+                        SaslProtection::Type minimum_protection = SaslProtection::kAuthentication,
+                        size_t max_recv_buf_size = kSaslMaxBufSize) WARN_UNUSED_RESULT;
+
+// Returns true if the SASL connection has been negotiated with auth-int or
+// auth-conf. 'sasl_conn' must already be negotiated.
+bool NeedsWrap(sasl_conn_t* sasl_conn);
+
+// Retrieves the negotiated maximum send buffer size for auth-int or auth-conf
+// protected channels.
+uint32_t GetMaxSendBufferSize(sasl_conn_t* sasl_conn) WARN_UNUSED_RESULT;
+
+// Encode the provided data.
+//
+// The plaintext data must not be longer than the negotiated maximum buffer size.
+//
+// The output 'ciphertext' slice is only valid until the next use of the SASL connection.
+Status SaslEncode(sasl_conn_t* conn,
+                  Slice plaintext,
+                  Slice* ciphertext) WARN_UNUSED_RESULT;
+
+// Decode the provided SASL-encoded data.
+//
+// The decoded plaintext must not be longer than the negotiated maximum buffer size.
+//
+// The output 'plaintext' slice is only valid until the next use of the SASL connection.
+Status SaslDecode(sasl_conn_t* conn,
+                  Slice ciphertext,
+                  Slice* plaintext) WARN_UNUSED_RESULT;
+
+// Deleter for sasl_conn_t instances, for use with gscoped_ptr after calling sasl_*_new()
+struct SaslDeleter {
+  inline void operator()(sasl_conn_t* conn) {
+    sasl_dispose(&conn);
+  }
+};
+
+// Internals exposed in the header for test purposes.
+namespace internal {
+void SaslSetMutex();
+} // namespace internal
+
+} // namespace rpc
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/sasl_helper.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/sasl_helper.cc b/be/src/kudu/rpc/sasl_helper.cc
new file mode 100644
index 0000000..765118e
--- /dev/null
+++ b/be/src/kudu/rpc/sasl_helper.cc
@@ -0,0 +1,134 @@
+// 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 "kudu/rpc/sasl_helper.h"
+
+#include <cstring>
+#include <ostream>
+#include <set>
+#include <string>
+
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+SaslHelper::SaslHelper(PeerType peer_type)
+  : peer_type_(peer_type),
+    global_mechs_(SaslListAvailableMechs()),
+    plain_enabled_(false),
+    gssapi_enabled_(false) {
+  tag_ = (peer_type_ == SERVER) ? "Server" : "Client";
+}
+
+void SaslHelper::set_server_fqdn(const string& domain_name) {
+  server_fqdn_ = domain_name;
+}
+const char* SaslHelper::server_fqdn() const {
+  return server_fqdn_.empty() ? nullptr : server_fqdn_.c_str();
+}
+
+const char* SaslHelper::EnabledMechsString() const {
+  enabled_mechs_string_ = JoinMapped(enabled_mechs_, SaslMechanism::name_of, " ");
+  return enabled_mechs_string_.c_str();
+}
+
+int SaslHelper::GetOptionCb(const char* plugin_name, const char* option,
+                            const char** result, unsigned* len) {
+  DVLOG(4) << tag_ << ": GetOption Callback called. ";
+  DVLOG(4) << tag_ << ": GetOption Plugin name: "
+                   << (plugin_name == nullptr ? "NULL" : plugin_name);
+  DVLOG(4) << tag_ << ": GetOption Option name: " << option;
+
+  if (PREDICT_FALSE(result == nullptr)) {
+    LOG(DFATAL) << tag_ << ": SASL Library passed NULL result out-param to GetOption callback!";
+    return SASL_BADPARAM;
+  }
+
+  if (plugin_name == nullptr) {
+    // SASL library option, not a plugin option
+    if (strcmp(option, "mech_list") == 0) {
+      *result = EnabledMechsString();
+      if (len != nullptr) *len = strlen(*result);
+      VLOG(4) << tag_ << ": Enabled mech list: " << *result;
+      return SASL_OK;
+    }
+    VLOG(4) << tag_ << ": GetOptionCb: Unknown library option: " << option;
+  } else {
+    VLOG(4) << tag_ << ": GetOptionCb: Unknown plugin: " << plugin_name;
+  }
+  return SASL_FAIL;
+}
+
+Status SaslHelper::EnablePlain() {
+  RETURN_NOT_OK(EnableMechanism(SaslMechanism::PLAIN));
+  plain_enabled_ = true;
+  return Status::OK();
+}
+
+Status SaslHelper::EnableGSSAPI() {
+  RETURN_NOT_OK(EnableMechanism(SaslMechanism::GSSAPI));
+  gssapi_enabled_ = true;
+  return Status::OK();
+}
+
+Status SaslHelper::EnableMechanism(SaslMechanism::Type mech) {
+  if (PREDICT_FALSE(!ContainsKey(global_mechs_, mech))) {
+    return Status::InvalidArgument("unable to find SASL plugin", SaslMechanism::name_of(mech));
+  }
+  enabled_mechs_.insert(mech);
+  return Status::OK();
+}
+
+bool SaslHelper::IsPlainEnabled() const {
+  return plain_enabled_;
+}
+
+Status SaslHelper::CheckNegotiateCallId(int32_t call_id) const {
+  if (call_id != kNegotiateCallId) {
+    Status s = Status::IllegalState(strings::Substitute(
+        "Received illegal call-id during negotiation; expected: $0, received: $1",
+        kNegotiateCallId, call_id));
+    LOG(DFATAL) << tag_ << ": " << s.ToString();
+    return s;
+  }
+  return Status::OK();
+}
+
+Status SaslHelper::ParseNegotiatePB(const Slice& param_buf, NegotiatePB* msg) {
+  if (!msg->ParseFromArray(param_buf.data(), param_buf.size())) {
+    return Status::IOError(tag_ + ": Invalid SASL message, missing fields",
+        msg->InitializationErrorString());
+  }
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/sasl_helper.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/sasl_helper.h b/be/src/kudu/rpc/sasl_helper.h
new file mode 100644
index 0000000..aa0c8bf
--- /dev/null
+++ b/be/src/kudu/rpc/sasl_helper.h
@@ -0,0 +1,109 @@
+// 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.
+
+#ifndef KUDU_RPC_SASL_HELPER_H
+#define KUDU_RPC_SASL_HELPER_H
+
+#include <cstdint>
+#include <set>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/sasl_common.h"
+
+namespace kudu {
+
+class Slice;
+class Status;
+
+namespace rpc {
+
+class NegotiatePB;
+
+// Helper class which contains functionality that is common to client and server
+// SASL negotiations. Most of these methods are convenience methods for
+// interacting with the libsasl2 library.
+class SaslHelper {
+ public:
+  enum PeerType {
+    CLIENT,
+    SERVER
+  };
+
+  explicit SaslHelper(PeerType peer_type);
+  ~SaslHelper() = default;
+
+  // Specify the fully-qualified domain name of the remote server.
+  void set_server_fqdn(const std::string& domain_name);
+  const char* server_fqdn() const;
+
+  // Globally-registered available SASL plugins.
+  const std::set<SaslMechanism::Type>& GlobalMechs() const {
+    return global_mechs_;
+  }
+
+  // Helper functions for managing the list of active SASL mechanisms.
+  const std::set<SaslMechanism::Type>& EnabledMechs() const {
+    return enabled_mechs_;
+  }
+
+  // Implements the client_mech_list / mech_list callbacks.
+  int GetOptionCb(const char* plugin_name, const char* option, const char** result, unsigned* len);
+
+  // Enable the PLAIN SASL mechanism.
+  Status EnablePlain();
+
+  // Enable the GSSAPI (Kerberos) mechanism.
+  Status EnableGSSAPI();
+
+  // Check for the PLAIN SASL mechanism.
+  bool IsPlainEnabled() const;
+
+  // Sanity check that the call ID is the negotiation call ID.
+  // Logs DFATAL if call_id does not match.
+  Status CheckNegotiateCallId(int32_t call_id) const;
+
+  // Parse msg from the given Slice.
+  Status ParseNegotiatePB(const Slice& param_buf, NegotiatePB* msg);
+
+ private:
+  Status EnableMechanism(SaslMechanism::Type mech);
+
+  // Returns space-delimited local mechanism list string suitable for passing
+  // to libsasl2, such as via "mech_list" callbacks.
+  // The returned pointer is valid only until the next call to EnabledMechsString().
+  const char* EnabledMechsString() const;
+
+  std::string server_fqdn_;
+
+  // Authentication types and data.
+  const PeerType peer_type_;
+  std::string tag_;
+  std::set<SaslMechanism::Type> global_mechs_;       // Cache of global mechanisms.
+  std::set<SaslMechanism::Type> enabled_mechs_;      // Active mechanisms.
+  mutable std::string enabled_mechs_string_; // Mechanism list string returned by callbacks.
+
+  bool plain_enabled_;
+  bool gssapi_enabled_;
+
+  DISALLOW_COPY_AND_ASSIGN(SaslHelper);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif  // KUDU_RPC_SASL_HELPER_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/serialization.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/serialization.cc b/be/src/kudu/rpc/serialization.cc
new file mode 100644
index 0000000..473a817
--- /dev/null
+++ b/be/src/kudu/rpc/serialization.cc
@@ -0,0 +1,223 @@
+// 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 "kudu/rpc/serialization.h"
+
+#include <limits>
+#include <ostream>
+#include <string>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <google/protobuf/message_lite.h>
+#include <google/protobuf/io/coded_stream.h>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+DECLARE_int64(rpc_max_message_size);
+
+using google::protobuf::MessageLite;
+using google::protobuf::io::CodedInputStream;
+using google::protobuf::io::CodedOutputStream;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+namespace serialization {
+
+enum {
+  kHeaderPosVersion = 0,
+  kHeaderPosServiceClass = 1,
+  kHeaderPosAuthProto = 2
+};
+
+void SerializeMessage(const MessageLite& message, faststring* param_buf,
+                        int additional_size, bool use_cached_size) {
+  DCHECK_GE(additional_size, 0);
+  int pb_size = use_cached_size ? message.GetCachedSize() : message.ByteSize();
+  DCHECK_EQ(message.ByteSize(), pb_size);
+  // Use 8-byte integers to avoid overflowing when additional_size approaches INT_MAX.
+  int64_t recorded_size = static_cast<int64_t>(pb_size) +
+      static_cast<int64_t>(additional_size);
+  int64_t size_with_delim = static_cast<int64_t>(pb_size) +
+      static_cast<int64_t>(CodedOutputStream::VarintSize32(recorded_size));
+  int64_t total_size = size_with_delim + static_cast<int64_t>(additional_size);
+  // The message format relies on an unsigned 32-bit integer to express the size, so
+  // the message must not exceed this size. Since additional_size is limited to INT_MAX,
+  // this is a safe limitation.
+  CHECK_LE(total_size, std::numeric_limits<uint32_t>::max());
+
+  if (total_size > FLAGS_rpc_max_message_size) {
+    LOG(WARNING) << Substitute("Serialized $0 ($1 bytes) is larger than the maximum configured "
+                               "RPC message size ($2 bytes). "
+                               "Sending anyway, but peer may reject the data.",
+                               message.GetTypeName(), total_size, FLAGS_rpc_max_message_size);
+  }
+
+  param_buf->resize(size_with_delim);
+  uint8_t* dst = param_buf->data();
+  dst = CodedOutputStream::WriteVarint32ToArray(recorded_size, dst);
+  dst = message.SerializeWithCachedSizesToArray(dst);
+  CHECK_EQ(dst, param_buf->data() + size_with_delim);
+}
+
+void SerializeHeader(const MessageLite& header,
+                     size_t param_len,
+                     faststring* header_buf) {
+
+  CHECK(header.IsInitialized())
+      << "RPC header missing fields: " << header.InitializationErrorString();
+
+  // Compute all the lengths for the packet.
+  size_t header_pb_len = header.ByteSize();
+  size_t header_tot_len = kMsgLengthPrefixLength        // Int prefix for the total length.
+      + CodedOutputStream::VarintSize32(header_pb_len)  // Varint delimiter for header PB.
+      + header_pb_len;                                  // Length for the header PB itself.
+  size_t total_size = header_tot_len + param_len;
+
+  header_buf->resize(header_tot_len);
+  uint8_t* dst = header_buf->data();
+
+  // 1. The length for the whole request, not including the 4-byte
+  // length prefix.
+  NetworkByteOrder::Store32(dst, total_size - kMsgLengthPrefixLength);
+  dst += sizeof(uint32_t);
+
+  // 2. The varint-prefixed RequestHeader PB
+  dst = CodedOutputStream::WriteVarint32ToArray(header_pb_len, dst);
+  dst = header.SerializeWithCachedSizesToArray(dst);
+
+  // We should have used the whole buffer we allocated.
+  CHECK_EQ(dst, header_buf->data() + header_tot_len);
+}
+
+Status ParseMessage(const Slice& buf,
+                    MessageLite* parsed_header,
+                    Slice* parsed_main_message) {
+
+  // First grab the total length
+  if (PREDICT_FALSE(buf.size() < kMsgLengthPrefixLength)) {
+    return Status::Corruption("Invalid packet: not enough bytes for length header",
+                              KUDU_REDACT(buf.ToDebugString()));
+  }
+
+  uint32_t total_len = NetworkByteOrder::Load32(buf.data());
+  DCHECK_EQ(total_len, buf.size() - kMsgLengthPrefixLength)
+    << "Got mis-sized buffer: " << KUDU_REDACT(buf.ToDebugString());
+
+  if (total_len > std::numeric_limits<int32_t>::max()) {
+    return Status::Corruption(Substitute("Invalid packet: message had a length of $0, "
+        "but we only support messages up to $1 bytes\n",
+        total_len, std::numeric_limits<int32_t>::max()));
+  }
+
+  CodedInputStream in(buf.data(), buf.size());
+  // Protobuf enforces a 64MB total bytes limit on CodedInputStream by default.
+  // Override this default with the actual size of the buffer to allow messages
+  // larger than 64MB.
+  in.SetTotalBytesLimit(buf.size(), -1);
+  in.Skip(kMsgLengthPrefixLength);
+
+  uint32_t header_len;
+  if (PREDICT_FALSE(!in.ReadVarint32(&header_len))) {
+    return Status::Corruption("Invalid packet: missing header delimiter",
+                              KUDU_REDACT(buf.ToDebugString()));
+  }
+
+  CodedInputStream::Limit l;
+  l = in.PushLimit(header_len);
+  if (PREDICT_FALSE(!parsed_header->ParseFromCodedStream(&in))) {
+    return Status::Corruption("Invalid packet: header too short",
+                              KUDU_REDACT(buf.ToDebugString()));
+  }
+  in.PopLimit(l);
+
+  uint32_t main_msg_len;
+  if (PREDICT_FALSE(!in.ReadVarint32(&main_msg_len))) {
+    return Status::Corruption("Invalid packet: missing main msg length",
+                              KUDU_REDACT(buf.ToDebugString()));
+  }
+
+  if (PREDICT_FALSE(!in.Skip(main_msg_len))) {
+    return Status::Corruption(
+        StringPrintf("Invalid packet: data too short, expected %d byte main_msg", main_msg_len),
+        KUDU_REDACT(buf.ToDebugString()));
+  }
+
+  if (PREDICT_FALSE(in.BytesUntilLimit() > 0)) {
+    return Status::Corruption(
+      StringPrintf("Invalid packet: %d extra bytes at end of packet", in.BytesUntilLimit()),
+      KUDU_REDACT(buf.ToDebugString()));
+  }
+
+  *parsed_main_message = Slice(buf.data() + buf.size() - main_msg_len,
+                              main_msg_len);
+  return Status::OK();
+}
+
+void SerializeConnHeader(uint8_t* buf) {
+  memcpy(reinterpret_cast<char *>(buf), kMagicNumber, kMagicNumberLength);
+  buf += kMagicNumberLength;
+  buf[kHeaderPosVersion] = kCurrentRpcVersion;
+  buf[kHeaderPosServiceClass] = 0; // TODO: implement
+  buf[kHeaderPosAuthProto] = 0; // TODO: implement
+}
+
+// validate the entire rpc header (magic number + flags)
+Status ValidateConnHeader(const Slice& slice) {
+  DCHECK_EQ(kMagicNumberLength + kHeaderFlagsLength, slice.size())
+    << "Invalid RPC header length";
+
+  // validate actual magic
+  if (!slice.starts_with(kMagicNumber)) {
+    if (slice.starts_with("GET ") ||
+        slice.starts_with("POST") ||
+        slice.starts_with("HEAD")) {
+      return Status::InvalidArgument("invalid negotation, appears to be an HTTP client on "
+                                     "the RPC port");
+    }
+    return Status::InvalidArgument("connection must begin with magic number", kMagicNumber);
+  }
+
+  const uint8_t *data = slice.data();
+  data += kMagicNumberLength;
+
+  // validate version
+  if (data[kHeaderPosVersion] != kCurrentRpcVersion) {
+    return Status::InvalidArgument("Unsupported RPC version",
+        StringPrintf("Received: %d, Supported: %d",
+            data[kHeaderPosVersion], kCurrentRpcVersion));
+  }
+
+  // TODO: validate additional header flags:
+  // RPC_SERVICE_CLASS
+  // RPC_AUTH_PROTOCOL
+
+  return Status::OK();
+}
+
+} // namespace serialization
+} // namespace rpc
+} // namespace kudu


[38/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc.h b/be/src/kudu/rpc/rpc.h
new file mode 100644
index 0000000..bd195dc
--- /dev/null
+++ b/be/src/kudu/rpc/rpc.h
@@ -0,0 +1,221 @@
+// 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.
+#ifndef KUDU_RPC_RPC_H
+#define KUDU_RPC_RPC_H
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+namespace rpc {
+
+class Messenger;
+class Rpc;
+
+// Result status of a retriable Rpc.
+//
+// TODO Consider merging this with ScanRpcStatus.
+struct RetriableRpcStatus {
+  enum Result {
+    // There was no error, i.e. the Rpc was successful.
+    OK,
+
+    // The Rpc got an error and it's not retriable.
+    NON_RETRIABLE_ERROR,
+
+    // The server couldn't be reached, i.e. there was a network error while
+    // reaching the replica or a DNS resolution problem.
+    SERVER_NOT_ACCESSIBLE,
+
+    // The server received the request but it was not ready to serve it right
+    // away. It might happen that the server was too busy and did not have
+    // necessary resources or information to serve the request but it
+    // anticipates it should be ready to serve the request really soon, so it's
+    // worth retrying the request at a later time.
+    SERVICE_UNAVAILABLE,
+
+    // For rpc's that are meant only for the leader of a shared resource, when the server
+    // we're interacting with is not the leader.
+    REPLICA_NOT_LEADER,
+
+    // The server doesn't know the resource we're interacting with. For instance a TabletServer
+    // is not part of the config for the tablet we're trying to write to.
+    RESOURCE_NOT_FOUND,
+
+    // The authentication token supplied with the operation was found invalid
+    // by the server. Most likely, the token has expired. If so, get a new token
+    // using client credentials and retry the operation with it.
+    INVALID_AUTHENTICATION_TOKEN,
+  };
+
+  Result result;
+  Status status;
+};
+
+// This class picks a server among a possible set of servers serving a given resource.
+//
+// TODO Currently this only picks the leader, though it wouldn't be unfeasible to have this
+// have an enum so that it can pick any server.
+template <class Server>
+class ServerPicker : public RefCountedThreadSafe<ServerPicker<Server>> {
+ public:
+  virtual ~ServerPicker() {}
+
+  typedef Callback<void(const Status& status, Server* server)> ServerPickedCallback;
+
+  // Picks the leader among the replicas serving a resource.
+  // If the leader was found, it calls the callback with Status::OK() and
+  // with 'server' set to the current leader, otherwise calls the callback
+  // with 'status' set to the failure reason, and 'server' set to nullptr.
+  // If picking a leader takes longer than 'deadline' the callback is called with
+  // Status::TimedOut().
+  virtual void PickLeader(const ServerPickedCallback& callback, const MonoTime& deadline) = 0;
+
+  // Marks a server as failed/unacessible.
+  virtual void MarkServerFailed(Server *server, const Status &status) = 0;
+
+  // Marks a server as not the leader of config serving the resource we're trying to interact with.
+  virtual void MarkReplicaNotLeader(Server* replica) = 0;
+
+  // Marks a server as not serving the resource we want.
+  virtual void MarkResourceNotFound(Server *replica) = 0;
+};
+
+// Provides utilities for retrying failed RPCs.
+//
+// All RPCs should use HandleResponse() to retry certain generic errors.
+class RpcRetrier {
+ public:
+  RpcRetrier(MonoTime deadline, std::shared_ptr<rpc::Messenger> messenger)
+      : attempt_num_(1),
+        deadline_(deadline),
+        messenger_(std::move(messenger)) {
+    if (deadline_.Initialized()) {
+      controller_.set_deadline(deadline_);
+    }
+    controller_.Reset();
+  }
+
+  // Tries to handle a failed RPC.
+  //
+  // If it was handled (e.g. scheduled for retry in the future), returns
+  // true. In this case, callers should ensure that 'rpc' remains alive.
+  //
+  // Otherwise, returns false and writes the controller status to
+  // 'out_status'.
+  bool HandleResponse(Rpc* rpc, Status* out_status);
+
+  // Retries an RPC at some point in the near future. If 'why_status' is not OK,
+  // records it as the most recent error causing the RPC to retry. This is
+  // reported to the caller eventually if the RPC never succeeds.
+  //
+  // If the RPC's deadline expires, the callback will fire with a timeout
+  // error when the RPC comes up for retrying. This is true even if the
+  // deadline has already expired at the time that Retry() was called.
+  //
+  // Callers should ensure that 'rpc' remains alive.
+  void DelayedRetry(Rpc* rpc, const Status& why_status);
+
+  RpcController* mutable_controller() { return &controller_; }
+  const RpcController& controller() const { return controller_; }
+
+  const MonoTime& deadline() const { return deadline_; }
+
+  const std::shared_ptr<Messenger>& messenger() const {
+    return messenger_;
+  }
+
+  int attempt_num() const { return attempt_num_; }
+
+  // Called when an RPC comes up for retrying. Actually sends the RPC.
+  void DelayedRetryCb(Rpc* rpc, const Status& status);
+
+ private:
+  // The next sent rpc will be the nth attempt (indexed from 1).
+  int attempt_num_;
+
+  // If the remote end is busy, the RPC will be retried (with a small
+  // delay) until this deadline is reached.
+  //
+  // May be uninitialized.
+  MonoTime deadline_;
+
+  // Messenger to use when sending the RPC.
+  std::shared_ptr<Messenger> messenger_;
+
+  // RPC controller to use when sending the RPC.
+  RpcController controller_;
+
+  // In case any retries have already happened, remembers the last error.
+  // Errors from the server take precedence over timeout errors.
+  Status last_error_;
+
+  DISALLOW_COPY_AND_ASSIGN(RpcRetrier);
+};
+
+// An in-flight remote procedure call to some server.
+class Rpc {
+ public:
+  Rpc(const MonoTime& deadline,
+      std::shared_ptr<rpc::Messenger> messenger)
+      : retrier_(deadline, std::move(messenger)) {
+  }
+
+  virtual ~Rpc() {}
+
+  // Asynchronously sends the RPC to the remote end.
+  //
+  // Subclasses should use SendRpcCb() below as the callback function.
+  virtual void SendRpc() = 0;
+
+  // Returns a string representation of the RPC.
+  virtual std::string ToString() const = 0;
+
+  // Returns the number of times this RPC has been sent. Will always be at
+  // least one.
+  int num_attempts() const { return retrier().attempt_num(); }
+
+ protected:
+  const RpcRetrier& retrier() const { return retrier_; }
+  RpcRetrier* mutable_retrier() { return &retrier_; }
+
+ private:
+  friend class RpcRetrier;
+
+  // Callback for SendRpc(). If 'status' is not OK, something failed
+  // before the RPC was sent.
+  virtual void SendRpcCb(const Status& status) = 0;
+
+  // Used to retry some failed RPCs.
+  RpcRetrier retrier_;
+
+  DISALLOW_COPY_AND_ASSIGN(Rpc);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif // KUDU_RPC_RPC_H

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_context.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_context.cc b/be/src/kudu/rpc/rpc_context.cc
new file mode 100644
index 0000000..97da445
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_context.cc
@@ -0,0 +1,217 @@
+// 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 "kudu/rpc/rpc_context.h"
+
+#include <memory>
+#include <ostream>
+#include <utility>
+
+#include <glog/logging.h>
+#include <google/protobuf/message.h>
+
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/result_tracker.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/trace.h"
+
+using google::protobuf::Message;
+using kudu::pb_util::SecureDebugString;
+using std::string;
+using std::unique_ptr;
+
+namespace kudu {
+
+class Slice;
+
+namespace rpc {
+
+RpcContext::RpcContext(InboundCall *call,
+                       const google::protobuf::Message *request_pb,
+                       google::protobuf::Message *response_pb,
+                       scoped_refptr<ResultTracker> result_tracker)
+  : call_(CHECK_NOTNULL(call)),
+    request_pb_(request_pb),
+    response_pb_(response_pb),
+    result_tracker_(std::move(result_tracker)) {
+  VLOG(4) << call_->remote_method().service_name() << ": Received RPC request for "
+          << call_->ToString() << ":" << std::endl << SecureDebugString(*request_pb_);
+  TRACE_EVENT_ASYNC_BEGIN2("rpc_call", "RPC", this,
+                           "call", call_->ToString(),
+                           "request", pb_util::PbTracer::TracePb(*request_pb_));
+}
+
+RpcContext::~RpcContext() {
+}
+
+void RpcContext::RespondSuccess() {
+  if (AreResultsTracked()) {
+    result_tracker_->RecordCompletionAndRespond(call_->header().request_id(),
+                                                response_pb_.get());
+  } else {
+    VLOG(4) << call_->remote_method().service_name() << ": Sending RPC success response for "
+        << call_->ToString() << ":" << std::endl << SecureDebugString(*response_pb_);
+    TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                           "response", pb_util::PbTracer::TracePb(*response_pb_),
+                           "trace", trace()->DumpToString());
+    call_->RespondSuccess(*response_pb_);
+    delete this;
+  }
+}
+
+void RpcContext::RespondNoCache() {
+  if (AreResultsTracked()) {
+    result_tracker_->FailAndRespond(call_->header().request_id(),
+                                    response_pb_.get());
+  } else {
+    VLOG(4) << call_->remote_method().service_name() << ": Sending RPC failure response for "
+        << call_->ToString() << ": " << SecureDebugString(*response_pb_);
+    TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                           "response", pb_util::PbTracer::TracePb(*response_pb_),
+                           "trace", trace()->DumpToString());
+    // This is a bit counter intuitive, but when we get the failure but set the error on the
+    // call's response we call RespondSuccess() instead of RespondFailure().
+    call_->RespondSuccess(*response_pb_);
+    delete this;
+  }
+}
+
+void RpcContext::RespondFailure(const Status &status) {
+  return RespondRpcFailure(ErrorStatusPB::ERROR_APPLICATION, status);
+}
+
+void RpcContext::RespondRpcFailure(ErrorStatusPB_RpcErrorCodePB err, const Status& status) {
+  if (AreResultsTracked()) {
+    result_tracker_->FailAndRespond(call_->header().request_id(),
+                                    err, status);
+  } else {
+    VLOG(4) << call_->remote_method().service_name() << ": Sending RPC failure response for "
+        << call_->ToString() << ": " << status.ToString();
+    TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                           "status", status.ToString(),
+                           "trace", trace()->DumpToString());
+    call_->RespondFailure(err, status);
+    delete this;
+  }
+}
+
+void RpcContext::RespondApplicationError(int error_ext_id, const std::string& message,
+                                         const Message& app_error_pb) {
+  if (AreResultsTracked()) {
+    result_tracker_->FailAndRespond(call_->header().request_id(),
+                                    error_ext_id, message, app_error_pb);
+  } else {
+    if (VLOG_IS_ON(4)) {
+      ErrorStatusPB err;
+      InboundCall::ApplicationErrorToPB(error_ext_id, message, app_error_pb, &err);
+      VLOG(4) << call_->remote_method().service_name()
+          << ": Sending application error response for " << call_->ToString()
+          << ":" << std::endl << SecureDebugString(err);
+    }
+    TRACE_EVENT_ASYNC_END2("rpc_call", "RPC", this,
+                           "response", pb_util::PbTracer::TracePb(app_error_pb),
+                           "trace", trace()->DumpToString());
+    call_->RespondApplicationError(error_ext_id, message, app_error_pb);
+    delete this;
+  }
+}
+
+const rpc::RequestIdPB* RpcContext::request_id() const {
+  return call_->header().has_request_id() ? &call_->header().request_id() : nullptr;
+}
+
+size_t RpcContext::GetTransferSize() const {
+  return call_->GetTransferSize();
+}
+
+Status RpcContext::AddOutboundSidecar(unique_ptr<RpcSidecar> car, int* idx) {
+  return call_->AddOutboundSidecar(std::move(car), idx);
+}
+
+Status RpcContext::GetInboundSidecar(int idx, Slice* slice) {
+  return call_->GetInboundSidecar(idx, slice);
+}
+
+const RemoteUser& RpcContext::remote_user() const {
+  return call_->remote_user();
+}
+
+bool RpcContext::is_confidential() const {
+  return call_->connection()->is_confidential();
+}
+
+void RpcContext::DiscardTransfer() {
+  call_->DiscardTransfer();
+}
+
+const Sockaddr& RpcContext::remote_address() const {
+  return call_->remote_address();
+}
+
+std::string RpcContext::requestor_string() const {
+  return call_->remote_user().ToString() + " at " +
+    call_->remote_address().ToString();
+}
+
+std::string RpcContext::method_name() const {
+  return call_->remote_method().method_name();
+}
+
+std::string RpcContext::service_name() const {
+  return call_->remote_method().service_name();
+}
+
+MonoTime RpcContext::GetClientDeadline() const {
+  return call_->GetClientDeadline();
+}
+
+MonoTime RpcContext::GetTimeReceived() const {
+  return call_->GetTimeReceived();
+}
+
+Trace* RpcContext::trace() {
+  return call_->trace();
+}
+
+void RpcContext::Panic(const char* filepath, int line_number, const string& message) {
+  // Use the LogMessage class directly so that the log messages appear to come from
+  // the line of code which caused the panic, not this code.
+#define MY_ERROR google::LogMessage(filepath, line_number, google::GLOG_ERROR).stream()
+#define MY_FATAL google::LogMessageFatal(filepath, line_number).stream()
+
+  MY_ERROR << "Panic handling " << call_->ToString() << ": " << message;
+  MY_ERROR << "Request:\n" << SecureDebugString(*request_pb_);
+  Trace* t = trace();
+  if (t) {
+    MY_ERROR << "RPC trace:";
+    t->Dump(&MY_ERROR, true);
+  }
+  MY_FATAL << "Exiting due to panic.";
+
+#undef MY_ERROR
+#undef MY_FATAL
+}
+
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_context.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_context.h b/be/src/kudu/rpc/rpc_context.h
new file mode 100644
index 0000000..c729d5e
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_context.h
@@ -0,0 +1,245 @@
+// 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.
+#ifndef KUDU_RPC_RPC_CONTEXT_H
+#define KUDU_RPC_RPC_CONTEXT_H
+
+#include <memory>
+#include <stddef.h>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+namespace google {
+namespace protobuf {
+class Message;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Slice;
+class Sockaddr;
+class Trace;
+
+namespace rpc {
+
+class InboundCall;
+class RemoteUser;
+class ResultTracker;
+class RpcSidecar;
+
+#define PANIC_RPC(rpc_context, message) \
+  do { \
+    if (rpc_context) {                              \
+      rpc_context->Panic(__FILE__, __LINE__, (message));  \
+    } else { \
+      LOG(FATAL) << message; \
+    } \
+  } while (0)
+
+// The context provided to a generated ServiceIf. This provides
+// methods to respond to the RPC. In the future, this will also
+// include methods to access information about the caller: e.g
+// authentication info, tracing info, and cancellation status.
+//
+// This is the server-side analogue to the RpcController class.
+class RpcContext {
+ public:
+  // Create an RpcContext. This is called only from generated code
+  // and is not a public API.
+  RpcContext(InboundCall *call,
+             const google::protobuf::Message *request_pb,
+             google::protobuf::Message *response_pb,
+             scoped_refptr<ResultTracker> result_tracker);
+
+  ~RpcContext();
+
+  // Return the trace buffer for this call.
+  Trace* trace();
+
+  // Send a response to the call. The service may call this method
+  // before or after returning from the original handler method,
+  // and it may call this method from a different thread.
+  //
+  // The response should be prepared already in the response PB pointer
+  // which was passed to the handler method.
+  //
+  // After this method returns, this RpcContext object is destroyed. The request
+  // and response protobufs are also destroyed.
+  void RespondSuccess();
+
+  // Like the above, but doesn't store the results of the service call, if results
+  // are being tracked.
+  // Used in cases where a call specific error was set on the response protobuf,
+  // the call should be considered failed, thus results shouldn't be cached.
+  void RespondNoCache();
+
+  // Respond with an error to the client. This sends back an error with the code
+  // ERROR_APPLICATION. Because there is no more specific error code passed back
+  // to the client, most applications should create a custom error PB extension
+  // and use RespondApplicationError(...) below. This method should only be used
+  // for unexpected errors where the server doesn't expect the client to do any
+  // more advanced handling.
+  //
+  // After this method returns, this RpcContext object is destroyed. The request
+  // and response protobufs are also destroyed.
+  void RespondFailure(const Status &status);
+
+  // Respond with an RPC-level error. This typically manifests to the client as
+  // a remote error, one whose handling is agnostic to the particulars of the
+  // sent RPC. For example, both ERROR_SERVER_TOO_BUSY and ERROR_UNAVAILABLE
+  // usually cause the client to retry the RPC at a later time.
+  //
+  // After this method returns, this RpcContext object is destroyed. The request
+  // and response protobufs are also destroyed.
+  void RespondRpcFailure(ErrorStatusPB_RpcErrorCodePB err, const Status& status);
+
+  // Respond with an application-level error. This causes the caller to get a
+  // RemoteError status with the provided string message. Additionally, a
+  // service-specific error extension is passed back to the client. The
+  // extension must be registered with the ErrorStatusPB protobuf. For
+  // example:
+  //
+  //   message MyServiceError {
+  //     extend kudu.rpc.ErrorStatusPB {
+  //       optional MyServiceError my_service_error_ext = 101;
+  //     }
+  //     // Add any extra fields or status codes you want to pass back to
+  //     // the client here.
+  //     required string extra_error_data = 1;
+  //   }
+  //
+  // NOTE: the numeric '101' above must be an integer greater than 101
+  // and must be unique across your code base.
+  //
+  // Given the above definition in your service protobuf file, you would
+  // use this method like:
+  //
+  //   MyServiceError err;
+  //   err.set_extra_error_data("foo bar");
+  //   ctx->RespondApplicationError(MyServiceError::my_service_error_ext.number(),
+  //                                "Some error occurred", err);
+  //
+  // The client side may then retreieve the error by calling:
+  //   const MyServiceError& err_details =
+  //     controller->error_response()->GetExtension(MyServiceError::my_service_error_ext);
+  //
+  // After this method returns, this RpcContext object is destroyed. The request
+  // and response protobufs are also destroyed.
+  void RespondApplicationError(int error_ext_id, const std::string& message,
+                               const google::protobuf::Message& app_error_pb);
+
+
+  // Adds an RpcSidecar to the response. This is the preferred method for
+  // transferring large amounts of binary data, because this avoids additional
+  // copies made by serializing the protobuf.
+  //
+  // Assumes no changes to the sidecar's data are made after insertion.
+  //
+  // Upon success, writes the index of the sidecar (necessary to be retrieved
+  // later) to 'idx'. Call may fail if all sidecars have already been used
+  // by the RPC response.
+  Status AddOutboundSidecar(std::unique_ptr<RpcSidecar> car, int* idx);
+
+  // Fills 'sidecar' with a sidecar sent by the client. Returns an error if 'idx' is out
+  // of bounds.
+  Status GetInboundSidecar(int idx, Slice* slice);
+
+  // Return the identity of remote user who made this call.
+  const RemoteUser& remote_user() const;
+
+  // Whether it's OK to pass confidential information between the client and the
+  // server in the context of the RPC call being handled.  In real world, this
+  // translates into properties of the connection between the client and the
+  // server. For example, this methods returns 'true' for a call over an
+  // encrypted connection.
+  bool is_confidential() const;
+
+  // Discards the memory associated with the inbound call's payload. All previously
+  // obtained sidecar slices will be invalidated by this call. It is an error to call
+  // GetInboundSidecar() after this method. request_pb() remains valid.
+  // This is useful in the case where the server wishes to delay responding to an RPC
+  // (perhaps to control the rate of RPC requests), but knows that the RPC payload itself
+  // won't be processed any further.
+  void DiscardTransfer();
+
+  // Return the remote IP address and port which sent the current RPC call.
+  const Sockaddr& remote_address() const;
+
+  // A string identifying the requestor -- both the user info and the IP address.
+  // Suitable for use in log messages.
+  std::string requestor_string() const;
+
+  // Return the name of the RPC service method being called.
+  std::string method_name() const;
+
+  // Return the name of the RPC service being called.
+  std::string service_name() const;
+
+  const google::protobuf::Message *request_pb() const { return request_pb_.get(); }
+  google::protobuf::Message *response_pb() const { return response_pb_.get(); }
+
+  // Return an upper bound on the client timeout deadline. This does not
+  // account for transmission delays between the client and the server.
+  // If the client did not specify a deadline, returns MonoTime::Max().
+  MonoTime GetClientDeadline() const;
+
+  // Return the time when the inbound call was received.
+  MonoTime GetTimeReceived() const;
+
+  // Whether the results of this RPC are tracked with a ResultTracker.
+  // If this returns true, both result_tracker() and request_id() should return non-null results.
+  bool AreResultsTracked() const { return result_tracker_.get() != nullptr; }
+
+  // Returns this call's result tracker, if it is set.
+  const scoped_refptr<ResultTracker>& result_tracker() const {
+    return result_tracker_;
+  }
+
+  // Returns this call's request id, if it is set.
+  const rpc::RequestIdPB* request_id() const;
+
+  // Returns the size of the transfer buffer that backs 'call_'. If the
+  // transfer buffer no longer exists (e.g. GetTransferSize() is called after
+  // DiscardTransfer()), returns 0.
+  size_t GetTransferSize() const;
+
+  // Panic the server. This logs a fatal error with the given message, and
+  // also includes the current RPC request, requestor, trace information, etc,
+  // to make it easier to debug.
+  //
+  // Call this via the PANIC_RPC() macro.
+  void Panic(const char* filepath, int line_number, const std::string& message)
+    __attribute__((noreturn));
+
+ private:
+  friend class ResultTracker;
+  InboundCall* const call_;
+  const gscoped_ptr<const google::protobuf::Message> request_pb_;
+  const gscoped_ptr<google::protobuf::Message> response_pb_;
+  scoped_refptr<ResultTracker> result_tracker_;
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_controller.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_controller.cc b/be/src/kudu/rpc/rpc_controller.cc
new file mode 100644
index 0000000..77c7ca4
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_controller.cc
@@ -0,0 +1,177 @@
+// 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 "kudu/rpc/rpc_controller.h"
+
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/slice.h"
+
+
+using std::unique_ptr;
+using strings::Substitute;
+namespace kudu {
+
+namespace rpc {
+
+RpcController::RpcController()
+    : credentials_policy_(CredentialsPolicy::ANY_CREDENTIALS), messenger_(nullptr) {
+  DVLOG(4) << "RpcController " << this << " constructed";
+}
+
+RpcController::~RpcController() {
+  DVLOG(4) << "RpcController " << this << " destroyed";
+}
+
+void RpcController::Swap(RpcController* other) {
+  // Cannot swap RPC controllers while they are in-flight.
+  if (call_) {
+    CHECK(finished());
+  }
+  if (other->call_) {
+    CHECK(other->finished());
+  }
+
+  std::swap(outbound_sidecars_, other->outbound_sidecars_);
+  std::swap(outbound_sidecars_total_bytes_, other->outbound_sidecars_total_bytes_);
+  std::swap(timeout_, other->timeout_);
+  std::swap(credentials_policy_, other->credentials_policy_);
+  std::swap(call_, other->call_);
+}
+
+void RpcController::Reset() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  if (call_) {
+    CHECK(finished());
+  }
+  call_.reset();
+  required_server_features_.clear();
+  credentials_policy_ = CredentialsPolicy::ANY_CREDENTIALS;
+  messenger_ = nullptr;
+  outbound_sidecars_total_bytes_ = 0;
+}
+
+bool RpcController::finished() const {
+  if (call_) {
+    return call_->IsFinished();
+  }
+  return false;
+}
+
+bool RpcController::negotiation_failed() const {
+  if (call_) {
+    DCHECK(finished());
+    return call_->IsNegotiationError();
+  }
+  return false;
+}
+
+Status RpcController::status() const {
+  if (call_) {
+    return call_->status();
+  }
+  return Status::OK();
+}
+
+const ErrorStatusPB* RpcController::error_response() const {
+  if (call_) {
+    return call_->error_pb();
+  }
+  return nullptr;
+}
+
+Status RpcController::GetInboundSidecar(int idx, Slice* sidecar) const {
+  return call_->call_response_->GetSidecar(idx, sidecar);
+}
+
+void RpcController::set_timeout(const MonoDelta& timeout) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  DCHECK(!call_ || call_->state() == OutboundCall::READY);
+  timeout_ = timeout;
+}
+
+void RpcController::set_deadline(const MonoTime& deadline) {
+  set_timeout(deadline - MonoTime::Now());
+}
+
+void RpcController::SetRequestIdPB(std::unique_ptr<RequestIdPB> request_id) {
+  request_id_ = std::move(request_id);
+}
+
+bool RpcController::has_request_id() const {
+  return request_id_ != nullptr;
+}
+
+const RequestIdPB& RpcController::request_id() const {
+  DCHECK(has_request_id());
+  return *request_id_;
+}
+
+void RpcController::RequireServerFeature(uint32_t feature) {
+  DCHECK(!call_ || call_->state() == OutboundCall::READY);
+  required_server_features_.insert(feature);
+}
+
+MonoDelta RpcController::timeout() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return timeout_;
+}
+
+Status RpcController::AddOutboundSidecar(unique_ptr<RpcSidecar> car, int* idx) {
+  if (outbound_sidecars_.size() >= TransferLimits::kMaxSidecars) {
+    return Status::RuntimeError("All available sidecars already used");
+  }
+  int64_t sidecar_bytes = car->AsSlice().size();
+  if (outbound_sidecars_total_bytes_ >
+      TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes) {
+    return Status::RuntimeError(Substitute("Total size of sidecars $0 would exceed limit $1",
+        static_cast<int64_t>(outbound_sidecars_total_bytes_) + sidecar_bytes,
+        TransferLimits::kMaxTotalSidecarBytes));
+  }
+
+  outbound_sidecars_.emplace_back(std::move(car));
+  outbound_sidecars_total_bytes_ += sidecar_bytes;
+  DCHECK_GE(outbound_sidecars_total_bytes_, 0);
+  *idx = outbound_sidecars_.size() - 1;
+  return Status::OK();
+}
+
+void RpcController::SetRequestParam(const google::protobuf::Message& req) {
+  DCHECK(call_ != nullptr);
+  call_->SetRequestPayload(req, std::move(outbound_sidecars_));
+}
+
+void RpcController::Cancel() {
+  DCHECK(call_);
+  DCHECK(messenger_);
+  messenger_->QueueCancellation(call_);
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_controller.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_controller.h b/be/src/kudu/rpc/rpc_controller.h
new file mode 100644
index 0000000..aa61d83
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_controller.h
@@ -0,0 +1,282 @@
+// 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.
+#ifndef KUDU_RPC_RPC_CONTROLLER_H
+#define KUDU_RPC_RPC_CONTROLLER_H
+
+#include <cstdint>
+#include <memory>
+#include <unordered_set>
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+namespace google {
+namespace protobuf {
+class Message;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Slice;
+
+namespace rpc {
+
+class ErrorStatusPB;
+class Messenger;
+class OutboundCall;
+class RequestIdPB;
+class RpcSidecar;
+
+// Authentication credentials policy for outbound RPCs. Some RPC methods
+// (e.g. MasterService::ConnectToMaster) behave differently depending on the
+// type of credentials used for authentication when establishing the connection.
+// The client expecting some particular results from the call should specify
+// the required policy on a per-call basis using RpcController. By default,
+// RpcController uses ANY_CREDENTIALS.
+enum class CredentialsPolicy {
+  // It's acceptable to use authentication credentials of any type, primary or
+  // secondary ones.
+  ANY_CREDENTIALS,
+
+  // Only primary credentials are acceptable. Primary credentials are Kerberos
+  // tickets, TLS certificate. Secondary credentials are authentication tokens:
+  // they are 'derived' in the sense that it's possible to acquire them using
+  // 'primary' credentials.
+  PRIMARY_CREDENTIALS,
+};
+
+// Controller for managing properties of a single RPC call, on the client side.
+//
+// An RpcController maps to exactly one call and is not thread-safe. The client
+// may use this class prior to sending an RPC in order to set properties such
+// as the call's timeout.
+//
+// After the call has been sent (e.g using Proxy::AsyncRequest()) the user
+// may invoke methods on the RpcController object in order to probe the status
+// of the call.
+class RpcController {
+ public:
+  RpcController();
+  ~RpcController();
+
+  // Swap the state of the controller (including ownership of sidecars, buffers,
+  // etc) with another one.
+  void Swap(RpcController* other);
+
+  // Reset this controller so it may be used with another call.
+  // Note that this resets the required server features.
+  void Reset();
+
+  // Return true if the call has finished.
+  // A call is finished if the server has responded, or if the call
+  // has timed out.
+  bool finished() const;
+
+  // Whether the call failed due to connection negotiation error.
+  bool negotiation_failed() const;
+
+  // Return the current status of a call.
+  //
+  // A call is "OK" status until it finishes, at which point it may
+  // either remain in "OK" status (if the call was successful), or
+  // change to an error status. Error status indicates that there was
+  // some RPC-layer issue with making the call, for example, one of:
+  //
+  // * failed to establish a connection to the server
+  // * the server was too busy to handle the request
+  // * the server was unable to interpret the request (eg due to a version
+  //   mismatch)
+  // * a network error occurred which caused the connection to be torn
+  //   down
+  // * the call timed out
+  Status status() const;
+
+  // If status() returns a RemoteError object, then this function returns
+  // the error response provided by the server. Service implementors may
+  // use protobuf Extensions to add application-specific data to this PB.
+  //
+  // If Status was not a RemoteError, this returns NULL.
+  // The returned pointer is only valid as long as the controller object.
+  const ErrorStatusPB* error_response() const;
+
+  // Set the timeout for the call to be made with this RPC controller.
+  //
+  // The configured timeout applies to the entire time period between
+  // the AsyncRequest() method call and getting a response. For example,
+  // if it takes too long to establish a connection to the remote host,
+  // or to DNS-resolve the remote host, those will be accounted as part
+  // of the timeout period.
+  //
+  // Timeouts must be set prior to making the request -- the timeout may
+  // not currently be adjusted for an already-sent call.
+  //
+  // Using an uninitialized timeout will result in a call which never
+  // times out (not recommended!)
+  void set_timeout(const MonoDelta& timeout);
+
+  // Like a timeout, but based on a fixed point in time instead of a delta.
+  //
+  // Using an uninitialized deadline means the call won't time out.
+  void set_deadline(const MonoTime& deadline);
+
+  // Allows setting the request id for the next request sent to the server.
+  // A request id allows the server to identify each request sent by the client uniquely,
+  // in some cases even when sent to multiple servers, enabling exactly once semantics.
+  void SetRequestIdPB(std::unique_ptr<RequestIdPB> request_id);
+
+  // Returns whether a request id has been set on RPC header.
+  bool has_request_id() const;
+
+  // Returns the currently set request id.
+  // When the request is sent to the server, it gets "moved" from RpcController
+  // so an absence of a request after send doesn't mean one wasn't sent.
+  // REQUIRES: the controller has a request ID set.
+  const RequestIdPB& request_id() const;
+
+  // Add a requirement that the server side must support a feature with the
+  // given identifier. The set of required features is sent to the server
+  // with the RPC call, and if any required feature is not supported, the
+  // call will fail with a NotSupported() status.
+  //
+  // This can be used when an RPC call changes in a way that is protobuf-compatible,
+  // but for which it would not be appropriate for the server to simply ignore
+  // an added field. For example, consider an API call like:
+  //
+  //   message DeleteAccount {
+  //     optional string username = 1;
+  //     optional bool dry_run = 2; // ADDED LATER!
+  //   }
+  //
+  // In this case, if a new client which supports the 'dry_run' flag sends the RPC
+  // to an old server, the old server will simply ignore the unrecognized parameter,
+  // with highly problematic results. To solve this problem, the new version can
+  // add a feature flag:
+  //
+  //   In .proto file
+  //   ----------------
+  //   enum MyFeatureFlags {
+  //     UNKNOWN = 0;
+  //     DELETE_ACCOUNT_SUPPORTS_DRY_RUN = 1;
+  //   }
+  //
+  //   In client code:
+  //   ---------------
+  //   if (dry_run) {
+  //     rpc.RequireServerFeature(DELETE_ACCOUNT_SUPPORTS_DRY_RUN);
+  //     req.set_dry_run(true);
+  //   }
+  //
+  // This has the effect of (a) maintaining compatibility when dry_run is not specified
+  // and (b) rejecting the RPC with a "NotSupported" error when it is.
+  //
+  // NOTE: 'feature' is an int rather than an enum type because each service
+  // must define its own enum of supported features, and protobuf doesn't support
+  // any ability to 'extend' enum types. Implementers should define an enum in the
+  // service's protobuf definition as shown above.
+  void RequireServerFeature(uint32_t feature);
+
+  // Executes the provided function with a reference to the required server
+  // features.
+  const std::unordered_set<uint32_t>& required_server_features() const {
+    return required_server_features_;
+  }
+
+  // Return the configured timeout.
+  MonoDelta timeout() const;
+
+  CredentialsPolicy credentials_policy() const {
+    return credentials_policy_;
+  }
+
+  void set_credentials_policy(CredentialsPolicy policy) {
+    credentials_policy_ = policy;
+  }
+
+  // Fills the 'sidecar' parameter with the slice pointing to the i-th
+  // sidecar upon success.
+  //
+  // Should only be called if the call's finished, but the controller has not
+  // been Reset().
+  //
+  // May fail if index is invalid.
+  Status GetInboundSidecar(int idx, Slice* sidecar) const;
+
+  // Adds a sidecar to the outbound request. The index of the sidecar is written to
+  // 'idx'. Returns an error if TransferLimits::kMaxSidecars have already been added
+  // to this request. Also returns an error if the total size of all sidecars would
+  // exceed TransferLimits::kMaxTotalSidecarBytes.
+  Status AddOutboundSidecar(std::unique_ptr<RpcSidecar> car, int* idx);
+
+  // Cancel the call associated with the RpcController. This function should only be
+  // called when there is an outstanding outbound call. It's always safe to call
+  // Cancel() after you've sent a call, so long as you haven't called Reset() yet.
+  // Caller is not responsible for synchronization between cancellation and the
+  // callback. (i.e. the callback may or may not be invoked yet when Cancel()
+  // is called).
+  //
+  // Cancellation is "best effort" - i.e. it's still possible the callback passed
+  // to the call will be fired with a success status. If cancellation succeeds,
+  // the callback will be invoked with a Aborted status. Cancellation is asynchronous
+  // so the callback will still be invoked from the reactor thread.
+  void Cancel();
+
+ private:
+  friend class OutboundCall;
+  friend class Proxy;
+
+  // Set the outbound call_'s request parameter, and transfer ownership of
+  // outbound_sidecars_ to call_ in preparation for serialization.
+  void SetRequestParam(const google::protobuf::Message& req);
+
+  // Set the messenger which contains the reactor thread handling the outbound call.
+  void SetMessenger(Messenger* messenger) { messenger_ = messenger; }
+
+  MonoDelta timeout_;
+  std::unordered_set<uint32_t> required_server_features_;
+
+  // RPC authentication policy for outbound calls.
+  CredentialsPolicy credentials_policy_;
+
+  mutable simple_spinlock lock_;
+
+  // The id of this request.
+  // Ownership is transferred to OutboundCall once the call is sent.
+  std::unique_ptr<RequestIdPB> request_id_;
+
+  // The messenger which contains the reactor thread for 'call_'.
+  // Set only when 'call_' is set.
+  Messenger* messenger_;
+
+  // Once the call is sent, it is tracked here.
+  std::shared_ptr<OutboundCall> call_;
+
+  std::vector<std::unique_ptr<RpcSidecar>> outbound_sidecars_;
+
+  // Total size of sidecars in outbound_sidecars_. This is limited to a maximum
+  // of TransferLimits::kMaxTotalSidecarBytes.
+  int32_t outbound_sidecars_total_bytes_ = 0;
+
+  DISALLOW_COPY_AND_ASSIGN(RpcController);
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_header.proto b/be/src/kudu/rpc/rpc_header.proto
new file mode 100644
index 0000000..1d55b6a
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_header.proto
@@ -0,0 +1,365 @@
+// 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.
+syntax = "proto2";
+
+option optimize_for = SPEED;
+
+package kudu.rpc;
+
+option java_package = "org.apache.kudu.rpc";
+
+import "google/protobuf/descriptor.proto";
+import "kudu/security/token.proto";
+import "kudu/util/pb_util.proto";
+
+// The Kudu RPC protocol is similar to the RPC protocol of Hadoop and HBase.
+// See the following for reference on those other protocols:
+//  - https://issues.apache.org/jira/browse/HBASE-7898
+//  - https://issues.apache.org/jira/browse/HADOOP-8990
+//
+// For a description of the Kudu protocol, see 'README' in this directory.
+
+// User Information proto.  Included in ConnectionContextPB on connection setup.
+message UserInformationPB {
+  optional string effective_user = 1;
+  required string real_user = 2;
+}
+
+// The connection context is sent as part of the connection establishment.
+// It establishes the context for ALL RPC calls within the connection.
+// This is sent on connection setup after the connection preamble is sent
+// and SASL has been negotiated.
+// No response is sent from the server to the client.
+message ConnectionContextPB {
+  // UserInfo beyond what is determined as part of security handshake
+  // at connection time (kerberos, tokens etc).
+  //
+  // DEPRECATED: No longer used in Kudu 1.1 and later.
+  // The 'real_user' should be taken from the SASL negotiation.
+  // Impersonation (effective user) was never supported, so we'll have
+  // to add that back at some point later.
+  optional UserInformationPB DEPRECATED_user_info = 2;
+
+  // If the server sends a nonce to the client during the SASL_SUCCESS
+  // negotiation step, the client is required to encode it with SASL integrity
+  // protection and return it in this field. The nonce protects the server
+  // against a Kerberos replay attack.
+  optional bytes encoded_nonce = 3 [(REDACT) = true];
+}
+
+// Features supported by the RPC system itself.
+//
+// Note that this should be used to evolve the RPC _system_, not the semantics
+// or compatibility of individual calls.
+//
+// For example, if we were to add a feature like call or response wire
+// compression in the future, we could add a flag here to indicate that the
+// client or server supports that feature. Optional features which may safely be
+// ignored by the receiver do not need a feature flag, instead the optional
+// field feature of ProtoBuf may be utilized.
+enum RpcFeatureFlag {
+  UNKNOWN = 0;
+
+  // The RPC system is required to support application feature flags in the
+  // request and response headers.
+  APPLICATION_FEATURE_FLAGS = 1;
+
+  // The RPC system supports TLS protected connections. If both sides support
+  // this flag, the connection will automatically be wrapped in a TLS protected
+  // channel following a TLS handshake.
+  TLS = 2;
+
+  // If both sides advertise TLS_AUTHENTICATION_ONLY, this means that they
+  // agree that, after handshaking TLS, they will *not* wrap the connection
+  // in a TLS-protected channel. Instead, they will use TLS only for its
+  // handshake-based authentication.
+  //
+  // This is currently used for loopback connections only, so that compute
+  // frameworks which schedule for locality don't pay encryption overhead.
+  TLS_AUTHENTICATION_ONLY = 3;
+};
+
+// An authentication type. This is modeled as a oneof in case any of these
+// authentication types, or any authentication types in the future, need to add
+// extra type-specific parameters during negotiation.
+message AuthenticationTypePB {
+  message Sasl {};
+  message Token {};
+  message Certificate {};
+
+  oneof type {
+    // The server and client mutually authenticate via SASL.
+    Sasl sasl = 1;
+
+    // The server authenticates the client via a signed token, and the client
+    // authenticates the server by verifying its certificate has been signed by
+    // a trusted CA.
+    //
+    // Token authentication requires the connection to be TLS encrypted.
+    Token token = 2;
+
+    // The server and client mutually authenticate by certificate.
+    //
+    // Certificate authentication requires the connection to be TLS encrypted.
+    Certificate certificate = 3;
+  }
+}
+
+// Message type passed back & forth for the SASL negotiation.
+message NegotiatePB {
+  enum NegotiateStep {
+    UNKNOWN        = 999;
+    NEGOTIATE      = 1;
+    SASL_SUCCESS   = 0;
+    SASL_INITIATE  = 2;
+    SASL_CHALLENGE = 3;
+    SASL_RESPONSE  = 4;
+    TLS_HANDSHAKE  = 5;
+    TOKEN_EXCHANGE = 6;
+  }
+
+  message SaslMechanism {
+    // The SASL mechanism, i.e. 'PLAIN' or 'GSSAPI'.
+    required string mechanism = 2;
+
+    // Deprecated: no longer used.
+    // optional string method = 1;
+    // optional bytes challenge = 5 [(REDACT) = true];
+  }
+
+  // When the client sends its NEGOTIATE step message, it sends its set of
+  // supported RPC system features. In the response to this message, the server
+  // sends back its own. This allows the two peers to agree on whether newer
+  // extensions of the RPC system may be used on this connection. We use a list
+  // of features rather than a simple version number to make it easier for the
+  // Java and C++ clients to implement features in different orders while still
+  // maintaining compatibility, as well as to simplify backporting of features
+  // out-of-order.
+  repeated RpcFeatureFlag supported_features = 1;
+
+  // The current negotiation step.
+  required NegotiateStep step  = 2;
+
+  // The SASL token, containing either the challenge during the SASL_CHALLENGE
+  // step, or the response during the SASL_RESPONSE step.
+  optional bytes token         = 3 [(REDACT) = true];
+
+  // During the TLS_HANDSHAKE step, contains the TLS handshake message.
+  optional bytes tls_handshake = 5 [(REDACT) = true];
+
+  // The tls-server-end-point channel bindings as specified in RFC 5929.  Sent
+  // from the server to the client during the SASL_SUCCESS step when the
+  // Kerberos (GSSAPI) SASL mechanism is used with TLS, in order to bind the
+  // Kerberos authenticated channel to the TLS channel. The value is integrity
+  // protected through SASL. The client is responsible for validating that the
+  // value matches the expected value.
+  optional bytes channel_bindings = 6 [(REDACT) = true];
+
+  // A random nonce sent from the server to the client during the SASL_SUCCESS
+  // step when the Kerberos (GSSAPI) SASL mechanism is used with TLS. The nonce
+  // must be sent back to the server, wrapped in SASL integrity protection, as
+  // part of the connection context.
+  optional bytes nonce = 9 [(REDACT) = true];
+
+  // During the NEGOTIATE step, contains the supported SASL mechanisms.
+  // During the SASL_INITIATE step, contains the single chosen SASL mechanism.
+  repeated SaslMechanism sasl_mechanisms = 4;
+
+  // During the client to server NEGOTIATE step, contains the supported authentication types.
+  // During the server to client NEGOTIATE step, contains the chosen authentication type.
+  repeated AuthenticationTypePB authn_types = 7;
+
+  // During the TOKEN_EXCHANGE step, contains the client's signed authentication token.
+  optional security.SignedTokenPB authn_token = 8;
+}
+
+message RemoteMethodPB {
+  // Service name for the RPC layer.
+  // The client created a proxy with this service name.
+  // Example: kudu.rpc_test.CalculatorService
+  required string service_name = 1;
+
+  // Name of the RPC method.
+  required string method_name = 2;
+};
+
+// The Id of a retriable RPC, whose results should be tracked on the server (see result_tracker.h).
+// This also includes some information that is useful for execution/garbage collection.
+message RequestIdPB {
+  // The (globally unique) id of the client performing this RPC.
+  required string client_id = 1;
+
+  // The (per-client unique) sequence number of this RPC.
+  required int64 seq_no = 2;
+
+  // The sequence number of the first RPC that has not been marked as completed by the client.
+  // Unset if there isn't an incomplete RPC.
+  required int64 first_incomplete_seq_no = 3;
+
+  // The number of times this RPC has been tried.
+  // Set to 1 in the first attempt.
+  required int64 attempt_no = 4;
+}
+
+// The header for the RPC request frame.
+message RequestHeader {
+  // A sequence number that uniquely identifies a call to a single remote server. This number is
+  // sent back in the Response and allows to match it to the original Request.
+  // Hadoop specifies a uint32 and casts it to a signed int. That is counterintuitive, so we use an
+  // int32 instead. Allowed values (inherited from Hadoop):
+  //   0 through INT32_MAX: Regular RPC call IDs.
+  //   -2: Invalid call ID.
+  //   -3: Connection context call ID.
+  //   -33: SASL negotiation call ID.
+  //
+  // NOTE: these calls must be increasing but may have gaps.
+  required int32 call_id = 3;
+
+  // RPC method being invoked.
+  // Not used for "connection setup" calls.
+  optional RemoteMethodPB remote_method = 6;
+
+  // Propagate the timeout as specified by the user. Note that, since there is some
+  // transit time between the client and server, if you wait exactly this amount of
+  // time and then respond, you are likely to cause a timeout on the client.
+  optional uint32 timeout_millis = 10;
+
+  // Feature flags that the service must support in order to properly interpret this
+  // request. The client can pass any set of flags, and if the server doesn't
+  // support any of them, then it will fail the request.
+  //
+  // NOTE: these are for evolving features at the level of the application, not
+  // the RPC framework. Hence, we have to use a generic int type rather than a
+  // particular enum.
+  // NOTE: the server will only interpret this field if it supports the
+  // APPLICATION_FEATURE_FLAGS flag.
+  repeated uint32 required_feature_flags = 11;
+
+  // The unique id of this request, if it's retriable and if the results are to be tracked.
+  // The request id is unique per logical request, i.e. retries of the same RPC must have the
+  // same request id.
+  // Note that this is different from 'call_id' in that a call_id is unique to a server while a
+  // request_id is unique to a logical request (i.e. the request_id remains the same when a request
+  // is retried on a different server).
+  // Optional for requests that are naturally idempotent or to maintain compatibility with
+  // older clients for requests that are not.
+  optional RequestIdPB request_id = 15;
+
+  // Byte offsets for side cars in the main body of the request message.
+  // These offsets are counted AFTER the message header, i.e., offset 0
+  // is the first byte after the bytes for this protobuf.
+  repeated uint32 sidecar_offsets = 16;
+}
+
+message ResponseHeader {
+  required int32 call_id = 1;
+
+  // If this is set, then this is an error response and the
+  // response message will be of type ErrorStatusPB instead of
+  // the expected response type.
+  optional bool is_error = 2 [ default = false ];
+
+  // Byte offsets for side cars in the main body of the response message.
+  // These offsets are counted AFTER the message header, i.e., offset 0
+  // is the first byte after the bytes for this protobuf.
+  repeated uint32 sidecar_offsets = 3;
+}
+
+// Sent as response when is_error == true.
+message ErrorStatusPB {
+
+  // These codes have all been inherited from Hadoop's RPC mechanism.
+  enum RpcErrorCodePB {
+    FATAL_UNKNOWN = 10;
+
+    // Non-fatal RPC errors. Connection should be left open for future RPC calls.
+    //------------------------------------------------------------
+    // The application generated an error status. See the message field for
+    // more details.
+    ERROR_APPLICATION = 1;
+
+    // The specified method was not valid.
+    ERROR_NO_SUCH_METHOD = 2;
+
+    // The specified service was not valid.
+    ERROR_NO_SUCH_SERVICE = 3;
+
+    // The server is overloaded - the client should try again shortly.
+    ERROR_SERVER_TOO_BUSY = 4;
+
+    // The request parameter was not parseable, was missing required fields,
+    // or the server does not support the required feature flags.
+    ERROR_INVALID_REQUEST = 5;
+
+    // The server might have previously received this request but its response is no
+    // longer cached. It's unknown whether the request was executed or not.
+    ERROR_REQUEST_STALE = 6;
+
+    // The server is not able to complete the connection or request at this
+    // time. The client may try again later.
+    ERROR_UNAVAILABLE = 7;
+
+    // FATAL_* errors indicate that the client should shut down the connection.
+    //------------------------------------------------------------
+    // The RPC server is already shutting down.
+    FATAL_SERVER_SHUTTING_DOWN = 11;
+    // Fields of RpcHeader are invalid.
+    FATAL_INVALID_RPC_HEADER = 12;
+    // Could not deserialize RPC request.
+    FATAL_DESERIALIZING_REQUEST = 13;
+    // IPC Layer version mismatch.
+    FATAL_VERSION_MISMATCH = 14;
+    // Auth failed.
+    FATAL_UNAUTHORIZED = 15;
+
+    // The authentication token is invalid or expired;
+    // the client should obtain a new one.
+    FATAL_INVALID_AUTHENTICATION_TOKEN = 16;
+  }
+
+  required string message = 1;
+
+  // TODO: Make code required?
+  optional RpcErrorCodePB code = 2;  // Specific error identifier.
+
+  // If the request is failed due to an unsupported feature flag, the particular
+  // flag(s) that were not supported will be sent back to the client.
+  repeated uint32 unsupported_feature_flags = 3;
+
+  // Allow extensions. When the RPC returns ERROR_APPLICATION, the server
+  // should also fill in exactly one of these extension fields, which contains
+  // more details on the service-specific error.
+  extensions 100 to max;
+}
+
+extend google.protobuf.MethodOptions {
+  // An option for RPC methods that allows to set whether that method's
+  // RPC results should be tracked with a ResultTracker.
+  optional bool track_rpc_result = 50006 [default=false];
+
+  // An option to set the authorization method for this particular
+  // RPC method. If this is not specified, the service's 'default_authz_method'
+  // is used.
+  optional string authz_method = 50007;
+}
+
+extend google.protobuf.ServiceOptions {
+  // Set the default authorization method for the RPCs in this service.
+  // If this is not set, then the default authorization is to allow all
+  // RPCs.
+  optional string default_authz_method = 50007;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_introspection.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_introspection.proto b/be/src/kudu/rpc/rpc_introspection.proto
new file mode 100644
index 0000000..7685903
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_introspection.proto
@@ -0,0 +1,110 @@
+// 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.
+//
+// Protobuf used for introspection of RPC services (eg listing in-flight RPCs,
+// reflection, etc)
+syntax = "proto2";
+
+package kudu.rpc;
+
+option java_package = "org.apache.kudu";
+
+import "kudu/rpc/rpc_header.proto";
+
+message RpcCallInProgressPB {
+  required RequestHeader header = 1;
+  optional string trace_buffer = 2;
+  optional uint64 micros_elapsed = 3;
+
+  enum State {
+    UNKNOWN = 999;
+
+    // States for OutboundCall
+    ON_OUTBOUND_QUEUE = 1;
+    SENDING = 2;
+    SENT = 3;
+    TIMED_OUT = 4;
+    FINISHED_ERROR = 5;
+    FINISHED_SUCCESS = 6;
+    NEGOTIATION_TIMED_OUT = 7;
+    FINISHED_NEGOTIATION_ERROR = 8;
+    CANCELLED = 9;
+
+    // TODO(todd): add states for InboundCall
+  }
+
+  optional State state = 4;
+}
+
+message RpcConnectionPB {
+  enum StateType {
+    UNKNOWN = 999;
+    NEGOTIATING = 0;  // Connection is still being negotiated.
+    OPEN = 1;         // Connection is active.
+  };
+
+  required string remote_ip = 1;
+  required StateType state = 2;
+  // TODO: swap out for separate fields
+  optional string remote_user_credentials = 3;
+  repeated RpcCallInProgressPB calls_in_flight = 4;
+  optional int64 outbound_queue_size = 5;
+}
+
+message DumpRunningRpcsRequestPB {
+  optional bool include_traces = 1 [ default = false ];
+}
+
+message DumpRunningRpcsResponsePB {
+  repeated RpcConnectionPB inbound_connections = 1;
+  repeated RpcConnectionPB outbound_connections = 2;
+}
+
+//------------------------------------------------------------
+
+// A particular TraceMetric key/value pair from a sampled RPC.
+message TraceMetricPB {
+  // A '.'-separated path through the parent-child trace hierarchy.
+  optional string child_path = 1;
+  optional string key = 2;
+  optional int64 value = 3;
+}
+
+// A single sampled RPC call.
+message RpczSamplePB {
+  // The original request header.
+  optional RequestHeader header = 1;
+  // The stringified request trace.
+  optional string trace = 2;
+  // The number of millis that this call took to complete.
+  optional int32 duration_ms = 3;
+  // The metrics from the sampled trace.
+  repeated TraceMetricPB metrics = 4;
+}
+
+// A set of samples for a particular RPC method.
+message RpczMethodPB {
+  required string method_name = 1;
+  repeated RpczSamplePB samples = 2;
+}
+
+// Request and response for dumping previously sampled RPC calls.
+message DumpRpczStoreRequestPB {
+}
+message DumpRpczStoreResponsePB {
+  repeated RpczMethodPB methods = 1;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_service.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_service.h b/be/src/kudu/rpc/rpc_service.h
new file mode 100644
index 0000000..dcaa9c1
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_service.h
@@ -0,0 +1,47 @@
+// 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.
+#ifndef KUDU_RPC_SERVICE_H_
+#define KUDU_RPC_SERVICE_H_
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace rpc {
+
+class RemoteMethod;
+struct RpcMethodInfo;
+class InboundCall;
+
+class RpcService : public RefCountedThreadSafe<RpcService> {
+ public:
+  virtual ~RpcService() {}
+
+  // Enqueue a call for processing.
+  // On failure, the RpcService::QueueInboundCall() implementation is
+  // responsible for responding to the client with a failure message.
+  virtual Status QueueInboundCall(gscoped_ptr<InboundCall> call) = 0;
+
+  virtual RpcMethodInfo* LookupMethod(const RemoteMethod& method) {
+    return nullptr;
+  }
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif // KUDU_RPC_SERVICE_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_sidecar.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_sidecar.cc b/be/src/kudu/rpc/rpc_sidecar.cc
new file mode 100644
index 0000000..b4de678
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_sidecar.cc
@@ -0,0 +1,115 @@
+// 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 "kudu/rpc/rpc_sidecar.h"
+
+#include <cstdint>
+#include <memory>
+#include <utility>
+
+#include <google/protobuf/repeated_field.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/status.h"
+
+using std::unique_ptr;
+
+namespace kudu {
+namespace rpc {
+
+// Sidecar that simply wraps a Slice. The data associated with the slice is therefore not
+// owned by this class, and it's the caller's responsibility to ensure it has a lifetime
+// at least as long as this sidecar.
+class SliceSidecar : public RpcSidecar {
+ public:
+  explicit SliceSidecar(Slice slice) : slice_(slice) { }
+  Slice AsSlice() const override { return slice_; }
+
+ private:
+  const Slice slice_;
+};
+
+class FaststringSidecar : public RpcSidecar {
+ public:
+  explicit FaststringSidecar(unique_ptr<faststring> data) : data_(std::move(data)) { }
+  Slice AsSlice() const override { return *data_; }
+
+ private:
+  const unique_ptr<faststring> data_;
+};
+
+unique_ptr<RpcSidecar> RpcSidecar::FromFaststring(unique_ptr<faststring> data) {
+  return unique_ptr<RpcSidecar>(new FaststringSidecar(std::move(data)));
+}
+
+unique_ptr<RpcSidecar> RpcSidecar::FromSlice(Slice slice) {
+  return unique_ptr<RpcSidecar>(new SliceSidecar(slice));
+}
+
+
+Status RpcSidecar::ParseSidecars(
+    const ::google::protobuf::RepeatedField<::google::protobuf::uint32>& offsets,
+    Slice buffer, Slice* sidecars) {
+  if (offsets.size() == 0) return Status::OK();
+
+  int last = offsets.size() - 1;
+  if (last >= TransferLimits::kMaxSidecars) {
+    return Status::Corruption(strings::Substitute(
+            "Received $0 additional payload slices, expected at most $1",
+            last, TransferLimits::kMaxSidecars));
+  }
+
+  if (buffer.size() > TransferLimits::kMaxTotalSidecarBytes) {
+    return Status::Corruption(strings::Substitute(
+            "Received $0 payload bytes, expected at most $1",
+            buffer.size(), TransferLimits::kMaxTotalSidecarBytes));
+  }
+
+  for (int i = 0; i < last; ++i) {
+    int64_t cur_offset = offsets.Get(i);
+    int64_t next_offset = offsets.Get(i + 1);
+    if (next_offset > buffer.size()) {
+      return Status::Corruption(strings::Substitute(
+              "Invalid sidecar offsets; sidecar $0 apparently starts at $1,"
+              " has length $2, but the entire message has length $3",
+              i, cur_offset, (next_offset - cur_offset), buffer.size()));
+    }
+    if (next_offset < cur_offset) {
+      return Status::Corruption(strings::Substitute(
+              "Invalid sidecar offsets; sidecar $0 apparently starts at $1,"
+              " but ends before that at offset $1.", i, cur_offset, next_offset));
+    }
+
+    sidecars[i] = Slice(buffer.data() + cur_offset, next_offset - cur_offset);
+  }
+
+  int64_t cur_offset = offsets.Get(last);
+  if (cur_offset > buffer.size()) {
+    return Status::Corruption(strings::Substitute("Invalid sidecar offsets: sidecar $0 "
+            "starts at offset $1after message ends (message length $2).", last,
+            cur_offset, buffer.size()));
+  }
+  sidecars[last] = Slice(buffer.data() + cur_offset, buffer.size() - cur_offset);
+
+  return Status::OK();
+}
+
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/rpc/rpc_sidecar.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/rpc/rpc_sidecar.h b/be/src/kudu/rpc/rpc_sidecar.h
new file mode 100644
index 0000000..bfbfcea
--- /dev/null
+++ b/be/src/kudu/rpc/rpc_sidecar.h
@@ -0,0 +1,73 @@
+// 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.
+#ifndef KUDU_RPC_RPC_SIDECAR_H
+#define KUDU_RPC_RPC_SIDECAR_H
+
+#include <memory>
+
+#include <google/protobuf/repeated_field.h> // IWYU pragma: keep
+#include <google/protobuf/stubs/port.h>
+
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+class Status;
+class faststring;
+
+namespace rpc {
+
+// An RpcSidecar is a mechanism which allows replies to RPCs to reference blocks of data
+// without extra copies. In other words, whenever a protobuf would have a large field
+// where additional copies become expensive, one may opt instead to use an RpcSidecar.
+//
+// The RpcSidecar saves on an additional copy to/from the protobuf on both the server and
+// client side. Both Inbound- and OutboundCall classes accept sidecars to be sent to the
+// client and server respectively. They are ignorant of the sidecar's format, requiring
+// only that it can be represented as a Slice. Data is copied from the Slice returned from
+// AsSlice() to the socket that is responding to the original RPC. The slice should remain
+// valid for as long as the call it is attached to takes to complete.
+//
+// In order to distinguish between separate sidecars, whenever a sidecar is added to the
+// RPC response on the server side, an index for that sidecar is returned. This index must
+// then in some way (i.e., via protobuf) be communicated to the recipient.
+//
+// After reconstructing the array of sidecars, servers and clients may retrieve the
+// sidecar data through the RpcContext or RpcController interfaces respectively.
+class RpcSidecar {
+ public:
+  static std::unique_ptr<RpcSidecar> FromFaststring(std::unique_ptr<faststring> data);
+  static std::unique_ptr<RpcSidecar> FromSlice(Slice slice);
+
+  // Utility method to parse a series of sidecar slices into 'sidecars' from 'buffer' and
+  // a set of offsets. 'sidecars' must have length >= TransferLimits::kMaxSidecars, and
+  // will be filled from index 0.
+  // TODO(henryr): Consider a vector instead here if there's no perf. impact.
+  static Status ParseSidecars(
+      const ::google::protobuf::RepeatedField<::google::protobuf::uint32>& offsets,
+      Slice buffer, Slice* sidecars);
+
+  // Returns a Slice representation of the sidecar's data.
+  virtual Slice AsSlice() const = 0;
+  virtual ~RpcSidecar() { }
+};
+
+} // namespace rpc
+} // namespace kudu
+
+
+#endif /* KUDU_RPC_RPC_SIDECAR_H */


[25/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/debug/trace_event_impl.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/debug/trace_event_impl.cc b/be/src/kudu/util/debug/trace_event_impl.cc
new file mode 100644
index 0000000..155faf7
--- /dev/null
+++ b/be/src/kudu/util/debug/trace_event_impl.cc
@@ -0,0 +1,2436 @@
+// Copyright (c) 2012 The Chromium Authors. All rights reserved.
+//
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "kudu/util/debug/trace_event_impl.h"
+
+#include <sched.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cinttypes>
+#include <cstdlib>
+#include <cstring>
+#include <list>
+#include <sstream>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/dynamic_annotations.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/mathlimits.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted_memory.h"
+#include "kudu/gutil/singleton.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/gutil/walltime.h"
+
+#include "kudu/util/atomic.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/debug/trace_event_synthetic_delay.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadlocal.h"
+
+DEFINE_string(trace_to_console, "",
+              "Trace pattern specifying which trace events should be dumped "
+              "directly to the console");
+TAG_FLAG(trace_to_console, experimental);
+
+// The thread buckets for the sampling profiler.
+BASE_EXPORT TRACE_EVENT_API_ATOMIC_WORD g_trace_state[3];
+
+namespace kudu {
+namespace debug {
+
+using base::SpinLockHolder;
+
+using strings::SubstituteAndAppend;
+using std::string;
+using std::vector;
+
+__thread TraceLog::PerThreadInfo* TraceLog::thread_local_info_ = nullptr;
+
+namespace {
+
+// Controls the number of trace events we will buffer in-memory
+// before throwing them away.
+const size_t kTraceBufferChunkSize = TraceBufferChunk::kTraceBufferChunkSize;
+const size_t kTraceEventVectorBufferChunks = 256000 / kTraceBufferChunkSize;
+const size_t kTraceEventRingBufferChunks = kTraceEventVectorBufferChunks / 4;
+const size_t kTraceEventBatchChunks = 1000 / kTraceBufferChunkSize;
+// Can store results for 30 seconds with 1 ms sampling interval.
+const size_t kMonitorTraceEventBufferChunks = 30000 / kTraceBufferChunkSize;
+// ECHO_TO_CONSOLE needs a small buffer to hold the unfinished COMPLETE events.
+const size_t kEchoToConsoleTraceEventBufferChunks = 256;
+
+const char kSyntheticDelayCategoryFilterPrefix[] = "DELAY(";
+
+#define MAX_CATEGORY_GROUPS 100
+
+// Parallel arrays g_category_groups and g_category_group_enabled are separate
+// so that a pointer to a member of g_category_group_enabled can be easily
+// converted to an index into g_category_groups. This allows macros to deal
+// only with char enabled pointers from g_category_group_enabled, and we can
+// convert internally to determine the category name from the char enabled
+// pointer.
+const char* g_category_groups[MAX_CATEGORY_GROUPS] = {
+  "toplevel",
+  "tracing already shutdown",
+  "tracing categories exhausted; must increase MAX_CATEGORY_GROUPS",
+  "__metadata"};
+
+// The enabled flag is char instead of bool so that the API can be used from C.
+unsigned char g_category_group_enabled[MAX_CATEGORY_GROUPS] = { 0 };
+// Indexes here have to match the g_category_groups array indexes above.
+const int kCategoryAlreadyShutdown = 1;
+const int kCategoryCategoriesExhausted = 2;
+const int kCategoryMetadata = 3;
+const int kNumBuiltinCategories = 4;
+// Skip default categories.
+AtomicWord g_category_index = kNumBuiltinCategories;
+
+// The name of the current thread. This is used to decide if the current
+// thread name has changed. We combine all the seen thread names into the
+// output name for the thread.
+__thread const char* g_current_thread_name = "";
+
+static void NOTIMPLEMENTED() {
+  LOG(FATAL);
+}
+
+class TraceBufferRingBuffer : public TraceBuffer {
+ public:
+  explicit TraceBufferRingBuffer(size_t max_chunks)
+      : max_chunks_(max_chunks),
+        recyclable_chunks_queue_(new size_t[queue_capacity()]),
+        queue_head_(0),
+        queue_tail_(max_chunks),
+        current_iteration_index_(0),
+        current_chunk_seq_(1) {
+    chunks_.reserve(max_chunks);
+    for (size_t i = 0; i < max_chunks; ++i)
+      recyclable_chunks_queue_[i] = i;
+  }
+
+  ~TraceBufferRingBuffer() {
+    STLDeleteElements(&chunks_);
+  }
+
+  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+    // Because the number of threads is much less than the number of chunks,
+    // the queue should never be empty.
+    DCHECK(!QueueIsEmpty());
+
+    *index = recyclable_chunks_queue_[queue_head_];
+    queue_head_ = NextQueueIndex(queue_head_);
+    current_iteration_index_ = queue_head_;
+
+    if (*index >= chunks_.size())
+      chunks_.resize(*index + 1);
+
+    TraceBufferChunk* chunk = chunks_[*index];
+    chunks_[*index] = nullptr;  // Put NULL in the slot of a in-flight chunk.
+    if (chunk)
+      chunk->Reset(current_chunk_seq_++);
+    else
+      chunk = new TraceBufferChunk(current_chunk_seq_++);
+
+    return gscoped_ptr<TraceBufferChunk>(chunk);
+  }
+
+  virtual void ReturnChunk(size_t index,
+                           gscoped_ptr<TraceBufferChunk> chunk) OVERRIDE {
+    // When this method is called, the queue should not be full because it
+    // can contain all chunks including the one to be returned.
+    DCHECK(!QueueIsFull());
+    DCHECK(chunk);
+    DCHECK_LT(index, chunks_.size());
+    DCHECK(!chunks_[index]);
+    chunks_[index] = chunk.release();
+    recyclable_chunks_queue_[queue_tail_] = index;
+    queue_tail_ = NextQueueIndex(queue_tail_);
+  }
+
+  virtual bool IsFull() const OVERRIDE {
+    return false;
+  }
+
+  virtual size_t Size() const OVERRIDE {
+    // This is approximate because not all of the chunks are full.
+    return chunks_.size() * kTraceBufferChunkSize;
+  }
+
+  virtual size_t Capacity() const OVERRIDE {
+    return max_chunks_ * kTraceBufferChunkSize;
+  }
+
+  virtual TraceEvent* GetEventByHandle(TraceEventHandle handle) OVERRIDE {
+    if (handle.chunk_index >= chunks_.size())
+      return nullptr;
+    TraceBufferChunk* chunk = chunks_[handle.chunk_index];
+    if (!chunk || chunk->seq() != handle.chunk_seq)
+      return nullptr;
+    return chunk->GetEventAt(handle.event_index);
+  }
+
+  virtual const TraceBufferChunk* NextChunk() OVERRIDE {
+    if (chunks_.empty())
+      return nullptr;
+
+    while (current_iteration_index_ != queue_tail_) {
+      size_t chunk_index = recyclable_chunks_queue_[current_iteration_index_];
+      current_iteration_index_ = NextQueueIndex(current_iteration_index_);
+      if (chunk_index >= chunks_.size()) // Skip uninitialized chunks.
+        continue;
+      DCHECK(chunks_[chunk_index]);
+      return chunks_[chunk_index];
+    }
+    return nullptr;
+  }
+
+  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+    gscoped_ptr<ClonedTraceBuffer> cloned_buffer(new ClonedTraceBuffer());
+    for (size_t queue_index = queue_head_; queue_index != queue_tail_;
+        queue_index = NextQueueIndex(queue_index)) {
+      size_t chunk_index = recyclable_chunks_queue_[queue_index];
+      if (chunk_index >= chunks_.size()) // Skip uninitialized chunks.
+        continue;
+      TraceBufferChunk* chunk = chunks_[chunk_index];
+      cloned_buffer->chunks_.push_back(chunk ? chunk->Clone().release() : nullptr);
+    }
+    return cloned_buffer.PassAs<TraceBuffer>();
+  }
+
+ private:
+  class ClonedTraceBuffer : public TraceBuffer {
+   public:
+    ClonedTraceBuffer() : current_iteration_index_(0) {}
+    ~ClonedTraceBuffer() {
+      STLDeleteElements(&chunks_);
+    }
+
+    // The only implemented method.
+    virtual const TraceBufferChunk* NextChunk() OVERRIDE {
+      return current_iteration_index_ < chunks_.size() ?
+          chunks_[current_iteration_index_++] : nullptr;
+    }
+
+    virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+      NOTIMPLEMENTED();
+      return gscoped_ptr<TraceBufferChunk>();
+    }
+    virtual void ReturnChunk(size_t index,
+                             gscoped_ptr<TraceBufferChunk>) OVERRIDE {
+      NOTIMPLEMENTED();
+    }
+    virtual bool IsFull() const OVERRIDE { return false; }
+    virtual size_t Size() const OVERRIDE { return 0; }
+    virtual size_t Capacity() const OVERRIDE { return 0; }
+    virtual TraceEvent* GetEventByHandle(TraceEventHandle handle) OVERRIDE {
+      return nullptr;
+    }
+    virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+      NOTIMPLEMENTED();
+      return gscoped_ptr<TraceBuffer>();
+    }
+
+    size_t current_iteration_index_;
+    vector<TraceBufferChunk*> chunks_;
+  };
+
+  bool QueueIsEmpty() const {
+    return queue_head_ == queue_tail_;
+  }
+
+  size_t QueueSize() const {
+    return queue_tail_ > queue_head_ ? queue_tail_ - queue_head_ :
+        queue_tail_ + queue_capacity() - queue_head_;
+  }
+
+  bool QueueIsFull() const {
+    return QueueSize() == queue_capacity() - 1;
+  }
+
+  size_t queue_capacity() const {
+    // One extra space to help distinguish full state and empty state.
+    return max_chunks_ + 1;
+  }
+
+  size_t NextQueueIndex(size_t index) const {
+    index++;
+    if (index >= queue_capacity())
+      index = 0;
+    return index;
+  }
+
+  size_t max_chunks_;
+  vector<TraceBufferChunk*> chunks_;
+
+  gscoped_ptr<size_t[]> recyclable_chunks_queue_;
+  size_t queue_head_;
+  size_t queue_tail_;
+
+  size_t current_iteration_index_;
+  uint32_t current_chunk_seq_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceBufferRingBuffer);
+};
+
+class TraceBufferVector : public TraceBuffer {
+ public:
+  TraceBufferVector()
+      : in_flight_chunk_count_(0),
+        current_iteration_index_(0) {
+    chunks_.reserve(kTraceEventVectorBufferChunks);
+  }
+  ~TraceBufferVector() {
+    STLDeleteElements(&chunks_);
+  }
+
+  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+    // This function may be called when adding normal events or indirectly from
+    // AddMetadataEventsWhileLocked(). We can not DECHECK(!IsFull()) because we
+    // have to add the metadata events and flush thread-local buffers even if
+    // the buffer is full.
+    *index = chunks_.size();
+    chunks_.push_back(nullptr);  // Put NULL in the slot of a in-flight chunk.
+    ++in_flight_chunk_count_;
+    // + 1 because zero chunk_seq is not allowed.
+    return gscoped_ptr<TraceBufferChunk>(
+        new TraceBufferChunk(static_cast<uint32_t>(*index) + 1));
+  }
+
+  virtual void ReturnChunk(size_t index,
+                           gscoped_ptr<TraceBufferChunk> chunk) OVERRIDE {
+    DCHECK_GT(in_flight_chunk_count_, 0u);
+    DCHECK_LT(index, chunks_.size());
+    DCHECK(!chunks_[index]);
+    --in_flight_chunk_count_;
+    chunks_[index] = chunk.release();
+  }
+
+  virtual bool IsFull() const OVERRIDE {
+    return chunks_.size() >= kTraceEventVectorBufferChunks;
+  }
+
+  virtual size_t Size() const OVERRIDE {
+    // This is approximate because not all of the chunks are full.
+    return chunks_.size() * kTraceBufferChunkSize;
+  }
+
+  virtual size_t Capacity() const OVERRIDE {
+    return kTraceEventVectorBufferChunks * kTraceBufferChunkSize;
+  }
+
+  virtual TraceEvent* GetEventByHandle(TraceEventHandle handle) OVERRIDE {
+    if (handle.chunk_index >= chunks_.size())
+      return nullptr;
+    TraceBufferChunk* chunk = chunks_[handle.chunk_index];
+    if (!chunk || chunk->seq() != handle.chunk_seq)
+      return nullptr;
+    return chunk->GetEventAt(handle.event_index);
+  }
+
+  virtual const TraceBufferChunk* NextChunk() OVERRIDE {
+    while (current_iteration_index_ < chunks_.size()) {
+      // Skip in-flight chunks.
+      const TraceBufferChunk* chunk = chunks_[current_iteration_index_++];
+      if (chunk)
+        return chunk;
+    }
+    return nullptr;
+  }
+
+  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+    NOTIMPLEMENTED();
+    return gscoped_ptr<TraceBuffer>();
+  }
+
+ private:
+  size_t in_flight_chunk_count_;
+  size_t current_iteration_index_;
+  vector<TraceBufferChunk*> chunks_;
+
+  DISALLOW_COPY_AND_ASSIGN(TraceBufferVector);
+};
+
+template <typename T>
+void InitializeMetadataEvent(TraceEvent* trace_event,
+                             int thread_id,
+                             const char* metadata_name, const char* arg_name,
+                             const T& value) {
+  if (!trace_event)
+    return;
+
+  int num_args = 1;
+  unsigned char arg_type;
+  uint64_t arg_value;
+  ::trace_event_internal::SetTraceValue(value, &arg_type, &arg_value);
+  trace_event->Initialize(thread_id,
+                          MicrosecondsInt64(0), MicrosecondsInt64(0), TRACE_EVENT_PHASE_METADATA,
+                          &g_category_group_enabled[kCategoryMetadata],
+                          metadata_name, ::trace_event_internal::kNoEventId,
+                          num_args, &arg_name, &arg_type, &arg_value, nullptr,
+                          TRACE_EVENT_FLAG_NONE);
+}
+
+// RAII object which marks '*dst' with a non-zero value while in scope.
+// This assumes that no other threads write to '*dst'.
+class MarkFlagInScope {
+ public:
+  explicit MarkFlagInScope(Atomic32* dst)
+      : dst_(dst) {
+    // We currently use Acquire_AtomicExchange here because it appears
+    // to be the cheapest way of getting an "Acquire_Store" barrier. Actually
+    // using Acquire_Store generates more assembly instructions and benchmarks
+    // slightly slower.
+    //
+    // TODO: it would be even faster to avoid the memory barrier here entirely,
+    // and do an asymmetric barrier, for example by having the flusher thread
+    // send a signal to every registered thread, or wait until every other thread
+    // has experienced at least one context switch. A number of options for this
+    // are outlined in:
+    // http://home.comcast.net/~pjbishop/Dave/Asymmetric-Dekker-Synchronization.txt
+    Atomic32 old_val = base::subtle::Acquire_AtomicExchange(dst_, 1);
+    DCHECK_EQ(old_val, 0);
+  }
+  ~MarkFlagInScope() {
+    base::subtle::Release_Store(dst_, 0);
+  }
+
+ private:
+  Atomic32* dst_;
+  DISALLOW_COPY_AND_ASSIGN(MarkFlagInScope);
+};
+}  // anonymous namespace
+
+TraceLog::ThreadLocalEventBuffer* TraceLog::PerThreadInfo::AtomicTakeBuffer() {
+  return reinterpret_cast<TraceLog::ThreadLocalEventBuffer*>(
+    base::subtle::Acquire_AtomicExchange(
+      reinterpret_cast<AtomicWord*>(&event_buffer_),
+      0));
+}
+
+void TraceBufferChunk::Reset(uint32_t new_seq) {
+  for (size_t i = 0; i < next_free_; ++i)
+    chunk_[i].Reset();
+  next_free_ = 0;
+  seq_ = new_seq;
+}
+
+TraceEvent* TraceBufferChunk::AddTraceEvent(size_t* event_index) {
+  DCHECK(!IsFull());
+  *event_index = next_free_++;
+  return &chunk_[*event_index];
+}
+
+gscoped_ptr<TraceBufferChunk> TraceBufferChunk::Clone() const {
+  gscoped_ptr<TraceBufferChunk> cloned_chunk(new TraceBufferChunk(seq_));
+  cloned_chunk->next_free_ = next_free_;
+  for (size_t i = 0; i < next_free_; ++i)
+    cloned_chunk->chunk_[i].CopyFrom(chunk_[i]);
+  return std::move(cloned_chunk);
+}
+
+// A helper class that allows the lock to be acquired in the middle of the scope
+// and unlocks at the end of scope if locked.
+class TraceLog::OptionalAutoLock {
+ public:
+  explicit OptionalAutoLock(base::SpinLock& lock) // NOLINT(google-runtime-references)
+      : lock_(lock),
+        locked_(false) {
+  }
+
+  ~OptionalAutoLock() {
+    if (locked_)
+      lock_.Unlock();
+  }
+
+  void EnsureAcquired() {
+    if (!locked_) {
+      lock_.Lock();
+      locked_ = true;
+    }
+  }
+
+ private:
+  base::SpinLock& lock_;
+  bool locked_;
+  DISALLOW_COPY_AND_ASSIGN(OptionalAutoLock);
+};
+
+// Use this function instead of TraceEventHandle constructor to keep the
+// overhead of ScopedTracer (trace_event.h) constructor minimum.
+void MakeHandle(uint32_t chunk_seq, size_t chunk_index, size_t event_index,
+                TraceEventHandle* handle) {
+  DCHECK(chunk_seq);
+  DCHECK(chunk_index < (1u << 16));
+  DCHECK(event_index < (1u << 16));
+  handle->chunk_seq = chunk_seq;
+  handle->chunk_index = static_cast<uint16_t>(chunk_index);
+  handle->event_index = static_cast<uint16_t>(event_index);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+//
+// TraceEvent
+//
+////////////////////////////////////////////////////////////////////////////////
+
+namespace {
+
+size_t GetAllocLength(const char* str) { return str ? strlen(str) + 1 : 0; }
+
+// Copies |*member| into |*buffer|, sets |*member| to point to this new
+// location, and then advances |*buffer| by the amount written.
+void CopyTraceEventParameter(char** buffer,
+                             const char** member,
+                             const char* end) {
+  if (*member) {
+    size_t written = strings::strlcpy(*buffer, *member, end - *buffer) + 1;
+    DCHECK_LE(static_cast<int>(written), end - *buffer);
+    *member = *buffer;
+    *buffer += written;
+  }
+}
+
+}  // namespace
+
+TraceEvent::TraceEvent()
+    : duration_(-1),
+      thread_duration_(-1),
+      id_(0u),
+      category_group_enabled_(nullptr),
+      name_(nullptr),
+      thread_id_(0),
+      phase_(TRACE_EVENT_PHASE_BEGIN),
+      flags_(0) {
+  for (auto& arg_name : arg_names_) {
+    arg_name = nullptr;
+  }
+  memset(arg_values_, 0, sizeof(arg_values_));
+}
+
+TraceEvent::~TraceEvent() {
+}
+
+void TraceEvent::CopyFrom(const TraceEvent& other) {
+  timestamp_ = other.timestamp_;
+  thread_timestamp_ = other.thread_timestamp_;
+  duration_ = other.duration_;
+  id_ = other.id_;
+  category_group_enabled_ = other.category_group_enabled_;
+  name_ = other.name_;
+  thread_id_ = other.thread_id_;
+  phase_ = other.phase_;
+  flags_ = other.flags_;
+  parameter_copy_storage_ = other.parameter_copy_storage_;
+
+  for (int i = 0; i < kTraceMaxNumArgs; ++i) {
+    arg_names_[i] = other.arg_names_[i];
+    arg_types_[i] = other.arg_types_[i];
+    arg_values_[i] = other.arg_values_[i];
+    convertable_values_[i] = other.convertable_values_[i];
+  }
+}
+
+void TraceEvent::Initialize(
+    int thread_id,
+    MicrosecondsInt64 timestamp,
+    MicrosecondsInt64 thread_timestamp,
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int num_args,
+    const char** arg_names,
+    const unsigned char* arg_types,
+    const uint64_t* arg_values,
+    const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+    unsigned char flags) {
+  timestamp_ = timestamp;
+  thread_timestamp_ = thread_timestamp;
+  duration_ = -1;;
+  id_ = id;
+  category_group_enabled_ = category_group_enabled;
+  name_ = name;
+  thread_id_ = thread_id;
+  phase_ = phase;
+  flags_ = flags;
+
+  // Clamp num_args since it may have been set by a third_party library.
+  num_args = (num_args > kTraceMaxNumArgs) ? kTraceMaxNumArgs : num_args;
+  int i = 0;
+  for (; i < num_args; ++i) {
+    arg_names_[i] = arg_names[i];
+    arg_types_[i] = arg_types[i];
+
+    if (arg_types[i] == TRACE_VALUE_TYPE_CONVERTABLE)
+      convertable_values_[i] = convertable_values[i];
+    else
+      arg_values_[i].as_uint = arg_values[i];
+  }
+  for (; i < kTraceMaxNumArgs; ++i) {
+    arg_names_[i] = nullptr;
+    arg_values_[i].as_uint = 0u;
+    convertable_values_[i] = nullptr;
+    arg_types_[i] = TRACE_VALUE_TYPE_UINT;
+  }
+
+  bool copy = !!(flags & TRACE_EVENT_FLAG_COPY);
+  size_t alloc_size = 0;
+  if (copy) {
+    alloc_size += GetAllocLength(name);
+    for (i = 0; i < num_args; ++i) {
+      alloc_size += GetAllocLength(arg_names_[i]);
+      if (arg_types_[i] == TRACE_VALUE_TYPE_STRING)
+        arg_types_[i] = TRACE_VALUE_TYPE_COPY_STRING;
+    }
+  }
+
+  bool arg_is_copy[kTraceMaxNumArgs];
+  for (i = 0; i < num_args; ++i) {
+    // No copying of convertable types, we retain ownership.
+    if (arg_types_[i] == TRACE_VALUE_TYPE_CONVERTABLE)
+      continue;
+
+    // We only take a copy of arg_vals if they are of type COPY_STRING.
+    arg_is_copy[i] = (arg_types_[i] == TRACE_VALUE_TYPE_COPY_STRING);
+    if (arg_is_copy[i])
+      alloc_size += GetAllocLength(arg_values_[i].as_string);
+  }
+
+  if (alloc_size) {
+    parameter_copy_storage_ = new RefCountedString;
+    parameter_copy_storage_->data().resize(alloc_size);
+    char* ptr = string_as_array(&parameter_copy_storage_->data());
+    const char* end = ptr + alloc_size;
+    if (copy) {
+      CopyTraceEventParameter(&ptr, &name_, end);
+      for (i = 0; i < num_args; ++i) {
+        CopyTraceEventParameter(&ptr, &arg_names_[i], end);
+      }
+    }
+    for (i = 0; i < num_args; ++i) {
+      if (arg_types_[i] == TRACE_VALUE_TYPE_CONVERTABLE)
+        continue;
+      if (arg_is_copy[i])
+        CopyTraceEventParameter(&ptr, &arg_values_[i].as_string, end);
+    }
+    DCHECK_EQ(end, ptr) << "Overrun by " << ptr - end;
+  }
+}
+
+void TraceEvent::Reset() {
+  // Only reset fields that won't be initialized in Initialize(), or that may
+  // hold references to other objects.
+  duration_ = -1;;
+  parameter_copy_storage_ = nullptr;
+  for (int i = 0; i < kTraceMaxNumArgs && arg_names_[i]; ++i)
+    convertable_values_[i] = nullptr;
+}
+
+void TraceEvent::UpdateDuration(const MicrosecondsInt64& now,
+                                const MicrosecondsInt64& thread_now) {
+  DCHECK(duration_ == -1);
+  duration_ = now - timestamp_;
+  thread_duration_ = thread_now - thread_timestamp_;
+}
+
+namespace {
+// Escape the given string using JSON rules.
+void JsonEscape(StringPiece s, string* out) {
+  out->reserve(out->size() + s.size() * 2);
+  const char* p_end = s.data() + s.size();
+  for (const char* p = s.data(); p != p_end; p++) {
+    // Only the following characters need to be escaped, according to json.org.
+    // In particular, it's illegal to escape the single-quote character, and
+    // JSON does not support the "\x" escape sequence like C/Java.
+    switch (*p) {
+      case '"':
+      case '\\':
+        out->push_back('\\');
+        out->push_back(*p);
+        break;
+      case '\b':
+        out->append("\\b");
+        break;
+      case '\f':
+        out->append("\\f");
+        break;
+      case '\n':
+        out->append("\\n");
+      case '\r':
+        out->append("\\r");
+        break;
+      case '\t':
+        out->append("\\t");
+        break;
+      default:
+        out->push_back(*p);
+    }
+  }
+}
+} // anonymous namespace
+
+// static
+void TraceEvent::AppendValueAsJSON(unsigned char type,
+                                   TraceEvent::TraceValue value,
+                                   std::string* out) {
+  switch (type) {
+    case TRACE_VALUE_TYPE_BOOL:
+      *out += value.as_bool ? "true" : "false";
+      break;
+    case TRACE_VALUE_TYPE_UINT:
+      SubstituteAndAppend(out, "$0", static_cast<uint64_t>(value.as_uint));
+      break;
+    case TRACE_VALUE_TYPE_INT:
+      SubstituteAndAppend(out, "$0", static_cast<int64_t>(value.as_int));
+      break;
+    case TRACE_VALUE_TYPE_DOUBLE: {
+      // FIXME: base/json/json_writer.cc is using the same code,
+      //        should be made into a common method.
+      std::string real;
+      double val = value.as_double;
+      if (MathLimits<double>::IsFinite(val)) {
+        real = strings::Substitute("$0", val);
+        // Ensure that the number has a .0 if there's no decimal or 'e'.  This
+        // makes sure that when we read the JSON back, it's interpreted as a
+        // real rather than an int.
+        if (real.find('.') == std::string::npos &&
+            real.find('e') == std::string::npos &&
+            real.find('E') == std::string::npos) {
+          real.append(".0");
+        }
+        // The JSON spec requires that non-integer values in the range (-1,1)
+        // have a zero before the decimal point - ".52" is not valid, "0.52" is.
+        if (real[0] == '.') {
+          real.insert(0, "0");
+        } else if (real.length() > 1 && real[0] == '-' && real[1] == '.') {
+          // "-.1" bad "-0.1" good
+          real.insert(1, "0");
+        }
+      } else if (MathLimits<double>::IsNaN(val)){
+        // The JSON spec doesn't allow NaN and Infinity (since these are
+        // objects in EcmaScript).  Use strings instead.
+        real = "\"NaN\"";
+      } else if (val < 0) {
+        real = "\"-Infinity\"";
+      } else {
+        real = "\"Infinity\"";
+      }
+      SubstituteAndAppend(out, "$0", real);
+      break;
+    }
+    case TRACE_VALUE_TYPE_POINTER:
+      // JSON only supports double and int numbers.
+      // So as not to lose bits from a 64-bit pointer, output as a hex string.
+      StringAppendF(out, "\"0x%" PRIx64 "\"", static_cast<uint64_t>(
+                                     reinterpret_cast<intptr_t>(
+                                     value.as_pointer)));
+      break;
+    case TRACE_VALUE_TYPE_STRING:
+    case TRACE_VALUE_TYPE_COPY_STRING:
+      *out += "\"";
+      JsonEscape(value.as_string ? value.as_string : "NULL", out);
+      *out += "\"";
+      break;
+    default:
+      LOG(FATAL) << "Don't know how to print this value";
+      break;
+  }
+}
+
+void TraceEvent::AppendAsJSON(std::string* out) const {
+  int64_t time_int64 = timestamp_;
+  int process_id = TraceLog::GetInstance()->process_id();
+  // Category group checked at category creation time.
+  DCHECK(!strchr(name_, '"'));
+  StringAppendF(out,
+      "{\"cat\":\"%s\",\"pid\":%i,\"tid\":%i,\"ts\":%" PRId64 ","
+      "\"ph\":\"%c\",\"name\":\"%s\",\"args\":{",
+      TraceLog::GetCategoryGroupName(category_group_enabled_),
+      process_id,
+      thread_id_,
+      time_int64,
+      phase_,
+      name_);
+
+  // Output argument names and values, stop at first NULL argument name.
+  for (int i = 0; i < kTraceMaxNumArgs && arg_names_[i]; ++i) {
+    if (i > 0)
+      *out += ",";
+    *out += "\"";
+    *out += arg_names_[i];
+    *out += "\":";
+
+    if (arg_types_[i] == TRACE_VALUE_TYPE_CONVERTABLE)
+      convertable_values_[i]->AppendAsTraceFormat(out);
+    else
+      AppendValueAsJSON(arg_types_[i], arg_values_[i], out);
+  }
+  *out += "}";
+
+  if (phase_ == TRACE_EVENT_PHASE_COMPLETE) {
+    int64_t duration = duration_;
+    if (duration != -1)
+      StringAppendF(out, ",\"dur\":%" PRId64, duration);
+    if (thread_timestamp_ >= 0) {
+      int64_t thread_duration = thread_duration_;
+      if (thread_duration != -1)
+        StringAppendF(out, ",\"tdur\":%" PRId64, thread_duration);
+    }
+  }
+
+  // Output tts if thread_timestamp is valid.
+  if (thread_timestamp_ >= 0) {
+    int64_t thread_time_int64 = thread_timestamp_;
+    StringAppendF(out, ",\"tts\":%" PRId64, thread_time_int64);
+  }
+
+  // If id_ is set, print it out as a hex string so we don't loose any
+  // bits (it might be a 64-bit pointer).
+  if (flags_ & TRACE_EVENT_FLAG_HAS_ID)
+    StringAppendF(out, ",\"id\":\"0x%" PRIx64 "\"", static_cast<uint64_t>(id_));
+
+  // Instant events also output their scope.
+  if (phase_ == TRACE_EVENT_PHASE_INSTANT) {
+    char scope = '?';
+    switch (flags_ & TRACE_EVENT_FLAG_SCOPE_MASK) {
+      case TRACE_EVENT_SCOPE_GLOBAL:
+        scope = TRACE_EVENT_SCOPE_NAME_GLOBAL;
+        break;
+
+      case TRACE_EVENT_SCOPE_PROCESS:
+        scope = TRACE_EVENT_SCOPE_NAME_PROCESS;
+        break;
+
+      case TRACE_EVENT_SCOPE_THREAD:
+        scope = TRACE_EVENT_SCOPE_NAME_THREAD;
+        break;
+    }
+    StringAppendF(out, ",\"s\":\"%c\"", scope);
+  }
+
+  *out += "}";
+}
+
+void TraceEvent::AppendPrettyPrinted(std::ostringstream* out) const {
+  *out << name_ << "[";
+  *out << TraceLog::GetCategoryGroupName(category_group_enabled_);
+  *out << "]";
+  if (arg_names_[0]) {
+    *out << ", {";
+    for (int i = 0; i < kTraceMaxNumArgs && arg_names_[i]; ++i) {
+      if (i > 0)
+        *out << ", ";
+      *out << arg_names_[i] << ":";
+      std::string value_as_text;
+
+      if (arg_types_[i] == TRACE_VALUE_TYPE_CONVERTABLE)
+        convertable_values_[i]->AppendAsTraceFormat(&value_as_text);
+      else
+        AppendValueAsJSON(arg_types_[i], arg_values_[i], &value_as_text);
+
+      *out << value_as_text;
+    }
+    *out << "}";
+  }
+}
+
+////////////////////////////////////////////////////////////////////////////////
+//
+// TraceResultBuffer
+//
+////////////////////////////////////////////////////////////////////////////////
+
+string TraceResultBuffer::FlushTraceLogToString() {
+  return DoFlush(false);
+}
+
+string TraceResultBuffer::FlushTraceLogToStringButLeaveBufferIntact() {
+  return DoFlush(true);
+}
+
+string TraceResultBuffer::DoFlush(bool leave_intact) {
+  TraceResultBuffer buf;
+  TraceLog* tl = TraceLog::GetInstance();
+  if (leave_intact) {
+    tl->FlushButLeaveBufferIntact(Bind(&TraceResultBuffer::Collect, Unretained(&buf)));
+  } else {
+    tl->Flush(Bind(&TraceResultBuffer::Collect, Unretained(&buf)));
+  }
+  buf.json_.append("]}\n");
+  return buf.json_;
+}
+
+TraceResultBuffer::TraceResultBuffer()
+  : first_(true) {
+}
+TraceResultBuffer::~TraceResultBuffer() {
+}
+
+void TraceResultBuffer::Collect(
+  const scoped_refptr<RefCountedString>& s,
+  bool has_more_events) {
+  if (first_) {
+    json_.append("{\"traceEvents\": [\n");
+    first_ = false;
+  } else if (!s->data().empty()) {
+    // Sometimes we get sent an empty chunk at the end,
+    // and we don't want to end up with an extra trailing ','
+    json_.append(",\n");
+  }
+  json_.append(s->data());
+}
+
+////////////////////////////////////////////////////////////////////////////////
+//
+// TraceSamplingThread
+//
+////////////////////////////////////////////////////////////////////////////////
+class TraceBucketData;
+typedef Callback<void(TraceBucketData*)> TraceSampleCallback;
+
+class TraceBucketData {
+ public:
+  TraceBucketData(AtomicWord* bucket,
+                  const char* name,
+                  TraceSampleCallback callback);
+  ~TraceBucketData();
+
+  TRACE_EVENT_API_ATOMIC_WORD* bucket;
+  const char* bucket_name;
+  TraceSampleCallback callback;
+};
+
+// This object must be created on the IO thread.
+class TraceSamplingThread {
+ public:
+  TraceSamplingThread();
+  virtual ~TraceSamplingThread();
+
+  void ThreadMain();
+
+  static void DefaultSamplingCallback(TraceBucketData* bucekt_data);
+
+  void Stop();
+
+ private:
+  friend class TraceLog;
+
+  void GetSamples();
+  // Not thread-safe. Once the ThreadMain has been called, this can no longer
+  // be called.
+  void RegisterSampleBucket(TRACE_EVENT_API_ATOMIC_WORD* bucket,
+                            const char* const name,
+                            TraceSampleCallback callback);
+  // Splits a combined "category\0name" into the two component parts.
+  static void ExtractCategoryAndName(const char* combined,
+                                     const char** category,
+                                     const char** name);
+  std::vector<TraceBucketData> sample_buckets_;
+  bool thread_running_;
+  AtomicBool cancellation_flag_;
+};
+
+
+TraceSamplingThread::TraceSamplingThread()
+  : thread_running_(false),
+    cancellation_flag_(false) {
+}
+
+TraceSamplingThread::~TraceSamplingThread() {
+}
+
+void TraceSamplingThread::ThreadMain() {
+  thread_running_ = true;
+  const MonoDelta sleepDelta = MonoDelta::FromMicroseconds(1000);
+  while (!cancellation_flag_.Load()) {
+    SleepFor(sleepDelta);
+    GetSamples();
+  }
+}
+
+// static
+void TraceSamplingThread::DefaultSamplingCallback(
+    TraceBucketData* bucket_data) {
+  TRACE_EVENT_API_ATOMIC_WORD category_and_name =
+      TRACE_EVENT_API_ATOMIC_LOAD(*bucket_data->bucket);
+  if (!category_and_name)
+    return;
+  const char* const combined =
+      reinterpret_cast<const char* const>(category_and_name);
+  const char* category_group;
+  const char* name;
+  ExtractCategoryAndName(combined, &category_group, &name);
+  TRACE_EVENT_API_ADD_TRACE_EVENT(TRACE_EVENT_PHASE_SAMPLE,
+      TraceLog::GetCategoryGroupEnabled(category_group),
+      name, 0, 0, nullptr, nullptr, nullptr, nullptr, 0);
+}
+
+void TraceSamplingThread::GetSamples() {
+  for (auto& sample_bucket : sample_buckets_) {
+    TraceBucketData* bucket_data = &sample_bucket;
+    bucket_data->callback.Run(bucket_data);
+  }
+}
+
+void TraceSamplingThread::RegisterSampleBucket(
+    TRACE_EVENT_API_ATOMIC_WORD* bucket,
+    const char* const name,
+    TraceSampleCallback callback) {
+  // Access to sample_buckets_ doesn't cause races with the sampling thread
+  // that uses the sample_buckets_, because it is guaranteed that
+  // RegisterSampleBucket is called before the sampling thread is created.
+  DCHECK(!thread_running_);
+  sample_buckets_.emplace_back(bucket, name, callback);
+}
+
+// static
+void TraceSamplingThread::ExtractCategoryAndName(const char* combined,
+                                                 const char** category,
+                                                 const char** name) {
+  *category = combined;
+  *name = &combined[strlen(combined) + 1];
+}
+
+void TraceSamplingThread::Stop() {
+  cancellation_flag_.Store(true);
+}
+
+TraceBucketData::TraceBucketData(AtomicWord* bucket, const char* name,
+                                 TraceSampleCallback callback)
+    : bucket(bucket), bucket_name(name), callback(std::move(callback)) {}
+
+TraceBucketData::~TraceBucketData() {
+}
+
+////////////////////////////////////////////////////////////////////////////////
+//
+// TraceLog
+//
+////////////////////////////////////////////////////////////////////////////////
+
+class TraceLog::ThreadLocalEventBuffer {
+ public:
+  explicit ThreadLocalEventBuffer(TraceLog* trace_log);
+  virtual ~ThreadLocalEventBuffer();
+
+  TraceEvent* AddTraceEvent(TraceEventHandle* handle);
+
+  TraceEvent* GetEventByHandle(TraceEventHandle handle) {
+    if (!chunk_ || handle.chunk_seq != chunk_->seq() ||
+        handle.chunk_index != chunk_index_)
+      return nullptr;
+
+    return chunk_->GetEventAt(handle.event_index);
+  }
+
+  int generation() const { return generation_; }
+
+  void Flush(int64_t tid);
+
+ private:
+  // Check that the current thread is the one that constructed this trace buffer.
+  void CheckIsOwnerThread() const {
+    DCHECK_EQ(kudu::Thread::UniqueThreadId(), owner_tid_);
+  }
+
+  // Since TraceLog is a leaky singleton, trace_log_ will always be valid
+  // as long as the thread exists.
+  TraceLog* trace_log_;
+  gscoped_ptr<TraceBufferChunk> chunk_;
+  size_t chunk_index_;
+  int generation_;
+
+  // The TID of the thread that constructed this event buffer. Only this thread
+  // may add trace events.
+  int64_t owner_tid_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadLocalEventBuffer);
+};
+
+TraceLog::ThreadLocalEventBuffer::ThreadLocalEventBuffer(TraceLog* trace_log)
+    : trace_log_(trace_log),
+      chunk_index_(0),
+      generation_(trace_log->generation()),
+      owner_tid_(kudu::Thread::UniqueThreadId()) {
+}
+
+TraceLog::ThreadLocalEventBuffer::~ThreadLocalEventBuffer() {
+}
+
+TraceEvent* TraceLog::ThreadLocalEventBuffer::AddTraceEvent(
+    TraceEventHandle* handle) {
+  CheckIsOwnerThread();
+
+  if (chunk_ && chunk_->IsFull()) {
+    SpinLockHolder lock(&trace_log_->lock_);
+    Flush(Thread::UniqueThreadId());
+    chunk_.reset();
+  }
+  if (!chunk_) {
+    SpinLockHolder lock(&trace_log_->lock_);
+    chunk_ = trace_log_->logged_events_->GetChunk(&chunk_index_);
+    trace_log_->CheckIfBufferIsFullWhileLocked();
+  }
+  if (!chunk_)
+    return nullptr;
+
+  size_t event_index;
+  TraceEvent* trace_event = chunk_->AddTraceEvent(&event_index);
+  if (trace_event && handle)
+    MakeHandle(chunk_->seq(), chunk_index_, event_index, handle);
+
+  return trace_event;
+}
+
+void TraceLog::ThreadLocalEventBuffer::Flush(int64_t tid) {
+  DCHECK(trace_log_->lock_.IsHeld());
+
+  if (!chunk_)
+    return;
+
+  if (trace_log_->CheckGeneration(generation_)) {
+    // Return the chunk to the buffer only if the generation matches.
+    trace_log_->logged_events_->ReturnChunk(chunk_index_, std::move(chunk_));
+  }
+}
+
+// static
+TraceLog* TraceLog::GetInstance() {
+  return Singleton<TraceLog>::get();
+}
+
+TraceLog::TraceLog()
+    : mode_(DISABLED),
+      num_traces_recorded_(0),
+      event_callback_(0),
+      dispatching_to_observer_list_(false),
+      process_sort_index_(0),
+      process_id_hash_(0),
+      process_id_(0),
+      time_offset_(0),
+      watch_category_(0),
+      trace_options_(RECORD_UNTIL_FULL),
+      sampling_thread_handle_(nullptr),
+      category_filter_(CategoryFilter::kDefaultCategoryFilterString),
+      event_callback_category_filter_(
+          CategoryFilter::kDefaultCategoryFilterString),
+      thread_shared_chunk_index_(0),
+      generation_(0) {
+  // Trace is enabled or disabled on one thread while other threads are
+  // accessing the enabled flag. We don't care whether edge-case events are
+  // traced or not, so we allow races on the enabled flag to keep the trace
+  // macros fast.
+  ANNOTATE_BENIGN_RACE_SIZED(g_category_group_enabled,
+                             sizeof(g_category_group_enabled),
+                             "trace_event category enabled");
+  for (int i = 0; i < MAX_CATEGORY_GROUPS; ++i) {
+    ANNOTATE_BENIGN_RACE(&g_category_group_enabled[i],
+                         "trace_event category enabled");
+  }
+  SetProcessID(static_cast<int>(getpid()));
+
+  string filter = FLAGS_trace_to_console;
+  if (!filter.empty()) {
+    SetEnabled(CategoryFilter(filter), RECORDING_MODE, ECHO_TO_CONSOLE);
+    LOG(ERROR) << "Tracing to console with CategoryFilter '" << filter << "'.";
+  }
+
+  logged_events_.reset(CreateTraceBuffer());
+}
+
+TraceLog::~TraceLog() {
+}
+
+const unsigned char* TraceLog::GetCategoryGroupEnabled(
+    const char* category_group) {
+  TraceLog* tracelog = GetInstance();
+  if (!tracelog) {
+    DCHECK(!g_category_group_enabled[kCategoryAlreadyShutdown]);
+    return &g_category_group_enabled[kCategoryAlreadyShutdown];
+  }
+  return tracelog->GetCategoryGroupEnabledInternal(category_group);
+}
+
+const char* TraceLog::GetCategoryGroupName(
+    const unsigned char* category_group_enabled) {
+  // Calculate the index of the category group by finding
+  // category_group_enabled in g_category_group_enabled array.
+  uintptr_t category_begin =
+      reinterpret_cast<uintptr_t>(g_category_group_enabled);
+  uintptr_t category_ptr = reinterpret_cast<uintptr_t>(category_group_enabled);
+  DCHECK(category_ptr >= category_begin &&
+         category_ptr < reinterpret_cast<uintptr_t>(
+             g_category_group_enabled + MAX_CATEGORY_GROUPS)) <<
+      "out of bounds category pointer";
+  uintptr_t category_index =
+      (category_ptr - category_begin) / sizeof(g_category_group_enabled[0]);
+  return g_category_groups[category_index];
+}
+
+void TraceLog::UpdateCategoryGroupEnabledFlag(int category_index) {
+  unsigned char enabled_flag = 0;
+  const char* category_group = g_category_groups[category_index];
+  if (mode_ == RECORDING_MODE &&
+      category_filter_.IsCategoryGroupEnabled(category_group))
+    enabled_flag |= ENABLED_FOR_RECORDING;
+  else if (mode_ == MONITORING_MODE &&
+      category_filter_.IsCategoryGroupEnabled(category_group))
+    enabled_flag |= ENABLED_FOR_MONITORING;
+  if (event_callback_ &&
+      event_callback_category_filter_.IsCategoryGroupEnabled(category_group))
+    enabled_flag |= ENABLED_FOR_EVENT_CALLBACK;
+  g_category_group_enabled[category_index] = enabled_flag;
+}
+
+void TraceLog::UpdateCategoryGroupEnabledFlags() {
+  int category_index = base::subtle::NoBarrier_Load(&g_category_index);
+  for (int i = 0; i < category_index; i++)
+    UpdateCategoryGroupEnabledFlag(i);
+}
+
+void TraceLog::UpdateSyntheticDelaysFromCategoryFilter() {
+  ResetTraceEventSyntheticDelays();
+  const CategoryFilter::StringList& delays =
+      category_filter_.GetSyntheticDelayValues();
+  CategoryFilter::StringList::const_iterator ci;
+  for (ci = delays.begin(); ci != delays.end(); ++ci) {
+    std::list<string> tokens = strings::Split(*ci, ";");
+    if (tokens.empty()) continue;
+
+    TraceEventSyntheticDelay* delay =
+        TraceEventSyntheticDelay::Lookup(tokens.front());
+    tokens.pop_front();
+    while (!tokens.empty()) {
+      std::string token = tokens.front();
+      tokens.pop_front();
+      char* duration_end;
+      double target_duration = strtod(token.c_str(), &duration_end);
+      if (duration_end != token.c_str()) {
+        delay->SetTargetDuration(MonoDelta::FromSeconds(target_duration));
+      } else if (token == "static") {
+        delay->SetMode(TraceEventSyntheticDelay::STATIC);
+      } else if (token == "oneshot") {
+        delay->SetMode(TraceEventSyntheticDelay::ONE_SHOT);
+      } else if (token == "alternating") {
+        delay->SetMode(TraceEventSyntheticDelay::ALTERNATING);
+      }
+    }
+  }
+}
+
+const unsigned char* TraceLog::GetCategoryGroupEnabledInternal(
+    const char* category_group) {
+  DCHECK(!strchr(category_group, '"')) <<
+      "Category groups may not contain double quote";
+  // The g_category_groups is append only, avoid using a lock for the fast path.
+  int current_category_index = base::subtle::Acquire_Load(&g_category_index);
+
+  // Search for pre-existing category group.
+  for (int i = 0; i < current_category_index; ++i) {
+    if (strcmp(g_category_groups[i], category_group) == 0) {
+      return &g_category_group_enabled[i];
+    }
+  }
+
+  unsigned char* category_group_enabled = nullptr;
+  // This is the slow path: the lock is not held in the case above, so more
+  // than one thread could have reached here trying to add the same category.
+  // Only hold to lock when actually appending a new category, and
+  // check the categories groups again.
+  SpinLockHolder lock(&lock_);
+  int category_index = base::subtle::Acquire_Load(&g_category_index);
+  for (int i = 0; i < category_index; ++i) {
+    if (strcmp(g_category_groups[i], category_group) == 0) {
+      return &g_category_group_enabled[i];
+    }
+  }
+
+  // Create a new category group.
+  DCHECK(category_index < MAX_CATEGORY_GROUPS) <<
+      "must increase MAX_CATEGORY_GROUPS";
+  if (category_index < MAX_CATEGORY_GROUPS) {
+    // Don't hold on to the category_group pointer, so that we can create
+    // category groups with strings not known at compile time (this is
+    // required by SetWatchEvent).
+    const char* new_group = strdup(category_group);
+    // NOTE: new_group is leaked, but this is a small finite amount of data
+    g_category_groups[category_index] = new_group;
+    DCHECK(!g_category_group_enabled[category_index]);
+    // Note that if both included and excluded patterns in the
+    // CategoryFilter are empty, we exclude nothing,
+    // thereby enabling this category group.
+    UpdateCategoryGroupEnabledFlag(category_index);
+    category_group_enabled = &g_category_group_enabled[category_index];
+    // Update the max index now.
+    base::subtle::Release_Store(&g_category_index, category_index + 1);
+  } else {
+    category_group_enabled =
+        &g_category_group_enabled[kCategoryCategoriesExhausted];
+  }
+  return category_group_enabled;
+}
+
+void TraceLog::GetKnownCategoryGroups(
+    std::vector<std::string>* category_groups) {
+  SpinLockHolder lock(&lock_);
+  int category_index = base::subtle::NoBarrier_Load(&g_category_index);
+  for (int i = kNumBuiltinCategories; i < category_index; i++)
+    category_groups->push_back(g_category_groups[i]);
+}
+
+void TraceLog::SetEnabled(const CategoryFilter& category_filter,
+                          Mode mode,
+                          Options options) {
+  std::vector<EnabledStateObserver*> observer_list;
+  {
+    SpinLockHolder lock(&lock_);
+
+    // Can't enable tracing when Flush() is in progress.
+    Options old_options = trace_options();
+
+    if (IsEnabled()) {
+      if (options != old_options) {
+        DLOG(ERROR) << "Attempting to re-enable tracing with a different "
+                    << "set of options.";
+      }
+
+      if (mode != mode_) {
+        DLOG(ERROR) << "Attempting to re-enable tracing with a different mode.";
+      }
+
+      category_filter_.Merge(category_filter);
+      UpdateCategoryGroupEnabledFlags();
+      return;
+    }
+
+    if (dispatching_to_observer_list_) {
+      DLOG(ERROR) <<
+          "Cannot manipulate TraceLog::Enabled state from an observer.";
+      return;
+    }
+
+    mode_ = mode;
+
+    if (options != old_options) {
+      base::subtle::NoBarrier_Store(&trace_options_, options);
+      UseNextTraceBuffer();
+    }
+
+    num_traces_recorded_++;
+
+    category_filter_ = CategoryFilter(category_filter);
+    UpdateCategoryGroupEnabledFlags();
+    UpdateSyntheticDelaysFromCategoryFilter();
+
+    if (options & ENABLE_SAMPLING) {
+      sampling_thread_.reset(new TraceSamplingThread);
+      sampling_thread_->RegisterSampleBucket(
+          &g_trace_state[0],
+          "bucket0",
+          Bind(&TraceSamplingThread::DefaultSamplingCallback));
+      sampling_thread_->RegisterSampleBucket(
+          &g_trace_state[1],
+          "bucket1",
+          Bind(&TraceSamplingThread::DefaultSamplingCallback));
+      sampling_thread_->RegisterSampleBucket(
+          &g_trace_state[2],
+          "bucket2",
+          Bind(&TraceSamplingThread::DefaultSamplingCallback));
+
+      Status s = Thread::Create("tracing", "sampler",
+                                &TraceSamplingThread::ThreadMain,
+                                sampling_thread_.get(),
+                                &sampling_thread_handle_);
+      if (!s.ok()) {
+        LOG(DFATAL) << "failed to create trace sampling thread: " << s.ToString();
+      }
+    }
+
+    dispatching_to_observer_list_ = true;
+    observer_list = enabled_state_observer_list_;
+  }
+  // Notify observers outside the lock in case they trigger trace events.
+  for (const auto& observer : observer_list)
+    observer->OnTraceLogEnabled();
+
+  {
+    SpinLockHolder lock(&lock_);
+    dispatching_to_observer_list_ = false;
+  }
+}
+
+CategoryFilter TraceLog::GetCurrentCategoryFilter() {
+  SpinLockHolder lock(&lock_);
+  return category_filter_;
+}
+
+void TraceLog::SetDisabled() {
+  SpinLockHolder lock(&lock_);
+  SetDisabledWhileLocked();
+}
+
+void TraceLog::SetDisabledWhileLocked() {
+  DCHECK(lock_.IsHeld());
+
+  if (!IsEnabled())
+    return;
+
+  if (dispatching_to_observer_list_) {
+    DLOG(ERROR)
+        << "Cannot manipulate TraceLog::Enabled state from an observer.";
+    return;
+  }
+
+  mode_ = DISABLED;
+
+  if (sampling_thread_.get()) {
+    // Stop the sampling thread.
+    sampling_thread_->Stop();
+    lock_.Unlock();
+    sampling_thread_handle_->Join();
+    lock_.Lock();
+    sampling_thread_handle_.reset();
+    sampling_thread_.reset();
+  }
+
+  category_filter_.Clear();
+  base::subtle::NoBarrier_Store(&watch_category_, 0);
+  watch_event_name_ = "";
+  UpdateCategoryGroupEnabledFlags();
+  AddMetadataEventsWhileLocked();
+
+  dispatching_to_observer_list_ = true;
+  std::vector<EnabledStateObserver*> observer_list =
+      enabled_state_observer_list_;
+
+  {
+    // Dispatch to observers outside the lock in case the observer triggers a
+    // trace event.
+    lock_.Unlock();
+    for (const auto& observer : observer_list)
+      observer->OnTraceLogDisabled();
+    lock_.Lock();
+  }
+  dispatching_to_observer_list_ = false;
+}
+
+int TraceLog::GetNumTracesRecorded() {
+  SpinLockHolder lock(&lock_);
+  if (!IsEnabled())
+    return -1;
+  return num_traces_recorded_;
+}
+
+void TraceLog::AddEnabledStateObserver(EnabledStateObserver* listener) {
+  enabled_state_observer_list_.push_back(listener);
+}
+
+void TraceLog::RemoveEnabledStateObserver(EnabledStateObserver* listener) {
+  auto it = std::find(enabled_state_observer_list_.begin(),
+                      enabled_state_observer_list_.end(), listener);
+  if (it != enabled_state_observer_list_.end())
+    enabled_state_observer_list_.erase(it);
+}
+
+bool TraceLog::HasEnabledStateObserver(EnabledStateObserver* listener) const {
+  auto it = std::find(enabled_state_observer_list_.begin(),
+                      enabled_state_observer_list_.end(), listener);
+  return it != enabled_state_observer_list_.end();
+}
+
+float TraceLog::GetBufferPercentFull() const {
+  SpinLockHolder lock(&lock_);
+  return static_cast<float>(static_cast<double>(logged_events_->Size()) /
+                            logged_events_->Capacity());
+}
+
+bool TraceLog::BufferIsFull() const {
+  SpinLockHolder lock(&lock_);
+  return logged_events_->IsFull();
+}
+
+TraceBuffer* TraceLog::CreateTraceBuffer() {
+  Options options = trace_options();
+  if (options & RECORD_CONTINUOUSLY)
+    return new TraceBufferRingBuffer(kTraceEventRingBufferChunks);
+  else if ((options & ENABLE_SAMPLING) && mode_ == MONITORING_MODE)
+    return new TraceBufferRingBuffer(kMonitorTraceEventBufferChunks);
+  else if (options & ECHO_TO_CONSOLE)
+    return new TraceBufferRingBuffer(kEchoToConsoleTraceEventBufferChunks);
+  return new TraceBufferVector();
+}
+
+TraceEvent* TraceLog::AddEventToThreadSharedChunkWhileLocked(
+    TraceEventHandle* handle, bool check_buffer_is_full) {
+  DCHECK(lock_.IsHeld());
+
+  if (thread_shared_chunk_ && thread_shared_chunk_->IsFull()) {
+    logged_events_->ReturnChunk(thread_shared_chunk_index_,
+                                std::move(thread_shared_chunk_));
+  }
+
+  if (!thread_shared_chunk_) {
+    thread_shared_chunk_ = logged_events_->GetChunk(
+        &thread_shared_chunk_index_);
+    if (check_buffer_is_full)
+      CheckIfBufferIsFullWhileLocked();
+  }
+  if (!thread_shared_chunk_)
+    return nullptr;
+
+  size_t event_index;
+  TraceEvent* trace_event = thread_shared_chunk_->AddTraceEvent(&event_index);
+  if (trace_event && handle) {
+    MakeHandle(thread_shared_chunk_->seq(), thread_shared_chunk_index_,
+               event_index, handle);
+  }
+  return trace_event;
+}
+
+void TraceLog::CheckIfBufferIsFullWhileLocked() {
+  DCHECK(lock_.IsHeld());
+  if (logged_events_->IsFull())
+    SetDisabledWhileLocked();
+}
+
+void TraceLog::SetEventCallbackEnabled(const CategoryFilter& category_filter,
+                                       EventCallback cb) {
+  SpinLockHolder lock(&lock_);
+  base::subtle::NoBarrier_Store(&event_callback_,
+                          reinterpret_cast<AtomicWord>(cb));
+  event_callback_category_filter_ = category_filter;
+  UpdateCategoryGroupEnabledFlags();
+};
+
+void TraceLog::SetEventCallbackDisabled() {
+  SpinLockHolder lock(&lock_);
+  base::subtle::NoBarrier_Store(&event_callback_, 0);
+  UpdateCategoryGroupEnabledFlags();
+}
+
+// Flush() works as the following:
+//
+// We ensure by taking the global lock that we have exactly one Flusher thread
+// (the caller of this function) and some number of "target" threads. We do
+// not want to block the target threads, since they are running application code,
+// so this implementation takes an approach based on asymmetric synchronization.
+//
+// For each active thread, we grab its PerThreadInfo object, which may contain
+// a pointer to its active trace chunk. We use an AtomicExchange to swap this
+// out for a null pointer. This ensures that, on the *next* TRACE call made by
+// that thread, it will see a NULL buffer and create a _new_ trace buffer. That
+// new buffer would be assigned the generation of the next collection and we don't
+// have to worry about it in the current Flush().
+//
+// However, the swap doesn't ensure that the thread doesn't already have a local copy of
+// the 'event_buffer_' that we are trying to flush. So, if the thread is in the
+// middle of a Trace call, we have to wait until it exits. We do that by spinning
+// on the 'is_in_trace_event_' member of that thread's thread-local structure.
+//
+// After we've swapped the buffer pointer and waited on the thread to exit any
+// concurrent Trace() call, we know that no other thread can hold a pointer to
+// the trace buffer, and we can safely flush it and delete it.
+void TraceLog::Flush(const TraceLog::OutputCallback& cb) {
+  if (IsEnabled()) {
+    // Can't flush when tracing is enabled because otherwise PostTask would
+    // - generate more trace events;
+    // - deschedule the calling thread on some platforms causing inaccurate
+    //   timing of the trace events.
+    scoped_refptr<RefCountedString> empty_result = new RefCountedString;
+    if (!cb.is_null())
+      cb.Run(empty_result, false);
+    LOG(WARNING) << "Ignored TraceLog::Flush called when tracing is enabled";
+    return;
+  }
+
+  int generation = this->generation();
+  {
+    // Holding the active threads lock ensures that no thread will exit and
+    // delete its own PerThreadInfo object.
+    MutexLock l(active_threads_lock_);
+    for (const ActiveThreadMap::value_type& entry : active_threads_) {
+      int64_t tid = entry.first;
+      PerThreadInfo* thr_info = entry.second;
+
+      // Swap out their buffer from their thread-local data.
+      // After this, any _future_ trace calls on that thread will create a new buffer
+      // and not use the one we obtain here.
+      ThreadLocalEventBuffer* buf = thr_info->AtomicTakeBuffer();
+
+      // If this thread hasn't traced anything since our last
+      // flush, we can skip it.
+      if (!buf) {
+        continue;
+      }
+
+      // The buffer may still be in use by that thread if they're in a call. Sleep until
+      // they aren't, so we can flush/delete their old buffer.
+      //
+      // It's important that we do not hold 'lock_' here, because otherwise we can get a
+      // deadlock: a thread may be in the middle of a trace event (is_in_trace_event_ ==
+      // true) and waiting to take lock_, while we are holding the lock and waiting for it
+      // to not be in the trace event.
+      while (base::subtle::Acquire_Load(&thr_info->is_in_trace_event_)) {
+        sched_yield();
+      }
+
+      {
+        SpinLockHolder lock(&lock_);
+        buf->Flush(tid);
+      }
+      delete buf;
+    }
+  }
+
+  {
+    SpinLockHolder lock(&lock_);
+
+    if (thread_shared_chunk_) {
+      logged_events_->ReturnChunk(thread_shared_chunk_index_,
+                                  std::move(thread_shared_chunk_));
+    }
+  }
+
+  FinishFlush(generation, cb);
+}
+
+void TraceLog::ConvertTraceEventsToTraceFormat(
+    gscoped_ptr<TraceBuffer> logged_events,
+    const TraceLog::OutputCallback& flush_output_callback) {
+
+  if (flush_output_callback.is_null())
+    return;
+
+  // The callback need to be called at least once even if there is no events
+  // to let the caller know the completion of flush.
+  bool has_more_events = true;
+  do {
+    scoped_refptr<RefCountedString> json_events_str_ptr =
+        new RefCountedString();
+
+    for (size_t i = 0; i < kTraceEventBatchChunks; ++i) {
+      const TraceBufferChunk* chunk = logged_events->NextChunk();
+      if (!chunk) {
+        has_more_events = false;
+        break;
+      }
+      for (size_t j = 0; j < chunk->size(); ++j) {
+        if (i > 0 || j > 0)
+          json_events_str_ptr->data().append(",");
+        chunk->GetEventAt(j)->AppendAsJSON(&(json_events_str_ptr->data()));
+      }
+    }
+
+    flush_output_callback.Run(json_events_str_ptr, has_more_events);
+  } while (has_more_events);
+  logged_events.reset();
+}
+
+void TraceLog::FinishFlush(int generation,
+                           const TraceLog::OutputCallback& flush_output_callback) {
+  gscoped_ptr<TraceBuffer> previous_logged_events;
+
+  if (!CheckGeneration(generation))
+    return;
+
+  {
+    SpinLockHolder lock(&lock_);
+
+    previous_logged_events.swap(logged_events_);
+    UseNextTraceBuffer();
+  }
+
+  ConvertTraceEventsToTraceFormat(std::move(previous_logged_events),
+                                  flush_output_callback);
+}
+
+void TraceLog::FlushButLeaveBufferIntact(
+    const TraceLog::OutputCallback& flush_output_callback) {
+  gscoped_ptr<TraceBuffer> previous_logged_events;
+  {
+    SpinLockHolder lock(&lock_);
+    if (mode_ == DISABLED || (trace_options_ & RECORD_CONTINUOUSLY) == 0) {
+      scoped_refptr<RefCountedString> empty_result = new RefCountedString;
+      flush_output_callback.Run(empty_result, false);
+      LOG(WARNING) << "Ignored TraceLog::FlushButLeaveBufferIntact when monitoring is not enabled";
+      return;
+    }
+
+    AddMetadataEventsWhileLocked();
+    if (thread_shared_chunk_) {
+      // Return the chunk to the main buffer to flush the sampling data.
+      logged_events_->ReturnChunk(thread_shared_chunk_index_,
+                                  std::move(thread_shared_chunk_));
+    }
+    previous_logged_events = logged_events_->CloneForIteration();
+  }
+
+  ConvertTraceEventsToTraceFormat(std::move(previous_logged_events),
+                                  flush_output_callback);
+}
+
+void TraceLog::UseNextTraceBuffer() {
+  logged_events_.reset(CreateTraceBuffer());
+  base::subtle::NoBarrier_AtomicIncrement(&generation_, 1);
+  thread_shared_chunk_.reset();
+  thread_shared_chunk_index_ = 0;
+}
+
+TraceEventHandle TraceLog::AddTraceEvent(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int num_args,
+    const char** arg_names,
+    const unsigned char* arg_types,
+    const uint64_t* arg_values,
+    const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+    unsigned char flags) {
+  int thread_id = static_cast<int>(kudu::Thread::UniqueThreadId());
+  MicrosecondsInt64 now = GetMonoTimeMicros();
+  return AddTraceEventWithThreadIdAndTimestamp(phase, category_group_enabled,
+                                               name, id, thread_id, now,
+                                               num_args, arg_names,
+                                               arg_types, arg_values,
+                                               convertable_values, flags);
+}
+
+TraceLog::PerThreadInfo* TraceLog::SetupThreadLocalBuffer() {
+  int64_t cur_tid = Thread::UniqueThreadId();
+
+  auto thr_info = new PerThreadInfo();
+  thr_info->event_buffer_ = nullptr;
+  thr_info->is_in_trace_event_ = 0;
+  thread_local_info_ = thr_info;
+
+  threadlocal::internal::AddDestructor(&TraceLog::ThreadExitingCB, this);
+
+  {
+    MutexLock lock(active_threads_lock_);
+    InsertOrDie(&active_threads_, cur_tid, thr_info);
+  }
+  return thr_info;
+}
+
+void TraceLog::ThreadExitingCB(void* arg) {
+  static_cast<TraceLog*>(arg)->ThreadExiting();
+}
+
+void TraceLog::ThreadExiting() {
+  PerThreadInfo* thr_info = thread_local_info_;
+  if (!thr_info) {
+    return;
+  }
+
+  int64_t cur_tid = Thread::UniqueThreadId();
+
+  // Flush our own buffer back to the central event buffer.
+  // We do the atomic exchange because a flusher thread may
+  // also be trying to flush us at the same time, and we need to avoid
+  // conflict.
+  ThreadLocalEventBuffer* buf = thr_info->AtomicTakeBuffer();
+  if (buf) {
+    SpinLockHolder lock(&lock_);
+    buf->Flush(Thread::UniqueThreadId());
+  }
+  delete buf;
+
+  {
+    MutexLock lock(active_threads_lock_);
+    active_threads_.erase(cur_tid);
+  }
+  delete thr_info;
+}
+
+TraceEventHandle TraceLog::AddTraceEventWithThreadIdAndTimestamp(
+    char phase,
+    const unsigned char* category_group_enabled,
+    const char* name,
+    uint64_t id,
+    int thread_id,
+    const MicrosecondsInt64& timestamp,
+    int num_args,
+    const char** arg_names,
+    const unsigned char* arg_types,
+    const uint64_t* arg_values,
+    const scoped_refptr<ConvertableToTraceFormat>* convertable_values,
+    unsigned char flags) {
+  TraceEventHandle handle = { 0, 0, 0 };
+  if (!*category_group_enabled)
+    return handle;
+
+  DCHECK(name);
+
+  if (flags & TRACE_EVENT_FLAG_MANGLE_ID)
+    id ^= process_id_hash_;
+
+  MicrosecondsInt64 now = OffsetTimestamp(timestamp);
+  MicrosecondsInt64 thread_now = GetThreadCpuTimeMicros();
+
+  PerThreadInfo* thr_info = thread_local_info_;
+  if (PREDICT_FALSE(!thr_info)) {
+    thr_info = SetupThreadLocalBuffer();
+  }
+
+  // Avoid re-entrance of AddTraceEvent. This may happen in GPU process when
+  // ECHO_TO_CONSOLE is enabled: AddTraceEvent -> LOG(ERROR) ->
+  // GpuProcessLogMessageHandler -> PostPendingTask -> TRACE_EVENT ...
+  if (base::subtle::NoBarrier_Load(&thr_info->is_in_trace_event_))
+    return handle;
+
+  MarkFlagInScope thread_is_in_trace_event(&thr_info->is_in_trace_event_);
+
+  ThreadLocalEventBuffer* thread_local_event_buffer =
+    reinterpret_cast<ThreadLocalEventBuffer*>(
+      base::subtle::NoBarrier_Load(
+        reinterpret_cast<AtomicWord*>(&thr_info->event_buffer_)));
+
+  // If we have an event buffer, but it's a left-over from a previous trace,
+  // delete it.
+  if (PREDICT_FALSE(thread_local_event_buffer &&
+                    !CheckGeneration(thread_local_event_buffer->generation()))) {
+    // We might also race against a flusher thread, so we have to atomically
+    // take the buffer.
+    thread_local_event_buffer = thr_info->AtomicTakeBuffer();
+    delete thread_local_event_buffer;
+    thread_local_event_buffer = nullptr;
+  }
+
+  // If there is no current buffer, create one for this event.
+  if (PREDICT_FALSE(!thread_local_event_buffer)) {
+    thread_local_event_buffer = new ThreadLocalEventBuffer(this);
+
+    base::subtle::NoBarrier_Store(
+      reinterpret_cast<AtomicWord*>(&thr_info->event_buffer_),
+      reinterpret_cast<AtomicWord>(thread_local_event_buffer));
+  }
+
+  // Check and update the current thread name only if the event is for the
+  // current thread to avoid locks in most cases.
+  if (thread_id == static_cast<int>(Thread::UniqueThreadId())) {
+    Thread* kudu_thr = Thread::current_thread();
+    if (kudu_thr) {
+      const char* new_name = kudu_thr->name().c_str();
+      // Check if the thread name has been set or changed since the previous
+      // call (if any), but don't bother if the new name is empty. Note this will
+      // not detect a thread name change within the same char* buffer address: we
+      // favor common case performance over corner case correctness.
+      if (PREDICT_FALSE(new_name != g_current_thread_name &&
+                        new_name && *new_name)) {
+        g_current_thread_name = new_name;
+
+        SpinLockHolder thread_info_lock(&thread_info_lock_);
+
+        auto existing_name = thread_names_.find(thread_id);
+        if (existing_name == thread_names_.end()) {
+          // This is a new thread id, and a new name.
+          thread_names_[thread_id] = new_name;
+        } else {
+          // This is a thread id that we've seen before, but potentially with a
+          // new name.
+          std::vector<StringPiece> existing_names = strings::Split(existing_name->second, ",");
+          bool found = std::find(existing_names.begin(),
+                                 existing_names.end(),
+                                 new_name) != existing_names.end();
+          if (!found) {
+            if (existing_names.size())
+              existing_name->second.push_back(',');
+            existing_name->second.append(new_name);
+          }
+        }
+      }
+    }
+  }
+
+  std::string console_message;
+  if (*category_group_enabled &
+      (ENABLED_FOR_RECORDING | ENABLED_FOR_MONITORING)) {
+    TraceEvent* trace_event = thread_local_event_buffer->AddTraceEvent(&handle);
+
+    if (trace_event) {
+      trace_event->Initialize(thread_id, now, thread_now, phase,
+                              category_group_enabled, name, id,
+                              num_args, arg_names, arg_types, arg_values,
+                              convertable_values, flags);
+
+#if defined(OS_ANDROID)
+      trace_event->SendToATrace();
+#endif
+    }
+
+    if (trace_options() & ECHO_TO_CONSOLE) {
+      console_message = EventToConsoleMessage(
+          phase == TRACE_EVENT_PHASE_COMPLETE ? TRACE_EVENT_PHASE_BEGIN : phase,
+          timestamp, trace_event);
+    }
+  }
+
+  if (PREDICT_FALSE(console_message.size()))
+    LOG(ERROR) << console_message;
+
+  if (PREDICT_FALSE(reinterpret_cast<const unsigned char*>(
+                      base::subtle::NoBarrier_Load(&watch_category_)) == category_group_enabled)) {
+    bool event_name_matches;
+    WatchEventCallback watch_event_callback_copy;
+    {
+      SpinLockHolder lock(&lock_);
+      event_name_matches = watch_event_name_ == name;
+      watch_event_callback_copy = watch_event_callback_;
+    }
+    if (event_name_matches) {
+      if (!watch_event_callback_copy.is_null())
+        watch_event_callback_copy.Run();
+    }
+  }
+
+  if (PREDICT_FALSE(*category_group_enabled & ENABLED_FOR_EVENT_CALLBACK)) {
+    EventCallback event_callback = reinterpret_cast<EventCallback>(
+      base::subtle::NoBarrier_Load(&event_callback_));
+    if (event_callback) {
+      event_callback(now,
+                     phase == TRACE_EVENT_PHASE_COMPLETE ?
+                         TRACE_EVENT_PHASE_BEGIN : phase,
+                     category_group_enabled, name, id,
+                     num_args, arg_names, arg_types, arg_values,
+                     flags);
+    }
+  }
+
+  return handle;
+}
+
+// May be called when a COMPELETE event ends and the unfinished event has been
+// recycled (phase == TRACE_EVENT_PHASE_END and trace_event == NULL).
+std::string TraceLog::EventToConsoleMessage(unsigned char phase,
+                                            const MicrosecondsInt64& timestamp,
+                                            TraceEvent* trace_event) {
+  SpinLockHolder thread_info_lock(&thread_info_lock_);
+
+  // The caller should translate TRACE_EVENT_PHASE_COMPLETE to
+  // TRACE_EVENT_PHASE_BEGIN or TRACE_EVENT_END.
+  DCHECK(phase != TRACE_EVENT_PHASE_COMPLETE);
+
+  MicrosecondsInt64 duration;
+  int thread_id = trace_event ?
+      trace_event->thread_id() : Thread::UniqueThreadId();
+  if (phase == TRACE_EVENT_PHASE_END) {
+    duration = timestamp - thread_event_start_times_[thread_id].top();
+    thread_event_start_times_[thread_id].pop();
+  }
+
+  std::string thread_name = thread_names_[thread_id];
+  if (thread_colors_.find(thread_name) == thread_colors_.end())
+    thread_colors_[thread_name] = (thread_colors_.size() % 6) + 1;
+
+  std::ostringstream log;
+  log << StringPrintf("%s: \x1b[0;3%dm",
+                            thread_name.c_str(),
+                            thread_colors_[thread_name]);
+
+  size_t depth = 0;
+  if (thread_event_start_times_.find(thread_id) !=
+      thread_event_start_times_.end())
+    depth = thread_event_start_times_[thread_id].size();
+
+  for (size_t i = 0; i < depth; ++i)
+    log << "| ";
+
+  if (trace_event)
+    trace_event->AppendPrettyPrinted(&log);
+  if (phase == TRACE_EVENT_PHASE_END)
+    log << StringPrintf(" (%.3f ms)", duration / 1000.0f);
+
+  log << "\x1b[0;m";
+
+  if (phase == TRACE_EVENT_PHASE_BEGIN)
+    thread_event_start_times_[thread_id].push(timestamp);
+
+  return log.str();
+}
+
+void TraceLog::AddTraceEventEtw(char phase,
+                                const char* name,
+                                const void* id,
+                                const char* extra) {
+#if defined(OS_WIN)
+  TraceEventETWProvider::Trace(name, phase, id, extra);
+#endif
+  INTERNAL_TRACE_EVENT_ADD(phase, "ETW Trace Event", name,
+                           TRACE_EVENT_FLAG_COPY, "id", id, "extra", extra);
+}
+
+void TraceLog::AddTraceEventEtw(char phase,
+                                const char* name,
+                                const void* id,
+                                const std::string& extra) {
+#if defined(OS_WIN)
+  TraceEventETWProvider::Trace(name, phase, id, extra);
+#endif
+  INTERNAL_TRACE_EVENT_ADD(phase, "ETW Trace Event", name,
+                           TRACE_EVENT_FLAG_COPY, "id", id, "extra", extra);
+}
+
+void TraceLog::UpdateTraceEventDuration(
+    const unsigned char* category_group_enabled,
+    const char* name,
+    TraceEventHandle handle) {
+
+  PerThreadInfo* thr_info = thread_local_info_;
+  if (!thr_info) {
+    thr_info = SetupThreadLocalBuffer();
+  }
+
+  // Avoid re-entrance of AddTraceEvent. This may happen in GPU process when
+  // ECHO_TO_CONSOLE is enabled: AddTraceEvent -> LOG(ERROR) ->
+  // GpuProcessLogMessageHandler -> PostPendingTask -> TRACE_EVENT ...
+  if (base::subtle::NoBarrier_Load(&thr_info->is_in_trace_event_))
+    return;
+  MarkFlagInScope thread_is_in_trace_event(&thr_info->is_in_trace_event_);
+
+  MicrosecondsInt64 thread_now = GetThreadCpuTimeMicros();
+  MicrosecondsInt64 now = OffsetNow();
+
+  std::string console_message;
+  if (*category_group_enabled & ENABLED_FOR_RECORDING) {
+    OptionalAutoLock lock(lock_);
+
+    TraceEvent* trace_event = GetEventByHandleInternal(handle, &lock);
+    if (trace_event) {
+      DCHECK(trace_event->phase() == TRACE_EVENT_PHASE_COMPLETE);
+      trace_event->UpdateDuration(now, thread_now);
+#if defined(OS_ANDROID)
+      trace_event->SendToATrace();
+#endif
+    }
+
+    if (trace_options() & ECHO_TO_CONSOLE) {
+      console_message = EventToConsoleMessage(TRACE_EVENT_PHASE_END,
+                                              now, trace_event);
+    }
+  }
+
+  if (console_message.size())
+    LOG(ERROR) << console_message;
+
+  if (*category_group_enabled & ENABLED_FOR_EVENT_CALLBACK) {
+    EventCallback event_callback = reinterpret_cast<EventCallback>(
+      base::subtle::NoBarrier_Load(&event_callback_));
+    if (event_callback) {
+      event_callback(now, TRACE_EVENT_PHASE_END, category_group_enabled, name,
+                     trace_event_internal::kNoEventId, 0, nullptr, nullptr, nullptr,
+                     TRACE_EVENT_FLAG_NONE);
+    }
+  }
+}
+
+void TraceLog::SetWatchEvent(const std::string& category_name,
+                             const std::string& event_name,
+                             const WatchEventCallback& callback) {
+  const unsigned char* category = GetCategoryGroupEnabled(
+      category_name.c_str());
+  SpinLockHolder lock(&lock_);
+  base::subtle::NoBarrier_Store(&watch_category_,
+                          reinterpret_cast<AtomicWord>(category));
+  watch_event_name_ = event_name;
+  watch_event_callback_ = callback;
+}
+
+void TraceLog::CancelWatchEvent() {
+  SpinLockHolder lock(&lock_);
+  base::subtle::NoBarrier_Store(&watch_category_, 0);
+  watch_event_name_ = "";
+  watch_event_callback_.Reset();
+}
+
+void TraceLog::AddMetadataEventsWhileLocked() {
+  DCHECK(lock_.IsHeld());
+
+#if !defined(OS_NACL)  // NaCl shouldn't expose the process id.
+  InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                          0,
+                          "num_cpus", "number",
+                          base::NumCPUs());
+#endif
+
+
+  int current_thread_id = static_cast<int>(kudu::Thread::UniqueThreadId());
+  if (process_sort_index_ != 0) {
+    InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                            current_thread_id,
+                            "process_sort_index", "sort_index",
+                            process_sort_index_);
+  }
+
+  if (process_name_.size()) {
+    InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                            current_thread_id,
+                            "process_name", "name",
+                            process_name_);
+  }
+
+  if (process_labels_.size() > 0) {
+    std::vector<std::string> labels;
+    for(auto& label : process_labels_) {
+      labels.push_back(label.second);
+    }
+    InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                            current_thread_id,
+                            "process_labels", "labels",
+                            JoinStrings(labels, ","));
+  }
+
+  // Thread sort indices.
+  for(auto& sort_index : thread_sort_indices_) {
+    if (sort_index.second == 0)
+      continue;
+    InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                            sort_index.first,
+                            "thread_sort_index", "sort_index",
+                            sort_index.second);
+  }
+
+  // Thread names.
+  SpinLockHolder thread_info_lock(&thread_info_lock_);
+  for(auto& name : thread_names_) {
+    if (name.second.empty())
+      continue;
+    InitializeMetadataEvent(AddEventToThreadSharedChunkWhileLocked(nullptr, false),
+                            name.first,
+                            "thread_name", "name",
+                            name.second);
+  }
+}
+
+
+TraceEvent* TraceLog::GetEventByHandle(TraceEventHandle handle) {
+  return GetEventByHandleInternal(handle, nullptr);
+}
+
+TraceEvent* TraceLog::GetEventByHandleInternal(TraceEventHandle handle,
+                                               OptionalAutoLock* lock) {
+  TraceLog::PerThreadInfo* thr_info = TraceLog::thread_local_info_;
+
+  if (!handle.chunk_seq)
+    return nullptr;
+
+  if (thr_info) {
+    ThreadLocalEventBuffer* buf =
+      reinterpret_cast<ThreadLocalEventBuffer*>(
+        base::subtle::NoBarrier_Load(
+          reinterpret_cast<AtomicWord*>(&thr_info->event_buffer_)));
+
+    if (buf) {
+      DCHECK_EQ(1, ANNOTATE_UNPROTECTED_READ(thr_info->is_in_trace_event_));
+
+      TraceEvent* trace_event = buf->GetEventByHandle(handle);
+      if (trace_event)
+        return trace_event;
+    }
+  }
+
+  // The event has been out-of-control of the thread local buffer.
+  // Try to get the event from the main buffer with a lock.
+  if (lock)
+    lock->EnsureAcquired();
+
+  if (thread_shared_chunk_ &&
+      handle.chunk_index == thread_shared_chunk_index_) {
+    return handle.chunk_seq == thread_shared_chunk_->seq() ?
+        thread_shared_chunk_->GetEventAt(handle.event_index) : nullptr;
+  }
+
+  return logged_events_->GetEventByHandle(handle);
+}
+
+ATTRIBUTE_NO_SANITIZE_INTEGER
+void TraceLog::SetProcessID(int process_id) {
+  process_id_ = process_id;
+  // Create a FNV hash from the process ID for XORing.
+  // See http://isthe.com/chongo/tech/comp/fnv/ for algorithm details.
+  uint64_t offset_basis = 14695981039346656037ull;
+  uint64_t fnv_prime = 1099511628211ull;
+  uint64_t pid = static_cast<uint64_t>(process_id_);
+  process_id_hash_ = (offset_basis ^ pid) * fnv_prime;
+}
+
+void TraceLog::SetProcessSortIndex(int sort_index) {
+  SpinLockHolder lock(&lock_);
+  process_sort_index_ = sort_index;
+}
+
+void TraceLog::SetProcessName(const std::string& process_name) {
+  SpinLockHolder lock(&lock_);
+  process_name_ = process_name;
+}
+
+void TraceLog::UpdateProcessLabel(
+    int label_id, const std::string& current_label) {
+  if(!current_label.length())
+    return RemoveProcessLabel(label_id);
+
+  SpinLockHolder lock(&lock_);
+  process_labels_[label_id] = current_label;
+}
+
+void TraceLog::RemoveProcessLabel(int label_id) {
+  SpinLockHolder lock(&lock_);
+  auto it = process_labels_.find(label_id);
+  if (it == process_labels_.end())
+    return;
+
+  process_labels_.erase(it);
+}
+
+void TraceLog::SetThreadSortIndex(int64_t thread_id, int sort_index) {
+  SpinLockHolder lock(&lock_);
+  thread_sort_indices_[static_cast<int>(thread_id)] = sort_index;
+}
+
+void TraceLog::SetTimeOffset(MicrosecondsInt64 offset) {
+  time_offset_ = offset;
+}
+
+size_t TraceLog::GetObserverCountForTest() const {
+  return enabled_state_observer_list_.size();
+}
+
+bool CategoryFilter::IsEmptyOrContainsLeadingOrTrailingWhitespace(
+    const std::string& str) {
+  return  str.empty() ||
+          str.at(0) == ' ' ||
+          str.at(str.length() - 1) == ' ';
+}
+
+bool CategoryFilter::DoesCategoryGroupContainCategory(
+    const char* category_group,
+    const char* category) const {
+  DCHECK(category);
+  vector<string> pieces = strings::Split(category_group, ",");
+  for (const string& category_group_token : pieces) {
+    // Don't allow empty tokens, nor tokens with leading or trailing space.
+    DCHECK(!CategoryFilter::IsEmptyOrContainsLeadingOrTrailingWhitespace(
+        category_group_token))
+        << "Disallowed category string";
+
+    if (MatchPattern(category_group_token.c_str(), category))
+      return true;
+  }
+  return false;
+}
+
+CategoryFilter::CategoryFilter(const std::string& filter_string) {
+  if (!filter_string.empty())
+    Initialize(filter_string);
+  else
+    Initialize(CategoryFilter::kDefaultCategoryFilterString);
+}
+
+CategoryFilter::CategoryFilter(const CategoryFilter& cf)
+    : included_(cf.included_),
+      disabled_(cf.disabled_),
+      excluded_(cf.excluded_),
+      delays_(cf.delays_) {
+}
+
+CategoryFilter::~CategoryFilter() {
+}
+
+CategoryFilter& CategoryFilter::operator=(const CategoryFilter& rhs) {
+  if (this == &rhs)
+    return *this;
+
+  included_ = rhs.included_;
+  disabled_ = rhs.disabled_;
+  excluded_ = rhs.excluded_;
+  delays_ = rhs.delays_;
+  return *this;
+}
+
+void CategoryFilter::Initialize(const std::string& filter_string) {
+  // Tokenize list of categories, delimited by ','.
+  vector<string> tokens = strings::Split(filter_string, ",");
+  // Add each token to the appropriate list (included_,excluded_).
+  for (string category : tokens) {
+    // Ignore empty categories.
+    if (category.empty())
+      continue;
+    // Synthetic delays are of the form 'DELAY(delay;option;option;...)'.
+    if (category.find(kSyntheticDelayCategoryFilterPrefix) == 0 &&
+        category.at(category.size() - 1) == ')') {
+      category = category.substr(
+          strlen(kSyntheticDelayCategoryFilterPrefix),
+          category.size() - strlen(kSyntheticDelayCategoryFilterPrefix) - 1);
+      size_t name_length = category.find(';');
+      if (name_length != std::string::npos && name_length > 0 &&
+          name_length != category.size() - 1) {
+        delays_.push_back(category);
+      }
+    } else if (category.at(0) == '-') {
+      // Excluded categories start with '-'.
+      // Remove '-' from category string.
+      category = category.substr(1);
+      excluded_.push_back(category);
+    } else if (category.compare(0, strlen(TRACE_DISABLED_BY_DEFAULT("")),
+                                TRACE_DISABLED_BY_DEFAULT("")) == 0) {
+      disabled_.push_back(category);
+    } else {
+      included_.push_back(category);
+    }
+  }
+}
+
+void CategoryFilter::WriteString(const StringList& values,
+                                 std::string* out,
+                                 bool included) const {
+  bool prepend_comma = !out->empty();
+  int token_cnt = 0;
+  for (const auto& value : values) {
+    if (token_cnt > 0 || prepend_comma)
+      StringAppendF(out, ",");
+    StringAppendF(out, "%s%s", (included ? "" : "-"), value.c_str());
+    ++token_cnt;
+  }
+}
+
+void CategoryFilter::WriteString(const StringList& delays,
+                                 std::string* out) const {
+  bool prepend_comma = !out->empty();
+  int token_cnt = 0;
+  for (const auto& delay : delays) {
+    if (token_cnt > 0 || prepend_comma)
+      StringAppendF(out, ",");
+    StringAppendF(out, "%s%s)", kSyntheticDelayCategoryFilterPrefix,
+                  delay.c_str());
+    ++token_cnt;
+  }
+}
+
+std::string CategoryFilter::ToString() const {
+  std::string filter_string;
+  WriteString(included_, &filter_string, true);
+  WriteString(disabled_, &filter_string, true);
+  WriteString(excluded_, &filter_string, false);
+  WriteString(delays_, &filter_string);
+  return filter_string;
+}
+
+bool CategoryFilter::IsCategoryGroupEnabled(
+    const char* category_group_name) const {
+  // TraceLog should call this method only as  part of enabling/disabling
+  // categories.
+  StringList::const_iterator ci;
+
+  // Check the disabled- filters and the disabled-* wildcard first so that a
+  // "*" filter does not include the disabled.
+  for (ci = disabled_.begin(); ci != disabled_.end(); ++ci) {
+    if (DoesCategoryGroupContainCategory(category_group_name, ci->c_str()))
+      return true;
+  }
+  if (DoesCategoryGroupContainCategory(category_group_name,
+                                       TRACE_DISABLED_BY_DEFAULT("*")))
+    return false;
+
+  for (ci = included_.begin(); ci != included_.end(); ++ci) {
+    if (DoesCategoryGroupContainCategory(category_group_name, ci->c_str()))
+      return true;
+  }
+
+  for (ci = excluded_.begin(); ci != excluded_.end(); ++ci) {
+    if (DoesCategoryGroupContainCategory(category_group_name, ci->c_str()))
+      return false;
+  }
+  // If the category group is not excluded, and there are no included patterns
+  // we consider this pattern enabled.
+  return included_.empty();
+}
+
+bool CategoryFilter::HasIncludedPatterns() const {
+  return !included_.empty();
+}
+
+void CategoryFilter::Merge(const CategoryFilter& nested_filter) {
+  // Keep included patterns only if both filters have an included entry.
+  // Otherwise, one of the filter was specifying "*" and we want to honour the
+  // broadest filter.
+  if (HasIncludedPatterns() && nested_filter.HasIncludedPatterns()) {
+    included_.insert(included_.end(),
+                     nested_filter.included_.begin(),
+                     nested_filter.included_.end());
+  } else {
+    included_.clear();
+  }
+
+  disabled_.insert(disabled_.end(),
+                   nested_filter.disabled_.begin(),
+                   nested_filter.disabled_.end());
+  excluded_.insert(excluded_.end(),
+                   nested_filter.excluded_.begin(),
+                   nested_filter.excluded_.end());
+  delays_.insert(delays_.end(),
+                 nested_filter.delays_.begin(),
+                 nested_filter.delays_.end());
+}
+
+void CategoryFilter::Clear() {
+  included_.clear();
+  disabled_.clear();
+  excluded_.clear();
+}
+
+const CategoryFilter::StringList&
+    CategoryFilter::GetSyntheticDelayValues() const {
+  return delays_;
+}
+
+}  // namespace debug
+}  // namespace kudu
+
+namespace trace_event_internal {
+
+ScopedTraceBinaryEfficient::ScopedTraceBinaryEfficient(
+    const char* category_group, const char* name) {
+  // The single atom works because for now the category_group can only be "gpu".
+  DCHECK(strcmp(category_group, "gpu") == 0);
+  static TRACE_EVENT_API_ATOMIC_WORD atomic = 0;
+  INTERNAL_TRACE_EVENT_GET_CATEGORY_INFO_CUSTOM_VARIABLES(
+      category_group, atomic, category_group_enabled_);
+  name_ = name;
+  if (*category_group_enabled_) {
+    event_handle_ =
+        TRACE_EVENT_API_ADD_TRACE_EVENT_WITH_THREAD_ID_AND_TIMESTAMP(
+            TRACE_EVENT_PHASE_COMPLETE, category_group_enabled_, name,
+            trace_event_internal::kNoEventId,
+            static_cast<int>(kudu::Thread::UniqueThreadId()),
+            GetMonoTimeMicros(),
+            0, nullptr, nullptr, nullptr, nullptr, TRACE_EVENT_FLAG_NONE);
+  }
+}
+
+ScopedTraceBinaryEfficient::~ScopedTraceBinaryEfficient() {
+  if (*category_group_enabled_) {
+    TRACE_EVENT_API_UPDATE_TRACE_EVENT_DURATION(category_group_enabled_,
+                                                name_, event_handle_);
+  }
+}
+
+}  // namespace trace_event_internal


[12/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/monotime.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/monotime.cc b/be/src/kudu/util/monotime.cc
new file mode 100644
index 0000000..89c795d
--- /dev/null
+++ b/be/src/kudu/util/monotime.cc
@@ -0,0 +1,334 @@
+// 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 "kudu/util/monotime.h"
+
+#include <sys/time.h>
+
+#include <ctime>
+#include <limits>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/integral_types.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/util/thread_restrictions.h"
+#if defined(__APPLE__)
+#include "kudu/gutil/walltime.h"
+#endif
+
+namespace kudu {
+
+#define MAX_MONOTONIC_SECONDS \
+  (((1ULL<<63) - 1ULL) /(int64_t)MonoTime::kNanosecondsPerSecond)
+
+
+///
+/// MonoDelta
+///
+
+const int64_t MonoDelta::kUninitialized = kint64min;
+
+MonoDelta MonoDelta::FromSeconds(double seconds) {
+  int64_t delta = seconds * MonoTime::kNanosecondsPerSecond;
+  return MonoDelta(delta);
+}
+
+MonoDelta MonoDelta::FromMilliseconds(int64_t ms) {
+  return MonoDelta(ms * MonoTime::kNanosecondsPerMillisecond);
+}
+
+MonoDelta MonoDelta::FromMicroseconds(int64_t us) {
+  return MonoDelta(us * MonoTime::kNanosecondsPerMicrosecond);
+}
+
+MonoDelta MonoDelta::FromNanoseconds(int64_t ns) {
+  return MonoDelta(ns);
+}
+
+MonoDelta::MonoDelta()
+  : nano_delta_(kUninitialized) {
+}
+
+bool MonoDelta::Initialized() const {
+  return nano_delta_ != kUninitialized;
+}
+
+bool MonoDelta::LessThan(const MonoDelta &rhs) const {
+  DCHECK(Initialized());
+  DCHECK(rhs.Initialized());
+  return nano_delta_ < rhs.nano_delta_;
+}
+
+bool MonoDelta::MoreThan(const MonoDelta &rhs) const {
+  DCHECK(Initialized());
+  DCHECK(rhs.Initialized());
+  return nano_delta_ > rhs.nano_delta_;
+}
+
+bool MonoDelta::Equals(const MonoDelta &rhs) const {
+  DCHECK(Initialized());
+  DCHECK(rhs.Initialized());
+  return nano_delta_ == rhs.nano_delta_;
+}
+
+std::string MonoDelta::ToString() const {
+  return StringPrintf("%.3fs", ToSeconds());
+}
+
+MonoDelta::MonoDelta(int64_t delta)
+  : nano_delta_(delta) {
+}
+
+double MonoDelta::ToSeconds() const {
+  DCHECK(Initialized());
+  double d(nano_delta_);
+  d /= MonoTime::kNanosecondsPerSecond;
+  return d;
+}
+
+int64_t MonoDelta::ToNanoseconds() const {
+  DCHECK(Initialized());
+  return nano_delta_;
+}
+
+int64_t MonoDelta::ToMicroseconds() const {
+  DCHECK(Initialized());
+ return nano_delta_ / MonoTime::kNanosecondsPerMicrosecond;
+}
+
+int64_t MonoDelta::ToMilliseconds() const {
+  DCHECK(Initialized());
+  return nano_delta_ / MonoTime::kNanosecondsPerMillisecond;
+}
+
+void MonoDelta::ToTimeVal(struct timeval *tv) const {
+  DCHECK(Initialized());
+  tv->tv_sec = nano_delta_ / MonoTime::kNanosecondsPerSecond;
+  tv->tv_usec = (nano_delta_ - (tv->tv_sec * MonoTime::kNanosecondsPerSecond))
+      / MonoTime::kNanosecondsPerMicrosecond;
+
+  // tv_usec must be between 0 and 999999.
+  // There is little use for negative timevals so wrap it in PREDICT_FALSE.
+  if (PREDICT_FALSE(tv->tv_usec < 0)) {
+    --(tv->tv_sec);
+    tv->tv_usec += 1000000;
+  }
+
+  // Catch positive corner case where we "round down" and could potentially set a timeout of 0.
+  // Make it 1 usec.
+  if (PREDICT_FALSE(tv->tv_usec == 0 && tv->tv_sec == 0 && nano_delta_ > 0)) {
+    tv->tv_usec = 1;
+  }
+
+  // Catch negative corner case where we "round down" and could potentially set a timeout of 0.
+  // Make it -1 usec (but normalized, so tv_usec is not negative).
+  if (PREDICT_FALSE(tv->tv_usec == 0 && tv->tv_sec == 0 && nano_delta_ < 0)) {
+    tv->tv_sec = -1;
+    tv->tv_usec = 999999;
+  }
+}
+
+
+void MonoDelta::NanosToTimeSpec(int64_t nanos, struct timespec* ts) {
+  ts->tv_sec = nanos / MonoTime::kNanosecondsPerSecond;
+  ts->tv_nsec = nanos - (ts->tv_sec * MonoTime::kNanosecondsPerSecond);
+
+  // tv_nsec must be between 0 and 999999999.
+  // There is little use for negative timespecs so wrap it in PREDICT_FALSE.
+  if (PREDICT_FALSE(ts->tv_nsec < 0)) {
+    --(ts->tv_sec);
+    ts->tv_nsec += MonoTime::kNanosecondsPerSecond;
+  }
+}
+
+void MonoDelta::ToTimeSpec(struct timespec *ts) const {
+  DCHECK(Initialized());
+  NanosToTimeSpec(nano_delta_, ts);
+}
+
+///
+/// MonoTime
+///
+
+MonoTime MonoTime::Now() {
+#if defined(__APPLE__)
+  return MonoTime(walltime_internal::GetMonoTimeNanos());
+# else
+  struct timespec ts;
+  PCHECK(clock_gettime(CLOCK_MONOTONIC, &ts) == 0);
+  return MonoTime(ts);
+#endif // defined(__APPLE__)
+}
+
+MonoTime MonoTime::Max() {
+  return MonoTime(std::numeric_limits<int64_t>::max());
+}
+
+MonoTime MonoTime::Min() {
+  return MonoTime(1);
+}
+
+const MonoTime& MonoTime::Earliest(const MonoTime& a, const MonoTime& b) {
+  if (b.nanos_ < a.nanos_) {
+    return b;
+  }
+  return a;
+}
+
+MonoTime::MonoTime()
+  : nanos_(0) {
+}
+
+bool MonoTime::Initialized() const {
+  return nanos_ != 0;
+}
+
+MonoDelta MonoTime::GetDeltaSince(const MonoTime &rhs) const {
+  DCHECK(Initialized());
+  DCHECK(rhs.Initialized());
+  int64_t delta(nanos_);
+  delta -= rhs.nanos_;
+  return MonoDelta(delta);
+}
+
+void MonoTime::AddDelta(const MonoDelta &delta) {
+  DCHECK(Initialized());
+  nanos_ += delta.nano_delta_;
+}
+
+bool MonoTime::ComesBefore(const MonoTime &rhs) const {
+  DCHECK(Initialized());
+  DCHECK(rhs.Initialized());
+  return nanos_ < rhs.nanos_;
+}
+
+std::string MonoTime::ToString() const {
+  return StringPrintf("%.3fs", ToSeconds());
+}
+
+void MonoTime::ToTimeSpec(struct timespec* ts) const {
+  DCHECK(Initialized());
+  MonoDelta::NanosToTimeSpec(nanos_, ts);
+}
+
+bool MonoTime::Equals(const MonoTime& other) const {
+  return nanos_ == other.nanos_;
+}
+
+MonoTime& MonoTime::operator+=(const MonoDelta& delta) {
+  this->AddDelta(delta);
+  return *this;
+}
+
+MonoTime& MonoTime::operator-=(const MonoDelta& delta) {
+  this->AddDelta(MonoDelta(-1 * delta.nano_delta_));
+  return *this;
+}
+
+MonoTime::MonoTime(const struct timespec &ts) {
+  // Monotonic time resets when the machine reboots.  The 64-bit limitation
+  // means that we can't represent times larger than 292 years, which should be
+  // adequate.
+  CHECK_LT(ts.tv_sec, MAX_MONOTONIC_SECONDS);
+  nanos_ = ts.tv_sec;
+  nanos_ *= MonoTime::kNanosecondsPerSecond;
+  nanos_ += ts.tv_nsec;
+}
+
+MonoTime::MonoTime(int64_t nanos)
+  : nanos_(nanos) {
+}
+
+double MonoTime::ToSeconds() const {
+  double d(nanos_);
+  d /= MonoTime::kNanosecondsPerSecond;
+  return d;
+}
+
+void SleepFor(const MonoDelta& delta) {
+  ThreadRestrictions::AssertWaitAllowed();
+  base::SleepForNanoseconds(delta.ToNanoseconds());
+}
+
+bool operator==(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return lhs.Equals(rhs);
+}
+
+bool operator!=(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return !lhs.Equals(rhs);
+}
+
+bool operator<(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return lhs.LessThan(rhs);
+}
+
+bool operator<=(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return lhs.LessThan(rhs) || lhs.Equals(rhs);
+}
+
+bool operator>(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return lhs.MoreThan(rhs);
+}
+
+bool operator>=(const MonoDelta &lhs, const MonoDelta &rhs) {
+  return lhs.MoreThan(rhs) || lhs.Equals(rhs);
+}
+
+bool operator==(const MonoTime& lhs, const MonoTime& rhs) {
+  return lhs.Equals(rhs);
+}
+
+bool operator!=(const MonoTime& lhs, const MonoTime& rhs) {
+  return !lhs.Equals(rhs);
+}
+
+bool operator<(const MonoTime& lhs, const MonoTime& rhs) {
+  return lhs.ComesBefore(rhs);
+}
+
+bool operator<=(const MonoTime& lhs, const MonoTime& rhs) {
+  return lhs.ComesBefore(rhs) || lhs.Equals(rhs);
+}
+
+bool operator>(const MonoTime& lhs, const MonoTime& rhs) {
+  return rhs.ComesBefore(lhs);
+}
+
+bool operator>=(const MonoTime& lhs, const MonoTime& rhs) {
+  return rhs.ComesBefore(lhs) || rhs.Equals(lhs);
+}
+
+MonoTime operator+(const MonoTime& t, const MonoDelta& delta) {
+  MonoTime tmp(t);
+  tmp.AddDelta(delta);
+  return tmp;
+}
+
+MonoTime operator-(const MonoTime& t, const MonoDelta& delta) {
+  MonoTime tmp(t);
+  tmp.AddDelta(MonoDelta::FromNanoseconds(-delta.ToNanoseconds()));
+  return tmp;
+}
+
+MonoDelta operator-(const MonoTime& t_end, const MonoTime& t_beg) {
+  return t_end.GetDeltaSince(t_beg);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/monotime.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/monotime.h b/be/src/kudu/util/monotime.h
new file mode 100644
index 0000000..bb8ec35
--- /dev/null
+++ b/be/src/kudu/util/monotime.h
@@ -0,0 +1,421 @@
+// 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.
+#ifndef KUDU_UTIL_MONOTIME_H
+#define KUDU_UTIL_MONOTIME_H
+
+// NOTE: using stdint.h instead of cstdint because this file is supposed
+//       to be processed by a compiler lacking C++11 support.
+#include <stdint.h>
+
+#include <string>
+
+#ifdef KUDU_HEADERS_NO_STUBS
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/port.h"
+#else
+// This is a poor module interdependency, but the stubs are header-only and
+// it's only for exported header builds, so we'll make an exception.
+#include "kudu/client/stubs.h"
+#endif
+
+#include "kudu/util/kudu_export.h"
+
+namespace kudu {
+
+/// @brief A representation of a time interval.
+///
+/// The MonoDelta class represents an elapsed duration of time -- i.e.
+/// the delta between two MonoTime instances.
+class KUDU_EXPORT MonoDelta {
+ public:
+  /// @name Converters from seconds representation (and ubiquitous SI prefixes).
+  ///
+  /// @param [in] seconds/ms/us/ns
+  ///   Time interval representation in seconds (with ubiquitous SI prefixes).
+  /// @return The resulting MonoDelta object initialized in accordance with
+  ///   the specified parameter.
+  ///
+  ///@{
+  static MonoDelta FromSeconds(double seconds);
+  static MonoDelta FromMilliseconds(int64_t ms);
+  static MonoDelta FromMicroseconds(int64_t us);
+  static MonoDelta FromNanoseconds(int64_t ns);
+  ///@}
+
+  /// Build a MonoDelta object.
+  ///
+  /// @note A MonoDelta instance built with the this default constructor is
+  ///   "uninitialized" and may not be used for any operation.
+  MonoDelta();
+
+  /// @return @c true iff this object is initialized.
+  bool Initialized() const;
+
+  /// Check whether this time interval is shorter than the specified one.
+  ///
+  /// @param [in] rhs
+  ///   A time interval for comparison.
+  /// @return @c true iff this time interval is strictly shorter
+  ///   than the specified one.
+  bool LessThan(const MonoDelta &rhs) const;
+
+  /// Check whether this time interval is longer than the specified one.
+  ///
+  /// @param [in] rhs
+  ///   A time interval for comparison.
+  /// @return @c true iff this time interval is strictly longer
+  ///   than the specified one.
+  bool MoreThan(const MonoDelta &rhs) const;
+
+  /// Check whether this time interval has the same duration
+  ///  as the specified one.
+  ///
+  /// @param [in] rhs
+  ///   A time interval for comparison.
+  /// @return @c true iff this time interval has the same duration as the
+  ///   the specified one.
+  bool Equals(const MonoDelta &rhs) const;
+
+  /// @return String representation of this interval's duration (in seconds).
+  std::string ToString() const;
+
+  /// @name Converters into seconds representation (and ubiquitous SI prefixes).
+  ///
+  /// @return Representation of the time interval in appropriate SI units.
+  ///
+  ///@{
+  double ToSeconds() const;
+  int64_t ToMilliseconds() const;
+  int64_t ToMicroseconds() const;
+  int64_t ToNanoseconds() const;
+  ///@}
+
+  /// Represent this time interval as a timeval structure, with microsecond
+  /// accuracy.
+  ///
+  /// @param [out] tv
+  ///   Placeholder for the result value.
+  void ToTimeVal(struct timeval *tv) const;
+
+  /// Represent this time interval as a timespec structure, with nanosecond
+  /// accuracy.
+  ///
+  /// @param [out] ts
+  ///   Placeholder for the result value.
+  void ToTimeSpec(struct timespec *ts) const;
+
+  /// Convert a nanosecond value to a timespec.
+  ///
+  /// @param [in] nanos
+  ///   Representation of a relative point in time in nanoseconds.
+  /// @param [out] ts
+  ///   Placeholder for the resulting timespec representation.
+  static void NanosToTimeSpec(int64_t nanos, struct timespec* ts);
+
+ private:
+  static const int64_t kUninitialized;
+
+  friend class MonoTime;
+  FRIEND_TEST(TestMonoTime, TestDeltaConversions);
+
+  explicit MonoDelta(int64_t delta);
+  int64_t nano_delta_;
+};
+
+/// @brief Representation of a particular point in time.
+///
+/// The MonoTime class represents a particular point in time,
+/// relative to some fixed but unspecified reference point.
+///
+/// This time is monotonic, meaning that if the user changes his or her system
+/// clock, the monotime does not change.
+class KUDU_EXPORT MonoTime {
+ public:
+  /// @name Conversion constants for ubiquitous time units.
+  ///
+  ///@{
+  static const int64_t kNanosecondsPerSecond = 1000000000L;
+  static const int64_t kNanosecondsPerMillisecond = 1000000L;
+  static const int64_t kNanosecondsPerMicrosecond = 1000L;
+
+  static const int64_t kMicrosecondsPerSecond = 1000000L;
+  ///@}
+
+  /// Get current time in MonoTime representation.
+  ///
+  /// @return Time specification for the moment of the method's invocation.
+  static MonoTime Now();
+
+  /// @return MonoTime equal to farthest possible time into the future.
+  static MonoTime Max();
+
+  /// @return MonoTime equal to farthest possible time into the past.
+  static MonoTime Min();
+
+  /// Select the earliest between the specified time points.
+  ///
+  /// @param [in] a
+  ///   The first MonoTime object to select from.
+  /// @param [in] b
+  ///   The second MonoTime object to select from.
+  /// @return The earliest (minimum) of the two monotimes.
+  static const MonoTime& Earliest(const MonoTime& a, const MonoTime& b)
+      ATTRIBUTE_DEPRECATED("use std::min() instead");
+
+  /// Build a MonoTime object. The resulting object is not initialized
+  /// and not ready to use.
+  MonoTime();
+
+  /// @return @c true iff the object is initialized.
+  bool Initialized() const;
+
+  /// Compute time interval between the point in time specified by this
+  /// and the specified object.
+  ///
+  /// @param [in] rhs
+  ///   The object that corresponds to the left boundary of the time interval,
+  ///   where this object corresponds to the right boundary of the interval.
+  /// @return The resulting time interval represented as a MonoDelta object.
+  MonoDelta GetDeltaSince(const MonoTime &rhs) const;
+
+  /// Advance this object's time specification by the specified interval.
+  ///
+  /// @param [in] delta
+  ///   The time interval to add.
+  void AddDelta(const MonoDelta &delta);
+
+  /// Check whether the point in time specified by this object is earlier
+  /// than the specified one.
+  ///
+  /// @param [in] rhs
+  ///   The other MonoTime object to compare with.
+  /// @return @c true iff the point in time represented by this MonoTime object
+  ///   is earlier then the point in time represented by the parameter.
+  bool ComesBefore(const MonoTime &rhs) const;
+
+  /// @return String representation of the object (in seconds).
+  std::string ToString() const;
+
+  /// Represent this point in time as a timespec structure, with nanosecond
+  /// accuracy.
+  ///
+  /// @param [out] ts
+  ///   Placeholder for the result value.
+  void ToTimeSpec(struct timespec* ts) const;
+
+  /// Check whether this object represents the same point in time as the other.
+  ///
+  /// @param [in] other
+  ///   The other MonoTime object to compare.
+  /// @return @c true iff the point in time represented by this MonoTime object
+  ///   is the same as the one represented by the other.
+  bool Equals(const MonoTime& other) const;
+
+  /// @name Syntactic sugar: increment/decrement operators for MonoTime.
+  ///@{
+  ///
+  /// Add a delta to the point in time represented by the object.
+  ///
+  /// @param [in] delta
+  ///   The delta to add.
+  /// @return Reference to the modified object.
+  MonoTime& operator+=(const MonoDelta& delta);
+
+  /// Substract a delta from the point in time represented by the object.
+  ///
+  /// @param [in] delta
+  ///   The delta to substract.
+  /// @return Reference to the modified object.
+  MonoTime& operator-=(const MonoDelta& delta);
+  ///@}
+
+ private:
+  friend class MonoDelta;
+  FRIEND_TEST(TestMonoTime, TestTimeSpec);
+  FRIEND_TEST(TestMonoTime, TestDeltaConversions);
+
+  explicit MonoTime(const struct timespec &ts);
+  explicit MonoTime(int64_t nanos);
+  double ToSeconds() const;
+  int64_t nanos_;
+};
+
+/// Sleep for an interval specified by a MonoDelta instance.
+///
+/// This is preferred over sleep(3), usleep(3), and nanosleep(3).
+/// It's less prone to mixups with units since it uses the MonoDelta for
+/// interval specification.
+/// Besides, it ignores signals/EINTR, so will reliably sleep at least for the
+/// MonoDelta duration.
+///
+/// @param [in] delta
+///   The time interval to sleep for.
+void KUDU_EXPORT SleepFor(const MonoDelta& delta);
+
+/// @name Syntactic sugar: binary operators for MonoDelta.
+///@{
+///
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is equal
+///   to the time interval represented by @c rhs.
+bool KUDU_EXPORT operator==(const MonoDelta &lhs, const MonoDelta &rhs);
+
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is not equal
+///   to the time interval represented by @c rhs.
+bool KUDU_EXPORT operator!=(const MonoDelta &lhs, const MonoDelta &rhs);
+
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is shorter
+///   than the time interval represented by @c rhs.
+bool KUDU_EXPORT operator<(const MonoDelta &lhs, const MonoDelta &rhs);
+
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is shorter
+///   than or equal to the time interval represented by @c rhs.
+bool KUDU_EXPORT operator<=(const MonoDelta &lhs, const MonoDelta &rhs);
+
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is longer
+///   than the time interval represented by @c rhs.
+bool KUDU_EXPORT operator>(const MonoDelta &lhs, const MonoDelta &rhs);
+
+/// @param [in] lhs
+///   A time interval for comparison: the left-hand operand.
+/// @param [in] rhs
+///   A time interval for comparison: the right-hand operand.
+/// @return @c true iff the time interval represented by @c lhs is longer
+///   than or equal to the time interval represented by @c rhs.
+bool KUDU_EXPORT operator>=(const MonoDelta &lhs, const MonoDelta &rhs);
+///@}
+
+/// @name Syntactic sugar: binary operators for MonoTime.
+///@{
+///
+/// Check if the specified objects represent the same point in time.
+///
+/// This is a handy operator which is semantically equivalent to
+/// MonoTime::Equals().
+///
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the given objects represent the same point in time.
+bool KUDU_EXPORT operator==(const MonoTime& lhs, const MonoTime& rhs);
+
+/// Check if the specified objects represent different points in time.
+///
+/// This is a handy operator which is semantically equivalent to the negation of
+/// MonoTime::Equals().
+///
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the given object represents a different point in time
+///   than the specified one.
+bool KUDU_EXPORT operator!=(const MonoTime& lhs, const MonoTime& rhs);
+
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the @c lhs object represents an earlier point in time
+///   than the @c rhs object.
+bool KUDU_EXPORT operator<(const MonoTime& lhs, const MonoTime& rhs);
+
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the @c lhs object represents an earlier than or
+///   the same point in time as the @c rhs object.
+bool KUDU_EXPORT operator<=(const MonoTime& lhs, const MonoTime& rhs);
+
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the @c lhs object represents a later point in time
+///   than the @c rhs object.
+bool KUDU_EXPORT operator>(const MonoTime& lhs, const MonoTime& rhs);
+
+/// @param [in] lhs
+///   The left-hand operand.
+/// @param [in] rhs
+///   The right-hand operand.
+/// @return @c true iff the @c lhs object represents a later than or
+///   the same point in time as the @c rhs object.
+bool KUDU_EXPORT operator>=(const MonoTime& lhs, const MonoTime& rhs);
+///@}
+
+/// @name Syntactic sugar: mixed binary operators for MonoTime/MonoDelta.
+///@{
+///
+/// Add the specified time interval to the given point in time.
+///
+/// @param [in] t
+///   A MonoTime object representing the given point in time.
+/// @param [in] delta
+///   A MonoDelta object representing the specified time interval.
+/// @return A MonoTime object representing the resulting point in time.
+MonoTime KUDU_EXPORT operator+(const MonoTime& t, const MonoDelta& delta);
+
+/// Subtract the specified time interval from the given point in time.
+///
+/// @param [in] t
+///   A MonoTime object representing the given point in time.
+/// @param [in] delta
+///   A MonoDelta object representing the specified time interval.
+/// @return A MonoTime object representing the resulting point in time.
+MonoTime KUDU_EXPORT operator-(const MonoTime& t, const MonoDelta& delta);
+
+/// Compute the time interval between the specified points in time.
+///
+/// Semantically, this is equivalent to t0.GetDeltaSince(t1).
+///
+/// @param [in] t_end
+///   The second point in time.  Semantically corresponds to the end
+///   of the resulting time interval.
+/// @param [in] t_beg
+///   The first point in time.  Semantically corresponds to the beginning
+///   of the resulting time interval.
+/// @return A MonoDelta object representing the time interval between the
+///   specified points in time.
+MonoDelta KUDU_EXPORT operator-(const MonoTime& t_end, const MonoTime& t_begin);
+///@}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mt-hdr_histogram-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mt-hdr_histogram-test.cc b/be/src/kudu/util/mt-hdr_histogram-test.cc
new file mode 100644
index 0000000..7221644
--- /dev/null
+++ b/be/src/kudu/util/mt-hdr_histogram-test.cc
@@ -0,0 +1,116 @@
+// 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 <cstdint>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/hdr_histogram.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+DEFINE_int32(histogram_test_num_threads, 16,
+    "Number of threads to spawn for mt-hdr_histogram test");
+DEFINE_uint64(histogram_test_num_increments_per_thread, 100000LU,
+    "Number of times to call Increment() per thread in mt-hdr_histogram test");
+
+using std::vector;
+
+namespace kudu {
+
+class MtHdrHistogramTest : public KuduTest {
+ public:
+  MtHdrHistogramTest() {
+    num_threads_ = FLAGS_histogram_test_num_threads;
+    num_times_ = FLAGS_histogram_test_num_increments_per_thread;
+  }
+
+ protected:
+  int num_threads_;
+  uint64_t num_times_;
+};
+
+// Increment a counter a bunch of times in the same bucket
+static void IncrementSameHistValue(HdrHistogram* hist, uint64_t value, uint64_t times) {
+  for (uint64_t i = 0; i < times; i++) {
+    hist->Increment(value);
+  }
+}
+
+TEST_F(MtHdrHistogramTest, ConcurrentWriteTest) {
+  const uint64_t kValue = 1LU;
+
+  HdrHistogram hist(100000LU, 3);
+
+  auto threads = new scoped_refptr<kudu::Thread>[num_threads_];
+  for (int i = 0; i < num_threads_; i++) {
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("thread-$0", i),
+        IncrementSameHistValue, &hist, kValue, num_times_, &threads[i]));
+  }
+  for (int i = 0; i < num_threads_; i++) {
+    CHECK_OK(ThreadJoiner(threads[i].get()).Join());
+  }
+
+  HdrHistogram snapshot(hist);
+  ASSERT_EQ(num_threads_ * num_times_, snapshot.CountInBucketForValue(kValue));
+
+  delete[] threads;
+}
+
+// Copy while writing, then iterate to ensure copies are consistent.
+TEST_F(MtHdrHistogramTest, ConcurrentCopyWhileWritingTest) {
+  const int kNumCopies = 10;
+  const uint64_t kValue = 1;
+
+  HdrHistogram hist(100000LU, 3);
+
+  auto threads = new scoped_refptr<kudu::Thread>[num_threads_];
+  for (int i = 0; i < num_threads_; i++) {
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("thread-$0", i),
+        IncrementSameHistValue, &hist, kValue, num_times_, &threads[i]));
+  }
+
+  // This is somewhat racy but the goal is to catch this issue at least
+  // most of the time. At the time of this writing, before fixing a bug where
+  // the total count stored in a copied histogram may not match its internal
+  // counts (under concurrent writes), this test fails for me on 100/100 runs.
+  vector<HdrHistogram *> snapshots;
+  ElementDeleter deleter(&snapshots);
+  for (int i = 0; i < kNumCopies; i++) {
+    snapshots.push_back(new HdrHistogram(hist));
+    SleepFor(MonoDelta::FromMicroseconds(100));
+  }
+  for (int i = 0; i < kNumCopies; i++) {
+    snapshots[i]->MeanValue(); // Will crash if underlying iterator is inconsistent.
+  }
+
+  for (int i = 0; i < num_threads_; i++) {
+    CHECK_OK(ThreadJoiner(threads[i].get()).Join());
+  }
+
+  delete[] threads;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mt-metrics-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mt-metrics-test.cc b/be/src/kudu/util/mt-metrics-test.cc
new file mode 100644
index 0000000..2480d57
--- /dev/null
+++ b/be/src/kudu/util/mt-metrics-test.cc
@@ -0,0 +1,128 @@
+// 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 <cstdint>
+#include <cstring>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/function.hpp> // IWYU pragma: keep
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/env.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+DEFINE_int32(mt_metrics_test_num_threads, 4,
+             "Number of threads to spawn in mt metrics tests");
+
+METRIC_DEFINE_entity(test_entity);
+
+namespace kudu {
+
+using debug::ScopedLeakCheckDisabler;
+using std::string;
+using std::vector;
+
+class MultiThreadedMetricsTest : public KuduTest {
+ public:
+  static void RegisterCounters(const scoped_refptr<MetricEntity>& metric_entity,
+                               const string& name_prefix, int num_counters);
+
+  MetricRegistry registry_;
+};
+
+// Call increment on a Counter a bunch of times.
+static void CountWithCounter(scoped_refptr<Counter> counter, int num_increments) {
+  for (int i = 0; i < num_increments; i++) {
+    counter->Increment();
+  }
+}
+
+// Helper function that spawns and then joins a bunch of threads.
+static void RunWithManyThreads(boost::function<void()>* f, int num_threads) {
+  vector<scoped_refptr<kudu::Thread> > threads;
+  for (int i = 0; i < num_threads; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("test", StringPrintf("thread%d", i),
+          *f, &new_thread));
+    threads.push_back(new_thread);
+  }
+  for (int i = 0; i < num_threads; i++) {
+    ASSERT_OK(ThreadJoiner(threads[i].get()).Join());
+  }
+}
+
+METRIC_DEFINE_counter(test_entity, test_counter, "Test Counter",
+                      MetricUnit::kRequests, "Test counter");
+
+// Ensure that incrementing a counter is thread-safe.
+TEST_F(MultiThreadedMetricsTest, CounterIncrementTest) {
+  scoped_refptr<Counter> counter = new Counter(&METRIC_test_counter);
+  int num_threads = FLAGS_mt_metrics_test_num_threads;
+  int num_increments = 1000;
+  boost::function<void()> f =
+      boost::bind(CountWithCounter, counter, num_increments);
+  RunWithManyThreads(&f, num_threads);
+  ASSERT_EQ(num_threads * num_increments, counter->value());
+}
+
+// Helper function to register a bunch of counters in a loop.
+void MultiThreadedMetricsTest::RegisterCounters(
+    const scoped_refptr<MetricEntity>& metric_entity,
+    const string& name_prefix,
+    int num_counters) {
+  uint64_t tid = Env::Default()->gettid();
+  for (int i = 0; i < num_counters; i++) {
+    // This loop purposefully leaks metrics prototypes, because the metrics system
+    // expects the prototypes and their names to live forever. This is the only
+    // place we dynamically generate them for the purposes of a test, so it's easier
+    // to just leak them than to figure out a way to manage lifecycle of objects that
+    // are typically static.
+    ScopedLeakCheckDisabler disabler;
+
+    string name = strings::Substitute("$0-$1-$2", name_prefix, tid, i);
+    auto proto = new CounterPrototype(MetricPrototype::CtorArgs(
+        "test_entity", strdup(name.c_str()), "Test Counter",
+        MetricUnit::kOperations, "test counter"));
+    proto->Instantiate(metric_entity)->Increment();
+  }
+}
+
+// Ensure that adding a counter to a registry is thread-safe.
+TEST_F(MultiThreadedMetricsTest, AddCounterToRegistryTest) {
+  scoped_refptr<MetricEntity> entity = METRIC_ENTITY_test_entity.Instantiate(&registry_, "my-test");
+  int num_threads = FLAGS_mt_metrics_test_num_threads;
+  int num_counters = 1000;
+  boost::function<void()> f =
+      boost::bind(RegisterCounters, entity, "prefix", num_counters);
+  RunWithManyThreads(&f, num_threads);
+  ASSERT_EQ(num_threads * num_counters, entity->UnsafeMetricsMapForTests().size());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mt-threadlocal-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mt-threadlocal-test.cc b/be/src/kudu/util/mt-threadlocal-test.cc
new file mode 100644
index 0000000..8f0b23b
--- /dev/null
+++ b/be/src/kudu/util/mt-threadlocal-test.cc
@@ -0,0 +1,357 @@
+// 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 <cstdint>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <vector>
+#include <unordered_set>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/env.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadlocal.h"
+#include "kudu/util/threadlocal_cache.h"
+
+using std::string;
+using std::unordered_set;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace threadlocal {
+
+class ThreadLocalTest : public KuduTest {};
+
+const int kTargetCounterVal = 1000000;
+
+class Counter;
+typedef unordered_set<Counter*> CounterPtrSet;
+typedef Mutex RegistryLockType;
+typedef simple_spinlock CounterLockType;
+
+// Registry to provide reader access to the thread-local Counters.
+// The methods are only thread-safe if the calling thread holds the lock.
+class CounterRegistry {
+ public:
+  CounterRegistry() {
+  }
+
+  RegistryLockType* get_lock() const {
+    return &lock_;
+  }
+
+  bool RegisterUnlocked(Counter* counter) {
+    LOG(INFO) << "Called RegisterUnlocked()";
+    return InsertIfNotPresent(&counters_, counter);
+  }
+
+  bool UnregisterUnlocked(Counter* counter) {
+    LOG(INFO) << "Called UnregisterUnlocked()";
+    return counters_.erase(counter) > 0;
+  }
+
+  CounterPtrSet* GetCountersUnlocked() {
+    return &counters_;
+  }
+
+ private:
+  mutable RegistryLockType lock_;
+  CounterPtrSet counters_;
+  DISALLOW_COPY_AND_ASSIGN(CounterRegistry);
+};
+
+// A simple Counter class that registers itself with a CounterRegistry.
+class Counter {
+ public:
+  Counter(CounterRegistry* registry, int val)
+    : tid_(Env::Default()->gettid()),
+      registry_(CHECK_NOTNULL(registry)),
+      val_(val) {
+    LOG(INFO) << "Counter::~Counter(): tid = " << tid_ << ", addr = " << this << ", val = " << val_;
+    std::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
+    CHECK(registry_->RegisterUnlocked(this));
+  }
+
+  ~Counter() {
+    LOG(INFO) << "Counter::~Counter(): tid = " << tid_ << ", addr = " << this << ", val = " << val_;
+    std::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
+    std::lock_guard<CounterLockType> self_lock(lock_);
+    LOG(INFO) << tid_ << ": deleting self from registry...";
+    CHECK(registry_->UnregisterUnlocked(this));
+  }
+
+  uint64_t tid() {
+    return tid_;
+  }
+
+  CounterLockType* get_lock() const {
+    return &lock_;
+  }
+
+  void IncrementUnlocked() {
+    val_++;
+  }
+
+  int GetValueUnlocked() {
+    return val_;
+  }
+
+ private:
+  // We expect that most of the time this lock will be uncontended.
+  mutable CounterLockType lock_;
+
+  // TID of thread that constructed this object.
+  const uint64_t tid_;
+
+  // Register / unregister ourselves with this on construction / destruction.
+  CounterRegistry* const registry_;
+
+  // Current value of the counter.
+  int val_;
+
+  DISALLOW_COPY_AND_ASSIGN(Counter);
+};
+
+// Create a new THREAD_LOCAL Counter and loop an increment operation on it.
+static void RegisterCounterAndLoopIncr(CounterRegistry* registry,
+                                       CountDownLatch* counters_ready,
+                                       CountDownLatch* reader_ready,
+                                       CountDownLatch* counters_done,
+                                       CountDownLatch* reader_done) {
+  BLOCK_STATIC_THREAD_LOCAL(Counter, counter, registry, 0);
+  // Inform the reader that we are alive.
+  counters_ready->CountDown();
+  // Let the reader initialize before we start counting.
+  reader_ready->Wait();
+  // Now rock & roll on the counting loop.
+  for (int i = 0; i < kTargetCounterVal; i++) {
+    std::lock_guard<CounterLockType> l(*counter->get_lock());
+    counter->IncrementUnlocked();
+  }
+  // Let the reader know we're ready for him to verify our counts.
+  counters_done->CountDown();
+  // Wait until the reader is done before we exit the thread, which will call
+  // delete on the Counter.
+  reader_done->Wait();
+}
+
+// Iterate over the registered counters and their values.
+static uint64_t Iterate(CounterRegistry* registry, int expected_counters) {
+  uint64_t sum = 0;
+  int seen_counters = 0;
+  std::lock_guard<RegistryLockType> l(*registry->get_lock());
+  for (Counter* counter : *registry->GetCountersUnlocked()) {
+    uint64_t value;
+    {
+      std::lock_guard<CounterLockType> l(*counter->get_lock());
+      value = counter->GetValueUnlocked();
+    }
+    LOG(INFO) << "tid " << counter->tid() << " (counter " << counter << "): " << value;
+    sum += value;
+    seen_counters++;
+  }
+  CHECK_EQ(expected_counters, seen_counters);
+  return sum;
+}
+
+static void TestThreadLocalCounters(CounterRegistry* registry, const int num_threads) {
+  LOG(INFO) << "Starting threads...";
+  vector<scoped_refptr<kudu::Thread> > threads;
+
+  CountDownLatch counters_ready(num_threads);
+  CountDownLatch reader_ready(1);
+  CountDownLatch counters_done(num_threads);
+  CountDownLatch reader_done(1);
+  for (int i = 0; i < num_threads; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+        &RegisterCounterAndLoopIncr, registry, &counters_ready, &reader_ready,
+        &counters_done, &reader_done, &new_thread));
+    threads.push_back(new_thread);
+  }
+
+  // Wait for all threads to start and register their Counters.
+  counters_ready.Wait();
+  CHECK_EQ(0, Iterate(registry, num_threads));
+  LOG(INFO) << "--";
+
+  // Let the counters start spinning.
+  reader_ready.CountDown();
+
+  // Try to catch them in the act, just for kicks.
+  for (int i = 0; i < 2; i++) {
+    Iterate(registry, num_threads);
+    LOG(INFO) << "--";
+    SleepFor(MonoDelta::FromMicroseconds(1));
+  }
+
+  // Wait until they're done and assure they sum up properly.
+  counters_done.Wait();
+  LOG(INFO) << "Checking Counter sums...";
+  CHECK_EQ(kTargetCounterVal * num_threads, Iterate(registry, num_threads));
+  LOG(INFO) << "Counter sums add up!";
+  reader_done.CountDown();
+
+  LOG(INFO) << "Joining & deleting threads...";
+  for (scoped_refptr<kudu::Thread> thread : threads) {
+    CHECK_OK(ThreadJoiner(thread.get()).Join());
+  }
+  LOG(INFO) << "Done.";
+}
+
+TEST_F(ThreadLocalTest, TestConcurrentCounters) {
+  // Run this multiple times to ensure we don't leave remnants behind in the
+  // CounterRegistry.
+  CounterRegistry registry;
+  for (int i = 0; i < 3; i++) {
+    TestThreadLocalCounters(&registry, 8);
+  }
+}
+
+// Test class that stores a string in a static thread local member.
+// This class cannot be instantiated. The methods are all static.
+class ThreadLocalString {
+ public:
+  static void set(std::string value);
+  static const std::string& get();
+ private:
+  ThreadLocalString() {
+  }
+  DECLARE_STATIC_THREAD_LOCAL(std::string, value_);
+  DISALLOW_COPY_AND_ASSIGN(ThreadLocalString);
+};
+
+DEFINE_STATIC_THREAD_LOCAL(std::string, ThreadLocalString, value_);
+
+void ThreadLocalString::set(std::string value) {
+  INIT_STATIC_THREAD_LOCAL(std::string, value_);
+  *value_ = value;
+}
+
+const std::string& ThreadLocalString::get() {
+  INIT_STATIC_THREAD_LOCAL(std::string, value_);
+  return *value_;
+}
+
+static void RunAndAssign(CountDownLatch* writers_ready,
+                         CountDownLatch *readers_ready,
+                         CountDownLatch *all_done,
+                         CountDownLatch *threads_exiting,
+                         const std::string& in,
+                         std::string* out) {
+  writers_ready->Wait();
+  // Ensure it starts off as an empty string.
+  CHECK_EQ("", ThreadLocalString::get());
+  ThreadLocalString::set(in);
+
+  readers_ready->Wait();
+  out->assign(ThreadLocalString::get());
+  all_done->Wait();
+  threads_exiting->CountDown();
+}
+
+TEST_F(ThreadLocalTest, TestTLSMember) {
+  const int num_threads = 8;
+
+  vector<CountDownLatch*> writers_ready;
+  vector<CountDownLatch*> readers_ready;
+  vector<std::string*> out_strings;
+  vector<scoped_refptr<kudu::Thread> > threads;
+
+  ElementDeleter writers_deleter(&writers_ready);
+  ElementDeleter readers_deleter(&readers_ready);
+  ElementDeleter out_strings_deleter(&out_strings);
+
+  CountDownLatch all_done(1);
+  CountDownLatch threads_exiting(num_threads);
+
+  LOG(INFO) << "Starting threads...";
+  for (int i = 0; i < num_threads; i++) {
+    writers_ready.push_back(new CountDownLatch(1));
+    readers_ready.push_back(new CountDownLatch(1));
+    out_strings.push_back(new std::string());
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+        &RunAndAssign, writers_ready[i], readers_ready[i],
+        &all_done, &threads_exiting, Substitute("$0", i), out_strings[i], &new_thread));
+    threads.push_back(new_thread);
+  }
+
+  // Unlatch the threads in order.
+  LOG(INFO) << "Writing to thread locals...";
+  for (int i = 0; i < num_threads; i++) {
+    writers_ready[i]->CountDown();
+  }
+  LOG(INFO) << "Reading from thread locals...";
+  for (int i = 0; i < num_threads; i++) {
+    readers_ready[i]->CountDown();
+  }
+  all_done.CountDown();
+  // threads_exiting acts as a memory barrier.
+  threads_exiting.Wait();
+  for (int i = 0; i < num_threads; i++) {
+    ASSERT_EQ(Substitute("$0", i), *out_strings[i]);
+    LOG(INFO) << "Read " << *out_strings[i];
+  }
+
+  LOG(INFO) << "Joining & deleting threads...";
+  for (scoped_refptr<kudu::Thread> thread : threads) {
+    CHECK_OK(ThreadJoiner(thread.get()).Join());
+  }
+}
+
+TEST_F(ThreadLocalTest, TestThreadLocalCache) {
+  using TLC = ThreadLocalCache<int, string>;
+  TLC* tlc = TLC::GetInstance();
+
+  // Lookup in an empty cache should return nullptr.
+  ASSERT_EQ(nullptr, tlc->Lookup(0));
+
+  // Insert more items than the cache capacity.
+  const int kLastItem = TLC::kItemCapacity * 2;
+  for (int i = 1; i <= kLastItem ; i++) {
+    auto* item = tlc->EmplaceNew(i);
+    ASSERT_NE(nullptr, item);
+    *item = Substitute("item $0", i);
+  }
+
+  // Looking up the most recent items should return them.
+  string* item = tlc->Lookup(kLastItem);
+  ASSERT_NE(nullptr, item);
+  EXPECT_EQ(*item, Substitute("item $0", kLastItem));
+
+  // Looking up evicted items should return nullptr.
+  ASSERT_EQ(nullptr, tlc->Lookup(1));
+}
+
+} // namespace threadlocal
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mutex.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mutex.cc b/be/src/kudu/util/mutex.cc
new file mode 100644
index 0000000..d55ccaf
--- /dev/null
+++ b/be/src/kudu/util/mutex.cc
@@ -0,0 +1,164 @@
+// 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.
+//
+// Portions (c) 2011 The Chromium Authors.
+
+#include "kudu/util/mutex.h"
+
+#include <cerrno>
+#include <cstdint>
+#include <cstring>
+#include <ostream>
+#include <string>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/debug-util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/trace.h"
+
+using std::string;
+using strings::Substitute;
+using strings::SubstituteAndAppend;
+
+#ifndef NDEBUG
+DEFINE_bool(debug_mutex_collect_stacktrace, false,
+            "Whether to collect a stacktrace on Mutex contention in a DEBUG build");
+TAG_FLAG(debug_mutex_collect_stacktrace, advanced);
+TAG_FLAG(debug_mutex_collect_stacktrace, hidden);
+#endif
+
+namespace kudu {
+
+Mutex::Mutex()
+#ifndef NDEBUG
+  : owning_tid_(0),
+    stack_trace_(new StackTrace())
+#endif
+{
+#ifndef NDEBUG
+  // In debug, setup attributes for lock error checking.
+  pthread_mutexattr_t mta;
+  int rv = pthread_mutexattr_init(&mta);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+  rv = pthread_mutexattr_settype(&mta, PTHREAD_MUTEX_ERRORCHECK);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+  rv = pthread_mutex_init(&native_handle_, &mta);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+  rv = pthread_mutexattr_destroy(&mta);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+#else
+  // In release, go with the default lock attributes.
+  pthread_mutex_init(&native_handle_, NULL);
+#endif
+}
+
+Mutex::~Mutex() {
+  int rv = pthread_mutex_destroy(&native_handle_);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+}
+
+bool Mutex::TryAcquire() {
+  int rv = pthread_mutex_trylock(&native_handle_);
+#ifndef NDEBUG
+  DCHECK(rv == 0 || rv == EBUSY) << ". " << strerror(rv) << ". " << GetOwnerThreadInfo();
+  if (rv == 0) {
+    CheckUnheldAndMark();
+  }
+#endif
+  return rv == 0;
+}
+
+void Mutex::Acquire() {
+  // Optimize for the case when mutexes are uncontended. If they
+  // are contended, we'll have to go to sleep anyway, so the extra
+  // cost of branch mispredictions is moot.
+  //
+  // TryAcquire() is implemented as a simple CompareAndSwap inside
+  // pthreads so this does not require a system call.
+  if (PREDICT_TRUE(TryAcquire())) {
+    return;
+  }
+
+  // If we weren't able to acquire the mutex immediately, then it's
+  // worth gathering timing information about the mutex acquisition.
+  MicrosecondsInt64 start_time = GetMonoTimeMicros();
+  int rv = pthread_mutex_lock(&native_handle_);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv)
+#ifndef NDEBUG
+                   << ". " << GetOwnerThreadInfo()
+#endif
+  ; // NOLINT(whitespace/semicolon)
+  MicrosecondsInt64 end_time = GetMonoTimeMicros();
+
+  int64_t wait_time = end_time - start_time;
+  if (wait_time > 0) {
+    TRACE_COUNTER_INCREMENT("mutex_wait_us", wait_time);
+  }
+
+#ifndef NDEBUG
+  CheckUnheldAndMark();
+#endif
+}
+
+void Mutex::Release() {
+#ifndef NDEBUG
+  CheckHeldAndUnmark();
+#endif
+  int rv = pthread_mutex_unlock(&native_handle_);
+  DCHECK_EQ(0, rv) << ". " << strerror(rv);
+}
+
+#ifndef NDEBUG
+void Mutex::AssertAcquired() const {
+  DCHECK_EQ(Env::Default()->gettid(), owning_tid_);
+}
+
+void Mutex::CheckHeldAndUnmark() {
+  AssertAcquired();
+  owning_tid_ = 0;
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    stack_trace_->Reset();
+  }
+}
+
+void Mutex::CheckUnheldAndMark() {
+  DCHECK_EQ(0, owning_tid_);
+  owning_tid_ = Env::Default()->gettid();
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    stack_trace_->Collect();
+  }
+}
+
+string Mutex::GetOwnerThreadInfo() const {
+  string str = Substitute("Owner tid: $0; Self tid: $1; ", owning_tid_, Env::Default()->gettid());
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    SubstituteAndAppend(&str, "Owner stack:\n$0", stack_trace_->Symbolize());
+  } else {
+    str += "To collect the owner stack trace, enable the flag --debug_mutex_collect_stacktrace";
+  }
+  return str;
+}
+
+#endif
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/mutex.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/mutex.h b/be/src/kudu/util/mutex.h
new file mode 100644
index 0000000..9277ac0
--- /dev/null
+++ b/be/src/kudu/util/mutex.h
@@ -0,0 +1,142 @@
+// 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.
+#ifndef KUDU_UTIL_MUTEX_H
+#define KUDU_UTIL_MUTEX_H
+
+#include <pthread.h>
+#include <sys/types.h>
+
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+class StackTrace;
+
+// A lock built around pthread_mutex_t. Does not allow recursion.
+//
+// The following checks will be performed in DEBUG mode:
+//   Acquire(), TryAcquire() - the lock isn't already held.
+//   Release() - the lock is already held by this thread.
+//
+class Mutex {
+ public:
+  Mutex();
+  ~Mutex();
+
+  void Acquire();
+  void Release();
+  bool TryAcquire();
+
+  void lock() { Acquire(); }
+  void unlock() { Release(); }
+  bool try_lock() { return TryAcquire(); }
+
+#ifndef NDEBUG
+  void AssertAcquired() const;
+#else
+  void AssertAcquired() const {}
+#endif
+
+ private:
+  friend class ConditionVariable;
+
+  pthread_mutex_t native_handle_;
+
+#ifndef NDEBUG
+  // Members and routines taking care of locks assertions.
+  void CheckHeldAndUnmark();
+  void CheckUnheldAndMark();
+  std::string GetOwnerThreadInfo() const;
+
+  // All private data is implicitly protected by native_handle_.
+  // Be VERY careful to only access members under that lock.
+  pid_t owning_tid_;
+  gscoped_ptr<StackTrace> stack_trace_;
+#endif
+
+  DISALLOW_COPY_AND_ASSIGN(Mutex);
+};
+
+// A helper class that acquires the given Lock while the MutexLock is in scope.
+class MutexLock {
+ public:
+  struct AlreadyAcquired {};
+
+  // Acquires 'lock' (must be unheld) and wraps around it.
+  //
+  // Sample usage:
+  // {
+  //   MutexLock l(lock_); // acquired
+  //   ...
+  // } // released
+  explicit MutexLock(Mutex& lock)
+    : lock_(&lock),
+      owned_(true) {
+    lock_->Acquire();
+  }
+
+  // Wraps around 'lock' (must already be held by this thread).
+  //
+  // Sample usage:
+  // {
+  //   lock_.Acquire(); // acquired
+  //   ...
+  //   MutexLock l(lock_, AlreadyAcquired());
+  //   ...
+  // } // released
+  MutexLock(Mutex& lock, const AlreadyAcquired&)
+    : lock_(&lock),
+      owned_(true) {
+    lock_->AssertAcquired();
+  }
+
+  void Lock() {
+    DCHECK(!owned_);
+    lock_->Acquire();
+    owned_ = true;
+  }
+
+  void Unlock() {
+    DCHECK(owned_);
+    lock_->AssertAcquired();
+    lock_->Release();
+    owned_ = false;
+  }
+
+  ~MutexLock() {
+    if (owned_) {
+      Unlock();
+    }
+  }
+
+  bool OwnsLock() const {
+    return owned_;
+  }
+
+ private:
+  Mutex* lock_;
+  bool owned_;
+  DISALLOW_COPY_AND_ASSIGN(MutexLock);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_MUTEX_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/dns_resolver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/dns_resolver-test.cc b/be/src/kudu/util/net/dns_resolver-test.cc
new file mode 100644
index 0000000..f08b089
--- /dev/null
+++ b/be/src/kudu/util/net/dns_resolver-test.cc
@@ -0,0 +1,59 @@
+// 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 "kudu/util/net/dns_resolver.h"
+
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/async_util.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::vector;
+
+namespace kudu {
+
+class DnsResolverTest : public KuduTest {
+ protected:
+  DnsResolver resolver_;
+};
+
+TEST_F(DnsResolverTest, TestResolution) {
+  vector<Sockaddr> addrs;
+  Synchronizer s;
+  {
+    HostPort hp("localhost", 12345);
+    resolver_.ResolveAddresses(hp, &addrs, s.AsStatusCallback());
+  }
+  ASSERT_OK(s.Wait());
+  ASSERT_TRUE(!addrs.empty());
+  for (const Sockaddr& addr : addrs) {
+    LOG(INFO) << "Address: " << addr.ToString();
+    EXPECT_TRUE(HasPrefixString(addr.ToString(), "127."));
+    EXPECT_TRUE(HasSuffixString(addr.ToString(), ":12345"));
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/dns_resolver.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/dns_resolver.cc b/be/src/kudu/util/net/dns_resolver.cc
new file mode 100644
index 0000000..4803688
--- /dev/null
+++ b/be/src/kudu/util/net/dns_resolver.cc
@@ -0,0 +1,65 @@
+// 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 "kudu/util/net/dns_resolver.h"
+
+#include <vector>
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/status.h"
+
+DEFINE_int32(dns_num_resolver_threads, 1, "The number of threads to use for DNS resolution");
+TAG_FLAG(dns_num_resolver_threads, advanced);
+
+using std::vector;
+
+namespace kudu {
+
+DnsResolver::DnsResolver() {
+  CHECK_OK(ThreadPoolBuilder("dns-resolver")
+           .set_max_threads(FLAGS_dns_num_resolver_threads)
+           .Build(&pool_));
+}
+
+DnsResolver::~DnsResolver() {
+  pool_->Shutdown();
+}
+
+namespace {
+void DoResolution(const HostPort &hostport, vector<Sockaddr>* addresses,
+                  const StatusCallback& cb) {
+  cb.Run(hostport.ResolveAddresses(addresses));
+}
+} // anonymous namespace
+
+void DnsResolver::ResolveAddresses(const HostPort& hostport,
+                                   vector<Sockaddr>* addresses,
+                                   const StatusCallback& cb) {
+  Status s = pool_->SubmitFunc(boost::bind(&DoResolution, hostport, addresses, cb));
+  if (!s.ok()) {
+    cb.Run(s);
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/dns_resolver.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/dns_resolver.h b/be/src/kudu/util/net/dns_resolver.h
new file mode 100644
index 0000000..06dfa48
--- /dev/null
+++ b/be/src/kudu/util/net/dns_resolver.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.
+#ifndef KUDU_UTIL_NET_DNS_RESOLVER_H
+#define KUDU_UTIL_NET_DNS_RESOLVER_H
+
+#include <vector>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/util/status_callback.h"
+
+namespace kudu {
+
+class HostPort;
+class Sockaddr;
+class ThreadPool;
+
+// DNS Resolver which supports async address resolution.
+class DnsResolver {
+ public:
+  DnsResolver();
+  ~DnsResolver();
+
+  // Resolve any addresses corresponding to this host:port pair.
+  // Note that a host may resolve to more than one IP address.
+  //
+  // 'addresses' may be NULL, in which case this function simply checks that
+  // the host/port pair can be resolved, without returning anything.
+  //
+  // When the result is available, or an error occurred, 'cb' is called
+  // with the result Status.
+  //
+  // NOTE: the callback should be fast since it is called by the DNS
+  // resolution thread.
+  // NOTE: in some rare cases, the callback may also be called inline
+  // from this function call, on the caller's thread.
+  void ResolveAddresses(const HostPort& hostport,
+                        std::vector<Sockaddr>* addresses,
+                        const StatusCallback& cb);
+
+ private:
+  gscoped_ptr<ThreadPool> pool_;
+
+  DISALLOW_COPY_AND_ASSIGN(DnsResolver);
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_NET_DNS_RESOLVER_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/net_util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/net_util-test.cc b/be/src/kudu/util/net/net_util-test.cc
new file mode 100644
index 0000000..202ec6b
--- /dev/null
+++ b/be/src/kudu/util/net/net_util-test.cc
@@ -0,0 +1,170 @@
+// 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 <algorithm>
+#include <cstdint>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+class NetUtilTest : public KuduTest {
+ protected:
+  Status DoParseBindAddresses(const string& input, string* result) {
+    vector<Sockaddr> addrs;
+    RETURN_NOT_OK(ParseAddressList(input, kDefaultPort, &addrs));
+    std::sort(addrs.begin(), addrs.end());
+
+    vector<string> addr_strs;
+    for (const Sockaddr& addr : addrs) {
+      addr_strs.push_back(addr.ToString());
+    }
+    *result = JoinStrings(addr_strs, ",");
+    return Status::OK();
+  }
+
+  static const uint16_t kDefaultPort = 7150;
+};
+
+TEST(SockaddrTest, Test) {
+  Sockaddr addr;
+  ASSERT_OK(addr.ParseString("1.1.1.1:12345", 12345));
+  ASSERT_EQ(12345, addr.port());
+  ASSERT_EQ("1.1.1.1", addr.host());
+}
+
+TEST_F(NetUtilTest, TestParseAddresses) {
+  string ret;
+  ASSERT_OK(DoParseBindAddresses("0.0.0.0:12345", &ret));
+  ASSERT_EQ("0.0.0.0:12345", ret);
+
+  ASSERT_OK(DoParseBindAddresses("0.0.0.0", &ret));
+  ASSERT_EQ("0.0.0.0:7150", ret);
+
+  ASSERT_OK(DoParseBindAddresses("0.0.0.0:12345, 0.0.0.0:12346", &ret));
+  ASSERT_EQ("0.0.0.0:12345,0.0.0.0:12346", ret);
+
+  // Test some invalid addresses.
+  Status s = DoParseBindAddresses("0.0.0.0:xyz", &ret);
+  ASSERT_STR_CONTAINS(s.ToString(), "Invalid port");
+
+  s = DoParseBindAddresses("0.0.0.0:100000", &ret);
+  ASSERT_STR_CONTAINS(s.ToString(), "Invalid port");
+
+  s = DoParseBindAddresses("0.0.0.0:", &ret);
+  ASSERT_STR_CONTAINS(s.ToString(), "Invalid port");
+}
+
+TEST_F(NetUtilTest, TestResolveAddresses) {
+  HostPort hp("localhost", 12345);
+  vector<Sockaddr> addrs;
+  ASSERT_OK(hp.ResolveAddresses(&addrs));
+  ASSERT_TRUE(!addrs.empty());
+  for (const Sockaddr& addr : addrs) {
+    LOG(INFO) << "Address: " << addr.ToString();
+    EXPECT_TRUE(HasPrefixString(addr.ToString(), "127."));
+    EXPECT_TRUE(HasSuffixString(addr.ToString(), ":12345"));
+    EXPECT_TRUE(addr.IsAnyLocalAddress());
+  }
+
+  ASSERT_OK(hp.ResolveAddresses(nullptr));
+}
+
+TEST_F(NetUtilTest, TestWithinNetwork) {
+  Sockaddr addr;
+  Network network;
+
+  ASSERT_OK(addr.ParseString("10.0.23.0:12345", 0));
+  ASSERT_OK(network.ParseCIDRString("10.0.0.0/8"));
+  EXPECT_TRUE(network.WithinNetwork(addr));
+
+  ASSERT_OK(addr.ParseString("172.28.3.4:0", 0));
+  ASSERT_OK(network.ParseCIDRString("172.16.0.0/12"));
+  EXPECT_TRUE(network.WithinNetwork(addr));
+
+  ASSERT_OK(addr.ParseString("192.168.0.23", 0));
+  ASSERT_OK(network.ParseCIDRString("192.168.1.14/16"));
+  EXPECT_TRUE(network.WithinNetwork(addr));
+
+  ASSERT_OK(addr.ParseString("8.8.8.8:0", 0));
+  ASSERT_OK(network.ParseCIDRString("0.0.0.0/0"));
+  EXPECT_TRUE(network.WithinNetwork(addr));
+
+  ASSERT_OK(addr.ParseString("192.169.0.23", 0));
+  ASSERT_OK(network.ParseCIDRString("192.168.0.0/16"));
+  EXPECT_FALSE(network.WithinNetwork(addr));
+}
+
+// Ensure that we are able to do a reverse DNS lookup on various IP addresses.
+// The reverse lookups should never fail, but may return numeric strings.
+TEST_F(NetUtilTest, TestReverseLookup) {
+  string host;
+  Sockaddr addr;
+  HostPort hp;
+  ASSERT_OK(addr.ParseString("0.0.0.0:12345", 0));
+  EXPECT_EQ(12345, addr.port());
+  ASSERT_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
+  EXPECT_NE("0.0.0.0", hp.host());
+  EXPECT_NE("", hp.host());
+  EXPECT_EQ(12345, hp.port());
+
+  ASSERT_OK(addr.ParseString("127.0.0.1:12345", 0));
+  ASSERT_OK(HostPortFromSockaddrReplaceWildcard(addr, &hp));
+  EXPECT_EQ("127.0.0.1", hp.host());
+  EXPECT_EQ(12345, hp.port());
+}
+
+TEST_F(NetUtilTest, TestLsof) {
+  Socket s;
+  ASSERT_OK(s.Init(0));
+
+  Sockaddr addr; // wildcard
+  ASSERT_OK(s.BindAndListen(addr, 1));
+
+  ASSERT_OK(s.GetSocketAddress(&addr));
+  ASSERT_NE(addr.port(), 0);
+  vector<string> lsof_lines;
+  TryRunLsof(addr, &lsof_lines);
+  SCOPED_TRACE(JoinStrings(lsof_lines, "\n"));
+
+  ASSERT_GE(lsof_lines.size(), 3);
+  ASSERT_STR_CONTAINS(lsof_lines[2], "net_util-test");
+}
+
+TEST_F(NetUtilTest, TestGetFQDN) {
+  string fqdn;
+  ASSERT_OK(GetFQDN(&fqdn));
+  LOG(INFO) << "fqdn is " << fqdn;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/net_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/net_util.cc b/be/src/kudu/util/net/net_util.cc
new file mode 100644
index 0000000..520882f
--- /dev/null
+++ b/be/src/kudu/util/net/net_util.cc
@@ -0,0 +1,402 @@
+// 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 <sys/socket.h>
+#include <ifaddrs.h>
+#include <limits.h>
+#include <netdb.h>
+#include <netinet/in.h>
+#include <unistd.h>
+
+#include <cerrno>
+#include <cstring>
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <boost/functional/hash/hash.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/trace.h"
+
+// Mac OS 10.9 does not appear to define HOST_NAME_MAX in unistd.h
+#ifndef HOST_NAME_MAX
+#define HOST_NAME_MAX 64
+#endif
+
+DEFINE_bool(fail_dns_resolution, false, "Whether to fail all dns resolution, for tests.");
+TAG_FLAG(fail_dns_resolution, hidden);
+
+using std::function;
+using std::string;
+using std::unordered_set;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+namespace {
+
+using AddrInfo = unique_ptr<addrinfo, function<void(addrinfo*)>>;
+
+// An utility wrapper around getaddrinfo() call to convert the return code
+// of the libc library function into Status.
+Status GetAddrInfo(const string& hostname,
+                   const addrinfo& hints,
+                   const string& op_description,
+                   AddrInfo* info) {
+  addrinfo* res = nullptr;
+  const int rc = getaddrinfo(hostname.c_str(), nullptr, &hints, &res);
+  const int err = errno; // preserving the errno from the getaddrinfo() call
+  AddrInfo result(res, ::freeaddrinfo);
+  if (rc == 0) {
+    if (info != nullptr) {
+      info->swap(result);
+    }
+    return Status::OK();
+  }
+  const string err_msg = Substitute("unable to $0", op_description);
+  if (rc == EAI_SYSTEM) {
+    return Status::NetworkError(err_msg, ErrnoToString(err), err);
+  }
+  return Status::NetworkError(err_msg, gai_strerror(rc));
+}
+
+} // anonymous namespace
+
+HostPort::HostPort()
+  : host_(""),
+    port_(0) {
+}
+
+HostPort::HostPort(std::string host, uint16_t port)
+    : host_(std::move(host)), port_(port) {}
+
+HostPort::HostPort(const Sockaddr& addr)
+  : host_(addr.host()),
+    port_(addr.port()) {
+}
+
+bool operator==(const HostPort& hp1, const HostPort& hp2) {
+  return hp1.port() == hp2.port() && hp1.host() == hp2.host();
+}
+
+size_t HostPort::HashCode() const {
+  size_t seed = 0;
+  boost::hash_combine(seed, host_);
+  boost::hash_combine(seed, port_);
+  return seed;
+}
+
+Status HostPort::ParseString(const string& str, uint16_t default_port) {
+  std::pair<string, string> p = strings::Split(str, strings::delimiter::Limit(":", 1));
+
+  // Strip any whitespace from the host.
+  StripWhiteSpace(&p.first);
+
+  // Parse the port.
+  uint32_t port;
+  if (p.second.empty() && strcount(str, ':') == 0) {
+    // No port specified.
+    port = default_port;
+  } else if (!SimpleAtoi(p.second, &port) ||
+             port > 65535) {
+    return Status::InvalidArgument("Invalid port", str);
+  }
+
+  host_.swap(p.first);
+  port_ = port;
+  return Status::OK();
+}
+
+Status HostPort::ResolveAddresses(vector<Sockaddr>* addresses) const {
+  TRACE_EVENT1("net", "HostPort::ResolveAddresses",
+               "host", host_);
+  TRACE_COUNTER_SCOPE_LATENCY_US("dns_us");
+  struct addrinfo hints;
+  memset(&hints, 0, sizeof(hints));
+  hints.ai_family = AF_INET;
+  hints.ai_socktype = SOCK_STREAM;
+  AddrInfo result;
+  const string op_description = Substitute("resolve address for $0", host_);
+  LOG_SLOW_EXECUTION(WARNING, 200, op_description) {
+    RETURN_NOT_OK(GetAddrInfo(host_, hints, op_description, &result));
+  }
+  for (const addrinfo* ai = result.get(); ai != nullptr; ai = ai->ai_next) {
+    CHECK_EQ(ai->ai_family, AF_INET);
+    struct sockaddr_in* addr = reinterpret_cast<struct sockaddr_in*>(ai->ai_addr);
+    addr->sin_port = htons(port_);
+    Sockaddr sockaddr(*addr);
+    if (addresses) {
+      addresses->push_back(sockaddr);
+    }
+    VLOG(2) << "Resolved address " << sockaddr.ToString()
+            << " for host/port " << ToString();
+  }
+  if (PREDICT_FALSE(FLAGS_fail_dns_resolution)) {
+    return Status::NetworkError("injected DNS resolution failure");
+  }
+  return Status::OK();
+}
+
+Status HostPort::ParseStrings(const string& comma_sep_addrs,
+                              uint16_t default_port,
+                              vector<HostPort>* res) {
+  vector<string> addr_strings = strings::Split(comma_sep_addrs, ",", strings::SkipEmpty());
+  for (const string& addr_string : addr_strings) {
+    HostPort host_port;
+    RETURN_NOT_OK(host_port.ParseString(addr_string, default_port));
+    res->push_back(host_port);
+  }
+  return Status::OK();
+}
+
+string HostPort::ToString() const {
+  return Substitute("$0:$1", host_, port_);
+}
+
+string HostPort::ToCommaSeparatedString(const vector<HostPort>& hostports) {
+  vector<string> hostport_strs;
+  for (const HostPort& hostport : hostports) {
+    hostport_strs.push_back(hostport.ToString());
+  }
+  return JoinStrings(hostport_strs, ",");
+}
+
+Network::Network()
+  : addr_(0),
+    netmask_(0) {
+}
+
+Network::Network(uint32_t addr, uint32_t netmask)
+  : addr_(addr), netmask_(netmask) {}
+
+bool Network::WithinNetwork(const Sockaddr& addr) const {
+  return ((addr.addr().sin_addr.s_addr & netmask_) ==
+          (addr_ & netmask_));
+}
+
+Status Network::ParseCIDRString(const string& addr) {
+  std::pair<string, string> p = strings::Split(addr, strings::delimiter::Limit("/", 1));
+
+  kudu::Sockaddr sockaddr;
+  Status s = sockaddr.ParseString(p.first, 0);
+
+  uint32_t bits;
+  bool success = SimpleAtoi(p.second, &bits);
+
+  if (!s.ok() || !success || bits > 32) {
+    return Status::NetworkError("Unable to parse CIDR address", addr);
+  }
+
+  // Netmask in network byte order
+  uint32_t netmask = NetworkByteOrder::FromHost32(~(0xffffffff >> bits));
+  addr_ = sockaddr.addr().sin_addr.s_addr;
+  netmask_ = netmask;
+  return Status::OK();
+}
+
+Status Network::ParseCIDRStrings(const string& comma_sep_addrs,
+                                 vector<Network>* res) {
+  vector<string> addr_strings = strings::Split(comma_sep_addrs, ",", strings::SkipEmpty());
+  for (const string& addr_string : addr_strings) {
+    Network network;
+    RETURN_NOT_OK(network.ParseCIDRString(addr_string));
+    res->push_back(network);
+  }
+  return Status::OK();
+}
+
+bool IsPrivilegedPort(uint16_t port) {
+  return port <= 1024 && port != 0;
+}
+
+Status ParseAddressList(const std::string& addr_list,
+                        uint16_t default_port,
+                        std::vector<Sockaddr>* addresses) {
+  vector<HostPort> host_ports;
+  RETURN_NOT_OK(HostPort::ParseStrings(addr_list, default_port, &host_ports));
+  if (host_ports.empty()) return Status::InvalidArgument("No address specified");
+  unordered_set<Sockaddr> uniqued;
+  for (const HostPort& host_port : host_ports) {
+    vector<Sockaddr> this_addresses;
+    RETURN_NOT_OK(host_port.ResolveAddresses(&this_addresses));
+
+    // Only add the unique ones -- the user may have specified
+    // some IP addresses in multiple ways
+    for (const Sockaddr& addr : this_addresses) {
+      if (InsertIfNotPresent(&uniqued, addr)) {
+        addresses->push_back(addr);
+      } else {
+        LOG(INFO) << "Address " << addr.ToString() << " for " << host_port.ToString()
+                  << " duplicates an earlier resolved entry.";
+      }
+    }
+  }
+  return Status::OK();
+}
+
+Status GetHostname(string* hostname) {
+  TRACE_EVENT0("net", "GetHostname");
+  char name[HOST_NAME_MAX];
+  int ret = gethostname(name, HOST_NAME_MAX);
+  if (ret != 0) {
+    return Status::NetworkError("Unable to determine local hostname",
+                                ErrnoToString(errno),
+                                errno);
+  }
+  *hostname = name;
+  return Status::OK();
+}
+
+Status GetLocalNetworks(std::vector<Network>* net) {
+  struct ifaddrs *ifap = nullptr;
+
+  int ret = getifaddrs(&ifap);
+  SCOPED_CLEANUP({
+    if (ifap) freeifaddrs(ifap);
+  });
+
+  if (ret != 0) {
+    return Status::NetworkError("Unable to determine local network addresses",
+                                ErrnoToString(errno),
+                                errno);
+  }
+
+  net->clear();
+  for (struct ifaddrs *ifa = ifap; ifa; ifa = ifa->ifa_next) {
+    if (ifa->ifa_addr == nullptr || ifa->ifa_netmask == nullptr) continue;
+
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      Sockaddr addr(*reinterpret_cast<struct sockaddr_in*>(ifa->ifa_addr));
+      Sockaddr netmask(*reinterpret_cast<struct sockaddr_in*>(ifa->ifa_netmask));
+      Network network(addr.addr().sin_addr.s_addr, netmask.addr().sin_addr.s_addr);
+      net->push_back(network);
+    }
+  }
+
+  return Status::OK();
+}
+
+Status GetFQDN(string* hostname) {
+  TRACE_EVENT0("net", "GetFQDN");
+  // Start with the non-qualified hostname
+  RETURN_NOT_OK(GetHostname(hostname));
+
+  struct addrinfo hints;
+  memset(&hints, 0, sizeof(hints));
+  hints.ai_socktype = SOCK_DGRAM;
+  hints.ai_flags = AI_CANONNAME;
+  AddrInfo result;
+  const string op_description =
+      Substitute("look up canonical hostname for localhost '$0'", *hostname);
+  LOG_SLOW_EXECUTION(WARNING, 200, op_description) {
+    TRACE_EVENT0("net", "getaddrinfo");
+    RETURN_NOT_OK(GetAddrInfo(*hostname, hints, op_description, &result));
+  }
+
+  *hostname = result->ai_canonname;
+  return Status::OK();
+}
+
+Status SockaddrFromHostPort(const HostPort& host_port, Sockaddr* addr) {
+  vector<Sockaddr> addrs;
+  RETURN_NOT_OK(host_port.ResolveAddresses(&addrs));
+  if (addrs.empty()) {
+    return Status::NetworkError("Unable to resolve address", host_port.ToString());
+  }
+  *addr = addrs[0];
+  if (addrs.size() > 1) {
+    VLOG(1) << "Hostname " << host_port.host() << " resolved to more than one address. "
+            << "Using address: " << addr->ToString();
+  }
+  return Status::OK();
+}
+
+Status HostPortFromSockaddrReplaceWildcard(const Sockaddr& addr, HostPort* hp) {
+  string host;
+  if (addr.IsWildcard()) {
+    RETURN_NOT_OK(GetFQDN(&host));
+  } else {
+    host = addr.host();
+  }
+  hp->set_host(host);
+  hp->set_port(addr.port());
+  return Status::OK();
+}
+
+void TryRunLsof(const Sockaddr& addr, vector<string>* log) {
+#if defined(__APPLE__)
+  string cmd = strings::Substitute(
+      "lsof -n -i 'TCP:$0' -sTCP:LISTEN ; "
+      "for pid in $$(lsof -F p -n -i 'TCP:$0' -sTCP:LISTEN | cut -f 2 -dp) ; do"
+      "  pstree $$pid || ps h -p $$pid;"
+      "done",
+      addr.port());
+#else
+  // Little inline bash script prints the full ancestry of any pid listening
+  // on the same port as 'addr'. We could use 'pstree -s', but that option
+  // doesn't exist on el6.
+  string cmd = strings::Substitute(
+      "export PATH=$$PATH:/usr/sbin ; "
+      "lsof -n -i 'TCP:$0' -sTCP:LISTEN ; "
+      "for pid in $$(lsof -F p -n -i 'TCP:$0' -sTCP:LISTEN | grep p | cut -f 2 -dp) ; do"
+      "  while [ $$pid -gt 1 ] ; do"
+      "    ps h -fp $$pid ;"
+      "    stat=($$(</proc/$$pid/stat)) ;"
+      "    pid=$${stat[3]} ;"
+      "  done ; "
+      "done",
+      addr.port());
+#endif // defined(__APPLE__)
+  LOG_STRING(WARNING, log)
+      << "Trying to use lsof to find any processes listening on "
+      << addr.ToString();
+  LOG_STRING(INFO, log) << "$ " << cmd;
+  vector<string> argv = { "bash", "-c", cmd };
+  string results;
+  Status s = Subprocess::Call(argv, "", &results);
+  if (PREDICT_FALSE(!s.ok())) {
+    LOG_STRING(WARNING, log) << s.ToString();
+  }
+  LOG_STRING(WARNING, log) << results;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/net/net_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/net/net_util.h b/be/src/kudu/util/net/net_util.h
new file mode 100644
index 0000000..c471ae8
--- /dev/null
+++ b/be/src/kudu/util/net/net_util.h
@@ -0,0 +1,166 @@
+// 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.
+#ifndef KUDU_UTIL_NET_NET_UTIL_H
+#define KUDU_UTIL_NET_NET_UTIL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Sockaddr;
+
+// A container for a host:port pair.
+class HostPort {
+ public:
+  HostPort();
+  HostPort(std::string host, uint16_t port);
+  explicit HostPort(const Sockaddr& addr);
+
+  bool Initialized() const {
+    return !host_.empty();
+  }
+
+  // Parse a "host:port" pair into this object.
+  // If there is no port specified in the string, then 'default_port' is used.
+  Status ParseString(const std::string& str, uint16_t default_port);
+
+  // Resolve any addresses corresponding to this host:port pair.
+  // Note that a host may resolve to more than one IP address.
+  //
+  // 'addresses' may be NULL, in which case this function simply checks that
+  // the host/port pair can be resolved, without returning anything.
+  Status ResolveAddresses(std::vector<Sockaddr>* addresses) const;
+
+  std::string ToString() const;
+
+  const std::string& host() const { return host_; }
+  void set_host(const std::string& host) { host_ = host; }
+
+  uint16_t port() const { return port_; }
+  void set_port(uint16_t port) { port_ = port; }
+
+  size_t HashCode() const;
+
+  // Parse a comma separated list of "host:port" pairs into a vector
+  // HostPort objects. If no port is specified for an entry in the
+  // comma separated list, 'default_port' is used for that entry's
+  // pair.
+  static Status ParseStrings(
+      const std::string& comma_sep_addrs, uint16_t default_port, std::vector<HostPort>* res);
+
+  // Takes a vector of HostPort objects and returns a comma separated
+  // string containing of "host:port" pairs. This method is the
+  // "inverse" of ParseStrings().
+  static std::string ToCommaSeparatedString(const std::vector<HostPort>& host_ports);
+
+ private:
+  std::string host_;
+  uint16_t port_;
+};
+
+bool operator==(const HostPort& hp1, const HostPort& hp2);
+
+// Hasher of HostPort objects for UnorderedAssociativeContainers.
+struct HostPortHasher {
+  size_t operator()(const HostPort& hp) const {
+    return hp.HashCode();
+  }
+};
+
+// Equality BinaryPredicate of HostPort objects for UnorderedAssociativeContainers.
+struct HostPortEqualityPredicate {
+  bool operator()(const HostPort& hp1, const HostPort& hp2) const {
+    return hp1 == hp2;
+  }
+};
+
+// A container for addr:mask pair.
+// Both addr and netmask are in big-endian byte order
+// (same as network byte order).
+class Network {
+ public:
+  Network();
+  Network(uint32_t addr, uint32_t netmask);
+
+  uint32_t addr() const { return addr_; }
+
+  uint32_t netmask() const { return netmask_; }
+
+  // Returns true if the address is within network.
+  bool WithinNetwork(const Sockaddr& addr) const;
+
+  // Parses a "addr/netmask" (CIDR notation) pair into this object.
+  Status ParseCIDRString(const std::string& addr);
+
+  // Parses a comma separated list of "addr/netmask" (CIDR notation)
+  // pairs into a vector of Network objects.
+  static Status ParseCIDRStrings(
+      const std::string& comma_sep_addrs, std::vector<Network>* res);
+ private:
+  uint32_t addr_;
+  uint32_t netmask_;
+};
+
+// Parse and resolve the given comma-separated list of addresses.
+//
+// The resulting addresses will be resolved, made unique, and added to
+// the 'addresses' vector.
+//
+// Any elements which do not include a port will be assigned 'default_port'.
+Status ParseAddressList(const std::string& addr_list,
+                        uint16_t default_port,
+                        std::vector<Sockaddr>* addresses);
+
+// Return true if the given port is likely to need root privileges to bind to.
+bool IsPrivilegedPort(uint16_t port);
+
+// Return the local machine's hostname.
+Status GetHostname(std::string* hostname);
+
+// Returns local subnets of all local network interfaces.
+Status GetLocalNetworks(std::vector<Network>* net);
+
+// Return the local machine's FQDN.
+Status GetFQDN(std::string* hostname);
+
+// Returns a single socket address from a HostPort.
+// If the hostname resolves to multiple addresses, returns the first in the
+// list and logs a message in verbose mode.
+Status SockaddrFromHostPort(const HostPort& host_port, Sockaddr* addr);
+
+// Converts the given Sockaddr into a HostPort, substituting the FQDN
+// in the case that the provided address is the wildcard.
+//
+// In the case of other addresses, the returned HostPort will contain just the
+// stringified form of the IP.
+Status HostPortFromSockaddrReplaceWildcard(const Sockaddr& addr, HostPort* hp);
+
+// Try to run 'lsof' to determine which process is preventing binding to
+// the given 'addr'. If pids can be determined, outputs full 'ps' and 'pstree'
+// output for that process.
+//
+// Output is issued to the log at WARNING level, or appended to 'log' if it
+// is non-NULL (mostly useful for testing).
+void TryRunLsof(const Sockaddr& addr, std::vector<std::string>* log = NULL);
+
+} // namespace kudu
+#endif


[28/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache.cc b/be/src/kudu/util/cache.cc
new file mode 100644
index 0000000..00f2e52
--- /dev/null
+++ b/be/src/kudu/util/cache.cc
@@ -0,0 +1,572 @@
+// Some portions copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "kudu/util/cache.h"
+
+#include <atomic>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/bits.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/hash/city.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/util/alignment.h"
+#include "kudu/util/cache_metrics.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/malloc.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/test_util_prod.h"
+
+#if !defined(__APPLE__)
+#include "kudu/util/nvm_cache.h"
+#endif
+
+// Useful in tests that require accurate cache capacity accounting.
+DEFINE_bool(cache_force_single_shard, false,
+            "Override all cache implementations to use just one shard");
+TAG_FLAG(cache_force_single_shard, hidden);
+
+DEFINE_double(cache_memtracker_approximation_ratio, 0.01,
+              "The MemTracker associated with a cache can accumulate error up to "
+              "this ratio to improve performance. For tests.");
+TAG_FLAG(cache_memtracker_approximation_ratio, hidden);
+
+using std::atomic;
+using std::shared_ptr;
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+Cache::~Cache() {
+}
+
+namespace {
+
+typedef simple_spinlock MutexType;
+
+// LRU cache implementation
+
+// An entry is a variable length heap-allocated structure.  Entries
+// are kept in a circular doubly linked list ordered by access time.
+struct LRUHandle {
+  Cache::EvictionCallback* eviction_callback;
+  LRUHandle* next_hash;
+  LRUHandle* next;
+  LRUHandle* prev;
+  size_t charge;      // TODO(opt): Only allow uint32_t?
+  uint32_t key_length;
+  uint32_t val_length;
+  std::atomic<int32_t> refs;
+  uint32_t hash;      // Hash of key(); used for fast sharding and comparisons
+
+  // The storage for the key/value pair itself. The data is stored as:
+  //   [key bytes ...] [padding up to 8-byte boundary] [value bytes ...]
+  uint8_t kv_data[1];   // Beginning of key/value pair
+
+  Slice key() const {
+    return Slice(kv_data, key_length);
+  }
+
+  uint8_t* mutable_val_ptr() {
+    int val_offset = KUDU_ALIGN_UP(key_length, sizeof(void*));
+    return &kv_data[val_offset];
+  }
+
+  const uint8_t* val_ptr() const {
+    return const_cast<LRUHandle*>(this)->mutable_val_ptr();
+  }
+
+  Slice value() const {
+    return Slice(val_ptr(), val_length);
+  }
+};
+
+// We provide our own simple hash table since it removes a whole bunch
+// of porting hacks and is also faster than some of the built-in hash
+// table implementations in some of the compiler/runtime combinations
+// we have tested.  E.g., readrandom speeds up by ~5% over the g++
+// 4.4.3's builtin hashtable.
+class HandleTable {
+ public:
+  HandleTable() : length_(0), elems_(0), list_(nullptr) { Resize(); }
+  ~HandleTable() { delete[] list_; }
+
+  LRUHandle* Lookup(const Slice& key, uint32_t hash) {
+    return *FindPointer(key, hash);
+  }
+
+  LRUHandle* Insert(LRUHandle* h) {
+    LRUHandle** ptr = FindPointer(h->key(), h->hash);
+    LRUHandle* old = *ptr;
+    h->next_hash = (old == nullptr ? nullptr : old->next_hash);
+    *ptr = h;
+    if (old == nullptr) {
+      ++elems_;
+      if (elems_ > length_) {
+        // Since each cache entry is fairly large, we aim for a small
+        // average linked list length (<= 1).
+        Resize();
+      }
+    }
+    return old;
+  }
+
+  LRUHandle* Remove(const Slice& key, uint32_t hash) {
+    LRUHandle** ptr = FindPointer(key, hash);
+    LRUHandle* result = *ptr;
+    if (result != nullptr) {
+      *ptr = result->next_hash;
+      --elems_;
+    }
+    return result;
+  }
+
+ private:
+  // The table consists of an array of buckets where each bucket is
+  // a linked list of cache entries that hash into the bucket.
+  uint32_t length_;
+  uint32_t elems_;
+  LRUHandle** list_;
+
+  // Return a pointer to slot that points to a cache entry that
+  // matches key/hash.  If there is no such cache entry, return a
+  // pointer to the trailing slot in the corresponding linked list.
+  LRUHandle** FindPointer(const Slice& key, uint32_t hash) {
+    LRUHandle** ptr = &list_[hash & (length_ - 1)];
+    while (*ptr != nullptr &&
+           ((*ptr)->hash != hash || key != (*ptr)->key())) {
+      ptr = &(*ptr)->next_hash;
+    }
+    return ptr;
+  }
+
+  void Resize() {
+    uint32_t new_length = 16;
+    while (new_length < elems_ * 1.5) {
+      new_length *= 2;
+    }
+    auto new_list = new LRUHandle*[new_length];
+    memset(new_list, 0, sizeof(new_list[0]) * new_length);
+    uint32_t count = 0;
+    for (uint32_t i = 0; i < length_; i++) {
+      LRUHandle* h = list_[i];
+      while (h != nullptr) {
+        LRUHandle* next = h->next_hash;
+        uint32_t hash = h->hash;
+        LRUHandle** ptr = &new_list[hash & (new_length - 1)];
+        h->next_hash = *ptr;
+        *ptr = h;
+        h = next;
+        count++;
+      }
+    }
+    DCHECK_EQ(elems_, count);
+    delete[] list_;
+    list_ = new_list;
+    length_ = new_length;
+  }
+};
+
+// A single shard of sharded cache.
+class LRUCache {
+ public:
+  explicit LRUCache(MemTracker* tracker);
+  ~LRUCache();
+
+  // Separate from constructor so caller can easily make an array of LRUCache
+  void SetCapacity(size_t capacity) {
+    capacity_ = capacity;
+    max_deferred_consumption_ = capacity * FLAGS_cache_memtracker_approximation_ratio;
+  }
+
+  void SetMetrics(CacheMetrics* metrics) { metrics_ = metrics; }
+
+  Cache::Handle* Insert(LRUHandle* handle, Cache::EvictionCallback* eviction_callback);
+  // Like Cache::Lookup, but with an extra "hash" parameter.
+  Cache::Handle* Lookup(const Slice& key, uint32_t hash, bool caching);
+  void Release(Cache::Handle* handle);
+  void Erase(const Slice& key, uint32_t hash);
+
+ private:
+  void LRU_Remove(LRUHandle* e);
+  void LRU_Append(LRUHandle* e);
+  // Just reduce the reference count by 1.
+  // Return true if last reference
+  bool Unref(LRUHandle* e);
+  // Call the user's eviction callback, if it exists, and free the entry.
+  void FreeEntry(LRUHandle* e);
+
+  // Update the memtracker's consumption by the given amount.
+  //
+  // This "buffers" the updates locally in 'deferred_consumption_' until the amount
+  // of accumulated delta is more than ~1% of the cache capacity. This improves
+  // performance under workloads with high eviction rates for a few reasons:
+  //
+  // 1) once the cache reaches its full capacity, we expect it to remain there
+  // in steady state. Each insertion is usually matched by an eviction, and unless
+  // the total size of the evicted item(s) is much different than the size of the
+  // inserted item, each eviction event is unlikely to change the total cache usage
+  // much. So, we expect that the accumulated error will mostly remain around 0
+  // and we can avoid propagating changes to the MemTracker at all.
+  //
+  // 2) because the cache implementation is sharded, we do this tracking in a bunch
+  // of different locations, avoiding bouncing cache-lines between cores. By contrast
+  // the MemTracker is a simple integer, so it doesn't scale as well under concurrency.
+  //
+  // Positive delta indicates an increased memory consumption.
+  void UpdateMemTracker(int64_t delta);
+
+  // Initialized before use.
+  size_t capacity_;
+
+  // mutex_ protects the following state.
+  MutexType mutex_;
+  size_t usage_;
+
+  // Dummy head of LRU list.
+  // lru.prev is newest entry, lru.next is oldest entry.
+  LRUHandle lru_;
+
+  HandleTable table_;
+
+  MemTracker* mem_tracker_;
+  atomic<int64_t> deferred_consumption_ { 0 };
+
+  // Initialized based on capacity_ to ensure an upper bound on the error on the
+  // MemTracker consumption.
+  int64_t max_deferred_consumption_;
+
+  CacheMetrics* metrics_;
+};
+
+LRUCache::LRUCache(MemTracker* tracker)
+ : usage_(0),
+   mem_tracker_(tracker),
+   metrics_(nullptr) {
+  // Make empty circular linked list
+  lru_.next = &lru_;
+  lru_.prev = &lru_;
+}
+
+LRUCache::~LRUCache() {
+  for (LRUHandle* e = lru_.next; e != &lru_; ) {
+    LRUHandle* next = e->next;
+    DCHECK_EQ(e->refs.load(std::memory_order_relaxed), 1)
+        << "caller has an unreleased handle";
+    if (Unref(e)) {
+      FreeEntry(e);
+    }
+    e = next;
+  }
+  mem_tracker_->Consume(deferred_consumption_);
+}
+
+bool LRUCache::Unref(LRUHandle* e) {
+  DCHECK_GT(e->refs.load(std::memory_order_relaxed), 0);
+  return e->refs.fetch_sub(1) == 1;
+}
+
+void LRUCache::FreeEntry(LRUHandle* e) {
+  DCHECK_EQ(e->refs.load(std::memory_order_relaxed), 0);
+  if (e->eviction_callback) {
+    e->eviction_callback->EvictedEntry(e->key(), e->value());
+  }
+  UpdateMemTracker(-static_cast<int64_t>(e->charge));
+  if (PREDICT_TRUE(metrics_)) {
+    metrics_->cache_usage->DecrementBy(e->charge);
+    metrics_->evictions->Increment();
+  }
+  delete [] e;
+}
+
+void LRUCache::UpdateMemTracker(int64_t delta) {
+  int64_t old_deferred = deferred_consumption_.fetch_add(delta);
+  int64_t new_deferred = old_deferred + delta;
+
+  if (new_deferred > max_deferred_consumption_ ||
+      new_deferred < -max_deferred_consumption_) {
+    int64_t to_propagate = deferred_consumption_.exchange(0, std::memory_order_relaxed);
+    mem_tracker_->Consume(to_propagate);
+  }
+}
+
+void LRUCache::LRU_Remove(LRUHandle* e) {
+  e->next->prev = e->prev;
+  e->prev->next = e->next;
+  usage_ -= e->charge;
+}
+
+void LRUCache::LRU_Append(LRUHandle* e) {
+  // Make "e" newest entry by inserting just before lru_
+  e->next = &lru_;
+  e->prev = lru_.prev;
+  e->prev->next = e;
+  e->next->prev = e;
+  usage_ += e->charge;
+}
+
+Cache::Handle* LRUCache::Lookup(const Slice& key, uint32_t hash, bool caching) {
+  LRUHandle* e;
+  {
+    std::lock_guard<MutexType> l(mutex_);
+    e = table_.Lookup(key, hash);
+    if (e != nullptr) {
+      e->refs.fetch_add(1, std::memory_order_relaxed);
+      LRU_Remove(e);
+      LRU_Append(e);
+    }
+  }
+
+  // Do the metrics outside of the lock.
+  if (metrics_) {
+    metrics_->lookups->Increment();
+    bool was_hit = (e != nullptr);
+    if (was_hit) {
+      if (caching) {
+        metrics_->cache_hits_caching->Increment();
+      } else {
+        metrics_->cache_hits->Increment();
+      }
+    } else {
+      if (caching) {
+        metrics_->cache_misses_caching->Increment();
+      } else {
+        metrics_->cache_misses->Increment();
+      }
+    }
+  }
+
+  return reinterpret_cast<Cache::Handle*>(e);
+}
+
+void LRUCache::Release(Cache::Handle* handle) {
+  LRUHandle* e = reinterpret_cast<LRUHandle*>(handle);
+  bool last_reference = Unref(e);
+  if (last_reference) {
+    FreeEntry(e);
+  }
+}
+
+Cache::Handle* LRUCache::Insert(LRUHandle* e, Cache::EvictionCallback *eviction_callback) {
+
+  // Set the remaining LRUHandle members which were not already allocated during
+  // Allocate().
+  e->eviction_callback = eviction_callback;
+  e->refs.store(2, std::memory_order_relaxed);  // One from LRUCache, one for the returned handle
+  UpdateMemTracker(e->charge);
+  if (PREDICT_TRUE(metrics_)) {
+    metrics_->cache_usage->IncrementBy(e->charge);
+    metrics_->inserts->Increment();
+  }
+
+  LRUHandle* to_remove_head = nullptr;
+  {
+    std::lock_guard<MutexType> l(mutex_);
+
+    LRU_Append(e);
+
+    LRUHandle* old = table_.Insert(e);
+    if (old != nullptr) {
+      LRU_Remove(old);
+      if (Unref(old)) {
+        old->next = to_remove_head;
+        to_remove_head = old;
+      }
+    }
+
+    while (usage_ > capacity_ && lru_.next != &lru_) {
+      LRUHandle* old = lru_.next;
+      LRU_Remove(old);
+      table_.Remove(old->key(), old->hash);
+      if (Unref(old)) {
+        old->next = to_remove_head;
+        to_remove_head = old;
+      }
+    }
+  }
+
+  // we free the entries here outside of mutex for
+  // performance reasons
+  while (to_remove_head != nullptr) {
+    LRUHandle* next = to_remove_head->next;
+    FreeEntry(to_remove_head);
+    to_remove_head = next;
+  }
+
+  return reinterpret_cast<Cache::Handle*>(e);
+}
+
+void LRUCache::Erase(const Slice& key, uint32_t hash) {
+  LRUHandle* e;
+  bool last_reference = false;
+  {
+    std::lock_guard<MutexType> l(mutex_);
+    e = table_.Remove(key, hash);
+    if (e != nullptr) {
+      LRU_Remove(e);
+      last_reference = Unref(e);
+    }
+  }
+  // mutex not held here
+  // last_reference will only be true if e != NULL
+  if (last_reference) {
+    FreeEntry(e);
+  }
+}
+
+// Determine the number of bits of the hash that should be used to determine
+// the cache shard. This, in turn, determines the number of shards.
+int DetermineShardBits() {
+  int bits = PREDICT_FALSE(FLAGS_cache_force_single_shard) ?
+      0 : Bits::Log2Ceiling(base::NumCPUs());
+  VLOG(1) << "Will use " << (1 << bits) << " shards for LRU cache.";
+  return bits;
+}
+
+class ShardedLRUCache : public Cache {
+ private:
+  shared_ptr<MemTracker> mem_tracker_;
+  gscoped_ptr<CacheMetrics> metrics_;
+  vector<LRUCache*> shards_;
+
+  // Number of bits of hash used to determine the shard.
+  const int shard_bits_;
+
+  // Protects 'metrics_'. Used only when metrics are set, to ensure
+  // that they are set only once in test environments.
+  MutexType metrics_lock_;
+
+  static inline uint32_t HashSlice(const Slice& s) {
+    return util_hash::CityHash64(
+      reinterpret_cast<const char *>(s.data()), s.size());
+  }
+
+  uint32_t Shard(uint32_t hash) {
+    // Widen to uint64 before shifting, or else on a single CPU,
+    // we would try to shift a uint32_t by 32 bits, which is undefined.
+    return static_cast<uint64_t>(hash) >> (32 - shard_bits_);
+  }
+
+ public:
+  explicit ShardedLRUCache(size_t capacity, const string& id)
+      : shard_bits_(DetermineShardBits()) {
+    // A cache is often a singleton, so:
+    // 1. We reuse its MemTracker if one already exists, and
+    // 2. It is directly parented to the root MemTracker.
+    mem_tracker_ = MemTracker::FindOrCreateGlobalTracker(
+        -1, strings::Substitute("$0-sharded_lru_cache", id));
+
+    int num_shards = 1 << shard_bits_;
+    const size_t per_shard = (capacity + (num_shards - 1)) / num_shards;
+    for (int s = 0; s < num_shards; s++) {
+      gscoped_ptr<LRUCache> shard(new LRUCache(mem_tracker_.get()));
+      shard->SetCapacity(per_shard);
+      shards_.push_back(shard.release());
+    }
+  }
+
+  virtual ~ShardedLRUCache() {
+    STLDeleteElements(&shards_);
+  }
+
+  virtual Handle* Insert(PendingHandle* handle,
+                         Cache::EvictionCallback* eviction_callback) OVERRIDE {
+    LRUHandle* h = reinterpret_cast<LRUHandle*>(DCHECK_NOTNULL(handle));
+    return shards_[Shard(h->hash)]->Insert(h, eviction_callback);
+  }
+  virtual Handle* Lookup(const Slice& key, CacheBehavior caching) OVERRIDE {
+    const uint32_t hash = HashSlice(key);
+    return shards_[Shard(hash)]->Lookup(key, hash, caching == EXPECT_IN_CACHE);
+  }
+  virtual void Release(Handle* handle) OVERRIDE {
+    LRUHandle* h = reinterpret_cast<LRUHandle*>(handle);
+    shards_[Shard(h->hash)]->Release(handle);
+  }
+  virtual void Erase(const Slice& key) OVERRIDE {
+    const uint32_t hash = HashSlice(key);
+    shards_[Shard(hash)]->Erase(key, hash);
+  }
+  virtual Slice Value(Handle* handle) OVERRIDE {
+    return reinterpret_cast<LRUHandle*>(handle)->value();
+  }
+  virtual void SetMetrics(const scoped_refptr<MetricEntity>& entity) OVERRIDE {
+    // TODO(KUDU-2165): reuse of the Cache singleton across multiple MiniCluster servers
+    // causes TSAN errors. So, we'll ensure that metrics only get attached once, from
+    // whichever server starts first. This has the downside that, in test builds, we won't
+    // get accurate cache metrics, but that's probably better than spurious failures.
+    std::lock_guard<simple_spinlock> l(metrics_lock_);
+    if (metrics_) {
+      CHECK(IsGTest()) << "Metrics should only be set once per Cache singleton";
+      return;
+    }
+    metrics_.reset(new CacheMetrics(entity));
+    for (LRUCache* cache : shards_) {
+      cache->SetMetrics(metrics_.get());
+    }
+  }
+
+  virtual PendingHandle* Allocate(Slice key, int val_len, int charge) OVERRIDE {
+    int key_len = key.size();
+    DCHECK_GE(key_len, 0);
+    DCHECK_GE(val_len, 0);
+    int key_len_padded = KUDU_ALIGN_UP(key_len, sizeof(void*));
+    uint8_t* buf = new uint8_t[sizeof(LRUHandle)
+                               + key_len_padded + val_len // the kv_data VLA data
+                               - 1 // (the VLA has a 1-byte placeholder)
+                               ];
+    LRUHandle* handle = reinterpret_cast<LRUHandle*>(buf);
+    handle->key_length = key_len;
+    handle->val_length = val_len;
+    handle->charge = (charge == kAutomaticCharge) ? kudu_malloc_usable_size(buf) : charge;
+    handle->hash = HashSlice(key);
+    memcpy(handle->kv_data, key.data(), key_len);
+
+    return reinterpret_cast<PendingHandle*>(handle);
+  }
+
+  virtual void Free(PendingHandle* h) OVERRIDE {
+    uint8_t* data = reinterpret_cast<uint8_t*>(h);
+    delete [] data;
+  }
+
+  virtual uint8_t* MutableValue(PendingHandle* h) OVERRIDE {
+    return reinterpret_cast<LRUHandle*>(h)->mutable_val_ptr();
+  }
+
+};
+
+}  // end anonymous namespace
+
+Cache* NewLRUCache(CacheType type, size_t capacity, const string& id) {
+  switch (type) {
+    case DRAM_CACHE:
+      return new ShardedLRUCache(capacity, id);
+#if defined(HAVE_LIB_VMEM)
+    case NVM_CACHE:
+      return NewLRUNvmCache(capacity, id);
+#endif
+    default:
+      LOG(FATAL) << "Unsupported LRU cache type: " << type;
+  }
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache.h b/be/src/kudu/util/cache.h
new file mode 100644
index 0000000..82ef8c9
--- /dev/null
+++ b/be/src/kudu/util/cache.h
@@ -0,0 +1,216 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+//
+// A Cache is an interface that maps keys to values.  It has internal
+// synchronization and may be safely accessed concurrently from
+// multiple threads.  It may automatically evict entries to make room
+// for new entries.  Values have a specified charge against the cache
+// capacity.  For example, a cache where the values are variable
+// length strings, may use the length of the string as the charge for
+// the string.
+//
+// This is taken from LevelDB and evolved to fit the kudu codebase.
+//
+// TODO: this is pretty lock-heavy. Would be good to sub out something
+// a little more concurrent.
+
+#ifndef KUDU_UTIL_CACHE_H_
+#define KUDU_UTIL_CACHE_H_
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/slice.h"
+
+namespace kudu {
+
+class Cache;
+class MetricEntity;
+
+enum CacheType {
+  DRAM_CACHE,
+  NVM_CACHE
+};
+
+// Create a new cache with a fixed size capacity.  This implementation
+// of Cache uses a least-recently-used eviction policy.
+Cache* NewLRUCache(CacheType type, size_t capacity, const std::string& id);
+
+class Cache {
+ public:
+  // Callback interface which is called when an entry is evicted from the
+  // cache.
+  class EvictionCallback {
+   public:
+    virtual void EvictedEntry(Slice key, Slice value) = 0;
+    virtual ~EvictionCallback() {}
+  };
+
+  Cache() { }
+
+  // Destroys all existing entries by calling the "deleter"
+  // function that was passed to the constructor.
+  virtual ~Cache();
+
+  // Opaque handle to an entry stored in the cache.
+  struct Handle { };
+
+  // Custom handle "deleter", primarily intended for use with std::unique_ptr.
+  //
+  // Sample usage:
+  //
+  //   Cache* cache = NewLRUCache(...);
+  //   ...
+  //   {
+  //     unique_ptr<Cache::Handle, Cache::HandleDeleter> h(
+  //       cache->Lookup(...), Cache::HandleDeleter(cache));
+  //     ...
+  //   } // 'h' is automatically released here
+  //
+  // Or:
+  //
+  //   Cache* cache = NewLRUCache(...);
+  //   ...
+  //   {
+  //     Cache::UniqueHandle h(cache->Lookup(...), Cache::HandleDeleter(cache));
+  //     ...
+  //   } // 'h' is automatically released here
+  //
+  class HandleDeleter {
+   public:
+    explicit HandleDeleter(Cache* c)
+        : c_(c) {
+    }
+
+    void operator()(Cache::Handle* h) const {
+      c_->Release(h);
+    }
+
+   private:
+    Cache* c_;
+  };
+  typedef std::unique_ptr<Handle, HandleDeleter> UniqueHandle;
+
+  // Passing EXPECT_IN_CACHE will increment the hit/miss metrics that track the number of times
+  // blocks were requested that the users were hoping to get the block from the cache, along with
+  // with the basic metrics.
+  // Passing NO_EXPECT_IN_CACHE will only increment the basic metrics.
+  // This helps in determining if we are effectively caching the blocks that matter the most.
+  enum CacheBehavior {
+    EXPECT_IN_CACHE,
+    NO_EXPECT_IN_CACHE
+  };
+
+  // If the cache has no mapping for "key", returns NULL.
+  //
+  // Else return a handle that corresponds to the mapping.  The caller
+  // must call this->Release(handle) when the returned mapping is no
+  // longer needed.
+  virtual Handle* Lookup(const Slice& key, CacheBehavior caching) = 0;
+
+  // Release a mapping returned by a previous Lookup().
+  // REQUIRES: handle must not have been released yet.
+  // REQUIRES: handle must have been returned by a method on *this.
+  virtual void Release(Handle* handle) = 0;
+
+  // Return the value encapsulated in a handle returned by a
+  // successful Lookup().
+  // REQUIRES: handle must not have been released yet.
+  // REQUIRES: handle must have been returned by a method on *this.
+  virtual Slice Value(Handle* handle) = 0;
+
+  // If the cache contains entry for key, erase it.  Note that the
+  // underlying entry will be kept around until all existing handles
+  // to it have been released.
+  virtual void Erase(const Slice& key) = 0;
+
+  // Pass a metric entity in order to start recoding metrics.
+  virtual void SetMetrics(const scoped_refptr<MetricEntity>& metric_entity) = 0;
+
+  // ------------------------------------------------------------
+  // Insertion path
+  // ------------------------------------------------------------
+  //
+  // Because some cache implementations (eg NVM) manage their own memory, and because we'd
+  // like to read blocks directly into cache-managed memory rather than causing an extra
+  // memcpy, the insertion of a new element into the cache requires two phases. First, a
+  // PendingHandle is allocated with space for the value, and then it is later inserted.
+  //
+  // For example:
+  //
+  //   PendingHandle* ph = cache_->Allocate("my entry", value_size, charge);
+  //   if (!ReadDataFromDisk(cache_->MutableValue(ph)).ok()) {
+  //     cache_->Free(ph);
+  //     ... error handling ...
+  //     return;
+  //   }
+  //   Handle* h = cache_->Insert(ph, my_eviction_callback);
+  //   ...
+  //   cache_->Release(h);
+
+  // Opaque handle to an entry which is being prepared to be added to
+  // the cache.
+  struct PendingHandle { };
+
+  // Indicates that the charge of an item in the cache should be calculated
+  // based on its memory consumption.
+  static constexpr int kAutomaticCharge = -1;
+
+  // Allocate space for a new entry to be inserted into the cache.
+  //
+  // The provided 'key' is copied into the resulting handle object.
+  // The allocated handle has enough space such that the value can
+  // be written into cache_->MutableValue(handle).
+  //
+  // If 'charge' is not 'kAutomaticCharge', then the cache capacity will be charged
+  // the explicit amount. This is useful when caching items that are small but need to
+  // maintain a bounded count (eg file descriptors) rather than caring about their actual
+  // memory usage.
+  //
+  // Note that this does not mutate the cache itself: lookups will
+  // not be able to find the provided key until it is inserted.
+  //
+  // It is possible that this will return NULL if the cache is above its capacity
+  // and eviction fails to free up enough space for the requested allocation.
+  //
+  // NOTE: the returned memory is not automatically freed by the cache: the
+  // caller must either free it using Free(), or insert it using Insert().
+  virtual PendingHandle* Allocate(Slice key, int val_len, int charge) = 0;
+
+  // Default 'charge' should be kAutomaticCharge.
+  // (default arguments on virtual functions are prohibited)
+  PendingHandle* Allocate(Slice key, int val_len) {
+    return Allocate(key, val_len, kAutomaticCharge);
+  }
+
+  virtual uint8_t* MutableValue(PendingHandle* handle) = 0;
+
+  // Commit a prepared entry into the cache.
+  //
+  // Returns a handle that corresponds to the mapping.  The caller
+  // must call this->Release(handle) when the returned mapping is no
+  // longer needed. This method always succeeds and returns a non-null
+  // entry, since the space was reserved above.
+  //
+  // The 'pending' entry passed here should have been allocated using
+  // Cache::Allocate() above.
+  //
+  // If 'eviction_callback' is non-NULL, then it will be called when the
+  // entry is later evicted or when the cache shuts down.
+  virtual Handle* Insert(PendingHandle* pending, EvictionCallback* eviction_callback) = 0;
+
+  // Free 'ptr', which must have been previously allocated using 'Allocate'.
+  virtual void Free(PendingHandle* ptr) = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(Cache);
+};
+
+}  // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache_metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache_metrics.cc b/be/src/kudu/util/cache_metrics.cc
new file mode 100644
index 0000000..ac2fadf
--- /dev/null
+++ b/be/src/kudu/util/cache_metrics.cc
@@ -0,0 +1,69 @@
+// 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 "kudu/util/cache_metrics.h"
+
+#include "kudu/util/metrics.h"
+
+METRIC_DEFINE_counter(server, block_cache_inserts,
+                      "Block Cache Inserts", kudu::MetricUnit::kBlocks,
+                      "Number of blocks inserted in the cache");
+METRIC_DEFINE_counter(server, block_cache_lookups,
+                      "Block Cache Lookups", kudu::MetricUnit::kBlocks,
+                      "Number of blocks looked up from the cache");
+METRIC_DEFINE_counter(server, block_cache_evictions,
+                      "Block Cache Evictions", kudu::MetricUnit::kBlocks,
+                      "Number of blocks evicted from the cache");
+METRIC_DEFINE_counter(server, block_cache_misses,
+                      "Block Cache Misses", kudu::MetricUnit::kBlocks,
+                      "Number of lookups that didn't yield a block");
+METRIC_DEFINE_counter(server, block_cache_misses_caching,
+                      "Block Cache Misses (Caching)", kudu::MetricUnit::kBlocks,
+                      "Number of lookups that were expecting a block that didn't yield one."
+                      "Use this number instead of cache_misses when trying to determine how "
+                      "efficient the cache is");
+METRIC_DEFINE_counter(server, block_cache_hits,
+                      "Block Cache Hits", kudu::MetricUnit::kBlocks,
+                      "Number of lookups that found a block");
+METRIC_DEFINE_counter(server, block_cache_hits_caching,
+                      "Block Cache Hits (Caching)", kudu::MetricUnit::kBlocks,
+                      "Number of lookups that were expecting a block that found one."
+                      "Use this number instead of cache_hits when trying to determine how "
+                      "efficient the cache is");
+
+METRIC_DEFINE_gauge_uint64(server, block_cache_usage, "Block Cache Memory Usage",
+                           kudu::MetricUnit::kBytes,
+                           "Memory consumed by the block cache");
+
+namespace kudu {
+
+#define MINIT(member, x) member(METRIC_##x.Instantiate(entity))
+#define GINIT(member, x) member(METRIC_##x.Instantiate(entity, 0))
+CacheMetrics::CacheMetrics(const scoped_refptr<MetricEntity>& entity)
+  : MINIT(inserts, block_cache_inserts),
+    MINIT(lookups, block_cache_lookups),
+    MINIT(evictions, block_cache_evictions),
+    MINIT(cache_hits, block_cache_hits),
+    MINIT(cache_hits_caching, block_cache_hits_caching),
+    MINIT(cache_misses, block_cache_misses),
+    MINIT(cache_misses_caching, block_cache_misses_caching),
+    GINIT(cache_usage, block_cache_usage) {
+}
+#undef MINIT
+#undef GINIT
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cache_metrics.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cache_metrics.h b/be/src/kudu/util/cache_metrics.h
new file mode 100644
index 0000000..04a546b
--- /dev/null
+++ b/be/src/kudu/util/cache_metrics.h
@@ -0,0 +1,42 @@
+// 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.
+#ifndef KUDU_UTIL_CACHE_METRICS_H
+#define KUDU_UTIL_CACHE_METRICS_H
+
+#include <cstdint>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/metrics.h"
+
+namespace kudu {
+
+struct CacheMetrics {
+  explicit CacheMetrics(const scoped_refptr<MetricEntity>& metric_entity);
+
+  scoped_refptr<Counter> inserts;
+  scoped_refptr<Counter> lookups;
+  scoped_refptr<Counter> evictions;
+  scoped_refptr<Counter> cache_hits;
+  scoped_refptr<Counter> cache_hits_caching;
+  scoped_refptr<Counter> cache_misses;
+  scoped_refptr<Counter> cache_misses_caching;
+
+  scoped_refptr<AtomicGauge<uint64_t> > cache_usage;
+};
+
+} // namespace kudu
+#endif /* KUDU_UTIL_CACHE_METRICS_H */

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/callback_bind-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/callback_bind-test.cc b/be/src/kudu/util/callback_bind-test.cc
new file mode 100644
index 0000000..392f496
--- /dev/null
+++ b/be/src/kudu/util/callback_bind-test.cc
@@ -0,0 +1,119 @@
+// 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 <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/bind.h"
+#include "kudu/gutil/bind_helpers.h"
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+
+namespace kudu {
+
+using std::string;
+
+static int Return5() {
+  return 5;
+}
+
+TEST(CallbackBindTest, TestFreeFunction) {
+  Callback<int(void)> func_cb = Bind(&Return5);
+  ASSERT_EQ(5, func_cb.Run());
+}
+
+class Ref : public RefCountedThreadSafe<Ref> {
+ public:
+  int Foo() { return 3; }
+};
+
+// Simple class that helps with verifying ref counting.
+// Not thread-safe.
+struct RefCountable {
+  RefCountable()
+      : refs(0) {
+  }
+  void AddRef() const {
+    refs++;
+  }
+  void Release() const {
+    refs--;
+  }
+  void Print() const {
+    LOG(INFO) << "Hello. Refs: " << refs;
+  }
+
+  mutable int refs;
+  DISALLOW_COPY_AND_ASSIGN(RefCountable);
+};
+
+TEST(CallbackBindTest, TestClassMethod) {
+  scoped_refptr<Ref> ref = new Ref();
+  Callback<int(void)> ref_cb = Bind(&Ref::Foo, ref);
+  ref = nullptr;
+  ASSERT_EQ(3, ref_cb.Run());
+}
+
+int ReturnI(int i, const char* str) {
+  return i;
+}
+
+TEST(CallbackBindTest, TestPartialBind) {
+  Callback<int(const char*)> cb = Bind(&ReturnI, 23);
+  ASSERT_EQ(23, cb.Run("hello world"));
+}
+
+char IncrementChar(gscoped_ptr<char> in) {
+  return *in + 1;
+}
+
+TEST(CallbackBindTest, TestCallScopedPtrArg) {
+  // Calling a function with a gscoped_ptr argument is just like any other
+  // function which takes gscoped_ptr:
+  gscoped_ptr<char> foo(new char('x'));
+  Callback<char(gscoped_ptr<char>)> cb = Bind(&IncrementChar);
+  ASSERT_EQ('y', cb.Run(std::move(foo)));
+}
+
+TEST(CallbackBindTest, TestBindScopedPtrArg) {
+  // Binding a function with a gscoped_ptr argument requires using Passed()
+  gscoped_ptr<char> foo(new char('x'));
+  Callback<char(void)> cb = Bind(&IncrementChar, Passed(&foo));
+  ASSERT_EQ('y', cb.Run());
+}
+
+// Test that the ref counting functionality works.
+TEST(CallbackBindTest, TestRefCounting) {
+  RefCountable countable;
+  {
+    ASSERT_EQ(0, countable.refs);
+    Closure cb = Bind(&RefCountable::Print, &countable);
+    ASSERT_EQ(1, countable.refs);
+    cb.Run();
+    ASSERT_EQ(1, countable.refs);
+  }
+  ASSERT_EQ(0, countable.refs);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/coding-inl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/coding-inl.h b/be/src/kudu/util/coding-inl.h
new file mode 100644
index 0000000..a47e9ce
--- /dev/null
+++ b/be/src/kudu/util/coding-inl.h
@@ -0,0 +1,120 @@
+// 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.
+// Some portions Copyright (c) 2011 The LevelDB Authors.
+//
+// Endian-neutral encoding:
+// * Fixed-length numbers are encoded with least-significant byte first
+// * In addition we support variable length "varint" encoding
+// * Strings are encoded prefixed by their length in varint format
+
+#ifndef KUDU_UTIL_CODING_INL_H
+#define KUDU_UTIL_CODING_INL_H
+
+#include <cstdint>
+#include <cstring>
+
+#include "kudu/gutil/port.h"  // IWYU pragma: keep
+// IWYU pragma: no_include <endian.h>
+
+namespace kudu {
+
+inline uint8_t *InlineEncodeVarint32(uint8_t *dst, uint32_t v) {
+  // Operate on characters as unsigneds
+  uint8_t *ptr = dst;
+  static const int B = 128;
+  if (v < (1<<7)) {
+    *(ptr++) = v;
+  } else if (v < (1<<14)) {
+    *(ptr++) = v | B;
+    *(ptr++) = v>>7;
+  } else if (v < (1<<21)) {
+    *(ptr++) = v | B;
+    *(ptr++) = (v>>7) | B;
+    *(ptr++) = v>>14;
+  } else if (v < (1<<28)) {
+    *(ptr++) = v | B;
+    *(ptr++) = (v>>7) | B;
+    *(ptr++) = (v>>14) | B;
+    *(ptr++) = v>>21;
+  } else {
+    *(ptr++) = v | B;
+    *(ptr++) = (v>>7) | B;
+    *(ptr++) = (v>>14) | B;
+    *(ptr++) = (v>>21) | B;
+    *(ptr++) = v>>28;
+  }
+  return ptr;
+}
+
+inline void InlineEncodeFixed32(uint8_t *buf, uint32_t value) {
+#if __BYTE_ORDER == __LITTLE_ENDIAN
+  memcpy(buf, &value, sizeof(value));
+#else
+  buf[0] = value & 0xff;
+  buf[1] = (value >> 8) & 0xff;
+  buf[2] = (value >> 16) & 0xff;
+  buf[3] = (value >> 24) & 0xff;
+#endif
+}
+
+inline void InlineEncodeFixed64(uint8_t *buf, uint64_t value) {
+#if __BYTE_ORDER == __LITTLE_ENDIAN
+  memcpy(buf, &value, sizeof(value));
+#else
+  buf[0] = value & 0xff;
+  buf[1] = (value >> 8) & 0xff;
+  buf[2] = (value >> 16) & 0xff;
+  buf[3] = (value >> 24) & 0xff;
+  buf[4] = (value >> 32) & 0xff;
+  buf[5] = (value >> 40) & 0xff;
+  buf[6] = (value >> 48) & 0xff;
+  buf[7] = (value >> 56) & 0xff;
+#endif
+}
+
+
+// Standard Put... routines append to a string
+template <class StrType>
+inline void InlinePutFixed32(StrType *dst, uint32_t value) {
+  uint8_t buf[sizeof(value)];
+  InlineEncodeFixed32(buf, value);
+  dst->append(buf, sizeof(buf));
+}
+
+template <class StrType>
+inline void InlinePutFixed64(StrType *dst, uint64_t value) {
+  uint8_t buf[sizeof(value)];
+  InlineEncodeFixed64(buf, value);
+  dst->append(buf, sizeof(buf));
+}
+
+template <class StrType>
+inline void InlinePutVarint32(StrType* dst, uint32_t v) {
+  // We resize the array and then size it back down as appropriate
+  // rather than using append(), since the generated code ends up
+  // being substantially shorter.
+  int old_size = dst->size();
+  dst->resize(old_size + 5);
+  uint8_t* p = &(*dst)[old_size];
+  uint8_t *ptr = InlineEncodeVarint32(p, v);
+
+  dst->resize(old_size + ptr - p);
+}
+
+} // namespace kudu
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/coding.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/coding.cc b/be/src/kudu/util/coding.cc
new file mode 100644
index 0000000..952af28
--- /dev/null
+++ b/be/src/kudu/util/coding.cc
@@ -0,0 +1,142 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "kudu/util/coding.h"
+#include "kudu/util/coding-inl.h"
+#include "kudu/util/faststring.h"
+
+namespace kudu {
+
+void PutVarint32(faststring* dst, uint32_t v) {
+  uint8_t buf[5];
+  uint8_t* ptr = InlineEncodeVarint32(buf, v);
+  dst->append(buf, ptr - buf);
+}
+
+uint8_t* EncodeVarint64(uint8_t* dst, uint64_t v) {
+  static const int B = 128;
+  while (v >= B) {
+    *(dst++) = (v & (B-1)) | B;
+    v >>= 7;
+  }
+  *(dst++) = static_cast<uint8_t>(v);
+  return dst;
+}
+
+void PutFixed32(faststring *dst, uint32_t value) {
+  InlinePutFixed32(dst, value);
+}
+
+void PutFixed64(faststring *dst, uint64_t value) {
+  InlinePutFixed64(dst, value);
+}
+
+void PutVarint64(faststring *dst, uint64_t v) {
+  uint8_t buf[10];
+  uint8_t* ptr = EncodeVarint64(buf, v);
+  dst->append(buf, ptr - buf);
+}
+
+void PutLengthPrefixedSlice(faststring* dst, const Slice& value) {
+  PutVarint32(dst, value.size());
+  dst->append(value.data(), value.size());
+}
+
+void PutFixed32LengthPrefixedSlice(faststring* dst, const Slice& value) {
+  PutFixed32(dst, value.size());
+  dst->append(value.data(), value.size());
+}
+
+int VarintLength(uint64_t v) {
+  int len = 1;
+  while (v >= 128) {
+    v >>= 7;
+    len++;
+  }
+  return len;
+}
+
+const uint8_t *GetVarint32PtrFallback(const uint8_t *p,
+                                   const uint8_t *limit,
+                                   uint32_t* value) {
+  uint32_t result = 0;
+  for (uint32_t shift = 0; shift <= 28 && p < limit; shift += 7) {
+    uint32_t byte = *p;
+    p++;
+    if (byte & 128) {
+      // More bytes are present
+      result |= ((byte & 127) << shift);
+    } else {
+      result |= (byte << shift);
+      *value = result;
+      return p;
+    }
+  }
+  return nullptr;
+}
+
+bool GetVarint32(Slice* input, uint32_t* value) {
+  const uint8_t *p = input->data();
+  const uint8_t *limit = p + input->size();
+  const uint8_t *q = GetVarint32Ptr(p, limit, value);
+  if (q == nullptr) {
+    return false;
+  } else {
+    *input = Slice(q, limit - q);
+    return true;
+  }
+}
+
+const uint8_t *GetVarint64Ptr(const uint8_t *p, const uint8_t *limit, uint64_t* value) {
+  uint64_t result = 0;
+  for (uint32_t shift = 0; shift <= 63 && p < limit; shift += 7) {
+    uint64_t byte = *p;
+    p++;
+    if (byte & 128) {
+      // More bytes are present
+      result |= ((byte & 127) << shift);
+    } else {
+      result |= (byte << shift);
+      *value = result;
+      return p;
+    }
+  }
+  return nullptr;
+}
+
+bool GetVarint64(Slice* input, uint64_t* value) {
+  const uint8_t *p = input->data();
+  const uint8_t *limit = p + input->size();
+  const uint8_t *q = GetVarint64Ptr(p, limit, value);
+  if (q == nullptr) {
+    return false;
+  } else {
+    *input = Slice(q, limit - q);
+    return true;
+  }
+}
+
+const uint8_t *GetLengthPrefixedSlice(const uint8_t *p, const uint8_t *limit,
+                                   Slice* result) {
+  uint32_t len = 0;
+  p = GetVarint32Ptr(p, limit, &len);
+  if (p == nullptr) return nullptr;
+  if (p + len > limit) return nullptr;
+  *result = Slice(p, len);
+  return p + len;
+}
+
+bool GetLengthPrefixedSlice(Slice* input, Slice* result) {
+  uint32_t len = 0;
+  if (GetVarint32(input, &len) &&
+      input->size() >= len) {
+    *result = Slice(input->data(), len);
+    input->remove_prefix(len);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/coding.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/coding.h b/be/src/kudu/util/coding.h
new file mode 100644
index 0000000..0612533
--- /dev/null
+++ b/be/src/kudu/util/coding.h
@@ -0,0 +1,113 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// Endian-neutral encoding:
+// * Fixed-length numbers are encoded with least-significant byte first
+// * In addition we support variable length "varint" encoding
+// * Strings are encoded prefixed by their length in varint format
+
+#ifndef STORAGE_LEVELDB_UTIL_CODING_H_
+#define STORAGE_LEVELDB_UTIL_CODING_H_
+
+#include <cstdint>
+#include <cstring>
+
+#include "kudu/util/slice.h"
+#include "kudu/gutil/port.h"  // IWYU pragma: keep
+// IWYU pragma: no_include <endian.h>
+
+namespace kudu {
+
+class faststring;
+
+extern void PutFixed32(faststring* dst, uint32_t value);
+extern void PutFixed64(faststring* dst, uint64_t value);
+extern void PutVarint32(faststring* dst, uint32_t value);
+extern void PutVarint64(faststring* dst, uint64_t value);
+
+// Put a length-prefixed Slice into the buffer. The length prefix
+// is varint-encoded.
+extern void PutLengthPrefixedSlice(faststring* dst, const Slice& value);
+
+// Put a length-prefixed Slice into the buffer. The length prefix
+// is 32-bit fixed encoded in little endian.
+extern void PutFixed32LengthPrefixedSlice(faststring* dst, const Slice& value);
+
+// Standard Get... routines parse a value from the beginning of a Slice
+// and advance the slice past the parsed value.
+extern bool GetVarint32(Slice* input, uint32_t* value);
+extern bool GetVarint64(Slice* input, uint64_t* value);
+extern bool GetLengthPrefixedSlice(Slice* input, Slice* result);
+
+// Pointer-based variants of GetVarint...  These either store a value
+// in *v and return a pointer just past the parsed value, or return
+// NULL on error.  These routines only look at bytes in the range
+// [p..limit-1]
+extern const uint8_t *GetVarint32Ptr(const uint8_t *p,const uint8_t *limit, uint32_t* v);
+extern const uint8_t *GetVarint64Ptr(const uint8_t *p,const uint8_t *limit, uint64_t* v);
+
+// Returns the length of the varint32 or varint64 encoding of "v"
+extern int VarintLength(uint64_t v);
+
+// Lower-level versions of Put... that write directly into a character buffer
+// REQUIRES: dst has enough space for the value being written
+extern void EncodeFixed32(uint8_t *dst, uint32_t value);
+extern void EncodeFixed64(uint8_t *dst, uint64_t value);
+
+// Lower-level versions of Put... that write directly into a character buffer
+// and return a pointer just past the last byte written.
+// REQUIRES: dst has enough space for the value being written
+extern uint8_t *EncodeVarint32(uint8_t *dst, uint32_t value);
+extern uint8_t *EncodeVarint64(uint8_t *dst, uint64_t value);
+
+// Lower-level versions of Get... that read directly from a character buffer
+// without any bounds checking.
+
+inline uint32_t DecodeFixed32(const uint8_t *ptr) {
+#if __BYTE_ORDER == __LITTLE_ENDIAN
+    // Load the raw bytes
+    uint32_t result;
+    memcpy(&result, ptr, sizeof(result));  // gcc optimizes this to a plain load
+    return result;
+#else
+    return ((static_cast<uint32_t>(static_cast<unsigned char>(ptr[0])))
+        | (static_cast<uint32_t>(static_cast<unsigned char>(ptr[1])) << 8)
+        | (static_cast<uint32_t>(static_cast<unsigned char>(ptr[2])) << 16)
+        | (static_cast<uint32_t>(static_cast<unsigned char>(ptr[3])) << 24));
+#endif
+}
+
+inline uint64_t DecodeFixed64(const uint8_t *ptr) {
+#if __BYTE_ORDER == __LITTLE_ENDIAN
+    // Load the raw bytes
+    uint64_t result;
+    memcpy(&result, ptr, sizeof(result));  // gcc optimizes this to a plain load
+    return result;
+#else
+    uint64_t lo = DecodeFixed32(ptr);
+    uint64_t hi = DecodeFixed32(ptr + 4);
+    return (hi << 32) | lo;
+#endif
+}
+
+// Internal routine for use by fallback path of GetVarint32Ptr
+extern const uint8_t *GetVarint32PtrFallback(const uint8_t *p,
+                                             const uint8_t *limit,
+                                             uint32_t* value);
+inline const uint8_t *GetVarint32Ptr(const uint8_t *p,
+                                     const uint8_t *limit,
+                                     uint32_t* value) {
+  if (PREDICT_TRUE(p < limit)) {
+    uint32_t result = *p;
+    if (PREDICT_TRUE((result & 128) == 0)) {
+      *value = result;
+      return p + 1;
+    }
+  }
+  return GetVarint32PtrFallback(p, limit, value);
+}
+
+}  // namespace kudu
+
+#endif  // STORAGE_LEVELDB_UTIL_CODING_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/compression/compression-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/compression/compression-test.cc b/be/src/kudu/util/compression/compression-test.cc
new file mode 100644
index 0000000..6b46a4f
--- /dev/null
+++ b/be/src/kudu/util/compression/compression-test.cc
@@ -0,0 +1,90 @@
+// 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 <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/util/compression/compression.pb.h"
+#include "kudu/util/compression/compression_codec.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+using std::vector;
+
+class TestCompression : public KuduTest {};
+
+static void TestCompressionCodec(CompressionType compression) {
+  const int kInputSize = 64;
+
+  const CompressionCodec* codec;
+  uint8_t ibuffer[kInputSize];
+  uint8_t ubuffer[kInputSize];
+  size_t compressed;
+
+  // Fill the test input buffer
+  memset(ibuffer, 'Z', kInputSize);
+
+  // Get the specified compression codec
+  ASSERT_OK(GetCompressionCodec(compression, &codec));
+
+  // Allocate the compression buffer
+  size_t max_compressed = codec->MaxCompressedLength(kInputSize);
+  ASSERT_LT(max_compressed, (kInputSize * 2));
+  gscoped_array<uint8_t> cbuffer(new uint8_t[max_compressed]);
+
+  // Compress and uncompress
+  ASSERT_OK(codec->Compress(Slice(ibuffer, kInputSize), cbuffer.get(), &compressed));
+  ASSERT_OK(codec->Uncompress(Slice(cbuffer.get(), compressed), ubuffer, kInputSize));
+  ASSERT_EQ(0, memcmp(ibuffer, ubuffer, kInputSize));
+
+  // Compress slices and uncompress
+  vector<Slice> v;
+  v.emplace_back(ibuffer, 1);
+  for (int i = 1; i <= kInputSize; i += 7)
+    v.emplace_back(ibuffer + i, 7);
+  ASSERT_OK(codec->Compress(Slice(ibuffer, kInputSize), cbuffer.get(), &compressed));
+  ASSERT_OK(codec->Uncompress(Slice(cbuffer.get(), compressed), ubuffer, kInputSize));
+  ASSERT_EQ(0, memcmp(ibuffer, ubuffer, kInputSize));
+}
+
+TEST_F(TestCompression, TestNoCompressionCodec) {
+  const CompressionCodec* codec;
+  ASSERT_OK(GetCompressionCodec(NO_COMPRESSION, &codec));
+  ASSERT_EQ(nullptr, codec);
+}
+
+TEST_F(TestCompression, TestSnappyCompressionCodec) {
+  TestCompressionCodec(SNAPPY);
+}
+
+TEST_F(TestCompression, TestLz4CompressionCodec) {
+  TestCompressionCodec(LZ4);
+}
+
+TEST_F(TestCompression, TestZlibCompressionCodec) {
+  TestCompressionCodec(ZLIB);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/compression/compression.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/compression/compression.proto b/be/src/kudu/util/compression/compression.proto
new file mode 100644
index 0000000..a0f5343
--- /dev/null
+++ b/be/src/kudu/util/compression/compression.proto
@@ -0,0 +1,29 @@
+// 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.
+syntax = "proto2";
+package kudu;
+
+option java_package = "org.apache.kudu";
+
+enum CompressionType {
+  UNKNOWN_COMPRESSION = 999;
+  DEFAULT_COMPRESSION = 0;
+  NO_COMPRESSION = 1;
+  SNAPPY = 2;
+  LZ4 = 3;
+  ZLIB = 4;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/compression/compression_codec.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/compression/compression_codec.cc b/be/src/kudu/util/compression/compression_codec.cc
new file mode 100644
index 0000000..a2231b6
--- /dev/null
+++ b/be/src/kudu/util/compression/compression_codec.cc
@@ -0,0 +1,286 @@
+// 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 "kudu/util/compression/compression_codec.h"
+
+#include <memory>
+#include <ostream>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+#include <lz4.h>
+#include <snappy-sinksource.h>
+#include <snappy.h>
+#include <zlib.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/singleton.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/string_case.h"
+
+namespace kudu {
+
+using std::vector;
+
+CompressionCodec::CompressionCodec() {
+}
+CompressionCodec::~CompressionCodec() {
+}
+
+class SlicesSource : public snappy::Source {
+ public:
+  explicit SlicesSource(const std::vector<Slice>& slices)
+    : slice_index_(0),
+      slice_offset_(0),
+      slices_(slices) {
+    available_ = TotalSize();
+  }
+
+  size_t Available() const OVERRIDE {
+    return available_;
+  }
+
+  const char* Peek(size_t* len) OVERRIDE {
+    if (available_ == 0) {
+      *len = 0;
+      return nullptr;
+    }
+
+    const Slice& data = slices_[slice_index_];
+    *len = data.size() - slice_offset_;
+    return reinterpret_cast<const char *>(data.data()) + slice_offset_;
+  }
+
+  void Skip(size_t n) OVERRIDE {
+    DCHECK_LE(n, Available());
+    if (n == 0) return;
+
+    available_ -= n;
+    if ((n + slice_offset_) < slices_[slice_index_].size()) {
+      slice_offset_ += n;
+    } else {
+      n -= slices_[slice_index_].size() - slice_offset_;
+      slice_index_++;
+      while (n > 0 && n >= slices_[slice_index_].size()) {
+        n -= slices_[slice_index_].size();
+        slice_index_++;
+      }
+      slice_offset_ = n;
+    }
+  }
+
+  void Dump(faststring *buffer) {
+    buffer->reserve(buffer->size() + TotalSize());
+    for (const Slice& block : slices_) {
+      buffer->append(block.data(), block.size());
+    }
+  }
+
+ private:
+  size_t TotalSize(void) const {
+    size_t size = 0;
+    for (const Slice& data : slices_) {
+      size += data.size();
+    }
+    return size;
+  }
+
+ private:
+  size_t available_;
+  size_t slice_index_;
+  size_t slice_offset_;
+  const vector<Slice>& slices_;
+};
+
+class SnappyCodec : public CompressionCodec {
+ public:
+  static SnappyCodec *GetSingleton() {
+    return Singleton<SnappyCodec>::get();
+  }
+
+  Status Compress(const Slice& input,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    snappy::RawCompress(reinterpret_cast<const char *>(input.data()), input.size(),
+                        reinterpret_cast<char *>(compressed), compressed_length);
+    return Status::OK();
+  }
+
+  Status Compress(const vector<Slice>& input_slices,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    SlicesSource source(input_slices);
+    snappy::UncheckedByteArraySink sink(reinterpret_cast<char *>(compressed));
+    if ((*compressed_length = snappy::Compress(&source, &sink)) <= 0) {
+      return Status::Corruption("unable to compress the buffer");
+    }
+    return Status::OK();
+  }
+
+  Status Uncompress(const Slice& compressed,
+                    uint8_t *uncompressed,
+                    size_t uncompressed_length) const OVERRIDE {
+    bool success = snappy::RawUncompress(reinterpret_cast<const char *>(compressed.data()),
+                                         compressed.size(), reinterpret_cast<char *>(uncompressed));
+    return success ? Status::OK() : Status::Corruption("unable to uncompress the buffer");
+  }
+
+  size_t MaxCompressedLength(size_t source_bytes) const OVERRIDE {
+    return snappy::MaxCompressedLength(source_bytes);
+  }
+
+  CompressionType type() const override {
+    return SNAPPY;
+  }
+};
+
+class Lz4Codec : public CompressionCodec {
+ public:
+  static Lz4Codec *GetSingleton() {
+    return Singleton<Lz4Codec>::get();
+  }
+
+  Status Compress(const Slice& input,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    int n = LZ4_compress(reinterpret_cast<const char *>(input.data()),
+                         reinterpret_cast<char *>(compressed), input.size());
+    *compressed_length = n;
+    return Status::OK();
+  }
+
+  Status Compress(const vector<Slice>& input_slices,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    if (input_slices.size() == 1) {
+      return Compress(input_slices[0], compressed, compressed_length);
+    }
+
+    SlicesSource source(input_slices);
+    faststring buffer;
+    source.Dump(&buffer);
+    return Compress(Slice(buffer.data(), buffer.size()), compressed, compressed_length);
+  }
+
+  Status Uncompress(const Slice& compressed,
+                    uint8_t *uncompressed,
+                    size_t uncompressed_length) const OVERRIDE {
+    int n = LZ4_decompress_fast(reinterpret_cast<const char *>(compressed.data()),
+                                reinterpret_cast<char *>(uncompressed), uncompressed_length);
+    if (n != compressed.size()) {
+      return Status::Corruption(
+        StringPrintf("unable to uncompress the buffer. error near %d, buffer", -n),
+                     KUDU_REDACT(compressed.ToDebugString(100)));
+    }
+    return Status::OK();
+  }
+
+  size_t MaxCompressedLength(size_t source_bytes) const OVERRIDE {
+    return LZ4_compressBound(source_bytes);
+  }
+
+  CompressionType type() const override {
+    return LZ4;
+  }
+};
+
+/**
+ * TODO: use a instance-local Arena and pass alloc/free into zlib
+ * so that it allocates from the arena.
+ */
+class ZlibCodec : public CompressionCodec {
+ public:
+  static ZlibCodec *GetSingleton() {
+    return Singleton<ZlibCodec>::get();
+  }
+
+  Status Compress(const Slice& input,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    *compressed_length = MaxCompressedLength(input.size());
+    int err = ::compress(compressed, compressed_length, input.data(), input.size());
+    return err == Z_OK ? Status::OK() : Status::IOError("unable to compress the buffer");
+  }
+
+  Status Compress(const vector<Slice>& input_slices,
+                  uint8_t *compressed, size_t *compressed_length) const OVERRIDE {
+    if (input_slices.size() == 1) {
+      return Compress(input_slices[0], compressed, compressed_length);
+    }
+
+    // TODO: use z_stream
+    SlicesSource source(input_slices);
+    faststring buffer;
+    source.Dump(&buffer);
+    return Compress(Slice(buffer.data(), buffer.size()), compressed, compressed_length);
+  }
+
+  Status Uncompress(const Slice& compressed,
+                    uint8_t *uncompressed, size_t uncompressed_length) const OVERRIDE {
+    int err = ::uncompress(uncompressed, &uncompressed_length,
+                           compressed.data(), compressed.size());
+    return err == Z_OK ? Status::OK() : Status::Corruption("unable to uncompress the buffer");
+  }
+
+  size_t MaxCompressedLength(size_t source_bytes) const OVERRIDE {
+    // one-time overhead of six bytes for the entire stream plus five bytes per 16 KB block
+    return source_bytes + (6 + (5 * ((source_bytes + 16383) >> 14)));
+  }
+
+  CompressionType type() const override {
+    return ZLIB;
+  }
+};
+
+Status GetCompressionCodec(CompressionType compression,
+                           const CompressionCodec** codec) {
+  switch (compression) {
+    case NO_COMPRESSION:
+      *codec = nullptr;
+      break;
+    case SNAPPY:
+      *codec = SnappyCodec::GetSingleton();
+      break;
+    case LZ4:
+      *codec = Lz4Codec::GetSingleton();
+      break;
+    case ZLIB:
+      *codec = ZlibCodec::GetSingleton();
+      break;
+    default:
+      return Status::NotFound("bad compression type");
+  }
+  return Status::OK();
+}
+
+CompressionType GetCompressionCodecType(const std::string& name) {
+  std::string uname;
+  ToUpperCase(name, &uname);
+
+  if (uname == "SNAPPY")
+    return SNAPPY;
+  if (uname == "LZ4")
+    return LZ4;
+  if (uname == "ZLIB")
+    return ZLIB;
+  if (uname == "NONE")
+    return NO_COMPRESSION;
+
+  LOG(WARNING) << "Unable to recognize the compression codec '" << name
+               << "' using no compression as default.";
+  return NO_COMPRESSION;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/compression/compression_codec.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/compression/compression_codec.h b/be/src/kudu/util/compression/compression_codec.h
new file mode 100644
index 0000000..4f81fd3
--- /dev/null
+++ b/be/src/kudu/util/compression/compression_codec.h
@@ -0,0 +1,78 @@
+// 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.
+#ifndef KUDU_CFILE_COMPRESSION_CODEC_H
+#define KUDU_CFILE_COMPRESSION_CODEC_H
+
+#include <cstddef>
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include <snappy-stubs-public.h>
+
+#include "kudu/util/compression/compression.pb.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class CompressionCodec {
+ public:
+  CompressionCodec();
+  virtual ~CompressionCodec();
+
+  // REQUIRES: "compressed" must point to an area of memory that is at
+  // least "MaxCompressedLength(input_length)" bytes in length.
+  //
+  // Takes the data stored in "input[0..input_length]" and stores
+  // it in the array pointed to by "compressed".
+  //
+  // returns the length of the compressed output.
+  virtual Status Compress(const Slice& input,
+                          uint8_t *compressed, size_t *compressed_length) const = 0;
+
+  virtual Status Compress(const std::vector<Slice>& input_slices,
+                          uint8_t *compressed, size_t *compressed_length) const = 0;
+
+  // Given data in "compressed[0..compressed_length-1]" generated by
+  // calling the Compress routine, this routine stores the uncompressed data
+  // to uncompressed[0..uncompressed_length-1]
+  // returns false if the message is corrupted and could not be uncompressed
+  virtual Status Uncompress(const Slice& compressed,
+                            uint8_t *uncompressed, size_t uncompressed_length) const = 0;
+
+  // Returns the maximal size of the compressed representation of
+  // input data that is "source_bytes" bytes in length.
+  virtual size_t MaxCompressedLength(size_t source_bytes) const = 0;
+
+  // Return the type of compression implemented by this codec.
+  virtual CompressionType type() const = 0;
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CompressionCodec);
+};
+
+// Returns the compression codec for the specified type.
+//
+// The returned codec is a singleton and should be not be destroyed.
+Status GetCompressionCodec(CompressionType compression,
+                           const CompressionCodec** codec);
+
+// Returns the compression codec type given the name
+CompressionType GetCompressionCodecType(const std::string& name);
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/condition_variable.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/condition_variable.cc b/be/src/kudu/util/condition_variable.cc
new file mode 100644
index 0000000..369d20d
--- /dev/null
+++ b/be/src/kudu/util/condition_variable.cc
@@ -0,0 +1,142 @@
+// Copyright (c) 2011 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "kudu/util/condition_variable.h"
+
+#include <sys/time.h>
+
+#include <cerrno>
+#include <cstdint>
+#include <ctime>
+#include <ostream>
+
+#include <glog/logging.h>
+
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/thread_restrictions.h"
+
+namespace kudu {
+
+ConditionVariable::ConditionVariable(Mutex* user_lock)
+    : user_mutex_(&user_lock->native_handle_)
+#if !defined(NDEBUG)
+    , user_lock_(user_lock)
+#endif
+{
+  int rv = 0;
+#if defined(__APPLE__)
+  rv = pthread_cond_init(&condition_, nullptr);
+#else
+  // On Linux we can't use relative times like on macOS; reconfiguring the
+  // condition variable to use the monotonic clock means we can use support
+  // WaitFor with our MonoTime implementation.
+  pthread_condattr_t attrs;
+  rv = pthread_condattr_init(&attrs);
+  DCHECK_EQ(0, rv);
+  pthread_condattr_setclock(&attrs, CLOCK_MONOTONIC);
+  rv = pthread_cond_init(&condition_, &attrs);
+  pthread_condattr_destroy(&attrs);
+#endif
+  DCHECK_EQ(0, rv);
+}
+
+ConditionVariable::~ConditionVariable() {
+  int rv = pthread_cond_destroy(&condition_);
+  DCHECK_EQ(0, rv);
+}
+
+void ConditionVariable::Wait() const {
+  ThreadRestrictions::AssertWaitAllowed();
+#if !defined(NDEBUG)
+  user_lock_->CheckHeldAndUnmark();
+#endif
+  int rv = pthread_cond_wait(&condition_, user_mutex_);
+  DCHECK_EQ(0, rv);
+#if !defined(NDEBUG)
+  user_lock_->CheckUnheldAndMark();
+#endif
+}
+
+bool ConditionVariable::WaitUntil(const MonoTime& until) const {
+  ThreadRestrictions::AssertWaitAllowed();
+
+  // Have we already timed out?
+  MonoTime now = MonoTime::Now();
+  if (now > until) {
+    return false;
+  }
+
+#if !defined(NDEBUG)
+  user_lock_->CheckHeldAndUnmark();
+#endif
+
+#if defined(__APPLE__)
+  // macOS does not provide a way to configure pthread_cond_timedwait() to use
+  // monotonic clocks, so we must convert the deadline into a delta and perform
+  // a relative wait.
+  MonoDelta delta = until - now;
+  struct timespec relative_time;
+  delta.ToTimeSpec(&relative_time);
+  int rv = pthread_cond_timedwait_relative_np(
+      &condition_, user_mutex_, &relative_time);
+#else
+  struct timespec absolute_time;
+  until.ToTimeSpec(&absolute_time);
+  int rv = pthread_cond_timedwait(&condition_, user_mutex_, &absolute_time);
+#endif
+  DCHECK(rv == 0 || rv == ETIMEDOUT)
+    << "unexpected pthread_cond_timedwait return value: " << rv;
+
+#if !defined(NDEBUG)
+  user_lock_->CheckUnheldAndMark();
+#endif
+  return rv == 0;
+}
+
+bool ConditionVariable::WaitFor(const MonoDelta& delta) const {
+  ThreadRestrictions::AssertWaitAllowed();
+
+  // Negative delta means we've already timed out.
+  int64_t nsecs = delta.ToNanoseconds();
+  if (nsecs < 0) {
+    return false;
+  }
+
+#if !defined(NDEBUG)
+  user_lock_->CheckHeldAndUnmark();
+#endif
+
+#if defined(__APPLE__)
+  struct timespec relative_time;
+  delta.ToTimeSpec(&relative_time);
+  int rv = pthread_cond_timedwait_relative_np(
+      &condition_, user_mutex_, &relative_time);
+#else
+  // The timeout argument to pthread_cond_timedwait is in absolute time.
+  struct timespec absolute_time;
+  MonoTime deadline = MonoTime::Now() + delta;
+  deadline.ToTimeSpec(&absolute_time);
+  int rv = pthread_cond_timedwait(&condition_, user_mutex_, &absolute_time);
+#endif
+
+  DCHECK(rv == 0 || rv == ETIMEDOUT)
+    << "unexpected pthread_cond_timedwait return value: " << rv;
+#if !defined(NDEBUG)
+  user_lock_->CheckUnheldAndMark();
+#endif
+  return rv == 0;
+}
+
+void ConditionVariable::Broadcast() {
+  int rv = pthread_cond_broadcast(&condition_);
+  DCHECK_EQ(0, rv);
+}
+
+void ConditionVariable::Signal() {
+  int rv = pthread_cond_signal(&condition_);
+  DCHECK_EQ(0, rv);
+}
+
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/condition_variable.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/condition_variable.h b/be/src/kudu/util/condition_variable.h
new file mode 100644
index 0000000..1245646
--- /dev/null
+++ b/be/src/kudu/util/condition_variable.h
@@ -0,0 +1,118 @@
+// Copyright (c) 2011 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+// ConditionVariable wraps pthreads condition variable synchronization or, on
+// Windows, simulates it.  This functionality is very helpful for having
+// several threads wait for an event, as is common with a thread pool managed
+// by a master.  The meaning of such an event in the (worker) thread pool
+// scenario is that additional tasks are now available for processing.  It is
+// used in Chrome in the DNS prefetching system to notify worker threads that
+// a queue now has items (tasks) which need to be tended to.  A related use
+// would have a pool manager waiting on a ConditionVariable, waiting for a
+// thread in the pool to announce (signal) that there is now more room in a
+// (bounded size) communications queue for the manager to deposit tasks, or,
+// as a second example, that the queue of tasks is completely empty and all
+// workers are waiting.
+//
+// USAGE NOTE 1: spurious signal events are possible with this and
+// most implementations of condition variables.  As a result, be
+// *sure* to retest your condition before proceeding.  The following
+// is a good example of doing this correctly:
+//
+// while (!work_to_be_done()) Wait(...);
+//
+// In contrast do NOT do the following:
+//
+// if (!work_to_be_done()) Wait(...);  // Don't do this.
+//
+// Especially avoid the above if you are relying on some other thread only
+// issuing a signal up *if* there is work-to-do.  There can/will
+// be spurious signals.  Recheck state on waiting thread before
+// assuming the signal was intentional. Caveat caller ;-).
+//
+// USAGE NOTE 2: Broadcast() frees up all waiting threads at once,
+// which leads to contention for the locks they all held when they
+// called Wait().  This results in POOR performance.  A much better
+// approach to getting a lot of threads out of Wait() is to have each
+// thread (upon exiting Wait()) call Signal() to free up another
+// Wait'ing thread.  Look at condition_variable_unittest.cc for
+// both examples.
+//
+// Broadcast() can be used nicely during teardown, as it gets the job
+// done, and leaves no sleeping threads... and performance is less
+// critical at that point.
+//
+// The semantics of Broadcast() are carefully crafted so that *all*
+// threads that were waiting when the request was made will indeed
+// get signaled.  Some implementations mess up, and don't signal them
+// all, while others allow the wait to be effectively turned off (for
+// a while while waiting threads come around).  This implementation
+// appears correct, as it will not "lose" any signals, and will guarantee
+// that all threads get signaled by Broadcast().
+//
+// This implementation offers support for "performance" in its selection of
+// which thread to revive.  Performance, in direct contrast with "fairness,"
+// assures that the thread that most recently began to Wait() is selected by
+// Signal to revive.  Fairness would (if publicly supported) assure that the
+// thread that has Wait()ed the longest is selected. The default policy
+// may improve performance, as the selected thread may have a greater chance of
+// having some of its stack data in various CPU caches.
+//
+// For a discussion of the many very subtle implementation details, see the FAQ
+// at the end of condition_variable_win.cc.
+
+#ifndef BASE_SYNCHRONIZATION_CONDITION_VARIABLE_H_
+#define BASE_SYNCHRONIZATION_CONDITION_VARIABLE_H_
+
+#include <pthread.h>
+
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+class MonoDelta;
+class MonoTime;
+class Mutex;
+
+class ConditionVariable {
+ public:
+  // Construct a cv for use with ONLY one user lock.
+  explicit ConditionVariable(Mutex* user_lock);
+
+  ~ConditionVariable();
+
+  // Wait() releases the caller's critical section atomically as it starts to
+  // sleep, and the reacquires it when it is signaled.
+  void Wait() const;
+
+  // Like Wait(), but only waits up to a certain point in time.
+  //
+  // Returns true if we were Signal()'ed, or false if we reached 'until'.
+  bool WaitUntil(const MonoTime& until) const;
+
+  // Like Wait(), but only waits up to a limited amount of time.
+  //
+  // Returns true if we were Signal()'ed, or false if 'delta' elapsed.
+  bool WaitFor(const MonoDelta& delta) const;
+
+  // Broadcast() revives all waiting threads.
+  void Broadcast();
+  // Signal() revives one waiting thread.
+  void Signal();
+
+ private:
+
+  mutable pthread_cond_t condition_;
+  pthread_mutex_t* user_mutex_;
+
+#if !defined(NDEBUG)
+  Mutex* user_lock_;     // Needed to adjust shadow lock state on wait.
+#endif
+
+  DISALLOW_COPY_AND_ASSIGN(ConditionVariable);
+};
+
+}  // namespace kudu
+
+#endif  // BASE_SYNCHRONIZATION_CONDITION_VARIABLE_H_

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/countdown_latch-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/countdown_latch-test.cc b/be/src/kudu/util/countdown_latch-test.cc
new file mode 100644
index 0000000..adb2623
--- /dev/null
+++ b/be/src/kudu/util/countdown_latch-test.cc
@@ -0,0 +1,74 @@
+// 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 <boost/bind.hpp> // IWYU pragma: keep
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadpool.h"
+
+namespace kudu {
+
+static void DecrementLatch(CountDownLatch* latch, int amount) {
+  if (amount == 1) {
+    latch->CountDown();
+    return;
+  }
+  latch->CountDown(amount);
+}
+
+// Tests that we can decrement the latch by arbitrary amounts, as well
+// as 1 by one.
+TEST(TestCountDownLatch, TestLatch) {
+
+  gscoped_ptr<ThreadPool> pool;
+  ASSERT_OK(ThreadPoolBuilder("cdl-test").set_max_threads(1).Build(&pool));
+
+  CountDownLatch latch(1000);
+
+  // Decrement the count by 1 in another thread, this should not fire the
+  // latch.
+  ASSERT_OK(pool->SubmitFunc(boost::bind(DecrementLatch, &latch, 1)));
+  ASSERT_FALSE(latch.WaitFor(MonoDelta::FromMilliseconds(200)));
+  ASSERT_EQ(999, latch.count());
+
+  // Now decrement by 1000 this should decrement to 0 and fire the latch
+  // (even though 1000 is one more than the current count).
+  ASSERT_OK(pool->SubmitFunc(boost::bind(DecrementLatch, &latch, 1000)));
+  latch.Wait();
+  ASSERT_EQ(0, latch.count());
+}
+
+// Test that resetting to zero while there are waiters lets the waiters
+// continue.
+TEST(TestCountDownLatch, TestResetToZero) {
+  CountDownLatch cdl(100);
+  scoped_refptr<Thread> t;
+  ASSERT_OK(Thread::Create("test", "cdl-test", &CountDownLatch::Wait, &cdl, &t));
+
+  // Sleep for a bit until it's likely the other thread is waiting on the latch.
+  SleepFor(MonoDelta::FromMilliseconds(10));
+  cdl.Reset(0);
+  t->Join();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/countdown_latch.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/countdown_latch.h b/be/src/kudu/util/countdown_latch.h
new file mode 100644
index 0000000..9a8000d
--- /dev/null
+++ b/be/src/kudu/util/countdown_latch.h
@@ -0,0 +1,137 @@
+// 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.
+#ifndef KUDU_UTIL_COUNTDOWN_LATCH_H
+#define KUDU_UTIL_COUNTDOWN_LATCH_H
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/thread_restrictions.h"
+
+namespace kudu {
+
+// This is a C++ implementation of the Java CountDownLatch
+// class.
+// See http://docs.oracle.com/javase/6/docs/api/java/util/concurrent/CountDownLatch.html
+class CountDownLatch {
+ public:
+  // Initialize the latch with the given initial count.
+  explicit CountDownLatch(int count)
+    : cond_(&lock_),
+      count_(count) {
+  }
+
+  // Decrement the count of this latch by 'amount'
+  // If the new count is less than or equal to zero, then all waiting threads are woken up.
+  // If the count is already zero, this has no effect.
+  void CountDown(int amount) {
+    DCHECK_GE(amount, 0);
+    MutexLock lock(lock_);
+    if (count_ == 0) {
+      return;
+    }
+
+    if (amount >= count_) {
+      count_ = 0;
+    } else {
+      count_ -= amount;
+    }
+
+    if (count_ == 0) {
+      // Latch has triggered.
+      cond_.Broadcast();
+    }
+  }
+
+  // Decrement the count of this latch.
+  // If the new count is zero, then all waiting threads are woken up.
+  // If the count is already zero, this has no effect.
+  void CountDown() {
+    CountDown(1);
+  }
+
+  // Wait until the count on the latch reaches zero.
+  // If the count is already zero, this returns immediately.
+  void Wait() const {
+    ThreadRestrictions::AssertWaitAllowed();
+    MutexLock lock(lock_);
+    while (count_ > 0) {
+      cond_.Wait();
+    }
+  }
+
+  // Waits for the count on the latch to reach zero, or until 'until' time is reached.
+  // Returns true if the count became zero, false otherwise.
+  bool WaitUntil(const MonoTime& when) const {
+    ThreadRestrictions::AssertWaitAllowed();
+    MutexLock lock(lock_);
+    while (count_ > 0) {
+      if (!cond_.WaitUntil(when)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  // Waits for the count on the latch to reach zero, or until 'delta' time elapses.
+  // Returns true if the count became zero, false otherwise.
+  bool WaitFor(const MonoDelta& delta) const {
+    return WaitUntil(MonoTime::Now() + delta);
+  }
+
+  // Reset the latch with the given count. This is equivalent to reconstructing
+  // the latch. If 'count' is 0, and there are currently waiters, those waiters
+  // will be triggered as if you counted down to 0.
+  void Reset(uint64_t count) {
+    MutexLock lock(lock_);
+    count_ = count;
+    if (count_ == 0) {
+      // Awake any waiters if we reset to 0.
+      cond_.Broadcast();
+    }
+  }
+
+  uint64_t count() const {
+    MutexLock lock(lock_);
+    return count_;
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CountDownLatch);
+  mutable Mutex lock_;
+  ConditionVariable cond_;
+
+  uint64_t count_;
+};
+
+// Utility class which calls latch->CountDown() in its destructor.
+class CountDownOnScopeExit {
+ public:
+  explicit CountDownOnScopeExit(CountDownLatch *latch) : latch_(latch) {}
+  ~CountDownOnScopeExit() {
+    latch_->CountDown();
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CountDownOnScopeExit);
+
+  CountDownLatch *latch_;
+};
+
+} // namespace kudu
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/util/cow_object.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/util/cow_object.cc b/be/src/kudu/util/cow_object.cc
new file mode 100644
index 0000000..a22393c
--- /dev/null
+++ b/be/src/kudu/util/cow_object.cc
@@ -0,0 +1,34 @@
+// 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 "kudu/util/cow_object.h"
+
+using std::ostream;
+
+namespace kudu {
+
+ostream& operator<<(ostream& o, LockMode m) {
+  switch (m) {
+    case LockMode::READ: o << "READ"; break;
+    case LockMode::WRITE: o << "WRITE"; break;
+    case LockMode::RELEASED: o << "RELEASED"; break;
+    default: o << "UNKNOWN"; break;
+  }
+  return o;
+}
+
+} // namespace kudu


[31/51] [abbrv] impala git commit: IMPALA-7006: Add KRPC folders from kudu@334ecafd

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/tls_socket.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/tls_socket.h b/be/src/kudu/security/tls_socket.h
new file mode 100644
index 0000000..a0d716c
--- /dev/null
+++ b/be/src/kudu/security/tls_socket.h
@@ -0,0 +1,60 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+
+#include "kudu/gutil/port.h"
+#include "kudu/security/openssl_util.h" // IWYU pragma: keep
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+struct iovec;
+typedef struct ssl_st SSL;
+
+namespace kudu {
+namespace security {
+
+class TlsSocket : public Socket {
+ public:
+
+  ~TlsSocket() override;
+
+  Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) override WARN_UNUSED_RESULT;
+
+  Status Writev(const struct ::iovec *iov,
+                int iov_len,
+                int64_t *nwritten) override WARN_UNUSED_RESULT;
+
+  Status Recv(uint8_t *buf, int32_t amt, int32_t *nread) override WARN_UNUSED_RESULT;
+
+  Status Close() override WARN_UNUSED_RESULT;
+
+ private:
+
+  friend class TlsHandshake;
+
+  TlsSocket(int fd, c_unique_ptr<SSL> ssl);
+
+  // Owned SSL handle.
+  c_unique_ptr<SSL> ssl_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token-test.cc b/be/src/kudu/security/token-test.cc
new file mode 100644
index 0000000..7172a51
--- /dev/null
+++ b/be/src/kudu/security/token-test.cc
@@ -0,0 +1,677 @@
+// 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 <cstdint>
+#include <deque>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/walltime.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_signer.h"
+#include "kudu/security/token_signing_key.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_int32(tsk_num_rsa_bits);
+
+using std::string;
+using std::make_shared;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+SignedTokenPB MakeUnsignedToken(int64_t expiration) {
+  SignedTokenPB ret;
+  TokenPB token;
+  token.set_expire_unix_epoch_seconds(expiration);
+  CHECK(token.SerializeToString(ret.mutable_token_data()));
+  return ret;
+}
+
+SignedTokenPB MakeIncompatibleToken() {
+  SignedTokenPB ret;
+  TokenPB token;
+  token.set_expire_unix_epoch_seconds(WallTime_Now() + 100);
+  token.add_incompatible_features(TokenPB::Feature_MAX + 1);
+  CHECK(token.SerializeToString(ret.mutable_token_data()));
+  return ret;
+}
+
+// Generate public key as a string in DER format for tests.
+Status GeneratePublicKeyStrDer(string* ret) {
+  PrivateKey private_key;
+  RETURN_NOT_OK(GeneratePrivateKey(512, &private_key));
+  PublicKey public_key;
+  RETURN_NOT_OK(private_key.GetPublicKey(&public_key));
+  string public_key_str_der;
+  RETURN_NOT_OK(public_key.ToString(&public_key_str_der, DataFormat::DER));
+  *ret = public_key_str_der;
+  return Status::OK();
+}
+
+// Generate token signing key with the specified parameters.
+Status GenerateTokenSigningKey(int64_t seq_num,
+                               int64_t expire_time_seconds,
+                               unique_ptr<TokenSigningPrivateKey>* tsk) {
+  {
+    unique_ptr<PrivateKey> private_key(new PrivateKey);
+    RETURN_NOT_OK(GeneratePrivateKey(512, private_key.get()));
+    tsk->reset(new TokenSigningPrivateKey(
+        seq_num, expire_time_seconds, std::move(private_key)));
+  }
+  return Status::OK();
+}
+
+void CheckAndAddNextKey(int iter_num,
+                        TokenSigner* signer,
+                        int64_t* key_seq_num) {
+  ASSERT_NE(nullptr, signer);
+  ASSERT_NE(nullptr, key_seq_num);
+  int64_t seq_num;
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer->CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    seq_num = key->key_seq_num();
+  }
+
+  for (int i = 0; i < iter_num; ++i) {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer->CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_EQ(seq_num, key->key_seq_num());
+    if (i + 1 == iter_num) {
+      // Finally, add the key to the TokenSigner.
+      ASSERT_OK(signer->AddKey(std::move(key)));
+    }
+  }
+  *key_seq_num = seq_num;
+}
+
+} // anonymous namespace
+
+class TokenTest : public KuduTest {
+};
+
+TEST_F(TokenTest, TestInit) {
+  TokenSigner signer(10, 10);
+  const TokenVerifier& verifier(signer.verifier());
+
+  SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+  Status s = signer.SignToken(&token);
+  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+
+  static const int64_t kKeySeqNum = 100;
+  PrivateKey private_key;
+  ASSERT_OK(GeneratePrivateKey(512, &private_key));
+  string private_key_str_der;
+  ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+  TokenSigningPrivateKeyPB pb;
+  pb.set_rsa_key_der(private_key_str_der);
+  pb.set_key_seq_num(kKeySeqNum);
+  pb.set_expire_unix_epoch_seconds(WallTime_Now() + 120);
+
+  ASSERT_OK(signer.ImportKeys({pb}));
+  vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
+  ASSERT_EQ(1, public_keys.size());
+  ASSERT_EQ(kKeySeqNum, public_keys[0].key_seq_num());
+
+  // It should be possible to sign tokens once the signer is initialized.
+  ASSERT_OK(signer.SignToken(&token));
+  ASSERT_TRUE(token.has_signature());
+}
+
+// Verify that TokenSigner does not allow 'holes' in the sequence numbers
+// of the generated keys. The idea is to not allow sequences like '1, 5, 6'.
+// In general, calling the CheckNeedKey() method multiple times and then calling
+// the AddKey() method once should advance the key sequence number only by 1
+// regardless of number CheckNeedKey() calls.
+//
+// This is to make sure that the sequence numbers are not sparse in case if
+// running scenarios CheckNeedKey()-try-to-store-key-AddKey() over and over
+// again, given that the 'try-to-store-key' part can fail sometimes.
+TEST_F(TokenTest, TestTokenSignerNonSparseSequenceNumbers) {
+  static const int kIterNum = 3;
+  static const int64_t kAuthnTokenValiditySeconds = 1;
+  static const int64_t kKeyRotationSeconds = 1;
+
+  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+
+  int64_t seq_num_first_key;
+  NO_FATALS(CheckAndAddNextKey(kIterNum, &signer, &seq_num_first_key));
+
+  SleepFor(MonoDelta::FromSeconds(kKeyRotationSeconds + 1));
+
+  int64_t seq_num_second_key;
+  NO_FATALS(CheckAndAddNextKey(kIterNum, &signer, &seq_num_second_key));
+
+  ASSERT_EQ(seq_num_first_key + 1, seq_num_second_key);
+}
+
+// Verify the behavior of the TokenSigner::ImportKeys() method. In general,
+// it should tolerate mix of expired and non-expired keys, even if their
+// sequence numbers are intermixed: keys with greater sequence numbers could
+// be already expired but keys with lesser sequence numbers could be still
+// valid. The idea is to correctly import TSKs generated with different
+// validity period settings. This is to address scenarios when the system
+// was run with long authn token validity interval and then switched to
+// a shorter one.
+//
+// After importing keys, the TokenSigner should contain only the valid ones.
+// In addition, the sequence number of the very first key generated after the
+// import should be greater than any sequence number the TokenSigner has seen
+// during the import.
+TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
+  static const int64_t kAuthnTokenValiditySeconds = 8;
+  static const int64_t kKeyRotationSeconds = 8;
+  static const int64_t kKeyValiditySeconds =
+      kAuthnTokenValiditySeconds + 2 * kKeyRotationSeconds;
+
+  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+  const TokenVerifier& verifier(signer.verifier());
+
+  static const int64_t kExpiredKeySeqNum = 100;
+  static const int64_t kKeySeqNum = kExpiredKeySeqNum - 1;
+  {
+    // First, try to import already expired key to check that internal key
+    // sequence number advances correspondingly.
+    PrivateKey private_key;
+    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kExpiredKeySeqNum);
+    pb.set_expire_unix_epoch_seconds(WallTime_Now() - 1);
+
+    ASSERT_OK(signer.ImportKeys({pb}));
+  }
+
+  {
+    // Check the result of importing keys: there should be no keys because
+    // the only one we tried to import was already expired.
+    vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
+    ASSERT_TRUE(public_keys.empty());
+  }
+
+  {
+    // Now import valid (not yet expired) key, but with sequence number less
+    // than of the expired key.
+    PrivateKey private_key;
+    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kKeySeqNum);
+    // Set the TSK's expiration time: make the key valid but past its activity
+    // interval.
+    pb.set_expire_unix_epoch_seconds(
+        WallTime_Now() + (kKeyValiditySeconds - 2 * kKeyRotationSeconds - 1));
+
+    ASSERT_OK(signer.ImportKeys({pb}));
+  }
+
+  {
+    // Check the result of importing keys.
+    vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
+    ASSERT_EQ(1, public_keys.size());
+    ASSERT_EQ(kKeySeqNum, public_keys[0].key_seq_num());
+  }
+
+  {
+    // The newly imported key should be used to sign tokens.
+    SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+    ASSERT_OK(signer.SignToken(&token));
+    ASSERT_TRUE(token.has_signature());
+    ASSERT_TRUE(token.has_signing_key_seq_num());
+    EXPECT_EQ(kKeySeqNum, token.signing_key_seq_num());
+  }
+
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_EQ(kExpiredKeySeqNum + 1, key->key_seq_num());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+    bool has_rotated = false;
+    ASSERT_OK(signer.TryRotateKey(&has_rotated));
+    ASSERT_TRUE(has_rotated);
+  }
+  {
+    // Check the result of generating the new key: the identifier of the new key
+    // should be +1 increment from the identifier of the expired imported key.
+    vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
+    ASSERT_EQ(2, public_keys.size());
+    EXPECT_EQ(kKeySeqNum, public_keys[0].key_seq_num());
+    EXPECT_EQ(kExpiredKeySeqNum + 1, public_keys[1].key_seq_num());
+  }
+
+  // At this point the new key should be used to sign tokens.
+  SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+  ASSERT_OK(signer.SignToken(&token));
+  ASSERT_TRUE(token.has_signature());
+  ASSERT_TRUE(token.has_signing_key_seq_num());
+  EXPECT_EQ(kExpiredKeySeqNum + 1, token.signing_key_seq_num());
+}
+
+// The AddKey() method should not allow to add a key with the sequence number
+// less or equal to the sequence number of the most 'recent' key.
+TEST_F(TokenTest, TestAddKeyConstraints) {
+  {
+    TokenSigner signer(1, 1);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+  {
+    TokenSigner signer(1, 1);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    const int64_t key_seq_num = key->key_seq_num();
+    key->key_seq_num_ = key_seq_num - 1;
+    Status s = signer.AddKey(std::move(key));
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        ": invalid key sequence number, should be at least ");
+  }
+  {
+    TokenSigner signer(1, 1);
+    static const int64_t kKeySeqNum = 100;
+    PrivateKey private_key;
+    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kKeySeqNum);
+    // Make the key already expired.
+    pb.set_expire_unix_epoch_seconds(WallTime_Now() - 1);
+    ASSERT_OK(signer.ImportKeys({pb}));
+
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    const int64_t key_seq_num = key->key_seq_num();
+    ASSERT_GT(key_seq_num, kKeySeqNum);
+    key->key_seq_num_ = kKeySeqNum;
+    Status s = signer.AddKey(std::move(key));
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        ": invalid key sequence number, should be at least ");
+  }
+}
+
+TEST_F(TokenTest, TestGenerateAuthTokenNoUserName) {
+  TokenSigner signer(10, 10);
+  SignedTokenPB signed_token_pb;
+  const Status& s = signer.GenerateAuthnToken("", &signed_token_pb);
+  EXPECT_TRUE(s.IsInvalidArgument()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "no username provided for authn token");
+}
+
+TEST_F(TokenTest, TestIsCurrentKeyValid) {
+  static const int64_t kAuthnTokenValiditySeconds = 1;
+  static const int64_t kKeyRotationSeconds = 1;
+  static const int64_t kKeyValiditySeconds =
+      kAuthnTokenValiditySeconds + 2 * kKeyRotationSeconds;
+
+  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+  EXPECT_FALSE(signer.IsCurrentKeyValid());
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+  EXPECT_TRUE(signer.IsCurrentKeyValid());
+  SleepFor(MonoDelta::FromSeconds(kKeyValiditySeconds));
+  // The key should expire after its validity interval.
+  EXPECT_FALSE(signer.IsCurrentKeyValid());
+
+  // Anyway, current implementation allows to use an expired key to sign tokens.
+  SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+  EXPECT_OK(signer.SignToken(&token));
+}
+
+TEST_F(TokenTest, TestTokenSignerAddKeys) {
+  {
+    TokenSigner signer(10, 10);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    // It's not time to add next key yet.
+    ASSERT_EQ(nullptr, key.get());
+  }
+
+  {
+    // Special configuration for TokenSigner: rotation interval is zero,
+    // so should be able to add two keys right away.
+    TokenSigner signer(10, 0);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+
+    // Should be able to add next key right away.
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+
+    // Active key and next key are already in place: no need for a new key.
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_EQ(nullptr, key.get());
+  }
+
+  if (AllowSlowTests()) {
+    // Special configuration for TokenSigner: short interval for key rotation.
+    // It should not need next key right away, but should need next key after
+    // the rotation interval.
+    static const int64_t kKeyRotationIntervalSeconds = 8;
+    TokenSigner signer(10, kKeyRotationIntervalSeconds);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+
+    // Should not need next key right away.
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_EQ(nullptr, key.get());
+
+    SleepFor(MonoDelta::FromSeconds(kKeyRotationIntervalSeconds));
+
+    // Should need next key after the rotation interval.
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+
+    // Active key and next key are already in place: no need for a new key.
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_EQ(nullptr, key.get());
+  }
+}
+
+// Test how key rotation works.
+TEST_F(TokenTest, TestTokenSignerSignVerifyExport) {
+  // Key rotation interval 0 allows adding 2 keys in a row with no delay.
+  TokenSigner signer(10, 0);
+  const TokenVerifier& verifier(signer.verifier());
+
+  // Should start off with no signing keys.
+  ASSERT_TRUE(verifier.ExportKeys().empty());
+
+  // Trying to sign a token when there is no TSK should give an error.
+  SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+  Status s = signer.SignToken(&token);
+  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+
+  // Generate and set a new key.
+  int64_t signing_key_seq_num;
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    signing_key_seq_num = key->key_seq_num();
+    ASSERT_GT(signing_key_seq_num, -1);
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+
+  // We should see the key now if we request TSKs starting at a
+  // lower sequence number.
+  ASSERT_EQ(1, verifier.ExportKeys().size());
+  // We should not see the key if we ask for the sequence number
+  // that it is assigned.
+  ASSERT_EQ(0, verifier.ExportKeys(signing_key_seq_num).size());
+
+  // We should be able to sign a token now.
+  ASSERT_OK(signer.SignToken(&token));
+  ASSERT_TRUE(token.has_signature());
+  ASSERT_EQ(signing_key_seq_num, token.signing_key_seq_num());
+
+  // Set next key and check that we return the right keys.
+  int64_t next_signing_key_seq_num;
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    next_signing_key_seq_num = key->key_seq_num();
+    ASSERT_GT(next_signing_key_seq_num, signing_key_seq_num);
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+  ASSERT_EQ(2, verifier.ExportKeys().size());
+  ASSERT_EQ(1, verifier.ExportKeys(signing_key_seq_num).size());
+  ASSERT_EQ(0, verifier.ExportKeys(next_signing_key_seq_num).size());
+
+  // The first key should be used for signing: the next one is saved
+  // for the next round.
+  {
+    SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+    ASSERT_OK(signer.SignToken(&token));
+    ASSERT_TRUE(token.has_signature());
+    ASSERT_EQ(signing_key_seq_num, token.signing_key_seq_num());
+  }
+}
+
+// Test that the TokenSigner can export its public keys in protobuf form
+// via bound TokenVerifier.
+TEST_F(TokenTest, TestExportKeys) {
+  // Test that the exported public keys don't contain private key material,
+  // and have an appropriate expiration.
+  const int64_t key_exp_seconds = 30;
+  const int64_t key_rotation_seconds = 10;
+  TokenSigner signer(key_exp_seconds - 2 * key_rotation_seconds,
+                     key_rotation_seconds);
+  int64_t key_seq_num;
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    key_seq_num = key->key_seq_num();
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+  const TokenVerifier& verifier(signer.verifier());
+  auto keys = verifier.ExportKeys();
+  ASSERT_EQ(1, keys.size());
+  const TokenSigningPublicKeyPB& key = keys[0];
+  ASSERT_TRUE(key.has_rsa_key_der());
+  ASSERT_EQ(key_seq_num, key.key_seq_num());
+  ASSERT_TRUE(key.has_expire_unix_epoch_seconds());
+  const int64_t now = WallTime_Now();
+  ASSERT_GT(key.expire_unix_epoch_seconds(), now);
+  ASSERT_LE(key.expire_unix_epoch_seconds(), now + key_exp_seconds);
+}
+
+// Test that the TokenVerifier can import keys exported by the TokenSigner
+// and then verify tokens signed by it.
+TEST_F(TokenTest, TestEndToEnd_Valid) {
+  TokenSigner signer(10, 10);
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+
+  // Make and sign a token.
+  SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() + 600);
+  ASSERT_OK(signer.SignToken(&signed_token));
+
+  // Try to verify it.
+  TokenVerifier verifier;
+  ASSERT_OK(verifier.ImportKeys(signer.verifier().ExportKeys()));
+  TokenPB token;
+  ASSERT_EQ(VerificationResult::VALID, verifier.VerifyTokenSignature(signed_token, &token));
+}
+
+// Test all of the possible cases covered by token verification.
+// See VerificationResult.
+TEST_F(TokenTest, TestEndToEnd_InvalidCases) {
+  // Key rotation interval 0 allows adding 2 keys in a row with no delay.
+  TokenSigner signer(10, 0);
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+  }
+
+  TokenVerifier verifier;
+  ASSERT_OK(verifier.ImportKeys(signer.verifier().ExportKeys()));
+
+  // Make and sign a token, but corrupt the data in it.
+  {
+    SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() + 600);
+    ASSERT_OK(signer.SignToken(&signed_token));
+    signed_token.set_token_data("xyz");
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::INVALID_TOKEN,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+
+  // Make and sign a token, but corrupt the signature.
+  {
+    SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() + 600);
+    ASSERT_OK(signer.SignToken(&signed_token));
+    signed_token.set_signature("xyz");
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::INVALID_SIGNATURE,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+
+  // Make and sign a token, but set it to be already expired.
+  {
+    SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() - 10);
+    ASSERT_OK(signer.SignToken(&signed_token));
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::EXPIRED_TOKEN,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+
+  // Make and sign a token which uses an incompatible feature flag.
+  {
+    SignedTokenPB signed_token = MakeIncompatibleToken();
+    ASSERT_OK(signer.SignToken(&signed_token));
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::INCOMPATIBLE_FEATURE,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+
+  // Set a new signing key, but don't inform the verifier of it yet. When we
+  // verify, we expect the verifier to complain the key is unknown.
+  {
+    {
+      std::unique_ptr<TokenSigningPrivateKey> key;
+      ASSERT_OK(signer.CheckNeedKey(&key));
+      ASSERT_NE(nullptr, key.get());
+      ASSERT_OK(signer.AddKey(std::move(key)));
+      bool has_rotated = false;
+      ASSERT_OK(signer.TryRotateKey(&has_rotated));
+      ASSERT_TRUE(has_rotated);
+    }
+    SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() + 600);
+    ASSERT_OK(signer.SignToken(&signed_token));
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::UNKNOWN_SIGNING_KEY,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+
+  // Set a new signing key which is already expired, and inform the verifier
+  // of all of the current keys. The verifier should recognize the key but
+  // know that it's expired.
+  {
+    {
+      unique_ptr<TokenSigningPrivateKey> tsk;
+      ASSERT_OK(GenerateTokenSigningKey(100, WallTime_Now() - 1, &tsk));
+      // This direct access is necessary because AddKey() does not allow to add
+      // an expired key.
+      TokenSigningPublicKeyPB tsk_public_pb;
+      tsk->ExportPublicKeyPB(&tsk_public_pb);
+      ASSERT_OK(verifier.ImportKeys({tsk_public_pb}));
+      signer.tsk_deque_.push_front(std::move(tsk));
+    }
+
+    SignedTokenPB signed_token = MakeUnsignedToken(WallTime_Now() + 600);
+    // Current implementation allows to use an expired key to sign tokens.
+    ASSERT_OK(signer.SignToken(&signed_token));
+    TokenPB token;
+    ASSERT_EQ(VerificationResult::EXPIRED_SIGNING_KEY,
+              verifier.VerifyTokenSignature(signed_token, &token));
+  }
+}
+
+// Test functionality of the TokenVerifier::ImportKeys() method.
+TEST_F(TokenTest, TestTokenVerifierImportKeys) {
+  TokenVerifier verifier;
+
+  // An attempt to import no keys is fine.
+  ASSERT_OK(verifier.ImportKeys({}));
+  ASSERT_TRUE(verifier.ExportKeys().empty());
+
+  TokenSigningPublicKeyPB tsk_public_pb;
+  const auto exp_time = WallTime_Now() + 600;
+  tsk_public_pb.set_key_seq_num(100500);
+  tsk_public_pb.set_expire_unix_epoch_seconds(exp_time);
+  string public_key_str_der;
+  ASSERT_OK(GeneratePublicKeyStrDer(&public_key_str_der));
+  tsk_public_pb.set_rsa_key_der(public_key_str_der);
+
+  ASSERT_OK(verifier.ImportKeys({ tsk_public_pb }));
+  {
+    const auto& exported_tsks_public_pb = verifier.ExportKeys();
+    ASSERT_EQ(1, exported_tsks_public_pb.size());
+    EXPECT_EQ(tsk_public_pb.SerializeAsString(),
+              exported_tsks_public_pb[0].SerializeAsString());
+  }
+
+  // Re-importing the same key again is fine, and the total number
+  // of exported keys should not increase.
+  ASSERT_OK(verifier.ImportKeys({ tsk_public_pb }));
+  {
+    const auto& exported_tsks_public_pb = verifier.ExportKeys();
+    ASSERT_EQ(1, exported_tsks_public_pb.size());
+    EXPECT_EQ(tsk_public_pb.SerializeAsString(),
+              exported_tsks_public_pb[0].SerializeAsString());
+  }
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token.proto
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token.proto b/be/src/kudu/security/token.proto
new file mode 100644
index 0000000..e27ccdb
--- /dev/null
+++ b/be/src/kudu/security/token.proto
@@ -0,0 +1,97 @@
+// 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.
+syntax = "proto2";
+package kudu.security;
+
+option java_package = "org.apache.kudu.security";
+
+import "kudu/util/pb_util.proto";
+
+message AuthnTokenPB {
+  optional string username = 1;
+};
+
+message AuthzTokenPB {
+};
+
+message TokenPB {
+  // The time at which this token expires, in seconds since the
+  // unix epoch.
+  optional int64 expire_unix_epoch_seconds = 1;
+
+  enum Feature {
+    // Protobuf doesn't let us define a enum with no values,
+    // so we've got this placeholder in here for now. When we add
+    // the first real feature flag, we can remove this.
+    UNUSED_PLACEHOLDER = 999;
+  };
+
+  // List of incompatible features used by this token. If a feature
+  // is listed in the token and a server verifying/authorizing the token
+  // sees an UNKNOWN value in this list, it should reject the token.
+  //
+  // This allows us to safely add "restrictive" content to tokens
+  // and have a "default deny" policy on servers that may not understand
+  // them.
+  //
+  // We use an int32 here but the values correspond to the 'Feature' enum
+  // above. This is to deal with protobuf's odd handling of unknown enum
+  // values (see KUDU-1850).
+  repeated int32 incompatible_features = 2;
+
+  oneof token {
+    AuthnTokenPB authn = 3;
+    AuthzTokenPB authz = 4;
+  }
+};
+
+message SignedTokenPB {
+  // The actual token data. This is a serialized TokenPB protobuf. However, we use a
+  // 'bytes' field, since protobuf doesn't guarantee that if two implementations serialize
+  // a protobuf, they'll necessary get bytewise identical results, particularly in the
+  // presence of unknown fields.
+  optional bytes token_data = 1;
+
+  // The cryptographic signature of 'token_contents'.
+  optional bytes signature = 2 [ (kudu.REDACT) = true ];
+
+  // The sequence number of the key which produced 'signature'.
+  optional int64 signing_key_seq_num = 3;
+};
+
+// A private key used to sign tokens.
+message TokenSigningPrivateKeyPB {
+  optional int64 key_seq_num = 1;
+
+  // The private key material, in DER format.
+  optional bytes rsa_key_der = 2 [ (kudu.REDACT) = true ];
+
+  // The time at which signatures made by this key should no longer be valid.
+  optional int64 expire_unix_epoch_seconds = 3;
+};
+
+// A public key corresponding to the private key used to sign tokens. Only
+// this part is necessary for token verification.
+message TokenSigningPublicKeyPB {
+  optional int64 key_seq_num = 1;
+
+  // The public key material, in DER format.
+  optional bytes rsa_key_der = 2;
+
+  // The time at which signatures made by this key should no longer be valid.
+  optional int64 expire_unix_epoch_seconds = 3;
+};

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_signer.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_signer.cc b/be/src/kudu/security/token_signer.cc
new file mode 100644
index 0000000..08c84be
--- /dev/null
+++ b/be/src/kudu/security/token_signer.cc
@@ -0,0 +1,299 @@
+// 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 "kudu/security/token_signer.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_signing_key.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/status.h"
+
+DEFINE_int32(tsk_num_rsa_bits, 2048,
+             "Number of bits in RSA keys used for token signing.");
+TAG_FLAG(tsk_num_rsa_bits, experimental);
+
+using std::lock_guard;
+using std::map;
+using std::shared_ptr;
+using std::string;
+using std::unique_lock;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+TokenSigner::TokenSigner(int64_t authn_token_validity_seconds,
+                         int64_t key_rotation_seconds,
+                         shared_ptr<TokenVerifier> verifier)
+    : verifier_(verifier ? std::move(verifier)
+                         : std::make_shared<TokenVerifier>()),
+      authn_token_validity_seconds_(authn_token_validity_seconds),
+      key_rotation_seconds_(key_rotation_seconds),
+      // The TSK propagation interval is equal to the rotation interval.
+      key_validity_seconds_(2 * key_rotation_seconds_ + authn_token_validity_seconds_),
+      last_key_seq_num_(-1) {
+  CHECK_GE(key_rotation_seconds_, 0);
+  CHECK_GE(authn_token_validity_seconds_, 0);
+  CHECK(verifier_);
+}
+
+TokenSigner::~TokenSigner() {
+}
+
+Status TokenSigner::ImportKeys(const vector<TokenSigningPrivateKeyPB>& keys) {
+  lock_guard<RWMutex> l(lock_);
+
+  const int64_t now = WallTime_Now();
+  map<int64_t, unique_ptr<TokenSigningPrivateKey>> tsk_by_seq;
+  vector<TokenSigningPublicKeyPB> public_keys_pb;
+  public_keys_pb.reserve(keys.size());
+  for (const auto& key : keys) {
+    // Check the input for consistency.
+    CHECK(key.has_key_seq_num());
+    CHECK(key.has_expire_unix_epoch_seconds());
+    CHECK(key.has_rsa_key_der());
+
+    const int64_t key_seq_num = key.key_seq_num();
+    unique_ptr<TokenSigningPrivateKey> tsk(new TokenSigningPrivateKey(key));
+
+    // Advance the key sequence number, if needed. For the use case when the
+    // history of keys sequence numbers is important, the generated keys are
+    // persisted when TokenSigner is active and then the keys are imported from
+    // the store when TokenSigner is initialized (e.g., on restart). It's
+    // crucial to take into account sequence numbers of all previously persisted
+    // keys even if they have expired at the moment of importing.
+    last_key_seq_num_ = std::max(last_key_seq_num_, key_seq_num);
+    const int64_t key_expire_time = tsk->expire_time();
+    if (key_expire_time <= now) {
+      // Do nothing else with an expired TSK.
+      continue;
+    }
+
+    // Need the public part of the key for the TokenVerifier.
+    {
+      TokenSigningPublicKeyPB public_key_pb;
+      tsk->ExportPublicKeyPB(&public_key_pb);
+      public_keys_pb.emplace_back(std::move(public_key_pb));
+    }
+
+    tsk_by_seq[key_seq_num] = std::move(tsk);
+    if (tsk_by_seq.size() > 2) {
+      tsk_by_seq.erase(tsk_by_seq.begin());
+    }
+  }
+  // Register the public parts of the imported keys with the TokenVerifier.
+  RETURN_NOT_OK(verifier_->ImportKeys(public_keys_pb));
+
+  // Use two most recent keys known so far (in terms of sequence numbers)
+  // for token signing.
+  for (auto& e : tsk_deque_) {
+    const int64_t seq_num = e->key_seq_num();
+    tsk_by_seq[seq_num] = std::move(e);
+  }
+  tsk_deque_.clear();
+  for (auto& e : tsk_by_seq) {
+    tsk_deque_.emplace_back(std::move(e.second));
+  }
+  while (tsk_deque_.size() > 2) {
+    tsk_deque_.pop_front();
+  }
+
+  return Status::OK();
+}
+
+Status TokenSigner::GenerateAuthnToken(string username,
+                                       SignedTokenPB* signed_token) const {
+  if (username.empty()) {
+    return Status::InvalidArgument("no username provided for authn token");
+  }
+  TokenPB token;
+  token.set_expire_unix_epoch_seconds(
+      WallTime_Now() + authn_token_validity_seconds_);
+  AuthnTokenPB* authn = token.mutable_authn();
+  authn->mutable_username()->assign(std::move(username));
+
+  SignedTokenPB ret;
+  if (!token.SerializeToString(ret.mutable_token_data())) {
+    return Status::RuntimeError("could not serialize authn token");
+  }
+
+  RETURN_NOT_OK(SignToken(&ret));
+  signed_token->Swap(&ret);
+  return Status::OK();
+}
+
+Status TokenSigner::SignToken(SignedTokenPB* token) const {
+  CHECK(token);
+  shared_lock<RWMutex> l(lock_);
+  if (tsk_deque_.empty()) {
+    return Status::IllegalState("no token signing key");
+  }
+  const TokenSigningPrivateKey* key = tsk_deque_.front().get();
+  RETURN_NOT_OK_PREPEND(key->Sign(token), "could not sign authn token");
+  return Status::OK();
+}
+
+bool TokenSigner::IsCurrentKeyValid() const {
+  shared_lock<RWMutex> l(lock_);
+  if (tsk_deque_.empty()) {
+    return false;
+  }
+  return (tsk_deque_.front()->expire_time() > WallTime_Now());
+}
+
+Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const {
+  CHECK(tsk);
+  const int64_t now = WallTime_Now();
+
+  unique_lock<RWMutex> l(lock_);
+  if (tsk_deque_.empty()) {
+    // No active key: need a new one.
+    const int64_t key_seq_num = last_key_seq_num_ + 1;
+    const int64_t key_expiration = now + key_validity_seconds_;
+    // Generation of cryptographically strong key takes many CPU cycles;
+    // do not want to block other parallel activity.
+    l.unlock();
+    return GenerateSigningKey(key_seq_num, key_expiration, tsk);
+  }
+
+  if (tsk_deque_.size() >= 2) {
+    // It does not make much sense to keep more than two keys in the queue.
+    // It's enough to have just one active key and next key ready to be
+    // activated when it's time to do so.  However, it does not mean the
+    // process of key refreshment is about to stop once there are two keys
+    // in the queue: the TryRotate() method (which should be called periodically
+    // along with CheckNeedKey()/AddKey() pair) will eventually pop the
+    // current key out of the keys queue once the key enters its inactive phase.
+    tsk->reset();
+    return Status::OK();
+  }
+
+  // The currently active key is in the front of the queue.
+  const auto* key = tsk_deque_.front().get();
+
+  // Check if it's time to generate a new token signing key.
+  //
+  //   <-----AAAAA===========>
+  //         ^
+  //        now
+  //
+  const auto key_creation_time = key->expire_time() - key_validity_seconds_;
+  if (key_creation_time + key_rotation_seconds_ <= now) {
+    // It's time to create and start propagating next key.
+    const int64_t key_seq_num = last_key_seq_num_ + 1;
+    const int64_t key_expiration = now + key_validity_seconds_;
+    // Generation of cryptographically strong key takes many CPU cycles:
+    // do not want to block other parallel activity.
+    l.unlock();
+    return GenerateSigningKey(key_seq_num, key_expiration, tsk);
+  }
+
+  // It's not yet time to generate a new key.
+  tsk->reset();
+  return Status::OK();
+}
+
+Status TokenSigner::AddKey(unique_ptr<TokenSigningPrivateKey> tsk) {
+  CHECK(tsk);
+  const int64_t key_seq_num = tsk->key_seq_num();
+  if (tsk->expire_time() <= WallTime_Now()) {
+    return Status::InvalidArgument("key has already expired");
+  }
+
+  lock_guard<RWMutex> l(lock_);
+  if (key_seq_num < last_key_seq_num_ + 1) {
+    // The AddKey() method is designed for adding new keys: that should be done
+    // using CheckNeedKey()/AddKey() sequence. Use the ImportKeys() method
+    // for importing keys in bulk.
+    return Status::InvalidArgument(
+        Substitute("$0: invalid key sequence number, should be at least $1",
+                   key_seq_num, last_key_seq_num_ + 1));
+  }
+  last_key_seq_num_ = std::max(last_key_seq_num_, key_seq_num);
+  // Register the public part of the key in TokenVerifier first.
+  TokenSigningPublicKeyPB public_key_pb;
+  tsk->ExportPublicKeyPB(&public_key_pb);
+  RETURN_NOT_OK(verifier_->ImportKeys({public_key_pb}));
+
+  tsk_deque_.emplace_back(std::move(tsk));
+
+  return Status::OK();
+}
+
+Status TokenSigner::TryRotateKey(bool* has_rotated) {
+  lock_guard<RWMutex> l(lock_);
+  if (has_rotated) {
+    *has_rotated = false;
+  }
+  if (tsk_deque_.size() < 2) {
+    // There isn't next key to rotate to.
+    return Status::OK();
+  }
+
+  const auto* key = tsk_deque_.front().get();
+  // Check if it's time to switch to next key. The key propagation interval
+  // is equal to the key rotation interval.
+  //
+  // current active key   <-----AAAAA===========>
+  //           next key        <-----AAAAA===========>
+  //                                 ^
+  //                                now
+  //
+  const auto key_creation_time = key->expire_time() - key_validity_seconds_;
+  if (key_creation_time + 2 * key_rotation_seconds_ <= WallTime_Now()) {
+    tsk_deque_.pop_front();
+    if (has_rotated) {
+      *has_rotated = true;
+    }
+  }
+  return Status::OK();
+}
+
+Status TokenSigner::GenerateSigningKey(int64_t key_seq_num,
+                                       int64_t key_expiration,
+                                       unique_ptr<TokenSigningPrivateKey>* tsk) {
+  unique_ptr<PrivateKey> key(new PrivateKey());
+  RETURN_NOT_OK_PREPEND(
+      GeneratePrivateKey(FLAGS_tsk_num_rsa_bits, key.get()),
+      "could not generate new RSA token-signing key");
+  tsk->reset(new TokenSigningPrivateKey(key_seq_num,
+                                        key_expiration,
+                                        std::move(key)));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_signer.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_signer.h b/be/src/kudu/security/token_signer.h
new file mode 100644
index 0000000..df1e3eb
--- /dev/null
+++ b/be/src/kudu/security/token_signer.h
@@ -0,0 +1,316 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <deque>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/rw_mutex.h"
+
+namespace kudu {
+class Status;
+
+namespace security {
+class SignedTokenPB;
+class TokenSigningPrivateKey;
+class TokenSigningPrivateKeyPB;
+class TokenVerifier;
+
+// Class responsible for managing Token Signing Keys (TSKs) and signing tokens.
+//
+// This class manages a set of private TSKs, each identified by a sequence
+// number. Callers can export their public TSK counterparts via the included
+// TokenVerifier, optionally transfer them to another node, and then import
+// them into a TokenVerifier.
+//
+// The class provides the ability to check whether it's time go generate and
+// activate a new key. Every generated private/public key pair is assigned a
+// sequence number. Note that, when signing tokens, the most recent key
+// (a.k.a. next key) is not used. Rather, the second-most-recent key, if exists,
+// is used. This ensures that there is plenty of time to transmit the public
+// part of the new TSK to all TokenVerifiers (e.g. on other servers via
+// heartbeats or by other means), before the new key enters usage.
+//
+// On a fresh instance, with only one key, there is no "second most recent"
+// key. Thus, we fall back to signing tokens with the only available key.
+//
+// Key rotation schedules and validity periods
+// ===========================================
+// The TokenSigner does not automatically handle the rotation of keys.
+// Rotation must be performed by an external caller using the combination of
+// 'CheckNeedKey()/AddKey()' and 'TryRotateKey()' methods. Typically,
+// key rotation is performed more frequently than the validity period
+// of the key, so that at any given point in time there are several valid keys.
+//
+// Below is the life cycle of a TSK (token signing key):
+//
+//      <---AAAAA===============>
+//      ^                       ^
+// creation time          expiration time
+//
+// Prior to the creation time the TSK does not exist in the system.
+//
+// '-' propagation interval
+//       The TSK is already created but not yet used to sign tokens. However,
+//       its public part is already being sent to the components which
+//       may be involved in validation of tokens signed by the key.
+//
+// 'A' activity interval
+//       The TSK is used to sign tokens. It's assumed that the components which
+//       are involved in token verification have already received
+//       the corresponding public part of the TSK.
+//
+// '=' inactivity interval
+//       The TSK is no longer used to sign tokens. However, it's still sent
+//       to other components which validate token signatures.
+//
+// Shortly after the TSK's expiration the token signing components stop
+// propagating its public part.
+//
+// The TSK is considered valid from its creation time until its expiration time.
+//
+// NOTE: The very first key created on the system bootstrap does not have
+//       propagation interval -- it turns active immediately.
+//
+// NOTE: One other result of the above is that the first key (Key 1) is actually
+//       active for longer than the rest. This has some potential security
+//       implications, so it's worth considering rolling twice at startup.
+//
+// For example, consider the following configuration for token signing keys:
+//   validity period:      4 days
+//   rotation interval:    1 days
+//   propagation interval: 1 day
+//
+// Day      1    2    3    4    5    6    7    8
+// ------------------------------------------------
+// Key 1:   <AAAAAAAAA==========>
+// Key 2:        <----AAAAA==========>
+// Key 3:             <----AAAAA==========>
+// Key 4:                  <----AAAAA==========>
+//                              ...............
+// authn token:                     <**********>
+//
+// 'A' indicates the 'Originator Usage Period' (a.k.a. 'Activity Interval'),
+// i.e. the period in which the key is being used to sign tokens.
+//
+// '<...>' indicates the 'Recipient Usage Period': the period in which
+// the verifier may get tokens signed by the TSK and should consider them
+// for verification. The start of the recipient usage period is not crucial
+// in that regard, but the end of that period is -- after the TSK is expired,
+// a verifier should consider tokens signed by that TSK invalid and stop
+// accepting them even if the token signature is correct and the expiration.
+//
+// '<***>' indicates the validity interval for an authn token.
+//
+// When configuring key rotation and authn token validity interval durations,
+// consider the following constraint:
+//
+//   max_token_validity < tsk_validity_period -
+//       (tsk_propagation_interval + tsk_rotation_interval)
+//
+// The idea is that the token validity interval should be contained in the
+// corresponding TSK's validity interval. If the TSK is already expired at the
+// time of token verification, the token is considered invalid and the
+// verification of the token fails. This means that no token may be issued with
+// a validity period longer than or equal to TSK inactivity interval, without
+// risking that the signing/verification key would expire before the token
+// itself. The edge case is demonstrated by the following scenario:
+//
+// * A TSK is issued at 00:00:00 on day 4.
+// * An authn token generated and signed by current/active TSK at 23:59:59 on
+//   day 6. That's at the very end of the TSK's activity interval.
+// * From the diagram above it's clear that if the authn token validity
+//   interval were set to something longer than TSK inactivity interval
+//   (which is 2 days with for the specified parameters), an attempt to verify
+//   the token at 00:00:00 on day 8 or later would fail due to the expiration
+//   the corresponding TSK.
+//
+// NOTE: Current implementation of TokenSigner assumes the propagation
+//       interval is equal to the rotation interval.
+//
+// Typical usage pattern:
+//
+//    TokenSigner ts(...);
+//    // Load existing TSKs from the system table.
+//    ...
+//    RETURN_NOT_OK(ts.ImportKeys(...));
+//
+//    // Check that there is a valid TSK to sign keys.
+//    {
+//      unique_ptr<TokenSigningPrivateKey> key;
+//      RETURN_NOT_OK(ts.CheckNeedKey(&key));
+//      if (key) {
+//        // Store the newly generated key into the system table.
+//        ...
+//
+//        // Add the key into the queue of the TokenSigner.
+//        RETURN_NOT_OK(ts.AddKey(std::move(key)));
+//      }
+//    }
+//    // Check and switch to the next key, if it's time.
+//    RETURN_NOT_OK(ts.TryRotateKey());
+//
+//    ...
+//    // Time to time (but much more often than TSK validity/rotation interval)
+//    // call the 'CheckNeedKey()/AddKey() followed by TryRotateKey()' sequence.
+//    // It's a good idea to dedicate a separate periodic task for that.
+//    ...
+//
+class TokenSigner {
+ public:
+  // The 'key_validity_seconds' and 'key_rotation_seconds' parameters define
+  // the schedule of TSK rotation. See the class comment above for details.
+  //
+  // Any newly imported or generated keys are automatically imported into the
+  // passed 'verifier'. If no verifier passed as a parameter, TokenSigner
+  // creates one on its own. In either case, it's possible to access
+  // the embedded TokenVerifier instance using the verifier() accessor.
+  //
+  // The 'authn_token_validity_seconds' parameter is used to specify validity
+  // interval for the generated authn tokens and with 'key_rotation_seconds'
+  // it defines validity interval of the newly generated TSK:
+  //   key_validity = 2 * key_rotation + authn_token_validity.
+  //
+  // That corresponds to the maximum possible token lifetime for the effective
+  // TSK validity and rotation intervals: see the class comment above for
+  // details.
+  TokenSigner(int64_t authn_token_validity_seconds,
+              int64_t key_rotation_seconds,
+              std::shared_ptr<TokenVerifier> verifier = nullptr);
+  ~TokenSigner();
+
+  // Import token signing keys in PB format, notifying TokenVerifier
+  // and updating internal key sequence number. This method can be called
+  // multiple times. Depending on the input keys and current time, the instance
+  // might not be ready to sign keys right after calling ImportKeys(),
+  // so additional cycle of CheckNeedKey/AddKey might be needed.
+  //
+  // See the class comment above for more information about the intended usage.
+  Status ImportKeys(const std::vector<TokenSigningPrivateKeyPB>& keys)
+      WARN_UNUSED_RESULT;
+
+  // Check whether it's time to generate and add a new key. If so, the new key
+  // is generated and output into the 'tsk' parameter so it's possible to
+  // examine and process the key as needed (e.g. store it). After that, use the
+  // AddKey() method to actually add the key into the TokenSigner's key queue.
+  //
+  // Every non-null key returned by this method has key sequence number.
+  // It's not a problem to call this method multiple times but call the AddKey()
+  // method only once, effectively discarding all the generated keys except for
+  // the key passed to the AddKey() call as a parameter. The key sequence number
+  // always increments with every newly added key (i.e. every successful call of
+  // the AddKey() method). The result key number sequence would not contain
+  // any 'holes'.
+  //
+  // In other words, sequence of calls like
+  //
+  //   CheckNeedKey(k);
+  //   CheckNeedKey(k);
+  //   ...
+  //   CheckNeedKey(k);
+  //   AddKey(k);
+  //
+  // would increase the key sequence number just by 1. Due to that fact, the
+  // following sequence of calls to CheckNeedKey()/AddKey() would work fine:
+  //
+  //   CheckNeedKey(k0);
+  //   AddKey(k0);
+  //   CheckNeedKey(k1);
+  //   AddKey(k1);
+  //
+  // but the sequence below would fail at AddKey(k1):
+  //
+  //   CheckNeedKey(k0);
+  //   CheckNeedKey(k1);
+  //   AddKey(k0);
+  //   AddKey(k1);
+  //
+  // See the class comment above for more information about the intended usage.
+  Status CheckNeedKey(std::unique_ptr<TokenSigningPrivateKey>* tsk) const
+      WARN_UNUSED_RESULT;
+
+  // Add the new key into the token signing keys queue. Call TryRotateKey()
+  // to make the newly added key active when it's time.
+  //
+  // See the class comment above for more information about the intended usage.
+  Status AddKey(std::unique_ptr<TokenSigningPrivateKey> tsk) WARN_UNUSED_RESULT;
+
+  // Check whether it's possible and it's time to switch to next signing key
+  // from the token signing keys queue. A key can be added using the
+  // CheckNeedKey()/AddKey() method pair. If there is next key to switch to
+  // and it's time to do so, the methods switches to the next key and reports
+  // on that via the 'has_rotated' parameter.
+  // The intended use case is to call TryRotateKey() periodically.
+  //
+  // See the class comment above for more information about the intended usage.
+  Status TryRotateKey(bool* has_rotated = nullptr) WARN_UNUSED_RESULT;
+
+  Status GenerateAuthnToken(std::string username,
+                            SignedTokenPB* signed_token) const WARN_UNUSED_RESULT;
+
+  Status SignToken(SignedTokenPB* token) const WARN_UNUSED_RESULT;
+
+  const TokenVerifier& verifier() const { return *verifier_; }
+
+  // Check if the current TSK is valid: return 'true' if current key is present
+  // and it's not yet expired, return 'false' otherwise.
+  bool IsCurrentKeyValid() const;
+
+ private:
+  FRIEND_TEST(TokenTest, TestEndToEnd_InvalidCases);
+
+  static Status GenerateSigningKey(int64_t key_seq_num,
+                                   int64_t key_expiration,
+                                   std::unique_ptr<TokenSigningPrivateKey>* tsk) WARN_UNUSED_RESULT;
+
+  std::shared_ptr<TokenVerifier> verifier_;
+
+  // Validity interval for the generated authn tokens.
+  const int64_t authn_token_validity_seconds_;
+
+  // TSK rotation interval: number of seconds between consecutive activations
+  // of new token signing keys. Note that in current implementation it defines
+  // the propagation interval as well, i.e. the TSK propagation interval is
+  // equal to the TSK rotation interval.
+  const int64_t key_rotation_seconds_;
+
+  // Period of validity for newly created token signing keys. In other words,
+  // the expiration time for a new key is set to (now + key_validity_seconds_).
+  const int64_t key_validity_seconds_;
+
+  // Protects next_seq_num_ and tsk_deque_ members.
+  mutable RWMutex lock_;
+
+  // The sequence number of the last generated/imported key.
+  int64_t last_key_seq_num_;
+
+  // The currently active key is in the front of the queue,
+  // the newly added ones are pushed into back of the queue.
+  std::deque<std::unique_ptr<TokenSigningPrivateKey>> tsk_deque_;
+
+  DISALLOW_COPY_AND_ASSIGN(TokenSigner);
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_signing_key.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_signing_key.cc b/be/src/kudu/security/token_signing_key.cc
new file mode 100644
index 0000000..38d49c6
--- /dev/null
+++ b/be/src/kudu/security/token_signing_key.cc
@@ -0,0 +1,110 @@
+// 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 "kudu/security/token_signing_key.h"
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/status.h"
+
+using std::unique_ptr;
+using std::string;
+
+namespace kudu {
+namespace security {
+
+TokenSigningPublicKey::TokenSigningPublicKey(TokenSigningPublicKeyPB pb)
+    : pb_(std::move(pb)) {
+}
+
+TokenSigningPublicKey::~TokenSigningPublicKey() {
+}
+
+Status TokenSigningPublicKey::Init() {
+  // This should be called only once.
+  CHECK(!key_.GetRawData());
+  if (!pb_.has_rsa_key_der()) {
+    return Status::RuntimeError("no key for token signing helper");
+  }
+  RETURN_NOT_OK(key_.FromString(pb_.rsa_key_der(), DataFormat::DER));
+  return Status::OK();
+}
+
+bool TokenSigningPublicKey::VerifySignature(const SignedTokenPB& token) const {
+  return key_.VerifySignature(DigestType::SHA256,
+      token.token_data(), token.signature()).ok();
+}
+
+TokenSigningPrivateKey::TokenSigningPrivateKey(
+    const TokenSigningPrivateKeyPB& pb)
+    : key_(new PrivateKey) {
+  CHECK_OK(key_->FromString(pb.rsa_key_der(), DataFormat::DER));
+  private_key_der_ = pb.rsa_key_der();
+  key_seq_num_ = pb.key_seq_num();
+  expire_time_ = pb.expire_unix_epoch_seconds();
+
+  PublicKey public_key;
+  CHECK_OK(key_->GetPublicKey(&public_key));
+  CHECK_OK(public_key.ToString(&public_key_der_, DataFormat::DER));
+}
+
+TokenSigningPrivateKey::TokenSigningPrivateKey(
+    int64_t key_seq_num, int64_t expire_time, unique_ptr<PrivateKey> key)
+    : key_(std::move(key)),
+      key_seq_num_(key_seq_num),
+      expire_time_(expire_time) {
+  CHECK_OK(key_->ToString(&private_key_der_, DataFormat::DER));
+  PublicKey public_key;
+  CHECK_OK(key_->GetPublicKey(&public_key));
+  CHECK_OK(public_key.ToString(&public_key_der_, DataFormat::DER));
+}
+
+TokenSigningPrivateKey::~TokenSigningPrivateKey() {
+}
+
+Status TokenSigningPrivateKey::Sign(SignedTokenPB* token) const {
+  string signature;
+  RETURN_NOT_OK(key_->MakeSignature(DigestType::SHA256,
+      token->token_data(), &signature));
+  token->mutable_signature()->assign(std::move(signature));
+  token->set_signing_key_seq_num(key_seq_num_);
+  return Status::OK();
+}
+
+void TokenSigningPrivateKey::ExportPB(TokenSigningPrivateKeyPB* pb) const {
+  pb->Clear();
+  pb->set_key_seq_num(key_seq_num_);
+  pb->set_rsa_key_der(private_key_der_);
+  pb->set_expire_unix_epoch_seconds(expire_time_);
+}
+
+void TokenSigningPrivateKey::ExportPublicKeyPB(TokenSigningPublicKeyPB* pb) const {
+  pb->Clear();
+  pb->set_key_seq_num(key_seq_num_);
+  pb->set_rsa_key_der(public_key_der_);
+  pb->set_expire_unix_epoch_seconds(expire_time_);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_signing_key.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_signing_key.h b/be/src/kudu/security/token_signing_key.h
new file mode 100644
index 0000000..67ecf95
--- /dev/null
+++ b/be/src/kudu/security/token_signing_key.h
@@ -0,0 +1,103 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+// Wrapper around a TokenSigningPublicKeyPB that provides useful functionality
+// to verify tokens.
+//
+// This represents a standalone public key useful for token verification.
+class TokenSigningPublicKey {
+ public:
+  explicit TokenSigningPublicKey(TokenSigningPublicKeyPB pb);
+  ~TokenSigningPublicKey();
+
+  const TokenSigningPublicKeyPB& pb() const {
+    return pb_;
+  }
+
+  // Initialize the object. Should be called only once.
+  Status Init() WARN_UNUSED_RESULT;
+
+  // Verify the signature in a given token.
+  // This method is thread-safe.
+  // NOTE: this does _not_ verify the expiration.
+  bool VerifySignature(const SignedTokenPB& token) const;
+
+ private:
+  const TokenSigningPublicKeyPB pb_;
+  // The 'key_' member is a parsed version of rsa_key_der() from pb_.
+  // In essence, the 'key_' is a public key for message signature verification.
+  PublicKey key_;
+
+  DISALLOW_COPY_AND_ASSIGN(TokenSigningPublicKey);
+};
+
+// Contains a private key used to sign tokens, along with its sequence
+// number and expiration date.
+class TokenSigningPrivateKey {
+ public:
+  explicit TokenSigningPrivateKey(const TokenSigningPrivateKeyPB& pb);
+  TokenSigningPrivateKey(int64_t key_seq_num,
+                         int64_t expire_time,
+                         std::unique_ptr<PrivateKey> key);
+  ~TokenSigningPrivateKey();
+
+  // Sign a token, and store the signature and signing key's sequence number.
+  Status Sign(SignedTokenPB* token) const WARN_UNUSED_RESULT;
+
+  // Export data into corresponding PB structure.
+  void ExportPB(TokenSigningPrivateKeyPB* pb) const;
+
+  // Export the public-key portion of this signing key.
+  void ExportPublicKeyPB(TokenSigningPublicKeyPB* pb) const;
+
+  int64_t key_seq_num() const { return key_seq_num_; }
+  int64_t expire_time() const { return expire_time_; }
+
+ private:
+  FRIEND_TEST(TokenTest, TestAddKeyConstraints);
+
+  std::unique_ptr<PrivateKey> key_;
+  // The 'private_key_der_' is a serialized 'key_' in DER format: just a cache.
+  std::string private_key_der_;
+  // The 'public_key_der_' is serialized public part of 'key_' in DER format;
+  // just a cache.
+  std::string public_key_der_;
+
+  int64_t key_seq_num_;
+  int64_t expire_time_;
+
+  DISALLOW_COPY_AND_ASSIGN(TokenSigningPrivateKey);
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_verifier.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_verifier.cc b/be/src/kudu/security/token_verifier.cc
new file mode 100644
index 0000000..1ae20db
--- /dev/null
+++ b/be/src/kudu/security/token_verifier.cc
@@ -0,0 +1,173 @@
+// 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 "kudu/security/token_verifier.h"
+
+#include <algorithm>
+#include <iterator>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_signing_key.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/status.h"
+
+using std::lock_guard;
+using std::string;
+using std::transform;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+TokenVerifier::TokenVerifier() {
+}
+
+TokenVerifier::~TokenVerifier() {
+}
+
+int64_t TokenVerifier::GetMaxKnownKeySequenceNumber() const {
+  shared_lock<RWMutex> l(lock_);
+  if (keys_by_seq_.empty()) {
+    return -1;
+  }
+
+  return keys_by_seq_.rbegin()->first;
+}
+
+// Import a set of public keys provided by the token signer (typically
+// running on another node).
+Status TokenVerifier::ImportKeys(const vector<TokenSigningPublicKeyPB>& keys) {
+  // Do the construction outside of the lock, to avoid holding the
+  // lock while doing lots of allocation.
+  vector<unique_ptr<TokenSigningPublicKey>> tsks;
+  for (const auto& pb : keys) {
+    // Sanity check the key.
+    if (!pb.has_rsa_key_der()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include the signing key");
+    }
+    if (!pb.has_key_seq_num()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include the signing key sequence number");
+    }
+    if (!pb.has_expire_unix_epoch_seconds()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include an expiration time");
+    }
+    tsks.emplace_back(new TokenSigningPublicKey { pb });
+    RETURN_NOT_OK(tsks.back()->Init());
+  }
+
+  lock_guard<RWMutex> l(lock_);
+  for (auto&& tsk_ptr : tsks) {
+    keys_by_seq_.emplace(tsk_ptr->pb().key_seq_num(), std::move(tsk_ptr));
+  }
+  return Status::OK();
+}
+
+std::vector<TokenSigningPublicKeyPB> TokenVerifier::ExportKeys(
+    int64_t after_sequence_number) const {
+  vector<TokenSigningPublicKeyPB> ret;
+  shared_lock<RWMutex> l(lock_);
+  ret.reserve(keys_by_seq_.size());
+  transform(keys_by_seq_.upper_bound(after_sequence_number),
+            keys_by_seq_.end(),
+            back_inserter(ret),
+            [](const KeysMap::value_type& e) { return e.second->pb(); });
+  return ret;
+}
+
+// Verify the signature on the given token.
+VerificationResult TokenVerifier::VerifyTokenSignature(const SignedTokenPB& signed_token,
+                                                       TokenPB* token) const {
+  if (!signed_token.has_signature() ||
+      !signed_token.has_signing_key_seq_num() ||
+      !signed_token.has_token_data()) {
+    return VerificationResult::INVALID_TOKEN;
+  }
+
+  if (!token->ParseFromString(signed_token.token_data()) ||
+      !token->has_expire_unix_epoch_seconds()) {
+    return VerificationResult::INVALID_TOKEN;
+  }
+
+  int64_t now = WallTime_Now();
+  if (token->expire_unix_epoch_seconds() < now) {
+    return VerificationResult::EXPIRED_TOKEN;
+  }
+
+  for (auto flag : token->incompatible_features()) {
+    if (!TokenPB::Feature_IsValid(flag)) {
+      KLOG_EVERY_N_SECS(WARNING, 60) << "received authentication token with unknown feature; "
+                                        "server needs to be updated";
+      return VerificationResult::INCOMPATIBLE_FEATURE;
+    }
+  }
+
+  {
+    shared_lock<RWMutex> l(lock_);
+    auto* tsk = FindPointeeOrNull(keys_by_seq_, signed_token.signing_key_seq_num());
+    if (!tsk) {
+      return VerificationResult::UNKNOWN_SIGNING_KEY;
+    }
+    if (tsk->pb().expire_unix_epoch_seconds() < now) {
+      return VerificationResult::EXPIRED_SIGNING_KEY;
+    }
+    if (!tsk->VerifySignature(signed_token)) {
+      return VerificationResult::INVALID_SIGNATURE;
+    }
+  }
+
+  return VerificationResult::VALID;
+}
+
+const char* VerificationResultToString(VerificationResult r) {
+  switch (r) {
+    case security::VerificationResult::VALID:
+      return "valid";
+    case security::VerificationResult::INVALID_TOKEN:
+      return "invalid authentication token";
+    case security::VerificationResult::INVALID_SIGNATURE:
+      return "invalid authentication token signature";
+    case security::VerificationResult::EXPIRED_TOKEN:
+      return "authentication token expired";
+    case security::VerificationResult::EXPIRED_SIGNING_KEY:
+      return "authentication token signing key expired";
+    case security::VerificationResult::UNKNOWN_SIGNING_KEY:
+      return "authentication token signed with unknown key";
+    case security::VerificationResult::INCOMPATIBLE_FEATURE:
+      return "authentication token uses incompatible feature";
+    default:
+      LOG(FATAL) << "unexpected VerificationResult value: "
+                 << static_cast<int>(r);
+  }
+}
+
+} // namespace security
+} // namespace kudu
+

http://git-wip-us.apache.org/repos/asf/impala/blob/fcf190c4/be/src/kudu/security/token_verifier.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/token_verifier.h b/be/src/kudu/security/token_verifier.h
new file mode 100644
index 0000000..8d5d176
--- /dev/null
+++ b/be/src/kudu/security/token_verifier.h
@@ -0,0 +1,126 @@
+// 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.
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/rw_mutex.h"
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+class SignedTokenPB;
+class TokenPB;
+class TokenSigningPublicKey;
+class TokenSigningPublicKeyPB;
+enum class VerificationResult;
+
+// Class responsible for verifying tokens provided to a server.
+//
+// This class manages a set of public keys, each identified by a sequence
+// number. It exposes the latest known sequence number, which can be sent
+// to a 'TokenSigner' running on another node. That node can then
+// export public keys, which are transferred back to this node and imported
+// into the 'TokenVerifier'.
+//
+// Each signed token also includes the key sequence number that signed it,
+// so this class can look up the correct key and verify the token's
+// validity and expiration.
+//
+// Note that this class does not perform any "business logic" around the
+// content of a token. It only verifies that the token has a valid signature
+// and is not yet expired. Any business rules around authorization or
+// authentication are left up to callers.
+//
+// NOTE: old tokens are never removed from the underlying storage of this
+// class. The assumption is that tokens rotate so infreqeuently that this
+// slow leak is not worrisome. If this class is adopted for any use cases
+// with frequent rotation, GC of expired tokens will need to be added.
+//
+// This class is thread-safe.
+class TokenVerifier {
+ public:
+  TokenVerifier();
+  ~TokenVerifier();
+
+  // Return the highest key sequence number known by this instance.
+  //
+  // If no keys are known, return -1.
+  int64_t GetMaxKnownKeySequenceNumber() const;
+
+  // Import a set of public keys provided by a TokenSigner instance
+  // (which might be running on a remote node). If any public keys already
+  // exist with matching key sequence numbers, they are replaced by
+  // the new keys.
+  Status ImportKeys(const std::vector<TokenSigningPublicKeyPB>& keys) WARN_UNUSED_RESULT;
+
+  // Export token signing public keys. Specifying the 'after_sequence_number'
+  // allows to get public keys with sequence numbers greater than
+  // 'after_sequence_number'. If the 'after_sequence_number' parameter is
+  // omitted, all known public keys are exported.
+  std::vector<TokenSigningPublicKeyPB> ExportKeys(
+      int64_t after_sequence_number = -1) const;
+
+  // Verify the signature on the given signed token, and deserialize the
+  // contents into 'token'.
+  VerificationResult VerifyTokenSignature(const SignedTokenPB& signed_token,
+                                          TokenPB* token) const;
+
+ private:
+  typedef std::map<int64_t, std::unique_ptr<TokenSigningPublicKey>> KeysMap;
+
+  // Lock protecting keys_by_seq_
+  mutable RWMutex lock_;
+  KeysMap keys_by_seq_;
+
+  DISALLOW_COPY_AND_ASSIGN(TokenVerifier);
+};
+
+// Result of a token verification.
+// Values added to this enum must also be added to VerificationResultToString().
+enum class VerificationResult {
+  // The signature is valid and the token is not expired.
+  VALID,
+  // The token itself is invalid (e.g. missing its signature or data,
+  // can't be deserialized, etc).
+  INVALID_TOKEN,
+  // The signature is invalid (i.e. cryptographically incorrect).
+  INVALID_SIGNATURE,
+  // The signature is valid, but the token has already expired.
+  EXPIRED_TOKEN,
+  // The signature is valid, but the signing key is no longer valid.
+  EXPIRED_SIGNING_KEY,
+  // The signing key used to sign this token is not available.
+  UNKNOWN_SIGNING_KEY,
+  // The token uses an incompatible feature which isn't supported by this
+  // version of the server. We reject the token to give a "default deny"
+  // policy.
+  INCOMPATIBLE_FEATURE
+};
+
+const char* VerificationResultToString(VerificationResult r);
+
+} // namespace security
+} // namespace kudu