You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2018/01/24 19:25:25 UTC

[1/3] kudu git commit: consensus: avoid extra thread wakeups for Peer::SignalRequest

Repository: kudu
Updated Branches:
  refs/heads/master 3b51bf337 -> c497c69bb


consensus: avoid extra thread wakeups for Peer::SignalRequest

Looking at a profile on a YCSB workload I see a significant amount of
CPU in submitting tasks to the raft threadpool for requests. My guess is
that, in a workload like this, we often call SignalRequest while the
request was already pending.

This patch avoids submitting the threadpool task if there is already a
task pending.

Change-Id: I62ec4ea91e976590ce2de4fa34db61eaf9aab92a
Reviewed-on: http://gerrit.cloudera.org:8080/9092
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <da...@gmail.com>


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

Branch: refs/heads/master
Commit: 9961902d8b4f18a5e5d68aeb624e27167651b837
Parents: 3b51bf3
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Jan 22 08:13:15 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Jan 24 19:07:17 2018 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_peers.cc | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9961902d/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 6d68528..187ed6f 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -167,6 +167,12 @@ Status Peer::SignalRequest(bool even_if_queue_empty) {
     return Status::IllegalState("Peer was closed.");
   }
 
+  // Only allow one request at a time. No sense waking up the
+  // raft thread pool if the task will just abort anyway.
+  if (request_pending_) {
+    return Status::OK();
+  }
+
   // Capture a weak_ptr reference into the submitted functor so that we can
   // safely handle the functor outliving its peer.
   weak_ptr<Peer> w_this = shared_from_this();


[3/3] kudu git commit: KUDU-2265 CA-signed server certs for non-leader masters

Posted by al...@apache.org.
KUDU-2265 CA-signed server certs for non-leader masters

This changelist addresses KUDU-2265.  Prior to this fix, a non-leader
master had a self-signed server TLS certificates if it hasn't ever
become a leader.

An integration test to verify the new behavior is added as well.

Change-Id: Ia3539d58d10ed319ad1d8686c1259c92822fb710
Reviewed-on: http://gerrit.cloudera.org:8080/9076
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Dan Burkert <da...@cloudera.com>


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

Branch: refs/heads/master
Commit: c497c69bb2128a8a62738b9b6bb97a5b83da0323
Parents: e0a743d
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jan 18 23:52:03 2018 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Jan 24 19:24:32 2018 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/CMakeLists.txt       |   1 +
 .../security-master-certificates-itest.cc       | 100 +++++++++++++++++++
 src/kudu/master/catalog_manager.cc              |  46 +++++++--
 src/kudu/master/catalog_manager.h               |  30 +++++-
 src/kudu/security/tls_context.cc                |  10 +-
 src/kudu/util/rw_mutex.cc                       |   6 ++
 src/kudu/util/rw_mutex.h                        |   2 +
 7 files changed, 177 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 1a10a7d..a6a6938 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -94,6 +94,7 @@ ADD_KUDU_TEST(raft_consensus-itest RUN_SERIAL true)
 ADD_KUDU_TEST(registration-test RESOURCE_LOCK "master-web-port")
 ADD_KUDU_TEST(security-faults-itest)
 ADD_KUDU_TEST(security-itest)
+ADD_KUDU_TEST(security-master-certificates-itest RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(security-unknown-tsk-itest)
 ADD_KUDU_TEST(stop_tablet-itest)
 ADD_KUDU_TEST(table_locations-itest)

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/integration-tests/security-master-certificates-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-master-certificates-itest.cc b/src/kudu/integration-tests/security-master-certificates-itest.cc
new file mode 100644
index 0000000..b521d65
--- /dev/null
+++ b/src/kudu/integration-tests/security-master-certificates-itest.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 <memory>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
+
+#include "kudu/consensus/raft_consensus.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/master/catalog_manager.h"
+#include "kudu/master/master.h"
+#include "kudu/master/mini_master.h"
+#include "kudu/master/sys_catalog.h"
+#include "kudu/mini-cluster/internal_mini_cluster.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/tablet/tablet_replica.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_bool(enable_leader_failure_detection);
+
+using kudu::cluster::InternalMiniCluster;
+using kudu::cluster::InternalMiniClusterOptions;
+using kudu::consensus::RaftConsensus;
+
+using std::unique_ptr;
+
+
+namespace kudu {
+
+class SecurityMasterCertsTest : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    // This test requires manual system catalog leader election.
+    FLAGS_enable_leader_failure_detection = false;
+
+    InternalMiniClusterOptions opts;
+    opts.master_rpc_ports = { 11010, 11011, 11012, 11013, 11014, };
+    opts.num_masters = opts.master_rpc_ports.size();
+    opts.num_tablet_servers = 0;
+    cluster_.reset(new InternalMiniCluster(env_, opts));
+    ASSERT_OK(cluster_->Start());
+  }
+
+  void TearDown() override {
+    cluster_->Shutdown();
+  }
+
+ protected:
+  unique_ptr<InternalMiniCluster> cluster_;
+};
+
+// This scenario verifies that follower masters get CA-signed certificates
+// even if they haven't run in the leader role yet. In this particular scenario,
+// only one of the masters has ever become a leader and the rest have always
+// been followers. This is a test to cover regressions of KUDU-2265, if any.
+TEST_F(SecurityMasterCertsTest, FollowerCertificates) {
+  for (auto i = 0; i < cluster_->num_masters(); ++i) {
+    const auto& tls = cluster_->mini_master(i)->master()->tls_context();
+    // Initially, all master servers have self-signed certs,
+    // but none has CA-signed cert.
+    ASSERT_FALSE(tls.has_signed_cert());
+    ASSERT_TRUE(tls.has_cert());
+  }
+
+  auto* consensus = cluster_->mini_master(0)->master()->catalog_manager()->
+      sys_catalog()->tablet_replica()->consensus();
+  ASSERT_OK(consensus->StartElection(
+      RaftConsensus::ELECT_EVEN_IF_LEADER_IS_ALIVE,
+      RaftConsensus::EXTERNAL_REQUEST));
+
+  // After some time, all master servers should have CA-signed certs.
+  ASSERT_EVENTUALLY([&] {
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      const auto& tls = cluster_->mini_master(i)->master()->tls_context();
+      ASSERT_TRUE(tls.has_signed_cert());
+    }
+  });
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 1248144..33fd194 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -536,6 +536,14 @@ void CatalogManagerBgTasks::Run() {
             LOG(FATAL) << err_msg;
           }
         }
+      } else if (catalog_manager_->NeedToPrepareFollower() && l.owns_lock()) {
+        // This is the case of a non-leader catalog manager that has some work
+        // to do in a preparation to run in its current role.
+        Status s = catalog_manager_->PrepareFollower();
+        if (!s.ok()) {
+          LOG(WARNING) << s.ToString()
+                       << ": failed to prepare follower catalog manager, will retry";
+        }
       }
     }
     // Wait for a notification or a timeout expiration.
@@ -788,17 +796,16 @@ Status CatalogManager::InitCertAuthority() {
 // private key and certificate.
 Status CatalogManager::InitCertAuthorityWith(
     unique_ptr<PrivateKey> key, unique_ptr<Cert> cert) {
-  leader_lock_.AssertAcquiredForWriting();
+  leader_lock_.AssertAcquired();
+
   auto* ca = master_->cert_authority();
   RETURN_NOT_OK_PREPEND(ca->Init(std::move(key), std::move(cert)),
                         "could not init master CA");
-
   auto* tls = master_->mutable_tls_context();
   RETURN_NOT_OK_PREPEND(tls->AddTrustedCertificate(ca->ca_cert()),
                         "could not trust master CA cert");
   // If we haven't signed our own server cert yet, do so.
-  boost::optional<security::CertSignRequest> csr =
-      tls->GetCsrIfNecessary();
+  boost::optional<security::CertSignRequest> csr = tls->GetCsrIfNecessary();
   if (csr) {
     Cert cert;
     RETURN_NOT_OK_PREPEND(ca->SignServerCSR(*csr, &cert),
@@ -811,7 +818,7 @@ Status CatalogManager::InitCertAuthorityWith(
 
 Status CatalogManager::LoadCertAuthorityInfo(unique_ptr<PrivateKey>* key,
                                              unique_ptr<Cert>* cert) {
-  leader_lock_.AssertAcquiredForWriting();
+  leader_lock_.AssertAcquired();
 
   SysCertAuthorityEntryPB info;
   RETURN_NOT_OK(sys_catalog_->GetCertAuthorityEntry(&info));
@@ -975,6 +982,29 @@ void CatalogManager::PrepareForLeadershipTask() {
   leader_ready_term_ = term;
 }
 
+bool CatalogManager::NeedToPrepareFollower() {
+  return !master_->tls_context().has_signed_cert();
+}
+
+Status CatalogManager::PrepareFollower() {
+  static const char* const kDescription =
+      "acquiring CA information for follower catalog manager";
+
+  leader_lock_.AssertAcquiredForReading();
+
+  unique_ptr<PrivateKey> key;
+  unique_ptr<Cert> cert;
+  Status s = LoadCertAuthorityInfo(&key, &cert).AndThen([&] {
+    return InitCertAuthorityWith(std::move(key), std::move(cert));
+  });
+  if (s.ok()) {
+    LOG_WITH_PREFIX(INFO) << kDescription << ": success";
+  } else {
+    LOG_WITH_PREFIX(WARNING) << kDescription << ": " << s.ToString();
+  }
+  return s;
+}
+
 Status CatalogManager::VisitTablesAndTabletsUnlocked() {
   leader_lock_.AssertAcquiredForWriting();
 
@@ -1013,10 +1043,8 @@ Status CatalogManager::VisitTablesAndTablets() {
 
 Status CatalogManager::InitSysCatalogAsync(bool is_first_run) {
   std::lock_guard<LockType> l(lock_);
-  unique_ptr<SysCatalogTable> new_catalog(
-      new SysCatalogTable(master_,
-                          Bind(&CatalogManager::ElectedAsLeaderCb,
-                               Unretained(this))));
+  unique_ptr<SysCatalogTable> new_catalog(new SysCatalogTable(
+      master_, Bind(&CatalogManager::ElectedAsLeaderCb, Unretained(this))));
   if (is_first_run) {
     RETURN_NOT_OK(new_catalog->CreateNew(master_->fs_manager()));
   } else {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 9261cf0..720e6ea 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -412,11 +412,18 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   //
   class ScopedLeaderSharedLock {
    public:
-    // Creates a new shared lock, acquiring the catalog manager's leader_lock_
-    // for reading in the process. The lock is released when this object is
-    // destroyed.
+    // Creates a new shared lock, trying to acquire the catalog manager's
+    // leader_lock_ for reading in the process. If acquired, the lock is
+    // released when this object is destroyed.
     //
-    // 'catalog' must outlive this object.
+    // In most common use cases, where write lock semantics is assumed, call
+    // CheckIsInitializedAndIsLeaderOrRespond() to verify that the leader_lock_
+    // has been acquired (as shown in the class-wide comment above). In rare
+    // cases, where both read and write semantics are applicable, use the
+    // combination of CheckIsInitializedOrRespond() and owns_lock() methods
+    // to verify that the leader_lock_ is acquired.
+    //
+    // The object pointed by the 'catalog' parameter must outlive this object.
     explicit ScopedLeaderSharedLock(CatalogManager* catalog);
 
     // General status of the catalog manager. If not OK (e.g. the catalog
@@ -440,6 +447,11 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
       return leader_status_;
     }
 
+    // Whether the underlying leader lock of the system catalog is acquired.
+    bool owns_lock() const {
+      return leader_shared_lock_.owns_lock();
+    }
+
     // Check whether the consensus configuration term has changed from the term
     // captured at object construction (initial_term_).
     // Requires: leader_status() returns OK().
@@ -660,6 +672,16 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // internal state of this object upon becoming the leader.
   void PrepareForLeadershipTask();
 
+  // Whether this catalog manager needs to prepare for running in the follower
+  // role.
+  bool NeedToPrepareFollower();
+
+  // Perform necessary work to prepare for running in the follower role.
+  // Currently, this includes reading the CA information from the system table,
+  // creating TLS server certificate request, signing it with the CA key,
+  // and installing the certificate TLS server certificates.
+  Status PrepareFollower();
+
   // Clears out the existing metadata ('table_names_map_', 'table_ids_map_',
   // and 'tablet_map_'), loads tables metadata into memory and if successful
   // loads the tablets metadata.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index e43522c..8e687f3 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -394,16 +394,16 @@ Status TlsContext::AdoptSignedCert(const Cert& cert) {
   SCOPED_OPENSSL_NO_PENDING_ERRORS;
   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));
-
   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;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/util/rw_mutex.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/rw_mutex.cc b/src/kudu/util/rw_mutex.cc
index 59139af..75fc1e7 100644
--- a/src/kudu/util/rw_mutex.cc
+++ b/src/kudu/util/rw_mutex.cc
@@ -140,6 +140,12 @@ bool RWMutex::TryWriteLock() {
 
 #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()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/util/rw_mutex.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/rw_mutex.h b/src/kudu/util/rw_mutex.h
index 84f77d2..bb6168c 100644
--- a/src/kudu/util/rw_mutex.h
+++ b/src/kudu/util/rw_mutex.h
@@ -65,9 +65,11 @@ class RWMutex {
   bool TryWriteLock();
 
 #ifndef NDEBUG
+  void AssertAcquired() const;
   void AssertAcquiredForReading() const;
   void AssertAcquiredForWriting() const;
 #else
+  void AssertAcquired() const {}
   void AssertAcquiredForReading() const {}
   void AssertAcquiredForWriting() const {}
 #endif


[2/3] kudu git commit: Upgrade to tcmalloc 2.6.3 and enabled sized-deallocation

Posted by al...@apache.org.
Upgrade to tcmalloc 2.6.3 and enabled sized-deallocation

This bumps to a new gperftools/tcmalloc version which ought to be a bit
faster. It also enables sized-deallocation support on compilers that
support it.

The upgrade is relatively straigthforward but for a few items:
- for whatever reason, we developed a conflict between our own
  base::SpinLock from gutil and tcmalloc's. This conflict has always
  existed, but apparently tcmalloc's implementation changed sizes
  slightly, meaning that now the linker complains about the duplicate
  symbol definition. I created a simple find-replace patch to change
  tcmalloc to use a 'tcmalloc::' namespace instead of 'base::'.
- our previous patches are all now included in tcmalloc upstream and no
  longer necessary.
- for unknown reasons, gperftools now fails to build with TSAN
  instrumentation: it shows duplicate symbols between the tcmalloc
  'operator new' and clang_rt's 'operator new'. We never actually needed
  to use gperftools in TSAN builds anyway, so this just removes the
  instrumented build and properly ifdefs out a few of the places where
  we'd included gperftools headers unconditionally.

Sized deallocation is a C++14 feature (which can be selectively enabled
on C++11) which adds a new 'operator delete(void*, size_t)'. In the
common case where we delete a Foo*, the compiler already knows
sizeof(Foo) and thus can pass this second argument to 'operator delete'.
This goes through an optimized code path inside tcmalloc -- namely, it
can skip the expensive lookup which otherwise would be required to map
the pointer back to its size class.

Unfortuantely, we can't always rely on the system allocator to provide
sized deallocation, and thus we can't use this in the exported variants
of our code (i.e. the client library). So, this patch only enables it
when building the Kudu binaries and not the client library.

I ran a quick performance check using full_stack-insert-scan-test twice
each before and after:

Before:
 Performance counter stats for 'build/latest/bin/full_stack-insert-scan-test --gtest_filter=*MRSOnlyStressTest* --inserts_per_client=200000 --concurrent_inserts=10 --rows_per_batch=1 --skip_scans':

     322623.135418      task-clock (msec)         #    4.964 CPUs utilized
        11,751,640      context-switches          #    0.036 M/sec
         3,619,010      cpu-migrations            #    0.011 M/sec
           117,706      page-faults               #    0.365 K/sec
 1,057,656,296,542      cycles                    #    3.278 GHz
   695,668,728,662      instructions              #    0.66  insn per cycle
   127,565,245,327      branches                  #  395.400 M/sec
     1,480,987,998      branch-misses             #    1.16% of all branches

      64.996506196 seconds time elapsed

     330206.351604      task-clock (msec)         #    5.330 CPUs utilized
        12,518,370      context-switches          #    0.038 M/sec
         3,695,790      cpu-migrations            #    0.011 M/sec
           113,580      page-faults               #    0.344 K/sec
 1,059,061,436,907      cycles                    #    3.207 GHz
   697,782,051,928      instructions              #    0.66  insn per cycle
   127,949,774,596      branches                  #  387.484 M/sec
     1,371,967,917      branch-misses             #    1.07% of all branches

      61.952217532 seconds time elapsed

After:

     310788.334202      task-clock (msec)         #    5.315 CPUs utilized
        12,486,276      context-switches          #    0.040 M/sec
         3,690,660      cpu-migrations            #    0.012 M/sec
           113,988      page-faults               #    0.367 K/sec
 1,027,033,932,375      cycles                    #    3.305 GHz
   663,508,168,985      instructions              #    0.65  insn per cycle
   125,864,966,052      branches                  #  404.986 M/sec
     1,442,683,495      branch-misses             #    1.15% of all branches

      58.479033228 seconds time elapsed

 Performance counter stats for 'build/latest/bin/full_stack-insert-scan-test.263 --gtest_filter=*MRSOnlyStressTest* --inserts_per_client=200000 --concurrent_inserts=10 --rows_per_batch=1 --skip_scans':

     317505.548908      task-clock (msec)         #    5.323 CPUs utilized
        12,461,003      context-switches          #    0.039 M/sec
         3,688,491      cpu-migrations            #    0.012 M/sec
           113,952      page-faults               #    0.359 K/sec
 1,029,595,155,391      cycles                    #    3.243 GHz
   663,514,269,656      instructions              #    0.64  insn per cycle
   125,865,138,975      branches                  #  396.419 M/sec
     1,419,717,877      branch-misses             #    1.13% of all branches

      59.642651558 seconds time elapsed

Seems like a small improvement in cycle count, wall clock, etc.

Change-Id: I3b76c9d71b29d58f0b6dade3fc33b32d0c3de23b
Reviewed-on: http://gerrit.cloudera.org:8080/9108
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@cloudera.com>


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

Branch: refs/heads/master
Commit: e0a743d24c8f46bbf0b921fc262dacc976d6cbb9
Parents: 9961902
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 23 11:57:05 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Jan 24 19:07:22 2018 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  |   23 +
 src/kudu/server/default_path_handlers.cc        |    2 +
 src/kudu/server/pprof_path_handlers.cc          |    2 +
 src/kudu/server/tcmalloc_metrics.cc             |    2 +
 src/kudu/util/mem_tracker.cc                    |    2 -
 src/kudu/util/process_memory.cc                 |    2 +
 thirdparty/build-thirdparty.sh                  |    4 -
 thirdparty/download-thirdparty.sh               |    6 +-
 ...efault-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch |   34 -
 ...namespace-base-with-namespace-tcmalloc.patch | 1670 ++++++++++++++++++
 ...nlock-on-OSX-instead-of-pthread_atfork.patch |   72 -
 ..._get_default_zone_to_osx_libc_override.patch |   48 -
 thirdparty/vars.sh                              |    2 +-
 13 files changed, 1704 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 1faab2d..e60170a 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -609,6 +609,13 @@ endif()
   # Exported variants may not use tcmalloc.
   set(EXPORTED_FLAGS "${EXPORTED_FLAGS} -UTCMALLOC_ENABLED")
 
+  # Exported variants should conform to the C++03 ABI, which doesn't
+  # include sized deallocation (new in C++14). This reverses the setting from
+  # non-exported (default) flags.
+  if(COMPILER_SUPPORTS_SIZED_DEALLOCATION)
+    set(EXPORTED_FLAGS "${EXPORTED_FLAGS} -fno-sized-deallocation")
+  endif()
+
   set_target_properties(${EXPORTED_LIB_NAME}
     PROPERTIES COMPILE_FLAGS "${ARG_COMPILE_FLAGS} ${EXPORTED_FLAGS}")
 
@@ -1072,6 +1079,22 @@ ADD_THIRDPARTY_LIB(boost_date_time
     SHARED_LIB "${BOOST_DATE_TIME_SHARED_LIB}")
 
 ############################################################
+# Enable sized deallocation where supported.
+# This happens down here instead of up with the rest of the
+# compiler options since we only do this when tcmalloc was
+# found and enabled.
+############################################################
+if (KUDU_TCMALLOC_AVAILABLE AND
+    (("${COMPILER_FAMILY}" STREQUAL "clang" AND
+      "${COMPILER_VERSION}" VERSION_GREATER "3.7") OR
+     ("${COMPILER_FAMILY}" STREQUAL "gcc" AND
+      "${COMPILER_VERSION}" VERSION_GREATER "5.0")))
+  set(COMPILER_SUPPORTS_SIZED_DEALLOCATION TRUE)
+  set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation")
+  message("Enabling sized-deallocation")
+endif()
+
+############################################################
 # Linker setup
 ############################################################
 set(KUDU_MIN_TEST_LIBS kudu_test_main kudu_test_util ${KUDU_BASE_LIBS})

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/src/kudu/server/default_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/default_path_handlers.cc b/src/kudu/server/default_path_handlers.cc
index 4db825d..8322017 100644
--- a/src/kudu/server/default_path_handlers.cc
+++ b/src/kudu/server/default_path_handlers.cc
@@ -34,7 +34,9 @@
 #include <gflags/gflags.h>
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#ifdef TCMALLOC_ENABLED
 #include <gperftools/malloc_extension.h>
+#endif
 
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/src/kudu/server/pprof_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/pprof_path_handlers.cc b/src/kudu/server/pprof_path_handlers.cc
index 1425941..040daa0 100644
--- a/src/kudu/server/pprof_path_handlers.cc
+++ b/src/kudu/server/pprof_path_handlers.cc
@@ -29,9 +29,11 @@
 
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#ifdef TCMALLOC_ENABLED
 #include <gperftools/heap-profiler.h>
 #include <gperftools/malloc_extension.h>
 #include <gperftools/profiler.h>
+#endif
 
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/numbers.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/src/kudu/server/tcmalloc_metrics.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/tcmalloc_metrics.cc b/src/kudu/server/tcmalloc_metrics.cc
index d6722b6..3c84e9b 100644
--- a/src/kudu/server/tcmalloc_metrics.cc
+++ b/src/kudu/server/tcmalloc_metrics.cc
@@ -21,7 +21,9 @@
 #include <ostream>
 
 #include <glog/logging.h>
+#ifdef TCMALLOC_ENABLED
 #include <gperftools/malloc_extension.h>
+#endif
 
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/bind_helpers.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/src/kudu/util/mem_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/mem_tracker.cc b/src/kudu/util/mem_tracker.cc
index a074354..4cf2866 100644
--- a/src/kudu/util/mem_tracker.cc
+++ b/src/kudu/util/mem_tracker.cc
@@ -25,8 +25,6 @@
 #include <memory>
 #include <ostream>
 
-#include <gperftools/malloc_extension.h>  // IWYU pragma: keep
-
 #include "kudu/gutil/once.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/src/kudu/util/process_memory.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/process_memory.cc b/src/kudu/util/process_memory.cc
index 090e012..224a656 100644
--- a/src/kudu/util/process_memory.cc
+++ b/src/kudu/util/process_memory.cc
@@ -21,7 +21,9 @@
 
 #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"

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/build-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index c570f9e..f91bc1f 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -505,10 +505,6 @@ if [ -n "$F_TSAN" -o -n "$F_GLOG" ]; then
   build_glog
 fi
 
-if [ -n "$F_TSAN" -o -n "$F_GPERFTOOLS" ]; then
-  build_gperftools
-fi
-
 if [ -n "$F_TSAN" -o -n "$F_GMOCK" ]; then
   build_gmock
 fi

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/download-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh
index decb2be..d699429 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -124,15 +124,13 @@ if [ ! -d $GFLAGS_SOURCE ]; then
   fetch_and_expand gflags-${GFLAGS_VERSION}.tar.gz
 fi
 
-GPERFTOOLS_PATCHLEVEL=3
+GPERFTOOLS_PATCHLEVEL=1
 delete_if_wrong_patchlevel $GPERFTOOLS_SOURCE $GPERFTOOLS_PATCHLEVEL
 if [ ! -d $GPERFTOOLS_SOURCE ]; then
   fetch_and_expand gperftools-${GPERFTOOLS_VERSION}.tar.gz
 
   pushd $GPERFTOOLS_SOURCE
-  patch -p1 < $TP_DIR/patches/gperftools-Change-default-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch
-  patch -p1 < $TP_DIR/patches/gperftools-hook-mi_force_unlock-on-OSX-instead-of-pthread_atfork.patch
-  patch -p1 < $TP_DIR/patches/gperftools-issue-827-add_get_default_zone_to_osx_libc_override.patch
+  patch -p1 < $TP_DIR/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch
   touch patchlevel-$GPERFTOOLS_PATCHLEVEL
   autoreconf -fvi
   popd

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/patches/gperftools-Change-default-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/gperftools-Change-default-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch b/thirdparty/patches/gperftools-Change-default-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch
deleted file mode 100644
index 353df16..0000000
--- a/thirdparty/patches/gperftools-Change-default-TCMALLOC_TRANSFER_NUM_OBJ-to-40.patch
+++ /dev/null
@@ -1,34 +0,0 @@
-From a36aea827b76b19c82b780de0762da30c230f19f Mon Sep 17 00:00:00 2001
-From: Todd Lipcon <to...@cloudera.com>
-Date: Tue, 23 Sep 2014 19:09:56 -0700
-Subject: [PATCH] Change default TCMALLOC_TRANSFER_NUM_OBJ to 40
-
-We found that this configuration was preventing good performance
-of the central freelist in benchmarks with a lot of threads.
-
-After discussion with Aliaksey Kandratsenka (the gperftools maintainer)
-we decided the best course of action was to revert to a number close to the old
-default, which seems to improve the performance of these high-concurrency
-benchmarks by about 4.5x.
-
-Any higher numbers seem to produce worse performance in our benchmarks.
----
- src/common.cc | 2 +-
- 1 file changed, 1 insertion(+), 1 deletion(-)
-
-diff --git a/src/common.cc b/src/common.cc
-index 9d48377..c2ad054 100644
---- a/src/common.cc
-+++ b/src/common.cc
-@@ -42,7 +42,7 @@ namespace tcmalloc {
- // thread and central caches.
- static int32 FLAGS_tcmalloc_transfer_num_objects;
- 
--static const int32 kDefaultTransferNumObjecs = 32768;
-+static const int32 kDefaultTransferNumObjecs = 40;
- 
- // The init function is provided to explicit initialize the variable value
- // from the env. var to avoid C++ global construction that might defer its
--- 
-1.8.3.2
-

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch b/thirdparty/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch
new file mode 100644
index 0000000..8660410
--- /dev/null
+++ b/thirdparty/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch
@@ -0,0 +1,1670 @@
+From c63fd16547503fb9d9b0a3e35e16a1a66fbfeaa0 Mon Sep 17 00:00:00 2001
+From: Todd Lipcon <to...@cloudera.com>
+Date: Tue, 23 Jan 2018 11:25:50 -0800
+Subject: [PATCH 1/1] Replace namespace base with namespace tcmalloc
+
+---
+ src/base/atomicops-internals-arm-generic.h |  6 +-
+ src/base/atomicops-internals-arm-v6plus.h  |  4 +-
+ src/base/atomicops-internals-gcc.h         |  6 +-
+ src/base/atomicops-internals-linuxppc.h    |  6 +-
+ src/base/atomicops-internals-macosx.h      |  8 +--
+ src/base/atomicops-internals-mips.h        |  6 +-
+ src/base/atomicops-internals-windows.h     | 14 ++---
+ src/base/atomicops-internals-x86.h         |  6 +-
+ src/base/atomicops.h                       | 92 +++++++++++++++---------------
+ src/base/basictypes.h                      |  8 +--
+ src/base/elf_mem_image.cc                  |  4 +-
+ src/base/elf_mem_image.h                   |  4 +-
+ src/base/spinlock.cc                       | 16 +++---
+ src/base/spinlock.h                        | 14 ++---
+ src/base/spinlock_internal.cc              | 18 +++---
+ src/base/spinlock_internal.h               |  4 +-
+ src/base/spinlock_linux-inl.h              |  6 +-
+ src/base/spinlock_posix-inl.h              |  6 +-
+ src/base/spinlock_win32-inl.h              |  6 +-
+ src/base/vdso_support.cc                   |  4 +-
+ src/base/vdso_support.h                    |  4 +-
+ src/central_freelist.h                     |  2 +-
+ src/emergency_malloc.cc                    |  2 +-
+ src/gperftools/malloc_extension.h          |  8 +--
+ src/heap-profile-table.cc                  |  2 +-
+ src/internal_logging.cc                    |  2 +-
+ src/malloc_hook-inl.h                      | 44 +++++++-------
+ src/malloc_hook.cc                         | 60 +++++++++----------
+ src/page_heap.cc                           | 10 ++--
+ src/page_heap.h                            |  4 +-
+ src/raw_printer.cc                         |  2 +-
+ src/raw_printer.h                          |  2 +-
+ src/stacktrace_powerpc-linux-inl.h         |  4 +-
+ src/stacktrace_x86-inl.h                   |  6 +-
+ src/tcmalloc.cc                            | 10 ++--
+ src/tests/atomicops_unittest.cc            | 36 ++++++------
+ src/tests/malloc_hook_test.cc              |  4 +-
+ src/tests/raw_printer_test.cc              |  8 +--
+ src/tests/tcmalloc_unittest.cc             | 28 ++++-----
+ 39 files changed, 238 insertions(+), 238 deletions(-)
+
+diff --git a/src/base/atomicops-internals-arm-generic.h b/src/base/atomicops-internals-arm-generic.h
+index d0f9413..c81f1e6 100644
+--- a/src/base/atomicops-internals-arm-generic.h
++++ b/src/base/atomicops-internals-arm-generic.h
+@@ -44,7 +44,7 @@
+ 
+ typedef int32_t Atomic32;
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ typedef int64_t Atomic64;
+@@ -222,7 +222,7 @@ inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,
+   return 0;
+ }
+ 
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_ARM_GENERIC_H_
+diff --git a/src/base/atomicops-internals-arm-v6plus.h b/src/base/atomicops-internals-arm-v6plus.h
+index 35f1048..c61ef24 100644
+--- a/src/base/atomicops-internals-arm-v6plus.h
++++ b/src/base/atomicops-internals-arm-v6plus.h
+@@ -52,7 +52,7 @@
+ 
+ typedef int32_t Atomic32;
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ typedef int64_t Atomic64;
+@@ -325,6 +325,6 @@ inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,
+ }
+ 
+ }  // namespace subtle ends
+-}  // namespace base ends
++}  // namespace tcmalloc ends
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_ARM_V6PLUS_H_
+diff --git a/src/base/atomicops-internals-gcc.h b/src/base/atomicops-internals-gcc.h
+index f8d2786..d633579 100644
+--- a/src/base/atomicops-internals-gcc.h
++++ b/src/base/atomicops-internals-gcc.h
+@@ -44,7 +44,7 @@
+ 
+ typedef int32_t Atomic32;
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ typedef int64_t Atomic64;
+@@ -197,7 +197,7 @@ inline Atomic64 Release_Load(volatile const Atomic64* ptr) {
+   return *ptr;
+ }
+ 
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_GCC_GENERIC_H_
+diff --git a/src/base/atomicops-internals-linuxppc.h b/src/base/atomicops-internals-linuxppc.h
+index b52fdf0..f174bf4 100644
+--- a/src/base/atomicops-internals-linuxppc.h
++++ b/src/base/atomicops-internals-linuxppc.h
+@@ -44,7 +44,7 @@ typedef int32_t Atomic32;
+ #define BASE_HAS_ATOMIC64 1
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ static inline void _sync(void) {
+@@ -431,7 +431,7 @@ inline Atomic64 Release_Load(volatile const Atomic64 *ptr) {
+ 
+ #endif
+ 
+-}   // namespace base::subtle
+-}   // namespace base
++}   // namespace tcmalloc::subtle
++}   // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_LINUXPPC_H_
+diff --git a/src/base/atomicops-internals-macosx.h b/src/base/atomicops-internals-macosx.h
+index b5130d4..2c53646 100644
+--- a/src/base/atomicops-internals-macosx.h
++++ b/src/base/atomicops-internals-macosx.h
+@@ -43,7 +43,7 @@ typedef int32_t Atomic32;
+ // AtomicWord and Atomic64 are always different.  Thus, we need explicit
+ // casting.
+ #ifdef __LP64__
+-#define AtomicWordCastType base::subtle::Atomic64
++#define AtomicWordCastType tcmalloc::subtle::Atomic64
+ #else
+ #define AtomicWordCastType Atomic32
+ #endif
+@@ -54,7 +54,7 @@ typedef int32_t Atomic32;
+ 
+ #include <libkern/OSAtomic.h>
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ #if !defined(__LP64__) && defined(__ppc__)
+@@ -364,7 +364,7 @@ inline Atomic64 Release_Load(volatile const Atomic64 *ptr) {
+ }
+ #endif  // __LP64__
+ 
+-}   // namespace base::subtle
+-}   // namespace base
++}   // namespace tcmalloc::subtle
++}   // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_MACOSX_H_
+diff --git a/src/base/atomicops-internals-mips.h b/src/base/atomicops-internals-mips.h
+index 4bfd7f6..612abb4 100644
+--- a/src/base/atomicops-internals-mips.h
++++ b/src/base/atomicops-internals-mips.h
+@@ -45,7 +45,7 @@
+ 
+ typedef int32_t Atomic32;
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ // Atomically execute:
+@@ -317,7 +317,7 @@ inline Atomic64 Release_Load(volatile const Atomic64* ptr)
+ 
+ #endif
+ 
+-}   // namespace base::subtle
+-}   // namespace base
++}   // namespace tcmalloc::subtle
++}   // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_MIPS_H_
+diff --git a/src/base/atomicops-internals-windows.h b/src/base/atomicops-internals-windows.h
+index 93ced87..4331b71 100644
+--- a/src/base/atomicops-internals-windows.h
++++ b/src/base/atomicops-internals-windows.h
+@@ -49,7 +49,7 @@ typedef int32 Atomic32;
+ #define BASE_HAS_ATOMIC64 1  // Use only in tests and base/atomic*
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ typedef int64 Atomic64;
+@@ -150,8 +150,8 @@ inline Atomic32 Release_AtomicExchange(volatile Atomic32* ptr,
+   return NoBarrier_AtomicExchange(ptr, new_value);
+ }
+ 
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ 
+ 
+ // In msvc8/vs2005, winnt.h already contains a definition for
+@@ -160,12 +160,12 @@ inline Atomic32 Release_AtomicExchange(volatile Atomic32* ptr,
+ #if !(defined(_MSC_VER) && _MSC_VER >= 1400)
+ inline void MemoryBarrier() {
+   Atomic32 value = 0;
+-  base::subtle::NoBarrier_AtomicExchange(&value, 0);
++  tcmalloc::subtle::NoBarrier_AtomicExchange(&value, 0);
+                         // actually acts as a barrier in thisd implementation
+ }
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ inline void MemoryBarrier() {
+@@ -451,7 +451,7 @@ inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,
+   return NoBarrier_CompareAndSwap(ptr, old_value, new_value);
+ }
+ 
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ 
+ #endif  // BASE_ATOMICOPS_INTERNALS_WINDOWS_H_
+diff --git a/src/base/atomicops-internals-x86.h b/src/base/atomicops-internals-x86.h
+index e441ac7..32751dc 100644
+--- a/src/base/atomicops-internals-x86.h
++++ b/src/base/atomicops-internals-x86.h
+@@ -64,7 +64,7 @@ extern struct AtomicOps_x86CPUFeatureStruct AtomicOps_Internalx86CPUFeatures;
+ #define ATOMICOPS_COMPILER_BARRIER() __asm__ __volatile__("" : : : "memory")
+ 
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ typedef int64_t Atomic64;
+@@ -383,8 +383,8 @@ inline Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr,
+   return NoBarrier_CompareAndSwap(ptr, old_value, new_value);
+ }
+ 
+-} // namespace base::subtle
+-} // namespace base
++} // namespace tcmalloc::subtle
++} // namespace tcmalloc
+ 
+ #undef ATOMICOPS_COMPILER_BARRIER
+ 
+diff --git a/src/base/atomicops.h b/src/base/atomicops.h
+index dac95be..069a495 100644
+--- a/src/base/atomicops.h
++++ b/src/base/atomicops.h
+@@ -67,12 +67,12 @@
+ //   NoBarrier_Store()
+ //   NoBarrier_Load()
+ // Although there are currently no compiler enforcement, you are encouraged
+-// to use these.  Moreover, if you choose to use base::subtle::Atomic64 type,
++// to use these.  Moreover, if you choose to use tcmalloc::subtle::Atomic64 type,
+ // you MUST use one of the Load or Store routines to get correct behavior
+ // on 32-bit platforms.
+ //
+ // The intent is eventually to put all of these routines in namespace
+-// base::subtle
++// tcmalloc::subtle
+ 
+ #ifndef THREAD_ATOMICOPS_H_
+ #define THREAD_ATOMICOPS_H_
+@@ -90,7 +90,7 @@
+ // should define the macro, AtomicWordCastType in a clause similar to the
+ // following:
+ // #if ...pointers are 64 bits...
+-// # define AtomicWordCastType base::subtle::Atomic64
++// # define AtomicWordCastType tcmalloc::subtle::Atomic64
+ // #else
+ // # define AtomicWordCastType Atomic32
+ // #endif
+@@ -143,7 +143,7 @@ typedef intptr_t AtomicWord;
+ // It also serves to document the AtomicWord interface.
+ // ------------------------------------------------------------------------
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ // Atomically execute:
+@@ -187,7 +187,7 @@ inline AtomicWord Release_AtomicExchange(volatile AtomicWord* ptr,
+ inline AtomicWord Acquire_CompareAndSwap(volatile AtomicWord* ptr,
+                                          AtomicWord old_value,
+                                          AtomicWord new_value) {
+-  return base::subtle::Acquire_CompareAndSwap(
++  return tcmalloc::subtle::Acquire_CompareAndSwap(
+       reinterpret_cast<volatile AtomicWordCastType*>(ptr),
+       old_value, new_value);
+ }
+@@ -195,7 +195,7 @@ inline AtomicWord Acquire_CompareAndSwap(volatile AtomicWord* ptr,
+ inline AtomicWord Release_CompareAndSwap(volatile AtomicWord* ptr,
+                                          AtomicWord old_value,
+                                          AtomicWord new_value) {
+-  return base::subtle::Release_CompareAndSwap(
++  return tcmalloc::subtle::Release_CompareAndSwap(
+       reinterpret_cast<volatile AtomicWordCastType*>(ptr),
+       old_value, new_value);
+ }
+@@ -206,12 +206,12 @@ inline void NoBarrier_Store(volatile AtomicWord *ptr, AtomicWord value) {
+ }
+ 
+ inline void Acquire_Store(volatile AtomicWord* ptr, AtomicWord value) {
+-  return base::subtle::Acquire_Store(
++  return tcmalloc::subtle::Acquire_Store(
+       reinterpret_cast<volatile AtomicWordCastType*>(ptr), value);
+ }
+ 
+ inline void Release_Store(volatile AtomicWord* ptr, AtomicWord value) {
+-  return base::subtle::Release_Store(
++  return tcmalloc::subtle::Release_Store(
+       reinterpret_cast<volatile AtomicWordCastType*>(ptr), value);
+ }
+ 
+@@ -221,17 +221,17 @@ inline AtomicWord NoBarrier_Load(volatile const AtomicWord *ptr) {
+ }
+ 
+ inline AtomicWord Acquire_Load(volatile const AtomicWord* ptr) {
+-  return base::subtle::Acquire_Load(
++  return tcmalloc::subtle::Acquire_Load(
+       reinterpret_cast<volatile const AtomicWordCastType*>(ptr));
+ }
+ 
+ inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {
+-  return base::subtle::Release_Load(
++  return tcmalloc::subtle::Release_Load(
+       reinterpret_cast<volatile const AtomicWordCastType*>(ptr));
+ }
+ 
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ #endif  // AtomicWordCastType
+ 
+ // ------------------------------------------------------------------------
+@@ -247,7 +247,7 @@ inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {
+ typedef int32_t Atomic32;
+ 
+ // Corresponding operations on Atomic32
+-namespace base {
++namespace tcmalloc {
+ namespace subtle {
+ 
+ // Signed 64-bit type that supports the atomic ops below, as well as atomic
+@@ -294,8 +294,8 @@ void Release_Store(volatile Atomic64* ptr, Atomic64 value);
+ Atomic64 NoBarrier_Load(volatile const Atomic64* ptr);
+ Atomic64 Acquire_Load(volatile const Atomic64* ptr);
+ Atomic64 Release_Load(volatile const Atomic64* ptr);
+-}  // namespace base::subtle
+-}  // namespace base
++}  // namespace tcmalloc::subtle
++}  // namespace tcmalloc
+ 
+ void MemoryBarrier();
+ 
+@@ -304,7 +304,7 @@ void MemoryBarrier();
+ 
+ // ------------------------------------------------------------------------
+ // The following are to be deprecated when all uses have been changed to
+-// use the base::subtle namespace.
++// use the tcmalloc::subtle namespace.
+ // ------------------------------------------------------------------------
+ 
+ #ifdef AtomicWordCastType
+@@ -312,29 +312,29 @@ void MemoryBarrier();
+ inline AtomicWord Acquire_CompareAndSwap(volatile AtomicWord* ptr,
+                                          AtomicWord old_value,
+                                          AtomicWord new_value) {
+-  return base::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
++  return tcmalloc::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
+ }
+ 
+ inline AtomicWord Release_CompareAndSwap(volatile AtomicWord* ptr,
+                                          AtomicWord old_value,
+                                          AtomicWord new_value) {
+-  return base::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
++  return tcmalloc::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
+ }
+ 
+ inline void Acquire_Store(volatile AtomicWord* ptr, AtomicWord value) {
+-  return base::subtle::Acquire_Store(ptr, value);
++  return tcmalloc::subtle::Acquire_Store(ptr, value);
+ }
+ 
+ inline void Release_Store(volatile AtomicWord* ptr, AtomicWord value) {
+-  return base::subtle::Release_Store(ptr, value);
++  return tcmalloc::subtle::Release_Store(ptr, value);
+ }
+ 
+ inline AtomicWord Acquire_Load(volatile const AtomicWord* ptr) {
+-  return base::subtle::Acquire_Load(ptr);
++  return tcmalloc::subtle::Acquire_Load(ptr);
+ }
+ 
+ inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {
+-  return base::subtle::Release_Load(ptr);
++  return tcmalloc::subtle::Release_Load(ptr);
+ }
+ #endif  // AtomicWordCastType
+ 
+@@ -343,55 +343,55 @@ inline AtomicWord Release_Load(volatile const AtomicWord* ptr) {
+ inline Atomic32 Acquire_CompareAndSwap(volatile Atomic32* ptr,
+                                        Atomic32 old_value,
+                                        Atomic32 new_value) {
+-  return base::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
++  return tcmalloc::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
+ }
+ inline Atomic32 Release_CompareAndSwap(volatile Atomic32* ptr,
+                                        Atomic32 old_value,
+                                        Atomic32 new_value) {
+-  return base::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
++  return tcmalloc::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
+ }
+ inline void Acquire_Store(volatile Atomic32* ptr, Atomic32 value) {
+-  base::subtle::Acquire_Store(ptr, value);
++  tcmalloc::subtle::Acquire_Store(ptr, value);
+ }
+ inline void Release_Store(volatile Atomic32* ptr, Atomic32 value) {
+-  return base::subtle::Release_Store(ptr, value);
++  return tcmalloc::subtle::Release_Store(ptr, value);
+ }
+ inline Atomic32 Acquire_Load(volatile const Atomic32* ptr) {
+-  return base::subtle::Acquire_Load(ptr);
++  return tcmalloc::subtle::Acquire_Load(ptr);
+ }
+ inline Atomic32 Release_Load(volatile const Atomic32* ptr) {
+-  return base::subtle::Release_Load(ptr);
++  return tcmalloc::subtle::Release_Load(ptr);
+ }
+ 
+ #ifdef BASE_HAS_ATOMIC64
+ 
+ // 64-bit Acquire/Release operations to be deprecated.
+ 
+-inline base::subtle::Atomic64 Acquire_CompareAndSwap(
+-    volatile base::subtle::Atomic64* ptr,
+-    base::subtle::Atomic64 old_value, base::subtle::Atomic64 new_value) {
+-  return base::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
++inline tcmalloc::subtle::Atomic64 Acquire_CompareAndSwap(
++    volatile tcmalloc::subtle::Atomic64* ptr,
++    tcmalloc::subtle::Atomic64 old_value, tcmalloc::subtle::Atomic64 new_value) {
++  return tcmalloc::subtle::Acquire_CompareAndSwap(ptr, old_value, new_value);
+ }
+-inline base::subtle::Atomic64 Release_CompareAndSwap(
+-    volatile base::subtle::Atomic64* ptr,
+-    base::subtle::Atomic64 old_value, base::subtle::Atomic64 new_value) {
+-  return base::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
++inline tcmalloc::subtle::Atomic64 Release_CompareAndSwap(
++    volatile tcmalloc::subtle::Atomic64* ptr,
++    tcmalloc::subtle::Atomic64 old_value, tcmalloc::subtle::Atomic64 new_value) {
++  return tcmalloc::subtle::Release_CompareAndSwap(ptr, old_value, new_value);
+ }
+ inline void Acquire_Store(
+-    volatile base::subtle::Atomic64* ptr, base::subtle::Atomic64 value) {
+-  base::subtle::Acquire_Store(ptr, value);
++    volatile tcmalloc::subtle::Atomic64* ptr, tcmalloc::subtle::Atomic64 value) {
++  tcmalloc::subtle::Acquire_Store(ptr, value);
+ }
+ inline void Release_Store(
+-    volatile base::subtle::Atomic64* ptr, base::subtle::Atomic64 value) {
+-  return base::subtle::Release_Store(ptr, value);
++    volatile tcmalloc::subtle::Atomic64* ptr, tcmalloc::subtle::Atomic64 value) {
++  return tcmalloc::subtle::Release_Store(ptr, value);
+ }
+-inline base::subtle::Atomic64 Acquire_Load(
+-    volatile const base::subtle::Atomic64* ptr) {
+-  return base::subtle::Acquire_Load(ptr);
++inline tcmalloc::subtle::Atomic64 Acquire_Load(
++    volatile const tcmalloc::subtle::Atomic64* ptr) {
++  return tcmalloc::subtle::Acquire_Load(ptr);
+ }
+-inline base::subtle::Atomic64 Release_Load(
+-    volatile const base::subtle::Atomic64* ptr) {
+-  return base::subtle::Release_Load(ptr);
++inline tcmalloc::subtle::Atomic64 Release_Load(
++    volatile const tcmalloc::subtle::Atomic64* ptr) {
++  return tcmalloc::subtle::Release_Load(ptr);
+ }
+ 
+ #endif  // BASE_HAS_ATOMIC64
+diff --git a/src/base/basictypes.h b/src/base/basictypes.h
+index 42dbe5c..b825290 100644
+--- a/src/base/basictypes.h
++++ b/src/base/basictypes.h
+@@ -421,15 +421,15 @@ union MemoryAligner {
+ // that the variable has static storage class, and that the constructor should
+ // do nothing to its state.  It indicates to the reader that it is legal to
+ // declare a static nistance of the class, provided the constructor is given
+-// the base::LINKER_INITIALIZED argument.  Normally, it is unsafe to declare a
++// the tcmalloc::LINKER_INITIALIZED argument.  Normally, it is unsafe to declare a
+ // static variable that has a constructor or a destructor because invocation
+ // order is undefined.  However, IF the type can be initialized by filling with
+ // zeroes (which the loader does for static variables), AND the destructor also
+ // does nothing to the storage, then a constructor declared as
+-//       explicit MyClass(base::LinkerInitialized x) {}
++//       explicit MyClass(tcmalloc::LinkerInitialized x) {}
+ // and invoked as
+-//       static MyClass my_variable_name(base::LINKER_INITIALIZED);
+-namespace base {
++//       static MyClass my_variable_name(tcmalloc::LINKER_INITIALIZED);
++namespace tcmalloc {
+ enum LinkerInitialized { LINKER_INITIALIZED };
+ }
+ 
+diff --git a/src/base/elf_mem_image.cc b/src/base/elf_mem_image.cc
+index d2ca1a5..fdfab4c 100644
+--- a/src/base/elf_mem_image.cc
++++ b/src/base/elf_mem_image.cc
+@@ -54,7 +54,7 @@
+ 
+ #define VERSYM_VERSION 0x7fff
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ namespace {
+ template <int N> class ElfClass {
+@@ -429,6 +429,6 @@ void ElfMemImage::SymbolIterator::Update(int increment) {
+   info_.symbol  = symbol;
+ }
+ 
+-}  // namespace base
++}  // namespace tcmalloc
+ 
+ #endif  // HAVE_ELF_MEM_IMAGE
+diff --git a/src/base/elf_mem_image.h b/src/base/elf_mem_image.h
+index 5fb00ff..4fef045 100644
+--- a/src/base/elf_mem_image.h
++++ b/src/base/elf_mem_image.h
+@@ -50,7 +50,7 @@
+ #include <stdlib.h>
+ #include <link.h>  // for ElfW
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ // An in-memory ELF image (may not exist on disk).
+ class ElfMemImage {
+@@ -128,7 +128,7 @@ class ElfMemImage {
+   ElfW(Addr) link_base_;     // Link-time base (p_vaddr of first PT_LOAD).
+ };
+ 
+-}  // namespace base
++}  // namespace tcmalloc
+ 
+ #endif  // __ELF__ and __GLIBC__ and !__native_client__
+ 
+diff --git a/src/base/spinlock.cc b/src/base/spinlock.cc
+index 85ff21e..6fbf226 100644
+--- a/src/base/spinlock.cc
++++ b/src/base/spinlock.cc
+@@ -45,8 +45,8 @@
+ 
+ static int adaptive_spin_count = 0;
+ 
+-const base::LinkerInitialized SpinLock::LINKER_INITIALIZED =
+-    base::LINKER_INITIALIZED;
++const tcmalloc::LinkerInitialized SpinLock::LINKER_INITIALIZED =
++    tcmalloc::LINKER_INITIALIZED;
+ 
+ namespace {
+ struct SpinLock_InitHelper {
+@@ -77,10 +77,10 @@ inline void SpinlockPause(void) {
+ // from the lock is returned from the method.
+ Atomic32 SpinLock::SpinLoop() {
+   int c = adaptive_spin_count;
+-  while (base::subtle::NoBarrier_Load(&lockword_) != kSpinLockFree && --c > 0) {
++  while (tcmalloc::subtle::NoBarrier_Load(&lockword_) != kSpinLockFree && --c > 0) {
+     SpinlockPause();
+   }
+-  return base::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
++  return tcmalloc::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
+                                               kSpinLockSleeper);
+ }
+ 
+@@ -95,7 +95,7 @@ void SpinLock::SlowLock() {
+       // Here, just "mark" that the thread is going to sleep.  Don't store the
+       // lock wait time in the lock as that will cause the current lock
+       // owner to think it experienced contention.
+-      lock_value = base::subtle::Acquire_CompareAndSwap(&lockword_,
++      lock_value = tcmalloc::subtle::Acquire_CompareAndSwap(&lockword_,
+                                                         kSpinLockHeld,
+                                                         kSpinLockSleeper);
+       if (lock_value == kSpinLockHeld) {
+@@ -107,7 +107,7 @@ void SpinLock::SlowLock() {
+         // Lock is free again, so try and acquire it before sleeping.  The
+         // new lock state will be the number of cycles this thread waited if
+         // this thread obtains the lock.
+-        lock_value = base::subtle::Acquire_CompareAndSwap(&lockword_,
++        lock_value = tcmalloc::subtle::Acquire_CompareAndSwap(&lockword_,
+                                                           kSpinLockFree,
+                                                           kSpinLockSleeper);
+         continue;  // skip the delay at the end of the loop
+@@ -115,7 +115,7 @@ void SpinLock::SlowLock() {
+     }
+ 
+     // Wait for an OS specific delay.
+-    base::internal::SpinLockDelay(&lockword_, lock_value,
++    tcmalloc::internal::SpinLockDelay(&lockword_, lock_value,
+                                   ++lock_wait_call_count);
+     // Spin again after returning from the wait routine to give this thread
+     // some chance of obtaining the lock.
+@@ -125,5 +125,5 @@ void SpinLock::SlowLock() {
+ 
+ void SpinLock::SlowUnlock() {
+   // wake waiter if necessary
+-  base::internal::SpinLockWake(&lockword_, false);
++  tcmalloc::internal::SpinLockWake(&lockword_, false);
+ }
+diff --git a/src/base/spinlock.h b/src/base/spinlock.h
+index 7243aea..3bf3ad9 100644
+--- a/src/base/spinlock.h
++++ b/src/base/spinlock.h
+@@ -51,14 +51,14 @@ class LOCKABLE SpinLock {
+ 
+   // Special constructor for use with static SpinLock objects.  E.g.,
+   //
+-  //    static SpinLock lock(base::LINKER_INITIALIZED);
++  //    static SpinLock lock(tcmalloc::LINKER_INITIALIZED);
+   //
+   // When intialized using this constructor, we depend on the fact
+   // that the linker has already initialized the memory appropriately.
+   // A SpinLock constructed like this can be freely used from global
+   // initializers without worrying about the order in which global
+   // initializers run.
+-  explicit SpinLock(base::LinkerInitialized /*x*/) {
++  explicit SpinLock(tcmalloc::LinkerInitialized /*x*/) {
+     // Does nothing; lockword_ is already initialized
+   }
+ 
+@@ -66,7 +66,7 @@ class LOCKABLE SpinLock {
+   // TODO(csilvers): uncomment the annotation when we figure out how to
+   //                 support this macro with 0 args (see thread_annotations.h)
+   inline void Lock() /*EXCLUSIVE_LOCK_FUNCTION()*/ {
+-    if (base::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
++    if (tcmalloc::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
+                                              kSpinLockHeld) != kSpinLockFree) {
+       SlowLock();
+     }
+@@ -79,7 +79,7 @@ class LOCKABLE SpinLock {
+   // will return true with high probability.
+   inline bool TryLock() EXCLUSIVE_TRYLOCK_FUNCTION(true) {
+     bool res =
+-        (base::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
++        (tcmalloc::subtle::Acquire_CompareAndSwap(&lockword_, kSpinLockFree,
+                                               kSpinLockHeld) == kSpinLockFree);
+     if (res) {
+       ANNOTATE_RWLOCK_ACQUIRED(this, 1);
+@@ -93,7 +93,7 @@ class LOCKABLE SpinLock {
+   inline void Unlock() /*UNLOCK_FUNCTION()*/ {
+     ANNOTATE_RWLOCK_RELEASED(this, 1);
+     uint64 prev_value = static_cast<uint64>(
+-        base::subtle::Release_AtomicExchange(&lockword_, kSpinLockFree));
++        tcmalloc::subtle::Release_AtomicExchange(&lockword_, kSpinLockFree));
+     if (prev_value != kSpinLockHeld) {
+       // Speed the wakeup of any waiter.
+       SlowUnlock();
+@@ -104,10 +104,10 @@ class LOCKABLE SpinLock {
+   // thread, true will always be returned. Intended to be used as
+   // CHECK(lock.IsHeld()).
+   inline bool IsHeld() const {
+-    return base::subtle::NoBarrier_Load(&lockword_) != kSpinLockFree;
++    return tcmalloc::subtle::NoBarrier_Load(&lockword_) != kSpinLockFree;
+   }
+ 
+-  static const base::LinkerInitialized LINKER_INITIALIZED;  // backwards compat
++  static const tcmalloc::LinkerInitialized LINKER_INITIALIZED;  // backwards compat
+  private:
+   enum { kSpinLockFree = 0 };
+   enum { kSpinLockHeld = 1 };
+diff --git a/src/base/spinlock_internal.cc b/src/base/spinlock_internal.cc
+index d962971..af6015d 100644
+--- a/src/base/spinlock_internal.cc
++++ b/src/base/spinlock_internal.cc
+@@ -30,7 +30,7 @@
+  */
+ 
+ // The OS-specific header included below must provide two calls:
+-// base::internal::SpinLockDelay() and base::internal::SpinLockWake().
++// tcmalloc::internal::SpinLockDelay() and tcmalloc::internal::SpinLockWake().
+ // See spinlock_internal.h for the spec of SpinLockWake().
+ 
+ // void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop)
+@@ -44,7 +44,7 @@
+ #include "base/spinlock_internal.h"
+ 
+ // forward declaration for use by spinlock_*-inl.h
+-namespace base { namespace internal { static int SuggestedDelayNS(int loop); }}
++namespace tcmalloc { namespace internal { static int SuggestedDelayNS(int loop); }}
+ 
+ #if defined(_WIN32)
+ #include "base/spinlock_win32-inl.h"
+@@ -54,7 +54,7 @@ namespace base { namespace internal { static int SuggestedDelayNS(int loop); }}
+ #include "base/spinlock_posix-inl.h"
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace internal {
+ 
+ // Return a suggested delay in nanoseconds for iteration number "loop"
+@@ -62,10 +62,10 @@ static int SuggestedDelayNS(int loop) {
+   // Weak pseudo-random number generator to get some spread between threads
+   // when many are spinning.
+ #ifdef BASE_HAS_ATOMIC64
+-  static base::subtle::Atomic64 rand;
+-  uint64 r = base::subtle::NoBarrier_Load(&rand);
++  static tcmalloc::subtle::Atomic64 rand;
++  uint64 r = tcmalloc::subtle::NoBarrier_Load(&rand);
+   r = 0x5deece66dLL * r + 0xb;   // numbers from nrand48()
+-  base::subtle::NoBarrier_Store(&rand, r);
++  tcmalloc::subtle::NoBarrier_Store(&rand, r);
+ 
+   r <<= 16;   // 48-bit random number now in top 48-bits.
+   if (loop < 0 || loop > 32) {   // limit loop to 0..32
+@@ -80,9 +80,9 @@ static int SuggestedDelayNS(int loop) {
+   return r >> (44 - (loop >> 3));
+ #else
+   static Atomic32 rand;
+-  uint32 r = base::subtle::NoBarrier_Load(&rand);
++  uint32 r = tcmalloc::subtle::NoBarrier_Load(&rand);
+   r = 0x343fd * r + 0x269ec3;   // numbers from MSVC++
+-  base::subtle::NoBarrier_Store(&rand, r);
++  tcmalloc::subtle::NoBarrier_Store(&rand, r);
+ 
+   r <<= 1;   // 31-bit random number now in top 31-bits.
+   if (loop < 0 || loop > 32) {   // limit loop to 0..32
+@@ -99,4 +99,4 @@ static int SuggestedDelayNS(int loop) {
+ }
+ 
+ } // namespace internal
+-} // namespace base
++} // namespace tcmalloc
+diff --git a/src/base/spinlock_internal.h b/src/base/spinlock_internal.h
+index aa47e67..657b7ca 100644
+--- a/src/base/spinlock_internal.h
++++ b/src/base/spinlock_internal.h
+@@ -40,12 +40,12 @@
+ #include "base/basictypes.h"
+ #include "base/atomicops.h"
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace internal {
+ 
+ void SpinLockWake(volatile Atomic32 *w, bool all);
+ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop);
+ 
+ } // namespace internal
+-} // namespace base
++} // namespace tcmalloc
+ #endif
+diff --git a/src/base/spinlock_linux-inl.h b/src/base/spinlock_linux-inl.h
+index aadf62a..ffd0e72 100644
+--- a/src/base/spinlock_linux-inl.h
++++ b/src/base/spinlock_linux-inl.h
+@@ -63,7 +63,7 @@ static struct InitModule {
+ }  // anonymous namespace
+ 
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace internal {
+ 
+ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+@@ -72,7 +72,7 @@ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+     struct timespec tm;
+     tm.tv_sec = 0;
+     if (have_futex) {
+-      tm.tv_nsec = base::internal::SuggestedDelayNS(loop);
++      tm.tv_nsec = tcmalloc::internal::SuggestedDelayNS(loop);
+     } else {
+       tm.tv_nsec = 2000001;   // above 2ms so linux 2.4 doesn't spin
+     }
+@@ -98,4 +98,4 @@ void SpinLockWake(volatile Atomic32 *w, bool all) {
+ }
+ 
+ } // namespace internal
+-} // namespace base
++} // namespace tcmalloc
+diff --git a/src/base/spinlock_posix-inl.h b/src/base/spinlock_posix-inl.h
+index e73a30f..2507ae2 100644
+--- a/src/base/spinlock_posix-inl.h
++++ b/src/base/spinlock_posix-inl.h
+@@ -39,7 +39,7 @@
+ #endif
+ #include <time.h>       /* For nanosleep() */
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace internal {
+ 
+ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+@@ -50,7 +50,7 @@ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+   } else {
+     struct timespec tm;
+     tm.tv_sec = 0;
+-    tm.tv_nsec = base::internal::SuggestedDelayNS(loop);
++    tm.tv_nsec = tcmalloc::internal::SuggestedDelayNS(loop);
+     nanosleep(&tm, NULL);
+   }
+   errno = save_errno;
+@@ -60,4 +60,4 @@ void SpinLockWake(volatile Atomic32 *w, bool all) {
+ }
+ 
+ } // namespace internal
+-} // namespace base
++} // namespace tcmalloc
+diff --git a/src/base/spinlock_win32-inl.h b/src/base/spinlock_win32-inl.h
+index 956b965..bbf630b 100644
+--- a/src/base/spinlock_win32-inl.h
++++ b/src/base/spinlock_win32-inl.h
+@@ -35,7 +35,7 @@
+ 
+ #include <windows.h>
+ 
+-namespace base {
++namespace tcmalloc {
+ namespace internal {
+ 
+ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+@@ -43,7 +43,7 @@ void SpinLockDelay(volatile Atomic32 *w, int32 value, int loop) {
+   } else if (loop == 1) {
+     Sleep(0);
+   } else {
+-    Sleep(base::internal::SuggestedDelayNS(loop) / 1000000);
++    Sleep(tcmalloc::internal::SuggestedDelayNS(loop) / 1000000);
+   }
+ }
+ 
+@@ -51,4 +51,4 @@ void SpinLockWake(volatile Atomic32 *w, bool all) {
+ }
+ 
+ } // namespace internal
+-} // namespace base
++} // namespace tcmalloc
+diff --git a/src/base/vdso_support.cc b/src/base/vdso_support.cc
+index 730df30..70aaef5 100644
+--- a/src/base/vdso_support.cc
++++ b/src/base/vdso_support.cc
+@@ -48,13 +48,13 @@
+ #include "base/dynamic_annotations.h"
+ #include "base/basictypes.h"  // for COMPILE_ASSERT
+ 
+-using base::subtle::MemoryBarrier;
++using tcmalloc::subtle::MemoryBarrier;
+ 
+ #ifndef AT_SYSINFO_EHDR
+ #define AT_SYSINFO_EHDR 33
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ const void *VDSOSupport::vdso_base_ = ElfMemImage::kInvalidBase;
+ VDSOSupport::VDSOSupport()
+diff --git a/src/base/vdso_support.h b/src/base/vdso_support.h
+index c1209a4..97545a2 100644
+--- a/src/base/vdso_support.h
++++ b/src/base/vdso_support.h
+@@ -65,7 +65,7 @@
+ 
+ #include <stdlib.h>     // for NULL
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ // NOTE: this class may be used from within tcmalloc, and can not
+ // use any memory allocation routines.
+@@ -125,7 +125,7 @@ class VDSOSupport {
+   DISALLOW_COPY_AND_ASSIGN(VDSOSupport);
+ };
+ 
+-}  // namespace base
++}  // namespace tcmalloc
+ 
+ #endif  // HAVE_ELF_MEM_IMAGE
+ 
+diff --git a/src/central_freelist.h b/src/central_freelist.h
+index 4148680..02bbaa0 100644
+--- a/src/central_freelist.h
++++ b/src/central_freelist.h
+@@ -52,7 +52,7 @@ class CentralFreeList {
+   // A CentralFreeList may be used before its constructor runs.
+   // So we prevent lock_'s constructor from doing anything to the
+   // lock_ state.
+-  CentralFreeList() : lock_(base::LINKER_INITIALIZED) { }
++  CentralFreeList() : lock_(tcmalloc::LINKER_INITIALIZED) { }
+ 
+   void Init(size_t cl);
+ 
+diff --git a/src/emergency_malloc.cc b/src/emergency_malloc.cc
+index 81c5554..0860c85 100644
+--- a/src/emergency_malloc.cc
++++ b/src/emergency_malloc.cc
+@@ -47,7 +47,7 @@ namespace tcmalloc {
+   __attribute__ ((visibility("internal"))) char *emergency_arena_start;
+   __attribute__ ((visibility("internal"))) uintptr_t emergency_arena_start_shifted;
+ 
+-  static CACHELINE_ALIGNED SpinLock emergency_malloc_lock(base::LINKER_INITIALIZED);
++  static CACHELINE_ALIGNED SpinLock emergency_malloc_lock(tcmalloc::LINKER_INITIALIZED);
+   static char *emergency_arena_end;
+   static LowLevelAlloc::Arena *emergency_arena;
+ 
+diff --git a/src/gperftools/malloc_extension.h b/src/gperftools/malloc_extension.h
+index 689b5f1..ec45fbd 100644
+--- a/src/gperftools/malloc_extension.h
++++ b/src/gperftools/malloc_extension.h
+@@ -67,7 +67,7 @@ static const int kMallocHistogramSize = 64;
+ // One day, we could support other types of writers (perhaps for C?)
+ typedef std::string MallocExtensionWriter;
+ 
+-namespace base {
++namespace tcmalloc {
+ struct MallocRange;
+ }
+ 
+@@ -144,7 +144,7 @@ class PERFTOOLS_DLL_DECL MallocExtension {
+   //
+   // This is a best-effort interface useful only for performance
+   // analysis.  The implementation may not call func at all.
+-  typedef void (RangeFunction)(void*, const base::MallocRange*);
++  typedef void (RangeFunction)(void*, const tcmalloc::MallocRange*);
+   virtual void Ranges(void* arg, RangeFunction func);
+ 
+   // -------------------------------------------------------------------
+@@ -406,7 +406,7 @@ class PERFTOOLS_DLL_DECL MallocExtension {
+   virtual void MarkThreadTemporarilyIdle();
+ };
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ // Information passed per range.  More fields may be added later.
+ struct MallocRange {
+@@ -429,6 +429,6 @@ struct MallocRange {
+   // - age when allocated (for inuse) or freed (if not in use)
+ };
+ 
+-} // namespace base
++} // namespace tcmalloc
+ 
+ #endif  // BASE_MALLOC_EXTENSION_H_
+diff --git a/src/heap-profile-table.cc b/src/heap-profile-table.cc
+index 7486468..6f6a1ee 100644
+--- a/src/heap-profile-table.cc
++++ b/src/heap-profile-table.cc
+@@ -593,7 +593,7 @@ void HeapProfileTable::Snapshot::ReportLeaks(const char* checker_name,
+     symbolization_table.Symbolize();
+   for (int i = 0; i < to_report; i++) {
+     const Entry& e = entries[i];
+-    base::RawPrinter printer(buffer, kBufSize);
++    tcmalloc::RawPrinter printer(buffer, kBufSize);
+     printer.Printf("Leak of %d bytes in %d objects allocated from:\n",
+                    e.bytes, e.count);
+     for (int j = 0; j < e.bucket->depth; j++) {
+diff --git a/src/internal_logging.cc b/src/internal_logging.cc
+index 708fa65..0f3a7be 100644
+--- a/src/internal_logging.cc
++++ b/src/internal_logging.cc
+@@ -47,7 +47,7 @@
+ 
+ // Variables for storing crash output.  Allocated statically since we
+ // may not be able to heap-allocate while crashing.
+-static SpinLock crash_lock(base::LINKER_INITIALIZED);
++static SpinLock crash_lock(tcmalloc::LINKER_INITIALIZED);
+ static bool crashed = false;
+ static const int kStatsBufferSize = 16 << 10;
+ static char stats_buffer[kStatsBufferSize] = { 0 };
+diff --git a/src/malloc_hook-inl.h b/src/malloc_hook-inl.h
+index 30375d6..1468566 100644
+--- a/src/malloc_hook-inl.h
++++ b/src/malloc_hook-inl.h
+@@ -46,7 +46,7 @@
+ 
+ #include "common.h" // for UNLIKELY
+ 
+-namespace base { namespace internal {
++namespace tcmalloc { namespace internal {
+ 
+ // Capacity of 8 means that HookList is 9 words.
+ static const int kHookListCapacity = 8;
+@@ -80,13 +80,13 @@ struct PERFTOOLS_DLL_DECL HookList {
+ 
+   // Fast inline implementation for fast path of Invoke*Hook.
+   bool empty() const {
+-    return base::subtle::NoBarrier_Load(&priv_end) == 0;
++    return tcmalloc::subtle::NoBarrier_Load(&priv_end) == 0;
+   }
+ 
+   // Used purely to handle deprecated singular hooks
+   T GetSingular() const {
+     const AtomicWord *place = &priv_data[kHookListSingularIdx];
+-    return bit_cast<T>(base::subtle::NoBarrier_Load(place));
++    return bit_cast<T>(tcmalloc::subtle::NoBarrier_Load(place));
+   }
+ 
+   T ExchangeSingular(T new_val);
+@@ -115,33 +115,33 @@ ATTRIBUTE_VISIBILITY_HIDDEN extern HookList<MallocHook::MremapHook> mremap_hooks
+ ATTRIBUTE_VISIBILITY_HIDDEN extern HookList<MallocHook::PreSbrkHook> presbrk_hooks_;
+ ATTRIBUTE_VISIBILITY_HIDDEN extern HookList<MallocHook::SbrkHook> sbrk_hooks_;
+ 
+-} }  // namespace base::internal
++} }  // namespace tcmalloc::internal
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::NewHook MallocHook::GetNewHook() {
+-  return base::internal::new_hooks_.GetSingular();
++  return tcmalloc::internal::new_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeNewHook(const void* p, size_t s) {
+-  if (PREDICT_FALSE(!base::internal::new_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::new_hooks_.empty())) {
+     InvokeNewHookSlow(p, s);
+   }
+ }
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::DeleteHook MallocHook::GetDeleteHook() {
+-  return base::internal::delete_hooks_.GetSingular();
++  return tcmalloc::internal::delete_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeDeleteHook(const void* p) {
+-  if (PREDICT_FALSE(!base::internal::delete_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::delete_hooks_.empty())) {
+     InvokeDeleteHookSlow(p);
+   }
+ }
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::PreMmapHook MallocHook::GetPreMmapHook() {
+-  return base::internal::premmap_hooks_.GetSingular();
++  return tcmalloc::internal::premmap_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokePreMmapHook(const void* start,
+@@ -150,14 +150,14 @@ inline void MallocHook::InvokePreMmapHook(const void* start,
+                                           int flags,
+                                           int fd,
+                                           off_t offset) {
+-  if (!base::internal::premmap_hooks_.empty()) {
++  if (!tcmalloc::internal::premmap_hooks_.empty()) {
+     InvokePreMmapHookSlow(start, size, protection, flags, fd, offset);
+   }
+ }
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::MmapHook MallocHook::GetMmapHook() {
+-  return base::internal::mmap_hooks_.GetSingular();
++  return tcmalloc::internal::mmap_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeMmapHook(const void* result,
+@@ -167,7 +167,7 @@ inline void MallocHook::InvokeMmapHook(const void* result,
+                                        int flags,
+                                        int fd,
+                                        off_t offset) {
+-  if (!base::internal::mmap_hooks_.empty()) {
++  if (!tcmalloc::internal::mmap_hooks_.empty()) {
+     InvokeMmapHookSlow(result, start, size, protection, flags, fd, offset);
+   }
+ }
+@@ -179,7 +179,7 @@ inline bool MallocHook::InvokeMmapReplacement(const void* start,
+                                               int fd,
+                                               off_t offset,
+                                               void** result) {
+-  if (!base::internal::mmap_replacement_.empty()) {
++  if (!tcmalloc::internal::mmap_replacement_.empty()) {
+     return InvokeMmapReplacementSlow(start, size,
+                                      protection, flags,
+                                      fd, offset,
+@@ -190,18 +190,18 @@ inline bool MallocHook::InvokeMmapReplacement(const void* start,
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::MunmapHook MallocHook::GetMunmapHook() {
+-  return base::internal::munmap_hooks_.GetSingular();
++  return tcmalloc::internal::munmap_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeMunmapHook(const void* p, size_t size) {
+-  if (!base::internal::munmap_hooks_.empty()) {
++  if (!tcmalloc::internal::munmap_hooks_.empty()) {
+     InvokeMunmapHookSlow(p, size);
+   }
+ }
+ 
+ inline bool MallocHook::InvokeMunmapReplacement(
+     const void* p, size_t size, int* result) {
+-  if (!base::internal::mmap_replacement_.empty()) {
++  if (!tcmalloc::internal::mmap_replacement_.empty()) {
+     return InvokeMunmapReplacementSlow(p, size, result);
+   }
+   return false;
+@@ -209,7 +209,7 @@ inline bool MallocHook::InvokeMunmapReplacement(
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::MremapHook MallocHook::GetMremapHook() {
+-  return base::internal::mremap_hooks_.GetSingular();
++  return tcmalloc::internal::mremap_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeMremapHook(const void* result,
+@@ -218,30 +218,30 @@ inline void MallocHook::InvokeMremapHook(const void* result,
+                                          size_t new_size,
+                                          int flags,
+                                          const void* new_addr) {
+-  if (!base::internal::mremap_hooks_.empty()) {
++  if (!tcmalloc::internal::mremap_hooks_.empty()) {
+     InvokeMremapHookSlow(result, old_addr, old_size, new_size, flags, new_addr);
+   }
+ }
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::PreSbrkHook MallocHook::GetPreSbrkHook() {
+-  return base::internal::presbrk_hooks_.GetSingular();
++  return tcmalloc::internal::presbrk_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokePreSbrkHook(ptrdiff_t increment) {
+-  if (!base::internal::presbrk_hooks_.empty() && increment != 0) {
++  if (!tcmalloc::internal::presbrk_hooks_.empty() && increment != 0) {
+     InvokePreSbrkHookSlow(increment);
+   }
+ }
+ 
+ // The following method is DEPRECATED
+ inline MallocHook::SbrkHook MallocHook::GetSbrkHook() {
+-  return base::internal::sbrk_hooks_.GetSingular();
++  return tcmalloc::internal::sbrk_hooks_.GetSingular();
+ }
+ 
+ inline void MallocHook::InvokeSbrkHook(const void* result,
+                                        ptrdiff_t increment) {
+-  if (!base::internal::sbrk_hooks_.empty() && increment != 0) {
++  if (!tcmalloc::internal::sbrk_hooks_.empty() && increment != 0) {
+     InvokeSbrkHookSlow(result, increment);
+   }
+ }
+diff --git a/src/malloc_hook.cc b/src/malloc_hook.cc
+index 64c2165..1b0dcd0 100644
+--- a/src/malloc_hook.cc
++++ b/src/malloc_hook.cc
+@@ -157,13 +157,13 @@ extern "C" void MallocHook_InitAtFirstAllocation_HeapLeakChecker() {
+   // Do nothing.
+ }
+ 
+-namespace base { namespace internal {
++namespace tcmalloc { namespace internal {
+ 
+ // This lock is shared between all implementations of HookList::Add & Remove.
+ // The potential for contention is very small.  This needs to be a SpinLock and
+ // not a Mutex since it's possible for Mutex locking to allocate memory (e.g.,
+ // per-thread allocation in debug builds), which could cause infinite recursion.
+-static SpinLock hooklist_spinlock(base::LINKER_INITIALIZED);
++static SpinLock hooklist_spinlock(tcmalloc::LINKER_INITIALIZED);
+ 
+ template <typename T>
+ bool HookList<T>::Add(T value_as_t) {
+@@ -175,28 +175,28 @@ bool HookList<T>::Add(T value_as_t) {
+   // Find the first slot in data that is 0.
+   int index = 0;
+   while ((index < kHookListMaxValues) &&
+-         (base::subtle::NoBarrier_Load(&priv_data[index]) != 0)) {
++         (tcmalloc::subtle::NoBarrier_Load(&priv_data[index]) != 0)) {
+     ++index;
+   }
+   if (index == kHookListMaxValues) {
+     return false;
+   }
+-  AtomicWord prev_num_hooks = base::subtle::Acquire_Load(&priv_end);
+-  base::subtle::NoBarrier_Store(&priv_data[index], value);
++  AtomicWord prev_num_hooks = tcmalloc::subtle::Acquire_Load(&priv_end);
++  tcmalloc::subtle::NoBarrier_Store(&priv_data[index], value);
+   if (prev_num_hooks <= index) {
+-    base::subtle::NoBarrier_Store(&priv_end, index + 1);
++    tcmalloc::subtle::NoBarrier_Store(&priv_end, index + 1);
+   }
+   return true;
+ }
+ 
+ template <typename T>
+ void HookList<T>::FixupPrivEndLocked() {
+-  AtomicWord hooks_end = base::subtle::NoBarrier_Load(&priv_end);
++  AtomicWord hooks_end = tcmalloc::subtle::NoBarrier_Load(&priv_end);
+   while ((hooks_end > 0) &&
+-         (base::subtle::NoBarrier_Load(&priv_data[hooks_end - 1]) == 0)) {
++         (tcmalloc::subtle::NoBarrier_Load(&priv_data[hooks_end - 1]) == 0)) {
+     --hooks_end;
+   }
+-  base::subtle::NoBarrier_Store(&priv_end, hooks_end);
++  tcmalloc::subtle::NoBarrier_Store(&priv_end, hooks_end);
+ }
+ 
+ template <typename T>
+@@ -205,26 +205,26 @@ bool HookList<T>::Remove(T value_as_t) {
+     return false;
+   }
+   SpinLockHolder l(&hooklist_spinlock);
+-  AtomicWord hooks_end = base::subtle::NoBarrier_Load(&priv_end);
++  AtomicWord hooks_end = tcmalloc::subtle::NoBarrier_Load(&priv_end);
+   int index = 0;
+   while (index < hooks_end && value_as_t != bit_cast<T>(
+-             base::subtle::NoBarrier_Load(&priv_data[index]))) {
++             tcmalloc::subtle::NoBarrier_Load(&priv_data[index]))) {
+     ++index;
+   }
+   if (index == hooks_end) {
+     return false;
+   }
+-  base::subtle::NoBarrier_Store(&priv_data[index], 0);
++  tcmalloc::subtle::NoBarrier_Store(&priv_data[index], 0);
+   FixupPrivEndLocked();
+   return true;
+ }
+ 
+ template <typename T>
+ int HookList<T>::Traverse(T* output_array, int n) const {
+-  AtomicWord hooks_end = base::subtle::Acquire_Load(&priv_end);
++  AtomicWord hooks_end = tcmalloc::subtle::Acquire_Load(&priv_end);
+   int actual_hooks_end = 0;
+   for (int i = 0; i < hooks_end && n > 0; ++i) {
+-    AtomicWord data = base::subtle::Acquire_Load(&priv_data[i]);
++    AtomicWord data = tcmalloc::subtle::Acquire_Load(&priv_data[i]);
+     if (data != 0) {
+       *output_array++ = bit_cast<T>(data);
+       ++actual_hooks_end;
+@@ -239,10 +239,10 @@ T HookList<T>::ExchangeSingular(T value_as_t) {
+   AtomicWord value = bit_cast<AtomicWord>(value_as_t);
+   AtomicWord old_value;
+   SpinLockHolder l(&hooklist_spinlock);
+-  old_value = base::subtle::NoBarrier_Load(&priv_data[kHookListSingularIdx]);
+-  base::subtle::NoBarrier_Store(&priv_data[kHookListSingularIdx], value);
++  old_value = tcmalloc::subtle::NoBarrier_Load(&priv_data[kHookListSingularIdx]);
++  tcmalloc::subtle::NoBarrier_Store(&priv_data[kHookListSingularIdx], value);
+   if (value != 0) {
+-    base::subtle::NoBarrier_Store(&priv_end, kHookListSingularIdx + 1);
++    tcmalloc::subtle::NoBarrier_Store(&priv_end, kHookListSingularIdx + 1);
+   } else {
+     FixupPrivEndLocked();
+   }
+@@ -277,19 +277,19 @@ HookList<MallocHook::MunmapReplacement> munmap_replacement_ = { 0 };
+ #undef INIT_HOOK_LIST_WITH_VALUE
+ #undef INIT_HOOK_LIST
+ 
+-} }  // namespace base::internal
+-
+-using base::internal::kHookListMaxValues;
+-using base::internal::new_hooks_;
+-using base::internal::delete_hooks_;
+-using base::internal::premmap_hooks_;
+-using base::internal::mmap_hooks_;
+-using base::internal::mmap_replacement_;
+-using base::internal::munmap_hooks_;
+-using base::internal::munmap_replacement_;
+-using base::internal::mremap_hooks_;
+-using base::internal::presbrk_hooks_;
+-using base::internal::sbrk_hooks_;
++} }  // namespace tcmalloc::internal
++
++using tcmalloc::internal::kHookListMaxValues;
++using tcmalloc::internal::new_hooks_;
++using tcmalloc::internal::delete_hooks_;
++using tcmalloc::internal::premmap_hooks_;
++using tcmalloc::internal::mmap_hooks_;
++using tcmalloc::internal::mmap_replacement_;
++using tcmalloc::internal::munmap_hooks_;
++using tcmalloc::internal::munmap_replacement_;
++using tcmalloc::internal::mremap_hooks_;
++using tcmalloc::internal::presbrk_hooks_;
++using tcmalloc::internal::sbrk_hooks_;
+ 
+ // These are available as C bindings as well as C++, hence their
+ // definition outside the MallocHook class.
+diff --git a/src/page_heap.cc b/src/page_heap.cc
+index 0dda58f..fa16001 100644
+--- a/src/page_heap.cc
++++ b/src/page_heap.cc
+@@ -554,7 +554,7 @@ void PageHeap::GetLargeSpanStats(LargeSpanStats* result) {
+   }
+ }
+ 
+-bool PageHeap::GetNextRange(PageID start, base::MallocRange* r) {
++bool PageHeap::GetNextRange(PageID start, tcmalloc::MallocRange* r) {
+   Span* span = reinterpret_cast<Span*>(pagemap_.Next(start));
+   if (span == NULL) {
+     return false;
+@@ -564,7 +564,7 @@ bool PageHeap::GetNextRange(PageID start, base::MallocRange* r) {
+   r->fraction = 0;
+   switch (span->location) {
+     case Span::IN_USE:
+-      r->type = base::MallocRange::INUSE;
++      r->type = tcmalloc::MallocRange::INUSE;
+       r->fraction = 1;
+       if (span->sizeclass > 0) {
+         // Only some of the objects in this span may be in use.
+@@ -573,13 +573,13 @@ bool PageHeap::GetNextRange(PageID start, base::MallocRange* r) {
+       }
+       break;
+     case Span::ON_NORMAL_FREELIST:
+-      r->type = base::MallocRange::FREE;
++      r->type = tcmalloc::MallocRange::FREE;
+       break;
+     case Span::ON_RETURNED_FREELIST:
+-      r->type = base::MallocRange::UNMAPPED;
++      r->type = tcmalloc::MallocRange::UNMAPPED;
+       break;
+     default:
+-      r->type = base::MallocRange::UNKNOWN;
++      r->type = tcmalloc::MallocRange::UNKNOWN;
+       break;
+   }
+   return true;
+diff --git a/src/page_heap.h b/src/page_heap.h
+index b688512..91b7587 100644
+--- a/src/page_heap.h
++++ b/src/page_heap.h
+@@ -65,7 +65,7 @@
+ # include <gperftools/stacktrace.h>
+ #endif
+ 
+-namespace base {
++namespace tcmalloc {
+ struct MallocRange;
+ }
+ 
+@@ -150,7 +150,7 @@ class PERFTOOLS_DLL_DECL PageHeap {
+ 
+   // If this page heap is managing a range with starting page # >= start,
+   // store info about the range in *r and return true.  Else return false.
+-  bool GetNextRange(PageID start, base::MallocRange* r);
++  bool GetNextRange(PageID start, tcmalloc::MallocRange* r);
+ 
+   // Page heap statistics
+   struct Stats {
+diff --git a/src/raw_printer.cc b/src/raw_printer.cc
+index 3cf028e..cdcf8a1 100644
+--- a/src/raw_printer.cc
++++ b/src/raw_printer.cc
+@@ -37,7 +37,7 @@
+ #include "raw_printer.h"
+ #include "base/logging.h"
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ RawPrinter::RawPrinter(char* buf, int length)
+     : base_(buf),
+diff --git a/src/raw_printer.h b/src/raw_printer.h
+index 9288bb5..b40c013 100644
+--- a/src/raw_printer.h
++++ b/src/raw_printer.h
+@@ -46,7 +46,7 @@
+ #include <config.h>
+ #include "base/basictypes.h"
+ 
+-namespace base {
++namespace tcmalloc {
+ 
+ class RawPrinter {
+  public:
+diff --git a/src/stacktrace_powerpc-linux-inl.h b/src/stacktrace_powerpc-linux-inl.h
+index a301a46..617fc05 100644
+--- a/src/stacktrace_powerpc-linux-inl.h
++++ b/src/stacktrace_powerpc-linux-inl.h
+@@ -149,8 +149,8 @@ static int GET_STACK_TRACE_OR_FRAMES {
+   skip_count++; // skip parent's frame due to indirection in
+                 // stacktrace.cc
+ 
+-  base::VDSOSupport vdso;
+-  base::ElfMemImage::SymbolInfo rt_sigreturn_symbol_info;
++  tcmalloc::VDSOSupport vdso;
++  tcmalloc::ElfMemImage::SymbolInfo rt_sigreturn_symbol_info;
+ #ifdef __PPC64__
+   const void *sigtramp64_vdso = 0;
+   if (vdso.LookupSymbol("__kernel_sigtramp_rt64", "LINUX_2.6.15", STT_NOTYPE,
+diff --git a/src/stacktrace_x86-inl.h b/src/stacktrace_x86-inl.h
+index 46eb5d8..ae3287b 100644
+--- a/src/stacktrace_x86-inl.h
++++ b/src/stacktrace_x86-inl.h
+@@ -154,10 +154,10 @@ static void **NextStackFrame(void **old_sp, const void *uc) {
+     static const unsigned char *kernel_rt_sigreturn_address = NULL;
+     static const unsigned char *kernel_vsyscall_address = NULL;
+     if (num_push_instructions == -1) {
+-      base::VDSOSupport vdso;
++      tcmalloc::VDSOSupport vdso;
+       if (vdso.IsPresent()) {
+-        base::VDSOSupport::SymbolInfo rt_sigreturn_symbol_info;
+-        base::VDSOSupport::SymbolInfo vsyscall_symbol_info;
++        tcmalloc::VDSOSupport::SymbolInfo rt_sigreturn_symbol_info;
++        tcmalloc::VDSOSupport::SymbolInfo vsyscall_symbol_info;
+         if (!vdso.LookupSymbol("__kernel_rt_sigreturn", "LINUX_2.5",
+                                STT_FUNC, &rt_sigreturn_symbol_info) ||
+             !vdso.LookupSymbol("__kernel_vsyscall", "LINUX_2.5",
+diff --git a/src/tcmalloc.cc b/src/tcmalloc.cc
+index be0a6ee..e506bd3 100644
+--- a/src/tcmalloc.cc
++++ b/src/tcmalloc.cc
+@@ -589,7 +589,7 @@ static void IterateOverRanges(void* arg, MallocExtension::RangeFunction func) {
+   while (!done) {
+     // Accumulate a small number of ranges in a local buffer
+     static const int kNumRanges = 16;
+-    static base::MallocRange ranges[kNumRanges];
++    static tcmalloc::MallocRange ranges[kNumRanges];
+     int n = 0;
+     {
+       SpinLockHolder h(Static::pageheap_lock());
+@@ -1823,7 +1823,7 @@ void* memalign_pages(size_t align, size_t size,
+ template <void* OOMHandler(size_t)>
+ ATTRIBUTE_ALWAYS_INLINE inline
+ static void * malloc_fast_path(size_t size) {
+-  if (PREDICT_FALSE(!base::internal::new_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::new_hooks_.empty())) {
+     return tcmalloc::dispatch_allocate_full<OOMHandler>(size);
+   }
+ 
+@@ -1874,7 +1874,7 @@ void* tc_malloc(size_t size) PERFTOOLS_NOTHROW {
+ 
+ static ATTRIBUTE_ALWAYS_INLINE inline
+ void free_fast_path(void *ptr) {
+-  if (PREDICT_FALSE(!base::internal::delete_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::delete_hooks_.empty())) {
+     tcmalloc::invoke_hooks_and_free(ptr);
+     return;
+   }
+@@ -1888,7 +1888,7 @@ void tc_free(void* ptr) PERFTOOLS_NOTHROW {
+ 
+ extern "C" PERFTOOLS_DLL_DECL CACHELINE_ALIGNED_FN
+ void tc_free_sized(void *ptr, size_t size) PERFTOOLS_NOTHROW {
+-  if (PREDICT_FALSE(!base::internal::delete_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::delete_hooks_.empty())) {
+     tcmalloc::invoke_hooks_and_free(ptr);
+     return;
+   }
+@@ -1990,7 +1990,7 @@ extern "C" PERFTOOLS_DLL_DECL void tc_delete_nothrow(void* p, const std::nothrow
+ TC_ALIAS(tc_free);
+ #else
+ {
+-  if (PREDICT_FALSE(!base::internal::delete_hooks_.empty())) {
++  if (PREDICT_FALSE(!tcmalloc::internal::delete_hooks_.empty())) {
+     tcmalloc::invoke_hooks_and_free(p);
+     return;
+   }
+diff --git a/src/tests/atomicops_unittest.cc b/src/tests/atomicops_unittest.cc
+index aa82a6b..3caa9f2 100644
+--- a/src/tests/atomicops_unittest.cc
++++ b/src/tests/atomicops_unittest.cc
+@@ -99,19 +99,19 @@ static void TestStore() {
+ 
+   AtomicType value;
+ 
+-  base::subtle::NoBarrier_Store(&value, kVal1);
++  tcmalloc::subtle::NoBarrier_Store(&value, kVal1);
+   ASSERT_EQ(kVal1, value);
+-  base::subtle::NoBarrier_Store(&value, kVal2);
++  tcmalloc::subtle::NoBarrier_Store(&value, kVal2);
+   ASSERT_EQ(kVal2, value);
+ 
+-  base::subtle::Acquire_Store(&value, kVal1);
++  tcmalloc::subtle::Acquire_Store(&value, kVal1);
+   ASSERT_EQ(kVal1, value);
+-  base::subtle::Acquire_Store(&value, kVal2);
++  tcmalloc::subtle::Acquire_Store(&value, kVal2);
+   ASSERT_EQ(kVal2, value);
+ 
+-  base::subtle::Release_Store(&value, kVal1);
++  tcmalloc::subtle::Release_Store(&value, kVal1);
+   ASSERT_EQ(kVal1, value);
+-  base::subtle::Release_Store(&value, kVal2);
++  tcmalloc::subtle::Release_Store(&value, kVal2);
+   ASSERT_EQ(kVal2, value);
+ }
+ 
+@@ -125,30 +125,30 @@ static void TestLoad() {
+   AtomicType value;
+ 
+   value = kVal1;
+-  ASSERT_EQ(kVal1, base::subtle::NoBarrier_Load(&value));
++  ASSERT_EQ(kVal1, tcmalloc::subtle::NoBarrier_Load(&value));
+   value = kVal2;
+-  ASSERT_EQ(kVal2, base::subtle::NoBarrier_Load(&value));
++  ASSERT_EQ(kVal2, tcmalloc::subtle::NoBarrier_Load(&value));
+ 
+   value = kVal1;
+-  ASSERT_EQ(kVal1, base::subtle::Acquire_Load(&value));
++  ASSERT_EQ(kVal1, tcmalloc::subtle::Acquire_Load(&value));
+   value = kVal2;
+-  ASSERT_EQ(kVal2, base::subtle::Acquire_Load(&value));
++  ASSERT_EQ(kVal2, tcmalloc::subtle::Acquire_Load(&value));
+ 
+   value = kVal1;
+-  ASSERT_EQ(kVal1, base::subtle::Release_Load(&value));
++  ASSERT_EQ(kVal1, tcmalloc::subtle::Release_Load(&value));
+   value = kVal2;
+-  ASSERT_EQ(kVal2, base::subtle::Release_Load(&value));
++  ASSERT_EQ(kVal2, tcmalloc::subtle::Release_Load(&value));
+ }
+ 
+ template <class AtomicType>
+ static void TestAtomicOps() {
+-  TestCompareAndSwap<AtomicType>(base::subtle::NoBarrier_CompareAndSwap);
+-  TestCompareAndSwap<AtomicType>(base::subtle::Acquire_CompareAndSwap);
+-  TestCompareAndSwap<AtomicType>(base::subtle::Release_CompareAndSwap);
++  TestCompareAndSwap<AtomicType>(tcmalloc::subtle::NoBarrier_CompareAndSwap);
++  TestCompareAndSwap<AtomicType>(tcmalloc::subtle::Acquire_CompareAndSwap);
++  TestCompareAndSwap<AtomicType>(tcmalloc::subtle::Release_CompareAndSwap);
+ 
+-  TestAtomicExchange<AtomicType>(base::subtle::NoBarrier_AtomicExchange);
+-  TestAtomicExchange<AtomicType>(base::subtle::Acquire_AtomicExchange);
+-  TestAtomicExchange<AtomicType>(base::subtle::Release_AtomicExchange);
++  TestAtomicExchange<AtomicType>(tcmalloc::subtle::NoBarrier_AtomicExchange);
++  TestAtomicExchange<AtomicType>(tcmalloc::subtle::Acquire_AtomicExchange);
++  TestAtomicExchange<AtomicType>(tcmalloc::subtle::Release_AtomicExchange);
+ 
+   TestStore<AtomicType>();
+   TestLoad<AtomicType>();
+diff --git a/src/tests/malloc_hook_test.cc b/src/tests/malloc_hook_test.cc
+index a5cd860..84fecca 100644
+--- a/src/tests/malloc_hook_test.cc
++++ b/src/tests/malloc_hook_test.cc
+@@ -91,12 +91,12 @@ void Sleep(int seconds) {
+ }
+ 
+ using std::min;
+-using base::internal::kHookListMaxValues;
++using tcmalloc::internal::kHookListMaxValues;
+ 
+ // Since HookList is a template and is defined in malloc_hook.cc, we can only
+ // use an instantiation of it from malloc_hook.cc.  We then reinterpret those
+ // values as integers for testing.
+-typedef base::internal::HookList<MallocHook::NewHook> TestHookList;
++typedef tcmalloc::internal::HookList<MallocHook::NewHook> TestHookList;
+ 
+ int TestHookList_Traverse(const TestHookList& list, uintptr_t* output_array, int n) {
+   MallocHook::NewHook values_as_hooks[kHookListMaxValues];
+diff --git a/src/tests/raw_printer_test.cc b/src/tests/raw_printer_test.cc
+index 2c7be6a..99a2f13 100644
+--- a/src/tests/raw_printer_test.cc
++++ b/src/tests/raw_printer_test.cc
+@@ -17,7 +17,7 @@ using std::string;
+ 
+ TEST(RawPrinter, Empty) {
+   char buffer[1];
+-  base::RawPrinter printer(buffer, arraysize(buffer));
++  tcmalloc::RawPrinter printer(buffer, arraysize(buffer));
+   CHECK_EQ(0, printer.length());
+   CHECK_EQ(string(""), buffer);
+   CHECK_EQ(0, printer.space_left());
+@@ -29,7 +29,7 @@ TEST(RawPrinter, Empty) {
+ 
+ TEST(RawPrinter, PartiallyFilled) {
+   char buffer[100];
+-  base::RawPrinter printer(buffer, arraysize(buffer));
++  tcmalloc::RawPrinter printer(buffer, arraysize(buffer));
+   printer.Printf("%s %s", "hello", "world");
+   CHECK_EQ(string("hello world"), string(buffer));
+   CHECK_EQ(11, printer.length());
+@@ -38,7 +38,7 @@ TEST(RawPrinter, PartiallyFilled) {
+ 
+ TEST(RawPrinter, Truncated) {
+   char buffer[3];
+-  base::RawPrinter printer(buffer, arraysize(buffer));
++  tcmalloc::RawPrinter printer(buffer, arraysize(buffer));
+   printer.Printf("%d", 12345678);
+   CHECK_EQ(string("12"), string(buffer));
+   CHECK_EQ(2, printer.length());
+@@ -47,7 +47,7 @@ TEST(RawPrinter, Truncated) {
+ 
+ TEST(RawPrinter, ExactlyFilled) {
+   char buffer[12];
+-  base::RawPrinter printer(buffer, arraysize(buffer));
++  tcmalloc::RawPrinter printer(buffer, arraysize(buffer));
+   printer.Printf("%s %s", "hello", "world");
+   CHECK_EQ(string("hello world"), string(buffer));
+   CHECK_EQ(11, printer.length());
+diff --git a/src/tests/tcmalloc_unittest.cc b/src/tests/tcmalloc_unittest.cc
+index 25b2e41..5b0a906 100644
+--- a/src/tests/tcmalloc_unittest.cc
++++ b/src/tests/tcmalloc_unittest.cc
+@@ -853,21 +853,21 @@ namespace {
+ 
+ struct RangeCallbackState {
+   uintptr_t ptr;
+-  base::MallocRange::Type expected_type;
++  tcmalloc::MallocRange::Type expected_type;
+   size_t min_size;
+   bool matched;
+ };
+ 
+-static void RangeCallback(void* arg, const base::MallocRange* r) {
++static void RangeCallback(void* arg, const tcmalloc::MallocRange* r) {
+   RangeCallbackState* state = reinterpret_cast<RangeCallbackState*>(arg);
+   if (state->ptr >= r->address &&
+       state->ptr < r->address + r->length) {
+-    if (state->expected_type == base::MallocRange::FREE) {
++    if (state->expected_type == tcmalloc::MallocRange::FREE) {
+       // We are expecting r->type == FREE, but ReleaseMemory
+       // may have already moved us to UNMAPPED state instead (this happens in
+       // approximately 0.1% of executions). Accept either state.
+-      CHECK(r->type == base::MallocRange::FREE ||
+-            r->type == base::MallocRange::UNMAPPED);
++      CHECK(r->type == tcmalloc::MallocRange::FREE ||
++            r->type == tcmalloc::MallocRange::UNMAPPED);
+     } else {
+       CHECK_EQ(r->type, state->expected_type);
+     }
+@@ -879,7 +879,7 @@ static void RangeCallback(void* arg, const base::MallocRange* r) {
+ // Check that at least one of the callbacks from Ranges() contains
+ // the specified address with the specified type, and has size
+ // >= min_size.
+-static void CheckRangeCallback(void* ptr, base::MallocRange::Type type,
++static void CheckRangeCallback(void* ptr, tcmalloc::MallocRange::Type type,
+                                size_t min_size) {
+   RangeCallbackState state;
+   state.ptr = reinterpret_cast<uintptr_t>(ptr);
+@@ -899,20 +899,20 @@ static void TestRanges() {
+   static const int MB = 1048576;
+   void* a = malloc(MB);
+   void* b = malloc(MB);
+-  base::MallocRange::Type releasedType =
+-      HaveSystemRelease ? base::MallocRange::UNMAPPED : base::MallocRange::FREE;
++  tcmalloc::MallocRange::Type releasedType =
++      HaveSystemRelease ? tcmalloc::MallocRange::UNMAPPED : tcmalloc::MallocRange::FREE;
+ 
+-  CheckRangeCallback(a, base::MallocRange::INUSE, MB);
+-  CheckRangeCallback(b, base::MallocRange::INUSE, MB);
++  CheckRangeCallback(a, tcmalloc::MallocRange::INUSE, MB);
++  CheckRangeCallback(b, tcmalloc::MallocRange::INUSE, MB);
+   free(a);
+-  CheckRangeCallback(a, base::MallocRange::FREE, MB);
+-  CheckRangeCallback(b, base::MallocRange::INUSE, MB);
++  CheckRangeCallback(a, tcmalloc::MallocRange::FREE, MB);
++  CheckRangeCallback(b, tcmalloc::MallocRange::INUSE, MB);
+   MallocExtension::instance()->ReleaseFreeMemory();
+   CheckRangeCallback(a, releasedType, MB);
+-  CheckRangeCallback(b, base::MallocRange::INUSE, MB);
++  CheckRangeCallback(b, tcmalloc::MallocRange::INUSE, MB);
+   free(b);
+   CheckRangeCallback(a, releasedType, MB);
+-  CheckRangeCallback(b, base::MallocRange::FREE, MB);
++  CheckRangeCallback(b, tcmalloc::MallocRange::FREE, MB);
+ }
+ 
+ #ifndef DEBUGALLOCATION
+-- 
+1.8.3.1
+

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/patches/gperftools-hook-mi_force_unlock-on-OSX-instead-of-pthread_atfork.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/gperftools-hook-mi_force_unlock-on-OSX-instead-of-pthread_atfork.patch b/thirdparty/patches/gperftools-hook-mi_force_unlock-on-OSX-instead-of-pthread_atfork.patch
deleted file mode 100644
index b8c9695..0000000
--- a/thirdparty/patches/gperftools-hook-mi_force_unlock-on-OSX-instead-of-pthread_atfork.patch
+++ /dev/null
@@ -1,72 +0,0 @@
-commit 7013b219970a329d1db58fbd7fa7c907bec8dbba
-Author: Aliaksey Kandratsenka <al...@tut.by>
-Date:   Sat May 9 12:48:11 2015 -0700
-
-    hook mi_force_{un,}lock on OSX instead of pthread_atfork
-    
-    This is patch by Anton Samokhvalov.
-    
-    Apparently it helps with locking around forking on OSX.
-
-diff --git a/src/libc_override_osx.h b/src/libc_override_osx.h
-index 26923e9..b801f22 100644
---- a/src/libc_override_osx.h
-+++ b/src/libc_override_osx.h
-@@ -85,6 +85,11 @@
- #include <AvailabilityMacros.h>
- #include <malloc/malloc.h>
- 
-+namespace tcmalloc {
-+  void CentralCacheLockAll();
-+  void CentralCacheUnlockAll();
-+}
-+
- // from AvailabilityMacros.h
- #if defined(MAC_OS_X_VERSION_10_6) && \
-     MAC_OS_X_VERSION_MAX_ALLOWED >= MAC_OS_X_VERSION_10_6
-@@ -169,11 +174,11 @@ void mi_log(malloc_zone_t *zone, void *address) {
- }
- 
- void mi_force_lock(malloc_zone_t *zone) {
--  // Hopefully unneeded by us!
-+  tcmalloc::CentralCacheLockAll();
- }
- 
- void mi_force_unlock(malloc_zone_t *zone) {
--  // Hopefully unneeded by us!
-+  tcmalloc::CentralCacheUnlockAll();
- }
- 
- void mi_statistics(malloc_zone_t *zone, malloc_statistics_t *stats) {
-diff --git a/src/static_vars.cc b/src/static_vars.cc
-index 197b3a1..09d2b59 100644
---- a/src/static_vars.cc
-+++ b/src/static_vars.cc
-@@ -51,7 +51,6 @@ namespace tcmalloc {
- // sure the central_cache locks remain in a consisten state in the forked
- // version of the thread.
- 
--static
- void CentralCacheLockAll()
- {
-   Static::pageheap_lock()->Lock();
-@@ -59,7 +58,6 @@ void CentralCacheLockAll()
-     Static::central_cache()[i].Lock();
- }
- 
--static
- void CentralCacheUnlockAll()
- {
-   for (int i = 0; i < kNumClasses; ++i)
-@@ -114,9 +112,11 @@ void Static::InitStaticVars() {
- static inline
- void SetupAtForkLocksHandler()
- {
-+#if !defined(__APPLE__)
-   pthread_atfork(CentralCacheLockAll,    // parent calls before fork
-                  CentralCacheUnlockAll,  // parent calls after fork
-                  CentralCacheUnlockAll); // child calls after fork
-+#endif
- }
- REGISTER_MODULE_INITIALIZER(tcmalloc_fork_handler, SetupAtForkLocksHandler());
- 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/patches/gperftools-issue-827-add_get_default_zone_to_osx_libc_override.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/gperftools-issue-827-add_get_default_zone_to_osx_libc_override.patch b/thirdparty/patches/gperftools-issue-827-add_get_default_zone_to_osx_libc_override.patch
deleted file mode 100644
index 5c3061a..0000000
--- a/thirdparty/patches/gperftools-issue-827-add_get_default_zone_to_osx_libc_override.patch
+++ /dev/null
@@ -1,48 +0,0 @@
-diff --git a/src/libc_override_osx.h b/src/libc_override_osx.h
-index b801f22..37c96c6 100644
---- a/src/libc_override_osx.h
-+++ b/src/libc_override_osx.h
-@@ -211,6 +211,34 @@ extern "C" {
-   size_t malloc_usable_size(void* p)     { return tc_malloc_size(p); }
- }  // extern "C"
-
-+static malloc_zone_t *get_default_zone()
-+{
-+   malloc_zone_t **zones = NULL;
-+   unsigned int num_zones = 0;
-+
-+   /*
-+    * On OSX 10.12, malloc_default_zone returns a special zone that is not
-+    * present in the list of registered zones. That zone uses a "lite zone"
-+    * if one is present (apparently enabled when malloc stack logging is
-+    * enabled), or the first registered zone otherwise. In practice this
-+    * means unless malloc stack logging is enabled, the first registered
-+    * zone is the default.
-+    * So get the list of zones to get the first one, instead of relying on
-+    * malloc_default_zone.
-+    */
-+        if (KERN_SUCCESS != malloc_get_all_zones(0, NULL, (vm_address_t**) &zones,
-+                                            &num_zones)) {
-+       /* Reset the value in case the failure happened after it was set. */
-+       num_zones = 0;
-+   }
-+
-+   if (num_zones)
-+       return zones[0];
-+
-+   return malloc_default_zone();
-+}
-+
-+
- static void ReplaceSystemAlloc() {
-   static malloc_introspection_t tcmalloc_introspection;
-   memset(&tcmalloc_introspection, 0, sizeof(tcmalloc_introspection));
-@@ -273,7 +301,7 @@ static void ReplaceSystemAlloc() {
-   // zone.  The default zone is then re-registered to ensure that
-   // allocations made from it earlier will be handled correctly.
-   // Things are not guaranteed to work that way, but it's how they work now.
--  malloc_zone_t *default_zone = malloc_default_zone();
-+  malloc_zone_t *default_zone = get_default_zone();
-   malloc_zone_unregister(default_zone);
-   malloc_zone_register(default_zone);
- }

http://git-wip-us.apache.org/repos/asf/kudu/blob/e0a743d2/thirdparty/vars.sh
----------------------------------------------------------------------
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index 15b8cba..472970c 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -46,7 +46,7 @@ GMOCK_VERSION=1.8.0
 GMOCK_NAME=googletest-release-$GMOCK_VERSION
 GMOCK_SOURCE=$TP_SOURCE_DIR/$GMOCK_NAME
 
-GPERFTOOLS_VERSION=2.2.1
+GPERFTOOLS_VERSION=2.6.3
 GPERFTOOLS_NAME=gperftools-$GPERFTOOLS_VERSION
 GPERFTOOLS_SOURCE=$TP_SOURCE_DIR/$GPERFTOOLS_NAME