You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2020/03/31 05:24:45 UTC

[kudu] branch master updated (8d1b6e1 -> 263c3aa)

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

adar pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 8d1b6e1  consensus: remove kudu::Bind usage from dirty callbacks
     new 1d28104  Add functionality to serialize a RowBlock into columnar format
     new 1de208d  [docs] added RHEL/CentOS 8 into the list of supported systems
     new 270746e  [util] Add special handling for nullptr in fast hash
     new 56ee422  gutil: remove callback and bind from the codebase
     new 37bda7b  Remove return types from various lambdas
     new ce82af1  KUDU-2059: add a TSAN suppression
     new 263c3aa  KUDU-3093: another band-aid for this DebugUtilTest.TestSignalStackTrace

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 build-support/release/rat_exclude_files.txt        |   10 -
 docs/contributing.adoc                             |   34 +-
 docs/installation.adoc                             |    5 +-
 .../main/java/org/apache/kudu/util/HashUtil.java   |   33 +-
 .../java/org/apache/kudu/util/TestFashHash.java    |   24 +-
 src/kudu/client/client.cc                          |    2 +-
 src/kudu/common/CMakeLists.txt                     |    2 +-
 src/kudu/common/columnar_serialization.cc          |  134 +-
 src/kudu/common/columnar_serialization.h           |   38 +-
 src/kudu/common/wire_protocol-test.cc              |  442 +++-
 src/kudu/fs/block_manager-test.cc                  |    4 +-
 src/kudu/fs/log_block_manager-test.cc              |    2 +-
 src/kudu/fs/log_block_manager.cc                   |    2 +-
 src/kudu/gutil/CMakeLists.txt                      |    1 -
 src/kudu/gutil/bind.h                              |  539 ----
 src/kudu/gutil/bind.h.pump                         |  153 --
 src/kudu/gutil/bind_helpers.h                      |  547 ----
 src/kudu/gutil/bind_internal.h                     | 2697 --------------------
 src/kudu/gutil/bind_internal.h.pump                |  464 ----
 src/kudu/gutil/callback.h                          |  765 ------
 src/kudu/gutil/callback.h.pump                     |  436 ----
 src/kudu/gutil/callback_forward.h                  |   17 -
 src/kudu/gutil/callback_internal.cc                |   38 -
 src/kudu/gutil/callback_internal.h                 |  175 --
 .../gutil/raw_scoped_refptr_mismatch_checker.h     |   63 -
 src/kudu/hms/hms_client-test.cc                    |    2 +-
 src/kudu/integration-tests/all_types-itest.cc      |    2 +-
 src/kudu/integration-tests/alter_table-test.cc     |   10 +-
 src/kudu/integration-tests/create-table-itest.cc   |    4 +-
 .../master_cert_authority-itest.cc                 |    2 +-
 .../integration-tests/tablet_history_gc-itest.cc   |    2 +-
 .../integration-tests/ts_tablet_manager-itest.cc   |    4 +-
 src/kudu/master/catalog_manager.cc                 |    2 +-
 src/kudu/master/master-test.cc                     |    2 +-
 src/kudu/sentry/sentry_client-test.cc              |    4 +-
 src/kudu/tablet/rowset_tree-test.cc                |    2 +-
 src/kudu/tablet/tablet_bootstrap.cc                |    2 +-
 src/kudu/tablet/tablet_history_gc-test.cc          |    2 +-
 src/kudu/tools/ksck.cc                             |    2 +-
 src/kudu/tools/kudu-admin-test.cc                  |    4 +-
 src/kudu/tools/kudu-tool-test.cc                   |    2 +-
 src/kudu/tools/tool_action_common.cc               |    2 +-
 src/kudu/tools/tool_action_perf.cc                 |    2 +-
 src/kudu/tools/tool_action_table.cc                |    3 +-
 src/kudu/tserver/tablet_server-test.cc             |   34 +-
 src/kudu/util/CMakeLists.txt                       |    1 -
 src/kudu/util/callback_bind-test.cc                |  119 -
 src/kudu/util/debug-util-test.cc                   |   17 +-
 src/kudu/util/env_posix.cc                         |    4 +-
 src/kudu/util/faststring.h                         |   10 +
 src/kudu/util/hash_util-test.cc                    |   12 +
 src/kudu/util/hash_util.h                          |   24 +-
 src/kudu/util/sanitizer_options.cc                 |    6 +-
 src/kudu/util/threadpool-test.cc                   |    2 +-
 src/kudu/util/trace.h                              |    2 +-
 55 files changed, 674 insertions(+), 6239 deletions(-)
 delete mode 100644 src/kudu/gutil/bind.h
 delete mode 100644 src/kudu/gutil/bind.h.pump
 delete mode 100644 src/kudu/gutil/bind_helpers.h
 delete mode 100644 src/kudu/gutil/bind_internal.h
 delete mode 100644 src/kudu/gutil/bind_internal.h.pump
 delete mode 100644 src/kudu/gutil/callback.h
 delete mode 100644 src/kudu/gutil/callback.h.pump
 delete mode 100644 src/kudu/gutil/callback_forward.h
 delete mode 100644 src/kudu/gutil/callback_internal.cc
 delete mode 100644 src/kudu/gutil/callback_internal.h
 delete mode 100644 src/kudu/gutil/raw_scoped_refptr_mismatch_checker.h
 delete mode 100644 src/kudu/util/callback_bind-test.cc


[kudu] 06/07: KUDU-2059: add a TSAN suppression

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ce82af1171099e00606c67a351362a0b68549141
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Mar 30 14:14:29 2020 -0700

    KUDU-2059: add a TSAN suppression
    
    No one is actively working on fixing this, so let's at least suppress it so
    that precommits aren't flaky.
    
    Change-Id: I86979e4c511bd4cbf027c629c867378cd0b8cd32
    Reviewed-on: http://gerrit.cloudera.org:8080/15603
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/util/sanitizer_options.cc | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/src/kudu/util/sanitizer_options.cc b/src/kudu/util/sanitizer_options.cc
index de7cf4f..8c8c0d3 100644
--- a/src/kudu/util/sanitizer_options.cc
+++ b/src/kudu/util/sanitizer_options.cc
@@ -165,7 +165,11 @@ SANITIZER_HOOK_ATTRIBUTE const char *__tsan_default_suppressions() {
   // This is carried out by OPENSSL_cleanup, but TSAN's unwinder doesn't
   // include any stack frame above the libcrypto lock destruction or memory release
   // call for some reason, so we have to do something more generic.
-  "called_from_lib:libcrypto.so\n";
+  "called_from_lib:libcrypto.so\n"
+
+  // KUDU-2059: there may be outstanding reactor threads in DnsResolver at the
+  // time that the KuduClient (and DnsResolver) is destroyed.
+  "race:kudu::DnsResolver::ResolveAddressesAsync\n";
 }
 #endif  // THREAD_SANITIZER
 


[kudu] 02/07: [docs] added RHEL/CentOS 8 into the list of supported systems

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1de208dcf19468bbf3f7a67818006ea7a98c1889
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Mar 30 07:51:55 2020 -0700

    [docs] added RHEL/CentOS 8 into the list of supported systems
    
    Change-Id: Ifaa6778e0fb1adc0657c518fa8077154cf471f64
    Reviewed-on: http://gerrit.cloudera.org:8080/15595
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 docs/installation.adoc | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/docs/installation.adoc b/docs/installation.adoc
index 08c5c34..415c597 100644
--- a/docs/installation.adoc
+++ b/docs/installation.adoc
@@ -45,8 +45,9 @@ only tolerate a single failure; two-master deployments cannot tolerate any failu
 
 .Operating System Requirements
 Linux::
-    - RHEL 6, RHEL 7, CentOS 6, CentOS 7, Ubuntu 14.04 (trusty), Ubuntu 16.04 (xenial),
-      Ubuntu 18.04 (bionic), Debian 8 (Jessie), or SLES 12.
+    - RHEL 6, RHEL 7, RHEL 8, CentOS 6, CentOS 7, CentOS 8,
+      Ubuntu 14.04 (trusty), Ubuntu 16.04 (xenial), Ubuntu 18.04 (bionic),
+      Debian 8 (Jessie), or SLES 12.
     - A kernel and filesystem that support _hole punching_. Hole punching is the use of the
       `fallocate(2)` system call with the `FALLOC_FL_PUNCH_HOLE` option set. See
       link:troubleshooting.html#req_hole_punching[troubleshooting hole punching] for more


[kudu] 05/07: Remove return types from various lambdas

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 37bda7be50f5b0e280612c3dfa0cf10bd8cb6dcf
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Mar 30 11:40:31 2020 -0700

    Remove return types from various lambdas
    
    Only the ones that were absolutely necessary (i.e. for implicit conversions)
    were retained.
    
    Change-Id: Idb6cb2570121d47dee15d9e63b677126c30de4f9
    Reviewed-on: http://gerrit.cloudera.org:8080/15599
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Bankim Bhavsar <ba...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/client/client.cc                          |  2 +-
 src/kudu/fs/block_manager-test.cc                  |  4 +--
 src/kudu/fs/log_block_manager-test.cc              |  2 +-
 src/kudu/fs/log_block_manager.cc                   |  2 +-
 src/kudu/hms/hms_client-test.cc                    |  2 +-
 src/kudu/integration-tests/all_types-itest.cc      |  2 +-
 src/kudu/integration-tests/alter_table-test.cc     | 10 +++----
 src/kudu/integration-tests/create-table-itest.cc   |  4 +--
 .../master_cert_authority-itest.cc                 |  2 +-
 .../integration-tests/tablet_history_gc-itest.cc   |  2 +-
 .../integration-tests/ts_tablet_manager-itest.cc   |  4 +--
 src/kudu/master/catalog_manager.cc                 |  2 +-
 src/kudu/master/master-test.cc                     |  2 +-
 src/kudu/sentry/sentry_client-test.cc              |  4 +--
 src/kudu/tablet/rowset_tree-test.cc                |  2 +-
 src/kudu/tablet/tablet_bootstrap.cc                |  2 +-
 src/kudu/tablet/tablet_history_gc-test.cc          |  2 +-
 src/kudu/tools/ksck.cc                             |  2 +-
 src/kudu/tools/kudu-admin-test.cc                  |  4 +--
 src/kudu/tools/kudu-tool-test.cc                   |  2 +-
 src/kudu/tools/tool_action_common.cc               |  2 +-
 src/kudu/tools/tool_action_perf.cc                 |  2 +-
 src/kudu/tools/tool_action_table.cc                |  3 +-
 src/kudu/tserver/tablet_server-test.cc             | 34 +++++++++++-----------
 src/kudu/util/env_posix.cc                         |  4 +--
 src/kudu/util/threadpool-test.cc                   |  2 +-
 src/kudu/util/trace.h                              |  2 +-
 27 files changed, 53 insertions(+), 54 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index e602eb6..571844a 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -551,7 +551,7 @@ Status KuduClient::GetTablet(const string& tablet_id, KuduTablet** tablet) {
 
   auto add_replica_func = [](const TSInfoPB& ts_info,
                              const RaftPeerPB::Role role,
-                             vector<const KuduReplica*>* replicas) -> Status {
+                             vector<const KuduReplica*>* replicas) {
     if (ts_info.rpc_addresses_size() == 0) {
       return Status::IllegalState(Substitute(
           "No RPC addresses found for tserver $0",
diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index 1aca697..a29b730 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -941,7 +941,7 @@ TYPED_TEST(BlockManagerTest, TestMetadataOkayDespiteFailure) {
 
   // Creates a block with the given 'test_data', writing the result
   // to 'out' on success.
-  auto create_a_block = [&](BlockId* out, const string& test_data) -> Status {
+  auto create_a_block = [&](BlockId* out, const string& test_data) {
     unique_ptr<WritableBlock> block;
     RETURN_NOT_OK(this->bm_->CreateBlock(this->test_block_opts_, &block));
     for (int i = 0; i < kNumAppends; i++) {
@@ -957,7 +957,7 @@ TYPED_TEST(BlockManagerTest, TestMetadataOkayDespiteFailure) {
   // Reads a block given by 'id', comparing its contents. Note that
   // we need to compare with both kLongTestData and kShortTestData as we
   // do not know the blocks' content ahead.
-  auto read_a_block = [&](const BlockId& id) -> Status {
+  auto read_a_block = [&](const BlockId& id) {
     unique_ptr<ReadableBlock> block;
     RETURN_NOT_OK(this->bm_->OpenBlock(id, &block));
     uint64_t size;
diff --git a/src/kudu/fs/log_block_manager-test.cc b/src/kudu/fs/log_block_manager-test.cc
index 37c13ec..0bf9cd3 100644
--- a/src/kudu/fs/log_block_manager-test.cc
+++ b/src/kudu/fs/log_block_manager-test.cc
@@ -1131,7 +1131,7 @@ TEST_F(LogBlockManagerTest, TestContainerBlockLimiting) {
   const int kNumBlocks = 1000;
 
   // Creates 'kNumBlocks' blocks with minimal data.
-  auto create_some_blocks = [&]() -> Status {
+  auto create_some_blocks = [&]() {
     for (int i = 0; i < kNumBlocks; i++) {
       unique_ptr<WritableBlock> block;
       RETURN_NOT_OK(bm_->CreateBlock(test_block_opts_, &block));
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 945d53f..eb47b08 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -1099,7 +1099,7 @@ Status LogBlockContainer::ProcessRecord(
 
 Status LogBlockContainer::DoCloseBlocks(const vector<LogWritableBlock*>& blocks,
                                         SyncMode mode) {
-  auto sync_blocks = [&]() -> Status {
+  auto sync_blocks = [&]() {
     if (mode == SYNC) {
       VLOG(3) << "Syncing data file " << data_file_->filename();
       RETURN_NOT_OK(SyncData());
diff --git a/src/kudu/hms/hms_client-test.cc b/src/kudu/hms/hms_client-test.cc
index 6343427..b2555bf 100644
--- a/src/kudu/hms/hms_client-test.cc
+++ b/src/kudu/hms/hms_client-test.cc
@@ -400,7 +400,7 @@ TEST_F(HmsClientTest, TestHmsConnect) {
   // client.
   options.verify_service_config = true;
 
-  auto start_client = [&options] (Sockaddr addr) -> Status {
+  auto start_client = [&options] (Sockaddr addr) {
     HmsClient client(HostPort(addr), options);
     return client.Start();
   };
diff --git a/src/kudu/integration-tests/all_types-itest.cc b/src/kudu/integration-tests/all_types-itest.cc
index 6362589..3b0b9d5 100644
--- a/src/kudu/integration-tests/all_types-itest.cc
+++ b/src/kudu/integration-tests/all_types-itest.cc
@@ -677,7 +677,7 @@ TYPED_TEST(AllTypesItest, TestAllKeyTypes) {
 TYPED_TEST(AllTypesItest, TestTimestampPadding) {
   vector<string> projection;
   this->SetupProjection(&projection);
-  auto scanner_setup = [&](KuduScanner* scanner) -> Status {
+  auto scanner_setup = [&](KuduScanner* scanner) {
     // Each time this function is called we shuffle the projection to get the chance
     // of having timestamps in different places of the projection and before/after
     // different types.
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 1a40a8b..714e923 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -1776,7 +1776,7 @@ TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
                          ->wait(true)->Alter());
 
   // Turns an optional value into a row for the table.
-  auto fill_row = [&] (boost::optional<int32_t> value) -> unique_ptr<KuduPartialRow> {
+  auto fill_row = [&] (boost::optional<int32_t> value) {
     unique_ptr<KuduPartialRow> row(schema_.NewRow());
     if (value) {
       CHECK_OK(row->SetInt32("c0", *value));
@@ -1786,7 +1786,7 @@ TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
 
   // Attempts to add a range partition to the table with the specified bounds.
   auto add_range_partition = [&] (boost::optional<int32_t> lower_bound,
-                                  boost::optional<int32_t> upper_bound) -> Status {
+                                  boost::optional<int32_t> upper_bound) {
     table_alterer.reset(client_->NewTableAlterer(table_name));
     return table_alterer->AddRangePartition(fill_row(lower_bound).release(),
                                             fill_row(upper_bound).release())
@@ -1796,7 +1796,7 @@ TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
 
   // Attempts to drop a range partition to the table with the specified bounds.
   auto drop_range_partition = [&] (boost::optional<int32_t> lower_bound,
-                                   boost::optional<int32_t> upper_bound) -> Status {
+                                   boost::optional<int32_t> upper_bound) {
     table_alterer.reset(client_->NewTableAlterer(table_name));
     return table_alterer->DropRangePartition(fill_row(lower_bound).release(),
                                              fill_row(upper_bound).release())
@@ -1808,7 +1808,7 @@ TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
   auto add_range_partitions = [&] (boost::optional<int32_t> a_lower_bound,
                                    boost::optional<int32_t> a_upper_bound,
                                    boost::optional<int32_t> b_lower_bound,
-                                   boost::optional<int32_t> b_upper_bound) -> Status {
+                                   boost::optional<int32_t> b_upper_bound) {
     table_alterer.reset(client_->NewTableAlterer(table_name));
     return table_alterer->AddRangePartition(fill_row(a_lower_bound).release(),
                                             fill_row(a_upper_bound).release())
@@ -1822,7 +1822,7 @@ TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
   auto add_drop_range_partitions = [&] (boost::optional<int32_t> a_lower_bound,
                                         boost::optional<int32_t> a_upper_bound,
                                         boost::optional<int32_t> b_lower_bound,
-                                        boost::optional<int32_t> b_upper_bound) -> Status {
+                                        boost::optional<int32_t> b_upper_bound) {
     table_alterer.reset(client_->NewTableAlterer(table_name));
     return table_alterer->AddRangePartition(fill_row(a_lower_bound).release(),
                                             fill_row(a_upper_bound).release())
diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc
index 9042252..dce0bd3 100644
--- a/src/kudu/integration-tests/create-table-itest.cc
+++ b/src/kudu/integration-tests/create-table-itest.cc
@@ -280,7 +280,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenlyWithDimension) {
                               const string& table_name,
                               int32_t range_lower_bound,
                               int32_t range_upper_bound,
-                              const string& dimension_label) -> Status {
+                              const string& dimension_label) {
     unique_ptr<client::KuduTableCreator> table_creator(client->NewTableCreator());
     unique_ptr<KuduPartialRow> lower_bound(client_schema->NewRow());
     RETURN_NOT_OK(lower_bound->SetInt32("key2", range_lower_bound));
@@ -301,7 +301,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenlyWithDimension) {
                              const string& table_name,
                              int32_t range_lower_bound,
                              int32_t range_upper_bound,
-                             const string& dimension_label) -> Status {
+                             const string& dimension_label) {
     unique_ptr<client::KuduTableAlterer> table_alterer(client->NewTableAlterer(table_name));
     unique_ptr<KuduPartialRow> lower_bound(client_schema->NewRow());
     RETURN_NOT_OK(lower_bound->SetInt32("key2", range_lower_bound));
diff --git a/src/kudu/integration-tests/master_cert_authority-itest.cc b/src/kudu/integration-tests/master_cert_authority-itest.cc
index 29fcde9..f933fb3 100644
--- a/src/kudu/integration-tests/master_cert_authority-itest.cc
+++ b/src/kudu/integration-tests/master_cert_authority-itest.cc
@@ -376,7 +376,7 @@ class ConnectToClusterBaseTest : public KuduTest {
     builder.default_rpc_timeout(timeout);
     client::sp::shared_ptr<KuduClient> client;
     ASSERT_OK(cluster_->CreateClient(&builder, &client));
-    ASSERT_EQ(1, client->data_->messenger_-> tls_context().trusted_cert_count_for_tests());
+    ASSERT_EQ(1, client->data_->messenger_->tls_context().trusted_cert_count_for_tests());
     ASSERT_NE(boost::none, client->data_->messenger_->authn_token());
   }
 
diff --git a/src/kudu/integration-tests/tablet_history_gc-itest.cc b/src/kudu/integration-tests/tablet_history_gc-itest.cc
index 1f5b684..c9a7d81 100644
--- a/src/kudu/integration-tests/tablet_history_gc-itest.cc
+++ b/src/kudu/integration-tests/tablet_history_gc-itest.cc
@@ -141,7 +141,7 @@ TEST_F(TabletHistoryGcITest, TestSnapshotScanBeforeAHM) {
   TestWorkload workload(cluster_.get());
   workload.Setup();
 
-  auto open_scanner_func = [](KuduClient* client) -> Status {
+  auto open_scanner_func = [](KuduClient* client) {
     shared_ptr<KuduTable> table;
     RETURN_NOT_OK(client->OpenTable(TestWorkload::kDefaultTableName, &table));
     KuduScanner scanner(table.get());
diff --git a/src/kudu/integration-tests/ts_tablet_manager-itest.cc b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
index 859295b..b03a011 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -717,7 +717,7 @@ TEST_F(TsTabletManagerITest, TestTableStats) {
       FLAGS_raft_heartbeat_interval_ms * FLAGS_leader_failure_max_missed_heartbeat_periods;
 
   // Get the LEADER master.
-  const auto GetLeaderMaster = [&] () -> Master* {
+  const auto GetLeaderMaster = [&] () {
     int idx = 0;
     Master* master = nullptr;
     Status s = cluster_->GetLeaderMasterIndex(&idx);
@@ -727,7 +727,7 @@ TEST_F(TsTabletManagerITest, TestTableStats) {
     return CHECK_NOTNULL(master);
   };
   // Get the LEADER master's service proxy.
-  const auto GetLeaderMasterServiceProxy = [&] () -> shared_ptr<MasterServiceProxy> {
+  const auto GetLeaderMasterServiceProxy = [&]() {
     const auto& addr = GetLeaderMaster()->first_rpc_address();
     shared_ptr<MasterServiceProxy> proxy(
         new MasterServiceProxy(client_messenger_, addr, addr.host()));
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 19218ce..4ddf84c 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1303,7 +1303,7 @@ Status CatalogManager::VisitTablesAndTablets() {
 Status CatalogManager::InitSysCatalogAsync(bool is_first_run) {
   std::lock_guard<LockType> l(lock_);
   unique_ptr<SysCatalogTable> new_catalog(new SysCatalogTable(
-      master_, [this]() -> Status { return this->ElectedAsLeaderCb(); }));
+      master_, [this]() { return this->ElectedAsLeaderCb(); }));
   if (is_first_run) {
     RETURN_NOT_OK(new_catalog->CreateNew(master_->fs_manager()));
   } else {
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 56434ce..52b8037 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -2012,7 +2012,7 @@ TEST_P(AuthzTokenMasterTest, TestGenerateAuthzTokens) {
   FLAGS_master_support_authz_tokens = supports_authz;
   const char* kTableName = "testtb";
   const Schema kTableSchema({ ColumnSchema("key", INT32) }, 1);
-  const auto send_req = [&] (GetTableSchemaResponsePB* resp) -> Status {
+  const auto send_req = [&] (GetTableSchemaResponsePB* resp) {
     RpcController rpc;
     GetTableSchemaRequestPB req;
     req.mutable_table()->set_table_name(kTableName);
diff --git a/src/kudu/sentry/sentry_client-test.cc b/src/kudu/sentry/sentry_client-test.cc
index a284568..c548473 100644
--- a/src/kudu/sentry/sentry_client-test.cc
+++ b/src/kudu/sentry/sentry_client-test.cc
@@ -72,8 +72,8 @@ TEST_P(SentryClientTest, TestMiniSentryLifecycle) {
 
   ASSERT_OK(client.Start(vector<HostPort>({sentry_->address()}),
                          sentry_client_opts));
-  auto smoketest = [&]() -> Status {
-    return client.Execute([](SentryClient* client) -> Status {
+  auto smoketest = [&]() {
+    return client.Execute([](SentryClient* client) {
         TCreateSentryRoleRequest create_req;
         create_req.requestorUserName = "test-admin";
         create_req.roleName = "test-role";
diff --git a/src/kudu/tablet/rowset_tree-test.cc b/src/kudu/tablet/rowset_tree-test.cc
index 400f2dc..62e7281 100644
--- a/src/kudu/tablet/rowset_tree-test.cc
+++ b/src/kudu/tablet/rowset_tree-test.cc
@@ -219,7 +219,7 @@ TEST_F(TestRowSetTree, TestTreeRandomized) {
     BOUND_GREATER_EQUAL,
     BOUND_EQUAL
   };
-  const auto& GetStringPair = [] (const BoundOperator op) -> std::pair<string, string> {
+  const auto& GetStringPair = [] (const BoundOperator op) {
     while (true) {
       string s1 = Substitute("$0", rand() % 100);
       string s2 = Substitute("$0", rand() % 100);
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index 7521d85..0d0f4e3 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -532,7 +532,7 @@ Status TabletBootstrap::Bootstrap(shared_ptr<Tablet>* rebuilt_tablet,
     auto cb = make_scoped_refptr(new FlushInflightsToLogCallback(
         rebuilt_tablet->get(), *rebuilt_log));
     tablet_meta_->SetPreFlushCallback(
-        [cb]() -> Status { return cb->WaitForInflightsAndFlushLog(); });
+        [cb]() { return cb->WaitForInflightsAndFlushLog(); });
   }
 
   // This will cause any pending TabletMetadata flush to be executed.
diff --git a/src/kudu/tablet/tablet_history_gc-test.cc b/src/kudu/tablet/tablet_history_gc-test.cc
index b92ee85..049d774 100644
--- a/src/kudu/tablet/tablet_history_gc-test.cc
+++ b/src/kudu/tablet/tablet_history_gc-test.cc
@@ -137,7 +137,7 @@ class TabletHistoryGcTest : public TabletTestBase<IntKeyTestSetup<INT64>> {
   // Returns a functor that returns whether all rows have 'expected_val' for
   // their values.
   static TestRowVerifier GenRowsEqualVerifier(int32_t expected_val) {
-    return [=](int32_t /*key*/, int32_t val) -> bool { return val == expected_val; };
+    return [=](int32_t /*key*/, int32_t val) { return val == expected_val; };
   }
   const TestRowVerifier kRowsEqual0 = GenRowsEqualVerifier(0);
   const TestRowVerifier kRowsEqual1 = GenRowsEqualVerifier(1);
diff --git a/src/kudu/tools/ksck.cc b/src/kudu/tools/ksck.cc
index 32a2af5..971deb1 100644
--- a/src/kudu/tools/ksck.cc
+++ b/src/kudu/tools/ksck.cc
@@ -869,7 +869,7 @@ HealthCheckResult Ksck::VerifyTablet(const shared_ptr<KsckTablet>& tablet,
                                  tablet->id(), tablet->table()->name());
 
   auto leader_it = std::find_if(tablet->replicas().cbegin(), tablet->replicas().cend(),
-      [](const shared_ptr<KsckTabletReplica>& r) -> bool { return r->is_leader(); });
+      [](const shared_ptr<KsckTabletReplica>& r) { return r->is_leader(); });
   boost::optional<string> leader_uuid;
   if (leader_it != tablet->replicas().cend()) {
     leader_uuid = (*leader_it)->ts_uuid();
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 4f6dde9..eae3b14 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -2509,7 +2509,7 @@ TEST_F(AdminCliTest, TestAddAndDropRangePartition) {
   const auto add_range_partition_using_CLI = [&] (const string& lower_bound_json,
                                                   const string& upper_bound_json,
                                                   const string& lower_bound_type,
-                                                  const string& upper_bound_type) -> Status {
+                                                  const string& upper_bound_type) {
     string error, out;
     Status s = RunKuduTool({
       "table",
@@ -2528,7 +2528,7 @@ TEST_F(AdminCliTest, TestAddAndDropRangePartition) {
   const auto drop_range_partition_using_CLI = [&] (const string& lower_bound_json,
                                                    const string& upper_bound_json,
                                                    const string& lower_bound_type,
-                                                   const string& upper_bound_type) -> Status {
+                                                   const string& upper_bound_type) {
     string error, out;
     Status s = RunKuduTool({
       "table",
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 6755183..4bc44c5 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -5333,7 +5333,7 @@ TEST_F(ToolTest, TestReplaceTablet) {
       ts = ts_map_[cluster_->tablet_server(i)->uuid()];
       ASSERT_OK(ListRunningTabletIds(ts, kTimeout, &tablet_ids));
       ASSERT_TRUE(std::none_of(tablet_ids.begin(), tablet_ids.end(),
-            [&](const string& tablet_id) -> bool { return tablet_id == old_tablet_id; }));
+            [&](const string& tablet_id) { return tablet_id == old_tablet_id; }));
     }
   });
 
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
index c725448..df71ca0 100644
--- a/src/kudu/tools/tool_action_common.cc
+++ b/src/kudu/tools/tool_action_common.cc
@@ -445,7 +445,7 @@ Status PrintServerFlags(const string& address, uint16_t default_port) {
 
   std::sort(flags.begin(), flags.end(),
       [](const GetFlagsResponsePB::Flag& left,
-         const GetFlagsResponsePB::Flag& right) -> bool {
+         const GetFlagsResponsePB::Flag& right) {
         return left.name() < right.name();
       });
   DataTable table({ "flag", "value", "default value?", "tags" });
diff --git a/src/kudu/tools/tool_action_perf.cc b/src/kudu/tools/tool_action_perf.cc
index 4a88105..a8f6951 100644
--- a/src/kudu/tools/tool_action_perf.cc
+++ b/src/kudu/tools/tool_action_perf.cc
@@ -543,7 +543,7 @@ void GeneratorThread(
   client::sp::shared_ptr<KuduSession> session(client->NewSession());
   int64_t idx = 0;
 
-  auto generator = [&]() -> Status {
+  auto generator = [&]() {
     const int64_t num_rows_per_gen = FLAGS_num_rows_per_thread;
     if (num_rows_per_gen == 0) {
       return Status::OK();
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index c6552f4..006cc71 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -630,8 +630,7 @@ Status ModifyRangePartition(const RunnerContext& context, PartitionAction action
 
   const auto convert_bounds_type = [&] (const string& range_bound,
                                         const string& flags_range_bound_type,
-                                        KuduTableCreator::RangePartitionBound* range_bound_type)
-      -> Status {
+                                        KuduTableCreator::RangePartitionBound* range_bound_type) {
     string inclusive_bound = boost::iequals(flags_range_bound_type, "INCLUSIVE_BOUND") ?
         "INCLUSIVE_BOUND" : "";
     string exclusive_bound = boost::iequals(flags_range_bound_type, "EXCLUSIVE_BOUND") ?
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index 65446cb..4c49e3f 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -302,11 +302,11 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "log_dir";
           }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "help";
           }));
   }
@@ -318,15 +318,15 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "log_dir";
           }));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "help";
           }));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "logemaillevel";
           }));
   }
@@ -338,15 +338,15 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "log_dir";
           }));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "help";
           }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-          [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+          [](const server::GetFlagsResponsePB::Flag& flag) {
             return flag.name() == "logemaillevel";
           }));
   }
@@ -360,15 +360,15 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-                            [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                            [](const server::GetFlagsResponsePB::Flag& flag) {
                                 return flag.name() == "log_dir";
                             }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "help";
                              }));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-                            [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                            [](const server::GetFlagsResponsePB::Flag& flag) {
                                 return flag.name() == "logemaillevel";
                             }));
   }
@@ -382,15 +382,15 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "log_dir";
                              }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "help";
                              }));
     EXPECT_TRUE(std::any_of(resp.flags().begin(), resp.flags().end(),
-                            [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                            [](const server::GetFlagsResponsePB::Flag& flag) {
                                 return flag.name() == "logemaillevel";
                             }));
   }
@@ -404,15 +404,15 @@ TEST_F(TabletServerTest, TestGetFlags) {
     ASSERT_OK(proxy.GetFlags(req, &resp, &controller));
     SCOPED_TRACE(SecureDebugString(resp));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "log_dir";
                              }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "help";
                              }));
     EXPECT_TRUE(std::none_of(resp.flags().begin(), resp.flags().end(),
-                             [](const server::GetFlagsResponsePB::Flag& flag) -> bool {
+                             [](const server::GetFlagsResponsePB::Flag& flag) {
                                  return flag.name() == "logemaillevel";
                              }));
   }
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 586b333..2657452 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -1333,7 +1333,7 @@ class PosixEnv : public Env {
   virtual Status DeleteRecursively(const string &name) OVERRIDE {
     return Walk(
         name, POST_ORDER,
-        [this](FileType type, const string& dirname, const string& basename) -> Status {
+        [this](FileType type, const string& dirname, const string& basename) {
           return this->DeleteRecursivelyCb(type, dirname, basename);
         });
   }
@@ -1377,7 +1377,7 @@ class PosixEnv : public Env {
     uint64_t total = 0;
     RETURN_NOT_OK(Walk(
         root, PRE_ORDER,
-        [this, &total](FileType type, const string& dirname, const string& basename) -> Status {
+        [this, &total](FileType type, const string& dirname, const string& basename) {
           return this->GetFileSizeOnDiskRecursivelyCb(&total, type, dirname, basename);
         }));
     *bytes_used = total;
diff --git a/src/kudu/util/threadpool-test.cc b/src/kudu/util/threadpool-test.cc
index 374b1bc..b8faac4 100644
--- a/src/kudu/util/threadpool-test.cc
+++ b/src/kudu/util/threadpool-test.cc
@@ -774,7 +774,7 @@ TEST_F(ThreadPoolTest, TestTokenConcurrency) {
   simple_spinlock lock;
 
   // Fetch a token from 'tokens' at random.
-  auto GetRandomToken = [&]() -> shared_ptr<ThreadPoolToken> {
+  auto GetRandomToken = [&]() {
     std::lock_guard<simple_spinlock> l(lock);
     int idx = rng.Uniform(kNumTokens);
     return tokens[idx];
diff --git a/src/kudu/util/trace.h b/src/kudu/util/trace.h
index 27d56dd..c30e241 100644
--- a/src/kudu/util/trace.h
+++ b/src/kudu/util/trace.h
@@ -99,7 +99,7 @@ class Trace;
 // Construct a constant C string counter name which acts as a sort of
 // coarse-grained histogram for trace metrics.
 #define BUCKETED_COUNTER_NAME(prefix, duration_us)      \
-  [=]() -> const char* {                                \
+  [=]() {                                               \
     if (duration_us >= 100 * 1000) {                    \
       return prefix "_gt_100_ms";                       \
     } else if (duration_us >= 10 * 1000) {              \


[kudu] 03/07: [util] Add special handling for nullptr in fast hash

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 270746e845317ed2d0a53acec94a1877798e9d88
Author: Bankim Bhavsar <ba...@cloudera.com>
AuthorDate: Mon Mar 30 11:24:37 2020 -0700

    [util] Add special handling for nullptr in fast hash
    
    Importing special handling for nullptr from Impala that helps distinguish
    between nullptr with possibly non-zero length input and empty object.
    https://github.com/apache/impala/blob/master/be/src/runtime/raw-value.inline.h#L40
    https://github.com/apache/impala/blob/master/be/src/runtime/raw-value-ir.cc#L179
    
    Fast hash already handles zero-length input and hence no special handling
    for zero length input.
    
    Fast hash is currently only used for BlockBloomFilter and hence no backward
    compatibility concerns. Don't plan to use Murmur2 hash with BlockBloomFilter
    and hence not importing the same logic for Murmur2 or other hash functions.
    Importing the same logic for other hash functions will require taking backward
    compatibility into account.
    
    Additionally this change updates Java implementation of Fast hash
    to drop the explicit "len" parameter as arrays in Java include length.
    
    Change-Id: Idf1ccff3dde7ccf54c4c2c6c2910915c69153316
    Reviewed-on: http://gerrit.cloudera.org:8080/15600
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 .../main/java/org/apache/kudu/util/HashUtil.java   | 33 ++++++++++++++++++----
 .../java/org/apache/kudu/util/TestFashHash.java    | 24 ++++++++++++----
 src/kudu/util/hash_util-test.cc                    | 12 ++++++++
 src/kudu/util/hash_util.h                          | 24 ++++++++++++++--
 4 files changed, 79 insertions(+), 14 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java b/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java
index d8dc526..9d27d4d 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java
@@ -21,18 +21,42 @@ package org.apache.kudu.util;
  * Hash utility functions.
  */
 public class HashUtil {
+  // Constant imported from Apache Impala used to compute hash values for special cases.
+  // It's an arbitrary constant obtained by taking lower bytes of generated UUID. Helps
+  // distinguish NULL values from empty objects.
+  // Impala uses the direct BlockBloomFilter C++ API and inserts hash value directly using
+  // its own implementation of the Fast hash. Hence the value must match with Impala.
+  // Though Impala will use C++ API, keeping the implementation of the Fast hash algorithm
+  // consistent across C++ and Java.
+  private static final int HASH_VAL_NULL = 0x58081667;
+  private static final byte[] HASH_VAL_NULL_BYTE_BUF = new byte[4];
+
+  static {
+    HASH_VAL_NULL_BYTE_BUF[0] = (byte) (HASH_VAL_NULL >>> 0);
+    HASH_VAL_NULL_BYTE_BUF[1] = (byte) (HASH_VAL_NULL >>> 8);
+    HASH_VAL_NULL_BYTE_BUF[2] = (byte) (HASH_VAL_NULL >>> 16);
+    HASH_VAL_NULL_BYTE_BUF[3] = (byte) (HASH_VAL_NULL >>> 24);
+  }
+
   /**
    * Compute 64-bit FastHash of the supplied data backed by byte array.
    *
    * FastHash is simple, robust, and efficient general-purpose hash function from Google.
    * Implementation is adapted from https://code.google.com/archive/p/fast-hash/
    *
+   * Adds special handling for null input.
+   *
    * @param buf the data to hash
-   * @param len length of the supplied data
    * @param seed seed to compute the hash
    * @return computed 64-bit hash value
    */
-  public static long fastHash64(final byte[] buf, int len, long seed) {
+  public static long fastHash64(byte[] buf, long seed) {
+    // Special handling for null input with possible non-zero length as could be the
+    // case with nullable column values.
+    if (buf == null) {
+      buf = HASH_VAL_NULL_BYTE_BUF;
+    }
+    final int len = buf.length;
     final long m = 0x880355f21e6d1965L;
     long h = seed ^ (len * m);
     long v;
@@ -89,15 +113,14 @@ public class HashUtil {
    * Implementation is adapted from https://code.google.com/archive/p/fast-hash/
    *
    * @param buf the data to compute the hash
-   * @param len length of the supplied data
    * @param seed seed to compute the hash
    * @return computed 32-bit hash value
    */
-  public static int fastHash32(final byte[] buf, int len, int seed) {
+  public static int fastHash32(byte[] buf, int seed) {
     // the following trick converts the 64-bit hashcode to Fermat
     // residue, which shall retain information from both the higher
     // and lower parts of hashcode.
-    long h = fastHash64(buf, len, seed);
+    long h = fastHash64(buf, seed);
     return (int)(h - (h >>> 32));
   }
 
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java
index fe08ba3..731a0f2 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java
@@ -39,27 +39,39 @@ public class TestFashHash {
   public void testFastHash64() {
     long hash;
 
-    hash = HashUtil.fastHash64("ab".getBytes(UTF_8), 2, 0);
+    hash = HashUtil.fastHash64("ab".getBytes(UTF_8), 0);
     assertEquals(Long.parseUnsignedLong("17293172613997361769"), hash);
 
-    hash = HashUtil.fastHash64("abcdefg".getBytes(UTF_8), 7, 0);
+    hash = HashUtil.fastHash64("abcdefg".getBytes(UTF_8), 0);
     assertEquals(Long.parseUnsignedLong("10206404559164245992"), hash);
 
-    hash = HashUtil.fastHash64("quick brown fox".getBytes(UTF_8), 15, 42);
+    hash = HashUtil.fastHash64("quick brown fox".getBytes(UTF_8), 42);
     assertEquals(Long.parseUnsignedLong("3757424404558187042"), hash);
+
+    hash = HashUtil.fastHash64(null, 0);
+    assertEquals(Long.parseUnsignedLong("12680076593665652444"), hash);
+
+    hash = HashUtil.fastHash64("".getBytes(UTF_8), 0);
+    assertEquals(0, hash);
   }
 
   @Test
   public void testFastHash32() {
     int hash;
 
-    hash = HashUtil.fastHash32("ab".getBytes(UTF_8), 2, 0);
+    hash = HashUtil.fastHash32("ab".getBytes(UTF_8), 0);
     assertEquals(Integer.parseUnsignedInt("2564147595"), hash);
 
-    hash = HashUtil.fastHash32("abcdefg".getBytes(UTF_8), 7, 0);
+    hash = HashUtil.fastHash32("abcdefg".getBytes(UTF_8), 0);
     assertEquals(Integer.parseUnsignedInt("1497700618"), hash);
 
-    hash = HashUtil.fastHash32("quick brown fox".getBytes(UTF_8), 15, 42);
+    hash = HashUtil.fastHash32("quick brown fox".getBytes(UTF_8), 42);
     assertEquals(Integer.parseUnsignedInt("1676541068"), hash);
+
+    hash = HashUtil.fastHash32(null, 0);
+    assertEquals(Integer.parseUnsignedInt("842467426"), hash);
+
+    hash = HashUtil.fastHash32("".getBytes(UTF_8), 0);
+    assertEquals(0, hash);
   }
 }
diff --git a/src/kudu/util/hash_util-test.cc b/src/kudu/util/hash_util-test.cc
index 9fadfcf..1e9298e 100644
--- a/src/kudu/util/hash_util-test.cc
+++ b/src/kudu/util/hash_util-test.cc
@@ -57,6 +57,12 @@ TEST(HashUtilTest, TestFastHash64) {
 
   hash = HashUtil::FastHash64("quick brown fox", 15, 42);
   ASSERT_EQ(3757424404558187042UL, hash);
+
+  hash = HashUtil::FastHash64(nullptr, 0, 0);
+  ASSERT_EQ(12680076593665652444UL, hash);
+
+  hash = HashUtil::FastHash64("", 0, 0);
+  ASSERT_EQ(0, hash);
 }
 
 TEST(HashUtilTest, TestFastHash32) {
@@ -70,6 +76,12 @@ TEST(HashUtilTest, TestFastHash32) {
 
   hash = HashUtil::FastHash32("quick brown fox", 15, 42);
   ASSERT_EQ(1676541068U, hash);
+
+  hash = HashUtil::FastHash32(nullptr, 0, 0);
+  ASSERT_EQ(842467426U, hash);
+
+  hash = HashUtil::FastHash32("", 0, 0);
+  ASSERT_EQ(0, hash);
 }
 
 TEST(HashUtilTest, TestComputeHash32Available) {
diff --git a/src/kudu/util/hash_util.h b/src/kudu/util/hash_util.h
index 71055e1..1664abd 100644
--- a/src/kudu/util/hash_util.h
+++ b/src/kudu/util/hash_util.h
@@ -29,11 +29,22 @@
 
 namespace kudu {
 
-/// Utility class to compute hash values.
+// Constant imported from Apache Impala used to compute hash values for special cases. It's an
+// arbitrary constant obtained by taking lower bytes of generated UUID. Helps distinguish NULL
+// values from empty objects.
+// Impala uses the direct BlockBloomFilter API and inserts hash value directly using its own
+// implementation of the Fast hash. Hence the value must match with Impala.
+//
+// Note: Since address of this static constexpr variable is used, declaring this as
+//       a member variable of HashUtil requires an explicit definition in .cc file
+//       and this class is completely defined in the header file to allow inlining.
+static constexpr uint32_t kHashValNull = 0x58081667;
+
+// Utility class to compute hash values.
 class HashUtil {
  public:
 
-  /// Murmur2 hash implementation returning 64-bit hashes.
+  // Murmur2 hash implementation returning 64-bit hashes.
   ATTRIBUTE_NO_SANITIZE_INTEGER
   static uint64_t MurmurHash2_64(const void* input, int len, uint64_t seed) {
     static constexpr uint64_t MURMUR_PRIME = 0xc6a4a7935bd1e995UL;
@@ -75,9 +86,17 @@ class HashUtil {
   // FastHash is simple, robust, and efficient general-purpose hash function from Google.
   // Implementation is adapted from https://code.google.com/archive/p/fast-hash/
   //
+  // Adds special handling for nullptr input.
+  //
   // Compute 64-bit FastHash.
   ATTRIBUTE_NO_SANITIZE_INTEGER
   static uint64_t FastHash64(const void* buf, size_t len, uint64_t seed) {
+    // Special handling for nullptr input with possible non-zero length as could be the
+    // case with nullable column values.
+    if (buf == nullptr) {
+      buf = &kHashValNull;
+      len = sizeof(kHashValNull);
+    }
     static constexpr uint64_t kMultiplier = 0x880355f21e6d1965UL;
     const uint64_t* pos = static_cast<const uint64_t*>(buf);
     const uint64_t* end = pos + (len / 8);
@@ -131,7 +150,6 @@ class HashUtil {
   // Compute 32-bit hash of the supplied data using the specified hash algorithm.
   // Must be kept in sync with IsComputeHash32Available() function.
   static uint32_t ComputeHash32(const Slice& data, HashAlgorithm hash_algorithm, uint32_t seed) {
-    // TODO(bankim): Consider adding special handling for zero-length/NULL objects.
     switch (hash_algorithm) {
       case FAST_HASH:
         return FastHash32(data.data(), data.size(), seed);


[kudu] 04/07: gutil: remove callback and bind from the codebase

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 56ee4225a304763cc6f5949e0b5497402860d614
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Sat Mar 28 01:53:04 2020 -0700

    gutil: remove callback and bind from the codebase
    
    It is fitting that 5.5 years after bringing this code into Kudu (see commit
    31f072096), I'm lucky enough to be the one burying it. There are no more
    usages of this functionality; everything has been migrated to std::function
    with lambdas for binding/capturing.
    
    Change-Id: Ib5fb53730f9c33b37415b22fcc775266467116b9
    Reviewed-on: http://gerrit.cloudera.org:8080/15583
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 build-support/release/rat_exclude_files.txt        |   10 -
 docs/contributing.adoc                             |   34 +-
 src/kudu/gutil/CMakeLists.txt                      |    1 -
 src/kudu/gutil/bind.h                              |  539 ----
 src/kudu/gutil/bind.h.pump                         |  153 --
 src/kudu/gutil/bind_helpers.h                      |  547 ----
 src/kudu/gutil/bind_internal.h                     | 2697 --------------------
 src/kudu/gutil/bind_internal.h.pump                |  464 ----
 src/kudu/gutil/callback.h                          |  765 ------
 src/kudu/gutil/callback.h.pump                     |  436 ----
 src/kudu/gutil/callback_forward.h                  |   17 -
 src/kudu/gutil/callback_internal.cc                |   38 -
 src/kudu/gutil/callback_internal.h                 |  175 --
 .../gutil/raw_scoped_refptr_mismatch_checker.h     |   63 -
 src/kudu/util/CMakeLists.txt                       |    1 -
 src/kudu/util/callback_bind-test.cc                |  119 -
 16 files changed, 12 insertions(+), 6047 deletions(-)

diff --git a/build-support/release/rat_exclude_files.txt b/build-support/release/rat_exclude_files.txt
index 9401276..cceb953 100644
--- a/build-support/release/rat_exclude_files.txt
+++ b/build-support/release/rat_exclude_files.txt
@@ -53,18 +53,8 @@ src/kudu/gutil/atomicops-internals-macosx.h
 src/kudu/gutil/atomicops-internals-tsan.h
 src/kudu/gutil/atomicops.h
 src/kudu/gutil/basictypes.h
-src/kudu/gutil/bind.h
-src/kudu/gutil/bind.h.pump
-src/kudu/gutil/bind_helpers.h
-src/kudu/gutil/bind_internal.h
-src/kudu/gutil/bind_internal.h.pump
 src/kudu/gutil/bits.cc
 src/kudu/gutil/bits.h
-src/kudu/gutil/callback.h
-src/kudu/gutil/callback.h.pump
-src/kudu/gutil/callback_forward.h
-src/kudu/gutil/callback_internal.cc
-src/kudu/gutil/callback_internal.h
 src/kudu/gutil/casts.h
 src/kudu/gutil/charmap.h
 src/kudu/gutil/cpu.cc
diff --git a/docs/contributing.adoc b/docs/contributing.adoc
index ea72369..4bb8339 100644
--- a/docs/contributing.adoc
+++ b/docs/contributing.adoc
@@ -317,28 +317,18 @@ in many places. When interfacing with that code, you can continue to use `shared
 
 ==== Function Binding and Callbacks
 
-Existing code uses `boost::bind` and `boost::function` to capture and manage
-functors. For new code, use `std::bind` and `std::function`, which are
-functionally equivalent.
-
-Alternatively, the `Bind` and `Callback` classes in `gutil` may also be used to
-capture functors. See _gutil/callback.h_ for more details and
-_util/callback_bind-test.cc_ for examples. While less full-featured, they
-provide different options from their counterparts by the way of argument
-lifecycle management. The benefits of each are described below:
-
-.`std::bind` and `std::function`
-[none]
-* natively supports binding `shared_ptr` and `weak_ptr` objects, so a bound
-smartpointer argument will increment its count during `bind`, and decrement it
-when the `function` leaves scope
-* supports argument placeholders, wrapped function pointers, and function objects
-
-.`Bind` and `Callback`
-[none]
-* natively supports binding `RefCounted` objects, so a bound argument whose
-class extends `RefCounted` will increment its count during `Bind` and decrement
-it when the `Callback` goes out of scope
+All code should use {cpp}11 lambdas to capture and manage functors. Functions that
+take a lambda as an argument should use `std::function` as the argument's
+type. Do not use `boost::bind` or `std::bind` to create functors. Lambdas offer
+the compiler greater opportunity to inline, and `std::bind` in particular is
+link:https://abseil.io/tips/108[error-prone] and has a proclivity towards heap
+allocation for storing bound parameters.
+
+Until Kudu is upgraded to {cpp}14, lambda support will be
+link:https://stackoverflow.com/a/17545183[somewhat incomplete]. For example, it
+is not possible in {cpp}11 to capture an argument by move. Nor is it possible
+to define new variables in the context of a lambda capture. Workarounds for
+these deficiencies exist, and they must be used in the interim.
 
 ==== GFlags
 
diff --git a/src/kudu/gutil/CMakeLists.txt b/src/kudu/gutil/CMakeLists.txt
index f24a41e..53bf62d 100644
--- a/src/kudu/gutil/CMakeLists.txt
+++ b/src/kudu/gutil/CMakeLists.txt
@@ -18,7 +18,6 @@
 set(GUTIL_SRCS
   atomicops-internals-x86.cc
   bits.cc
-  callback_internal.cc
   cpu.cc
   dynamic_annotations.c
   hash/city.cc
diff --git a/src/kudu/gutil/bind.h b/src/kudu/gutil/bind.h
deleted file mode 100644
index 8875f70..0000000
--- a/src/kudu/gutil/bind.h
+++ /dev/null
@@ -1,539 +0,0 @@
-// This file was GENERATED by command:
-//     pump.py bind.h.pump
-// DO NOT EDIT BY HAND!!!
-
-
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_BIND_H_
-#define KUDU_GUTIL_BIND_H_
-
-#include "kudu/gutil/bind_internal.h"
-#include "kudu/gutil/callback_internal.h"
-
-// -----------------------------------------------------------------------------
-// Usage documentation
-// -----------------------------------------------------------------------------
-//
-// See kudu/gutil/callback.h for documentation.
-//
-//
-// -----------------------------------------------------------------------------
-// Implementation notes
-// -----------------------------------------------------------------------------
-//
-// If you're reading the implementation, before proceeding further, you should
-// read the top comment of kudu/gutil/bind_internal.h for a definition of common
-// terms and concepts.
-//
-// RETURN TYPES
-//
-// Though Bind()'s result is meant to be stored in a Callback<> type, it
-// cannot actually return the exact type without requiring a large amount
-// of extra template specializations. The problem is that in order to
-// discern the correct specialization of Callback<>, Bind would need to
-// unwrap the function signature to determine the signature's arity, and
-// whether or not it is a method.
-//
-// Each unique combination of (arity, function_type, num_prebound) where
-// function_type is one of {function, method, const_method} would require
-// one specialization.  We eventually have to do a similar number of
-// specializations anyways in the implementation (see the Invoker<>,
-// classes).  However, it is avoidable in Bind if we return the result
-// via an indirection like we do below.
-//
-// TODO(ajwong): We might be able to avoid this now, but need to test.
-//
-// It is possible to move most of the COMPILE_ASSERT asserts into BindState<>,
-// but it feels a little nicer to have the asserts here so people do not
-// need to crack open bind_internal.h.  On the other hand, it makes Bind()
-// harder to read.
-
-namespace kudu {
-
-template <typename Functor>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void()>
-            ::UnboundRunType>
-Bind(Functor functor) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  typedef internal::BindState<RunnableType, RunType, void()> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor)));
-}
-
-template <typename Functor, typename P1>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1));
-}
-
-template <typename Functor, typename P1, typename P2>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2));
-}
-
-template <typename Functor, typename P1, typename P2, typename P3>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType,
-            typename internal::CallbackParamTraits<P3>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2, const P3& p3) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A3Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P3>::value,
-                 p3_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType,
-      typename internal::CallbackParamTraits<P3>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2, p3));
-}
-
-template <typename Functor, typename P1, typename P2, typename P3, typename P4>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType,
-            typename internal::CallbackParamTraits<P3>::StorageType,
-            typename internal::CallbackParamTraits<P4>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2, const P3& p3, const P4& p4) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A3Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A4Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P3>::value,
-                 p3_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P4>::value,
-                 p4_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType,
-      typename internal::CallbackParamTraits<P3>::StorageType,
-      typename internal::CallbackParamTraits<P4>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2, p3, p4));
-}
-
-template <typename Functor, typename P1, typename P2, typename P3, typename P4,
-    typename P5>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType,
-            typename internal::CallbackParamTraits<P3>::StorageType,
-            typename internal::CallbackParamTraits<P4>::StorageType,
-            typename internal::CallbackParamTraits<P5>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2, const P3& p3, const P4& p4,
-    const P5& p5) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A3Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A4Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A5Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P3>::value,
-                 p3_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P4>::value,
-                 p4_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P5>::value,
-                 p5_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType,
-      typename internal::CallbackParamTraits<P3>::StorageType,
-      typename internal::CallbackParamTraits<P4>::StorageType,
-      typename internal::CallbackParamTraits<P5>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2, p3, p4, p5));
-}
-
-template <typename Functor, typename P1, typename P2, typename P3, typename P4,
-    typename P5, typename P6>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType,
-            typename internal::CallbackParamTraits<P3>::StorageType,
-            typename internal::CallbackParamTraits<P4>::StorageType,
-            typename internal::CallbackParamTraits<P5>::StorageType,
-            typename internal::CallbackParamTraits<P6>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2, const P3& p3, const P4& p4,
-    const P5& p5, const P6& p6) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A3Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A4Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A5Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A6Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P3>::value,
-                 p3_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P4>::value,
-                 p4_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P5>::value,
-                 p5_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P6>::value,
-                 p6_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType,
-      typename internal::CallbackParamTraits<P3>::StorageType,
-      typename internal::CallbackParamTraits<P4>::StorageType,
-      typename internal::CallbackParamTraits<P5>::StorageType,
-      typename internal::CallbackParamTraits<P6>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2, p3, p4, p5, p6));
-}
-
-template <typename Functor, typename P1, typename P2, typename P3, typename P4,
-    typename P5, typename P6, typename P7>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void(typename internal::CallbackParamTraits<P1>::StorageType,
-            typename internal::CallbackParamTraits<P2>::StorageType,
-            typename internal::CallbackParamTraits<P3>::StorageType,
-            typename internal::CallbackParamTraits<P4>::StorageType,
-            typename internal::CallbackParamTraits<P5>::StorageType,
-            typename internal::CallbackParamTraits<P6>::StorageType,
-            typename internal::CallbackParamTraits<P7>::StorageType)>
-            ::UnboundRunType>
-Bind(Functor functor, const P1& p1, const P2& p2, const P3& p3, const P4& p4,
-    const P5& p5, const P6& p6, const P7& p7) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !(base::is_non_const_reference<typename
-          BoundFunctorTraits::A1Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A2Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A3Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A4Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A5Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A6Type>::value ||
-          base::is_non_const_reference<typename
-          BoundFunctorTraits::A7Type>::value ),
-      do_not_bind_functions_with_nonconst_ref);
-
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P1>::value,
-      p1_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P1>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P2>::value,
-                 p2_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P3>::value,
-                 p3_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P4>::value,
-                 p4_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P5>::value,
-                 p5_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P6>::value,
-                 p6_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P7>::value,
-                 p7_is_refcounted_type_and_needs_scoped_refptr);
-  typedef internal::BindState<RunnableType, RunType,
-      void(typename internal::CallbackParamTraits<P1>::StorageType,
-      typename internal::CallbackParamTraits<P2>::StorageType,
-      typename internal::CallbackParamTraits<P3>::StorageType,
-      typename internal::CallbackParamTraits<P4>::StorageType,
-      typename internal::CallbackParamTraits<P5>::StorageType,
-      typename internal::CallbackParamTraits<P6>::StorageType,
-      typename internal::CallbackParamTraits<P7>::StorageType)> BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor), p1, p2, p3, p4, p5, p6,
-          p7));
-}
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_BIND_H_
diff --git a/src/kudu/gutil/bind.h.pump b/src/kudu/gutil/bind.h.pump
deleted file mode 100644
index 2bf6a91..0000000
--- a/src/kudu/gutil/bind.h.pump
+++ /dev/null
@@ -1,153 +0,0 @@
-$$ This is a pump file for generating file templates.  Pump is a python
-$$ script that is part of the Google Test suite of utilities.  Description
-$$ can be found here:
-$$
-$$ http://code.google.com/p/googletest/wiki/PumpManual
-$$
-
-$$
-$$ MAX_ARITY controls the number of arguments that Bind() supports.
-$$ The amount of code, and more importantly, the number of template types
-$$ generated by pump grows at O(MAX_ARITY^2).
-$$
-$$ We tried going to 11 and found it imposed an extra 10 penalty on windows
-$$ cycle times compared to our original baseline of 6.
-$$
-$$ Currently 7 is chosen as a compromise between supporting a convenient
-$$ number of arguments and keeping compile times low.  At 7, we have 115
-$$ templates being generated by pump.
-$$
-$$ Be careful when adjusting this number.  If people find a need to bind
-$$ a larger number of arguments, consider refactoring the function to use
-$$ a param struct instead of raising the MAX_ARITY.
-$$
-$$ See http://crbug.com/98542 for more context.
-$$
-$var MAX_ARITY = 7
-
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_BIND_H_
-#define KUDU_GUTIL_BIND_H_
-
-#include "kudu/gutil/bind_internal.h"
-#include "kudu/gutil/callback_internal.h"
-
-// -----------------------------------------------------------------------------
-// Usage documentation
-// -----------------------------------------------------------------------------
-//
-// See kudu/gutil/callback.h for documentation.
-//
-//
-// -----------------------------------------------------------------------------
-// Implementation notes
-// -----------------------------------------------------------------------------
-//
-// If you're reading the implementation, before proceeding further, you should
-// read the top comment of kudu/gutil/bind_internal.h for a definition of common
-// terms and concepts.
-//
-// RETURN TYPES
-//
-// Though Bind()'s result is meant to be stored in a Callback<> type, it
-// cannot actually return the exact type without requiring a large amount
-// of extra template specializations. The problem is that in order to
-// discern the correct specialization of Callback<>, Bind would need to
-// unwrap the function signature to determine the signature's arity, and
-// whether or not it is a method.
-//
-// Each unique combination of (arity, function_type, num_prebound) where
-// function_type is one of {function, method, const_method} would require
-// one specialization.  We eventually have to do a similar number of
-// specializations anyways in the implementation (see the Invoker<>,
-// classes).  However, it is avoidable in Bind if we return the result
-// via an indirection like we do below.
-//
-// TODO(ajwong): We might be able to avoid this now, but need to test.
-//
-// It is possible to move most of the COMPILE_ASSERT asserts into BindState<>,
-// but it feels a little nicer to have the asserts here so people do not
-// need to crack open bind_internal.h.  On the other hand, it makes Bind()
-// harder to read.
-
-namespace kudu {
-
-$range ARITY 0..MAX_ARITY
-$for ARITY [[
-$range ARG 1..ARITY
-
-template <typename Functor[[]]
-$if ARITY > 0 [[, ]] $for ARG , [[typename P$(ARG)]]>
-Callback<
-    typename internal::BindState<
-        typename internal::FunctorTraits<Functor>::RunnableType,
-        typename internal::FunctorTraits<Functor>::RunType,
-        void($for ARG , [[typename internal::CallbackParamTraits<P$(ARG)>::StorageType]])>
-            ::UnboundRunType>
-Bind(Functor functor
-$if ARITY > 0 [[, ]] $for ARG , [[const P$(ARG)& p$(ARG)]]) {
-  // Typedefs for how to store and run the functor.
-  typedef typename internal::FunctorTraits<Functor>::RunnableType RunnableType;
-  typedef typename internal::FunctorTraits<Functor>::RunType RunType;
-
-$if ARITY > 0 [[
-
-  // Use RunnableType::RunType instead of RunType above because our
-  // checks should below for bound references need to know what the actual
-  // functor is going to interpret the argument as.
-  typedef internal::FunctionTraits<typename RunnableType::RunType>
-      BoundFunctorTraits;
-
-  // Do not allow binding a non-const reference parameter. Non-const reference
-  // parameters are disallowed by the Google style guide.  Also, binding a
-  // non-const reference parameter can make for subtle bugs because the
-  // invoked function will receive a reference to the stored copy of the
-  // argument and not the original.
-  COMPILE_ASSERT(
-      !($for ARG || [[
-base::is_non_const_reference<typename BoundFunctorTraits::A$(ARG)Type>::value ]]),
-      do_not_bind_functions_with_nonconst_ref);
-
-]]
-
-
-$for ARG [[
-
-
-$if ARG == 1 [[
-  // For methods, we need to be careful for parameter 1.  We do not require
-  // a scoped_refptr because BindState<> itself takes care of AddRef() for
-  // methods. We also disallow binding of an array as the method's target
-  // object.
-  COMPILE_ASSERT(
-      internal::HasIsMethodTag<RunnableType>::value ||
-          !internal::NeedsScopedRefptrButGetsRawPtr<P$(ARG)>::value,
-      p$(ARG)_is_refcounted_type_and_needs_scoped_refptr);
-  COMPILE_ASSERT(!internal::HasIsMethodTag<RunnableType>::value ||
-                     !base::is_array<P$(ARG)>::value,
-                 first_bound_argument_to_method_cannot_be_array);
-]] $else [[
-  COMPILE_ASSERT(!internal::NeedsScopedRefptrButGetsRawPtr<P$(ARG)>::value,
-                 p$(ARG)_is_refcounted_type_and_needs_scoped_refptr);
-]]  $$ $if ARG
-
-]]  $$ $for ARG
-
-  typedef internal::BindState<RunnableType, RunType, [[]]
-void($for ARG , [[typename internal::CallbackParamTraits<P$(ARG)>::StorageType]])> [[]]
-BindState;
-
-
-  return Callback<typename BindState::UnboundRunType>(
-      new BindState(internal::MakeRunnable(functor)[[]]
-$if ARITY > 0 [[, ]] $for ARG , [[p$(ARG)]]));
-}
-
-]]  $$ for ARITY
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_BIND_H_
diff --git a/src/kudu/gutil/bind_helpers.h b/src/kudu/gutil/bind_helpers.h
deleted file mode 100644
index f8dabe6..0000000
--- a/src/kudu/gutil/bind_helpers.h
+++ /dev/null
@@ -1,547 +0,0 @@
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-// This defines a set of argument wrappers and related factory methods that
-// can be used specify the refcounting and reference semantics of arguments
-// that are bound by the Bind() function in kudu/gutil/bind.h.
-//
-// It also defines a set of simple functions and utilities that people want
-// when using Callback<> and Bind().
-//
-//
-// ARGUMENT BINDING WRAPPERS
-//
-// The wrapper functions are kudu::Unretained(), kudu::Owned(), kudu::Passed(),
-// kudu::ConstRef(), and kudu::IgnoreResult().
-//
-// Unretained() allows Bind() to bind a non-refcounted class, and to disable
-// refcounting on arguments that are refcounted objects.
-//
-// Owned() transfers ownership of an object to the Callback resulting from
-// bind; the object will be deleted when the Callback is deleted.
-//
-// Passed() is for transferring movable-but-not-copyable types (eg. scoped_ptr)
-// through a Callback. Logically, this signifies a destructive transfer of
-// the state of the argument into the target function.  Invoking
-// Callback::Run() twice on a Callback that was created with a Passed()
-// argument will CHECK() because the first invocation would have already
-// transferred ownership to the target function.
-//
-// ConstRef() allows binding a constant reference to an argument rather
-// than a copy.
-//
-// IgnoreResult() is used to adapt a function or Callback with a return type to
-// one with a void return. This is most useful if you have a function with,
-// say, a pesky ignorable bool return that you want to use with PostTask or
-// something else that expect a Callback with a void return.
-//
-// EXAMPLE OF Unretained():
-//
-//   class Foo {
-//    public:
-//     void func() { cout << "Foo:f" << endl; }
-//   };
-//
-//   // In some function somewhere.
-//   Foo foo;
-//   Closure foo_callback =
-//       Bind(&Foo::func, Unretained(&foo));
-//   foo_callback.Run();  // Prints "Foo:f".
-//
-// Without the Unretained() wrapper on |&foo|, the above call would fail
-// to compile because Foo does not support the AddRef() and Release() methods.
-//
-//
-// EXAMPLE OF Owned():
-//
-//   void foo(int* arg) { cout << *arg << endl }
-//
-//   int* pn = new int(1);
-//   Closure foo_callback = Bind(&foo, Owned(pn));
-//
-//   foo_callback.Run();  // Prints "1"
-//   foo_callback.Run();  // Prints "1"
-//   *n = 2;
-//   foo_callback.Run();  // Prints "2"
-//
-//   foo_callback.Reset();  // |pn| is deleted.  Also will happen when
-//                          // |foo_callback| goes out of scope.
-//
-// Without Owned(), someone would have to know to delete |pn| when the last
-// reference to the Callback is deleted.
-//
-//
-// EXAMPLE OF ConstRef():
-//
-//   void foo(int arg) { cout << arg << endl }
-//
-//   int n = 1;
-//   Closure no_ref = Bind(&foo, n);
-//   Closure has_ref = Bind(&foo, ConstRef(n));
-//
-//   no_ref.Run();  // Prints "1"
-//   has_ref.Run();  // Prints "1"
-//
-//   n = 2;
-//   no_ref.Run();  // Prints "1"
-//   has_ref.Run();  // Prints "2"
-//
-// Note that because ConstRef() takes a reference on |n|, |n| must outlive all
-// its bound callbacks.
-//
-//
-// EXAMPLE OF IgnoreResult():
-//
-//   int DoSomething(int arg) { cout << arg << endl; }
-//
-//   // Assign to a Callback with a void return type.
-//   Callback<void(int)> cb = Bind(IgnoreResult(&DoSomething));
-//   cb->Run(1);  // Prints "1".
-//
-//   // Prints "1" on |ml|.
-//   ml->PostTask(FROM_HERE, Bind(IgnoreResult(&DoSomething), 1);
-//
-//
-// EXAMPLE OF Passed():
-//
-//   void TakesOwnership(scoped_ptr<Foo> arg) { }
-//   scoped_ptr<Foo> CreateFoo() { return scoped_ptr<Foo>(new Foo()); }
-//
-//   scoped_ptr<Foo> f(new Foo());
-//
-//   // |cb| is given ownership of Foo(). |f| is now NULL.
-//   // You can use f.Pass() in place of &f, but it's more verbose.
-//   Closure cb = Bind(&TakesOwnership, Passed(&f));
-//
-//   // Run was never called so |cb| still owns Foo() and deletes
-//   // it on Reset().
-//   cb.Reset();
-//
-//   // |cb| is given a new Foo created by CreateFoo().
-//   cb = Bind(&TakesOwnership, Passed(CreateFoo()));
-//
-//   // |arg| in TakesOwnership() is given ownership of Foo(). |cb|
-//   // no longer owns Foo() and, if reset, would not delete Foo().
-//   cb.Run();  // Foo() is now transferred to |arg| and deleted.
-//   cb.Run();  // This CHECK()s since Foo() already been used once.
-//
-// Passed() is particularly useful with PostTask() when you are transferring
-// ownership of an argument into a task, but don't necessarily know if the
-// task will always be executed. This can happen if the task is cancellable
-// or if it is posted to a MessageLoopProxy.
-//
-//
-// SIMPLE FUNCTIONS AND UTILITIES.
-//
-//   DoNothing() - Useful for creating a Closure that does nothing when called.
-//   DeletePointer<T>() - Useful for creating a Closure that will delete a
-//                        pointer when invoked. Only use this when necessary.
-//                        In most cases MessageLoop::DeleteSoon() is a better
-//                        fit.
-
-#ifndef KUDU_GUTIL_BIND_HELPERS_H_
-#define KUDU_GUTIL_BIND_HELPERS_H_
-
-#include <assert.h>
-
-#include "kudu/gutil/basictypes.h"
-#include "kudu/gutil/callback.h"
-#include "kudu/gutil/template_util.h"
-
-namespace kudu {
-namespace internal {
-
-// Use the Substitution Failure Is Not An Error (SFINAE) trick to inspect T
-// for the existence of AddRef() and Release() functions of the correct
-// signature.
-//
-// http://en.wikipedia.org/wiki/Substitution_failure_is_not_an_error
-// http://stackoverflow.com/questions/257288/is-it-possible-to-write-a-c-template-to-check-for-a-functions-existence
-// http://stackoverflow.com/questions/4358584/sfinae-approach-comparison
-// http://stackoverflow.com/questions/1966362/sfinae-to-check-for-inherited-member-functions
-//
-// The last link in particular show the method used below.
-//
-// For SFINAE to work with inherited methods, we need to pull some extra tricks
-// with multiple inheritance.  In the more standard formulation, the overloads
-// of Check would be:
-//
-//   template <typename C>
-//   Yes NotTheCheckWeWant(Helper<&C::TargetFunc>*);
-//
-//   template <typename C>
-//   No NotTheCheckWeWant(...);
-//
-//   static const bool value = sizeof(NotTheCheckWeWant<T>(0)) == sizeof(Yes);
-//
-// The problem here is that template resolution will not match
-// C::TargetFunc if TargetFunc does not exist directly in C.  That is, if
-// TargetFunc in inherited from an ancestor, &C::TargetFunc will not match,
-// |value| will be false.  This formulation only checks for whether or
-// not TargetFunc exist directly in the class being introspected.
-//
-// To get around this, we play a dirty trick with multiple inheritance.
-// First, We create a class BaseMixin that declares each function that we
-// want to probe for.  Then we create a class Base that inherits from both T
-// (the class we wish to probe) and BaseMixin.  Note that the function
-// signature in BaseMixin does not need to match the signature of the function
-// we are probing for; thus it's easiest to just use void(void).
-//
-// Now, if TargetFunc exists somewhere in T, then &Base::TargetFunc has an
-// ambiguous resolution between BaseMixin and T.  This lets us write the
-// following:
-//
-//   template <typename C>
-//   No GoodCheck(Helper<&C::TargetFunc>*);
-//
-//   template <typename C>
-//   Yes GoodCheck(...);
-//
-//   static const bool value = sizeof(GoodCheck<Base>(0)) == sizeof(Yes);
-//
-// Notice here that the variadic version of GoodCheck() returns Yes here
-// instead of No like the previous one. Also notice that we calculate |value|
-// by specializing GoodCheck() on Base instead of T.
-//
-// We've reversed the roles of the variadic, and Helper overloads.
-// GoodCheck(Helper<&C::TargetFunc>*), when C = Base, fails to be a valid
-// substitution if T::TargetFunc exists. Thus GoodCheck<Base>(0) will resolve
-// to the variadic version if T has TargetFunc.  If T::TargetFunc does not
-// exist, then &C::TargetFunc is not ambiguous, and the overload resolution
-// will prefer GoodCheck(Helper<&C::TargetFunc>*).
-//
-// This method of SFINAE will correctly probe for inherited names, but it cannot
-// typecheck those names.  It's still a good enough sanity check though.
-//
-// Works on gcc-4.2, gcc-4.4, and Visual Studio 2008.
-//
-// TODO(ajwong): Move to ref_counted.h or template_util.h when we've vetted
-// this works well.
-//
-// TODO(ajwong): Make this check for Release() as well.
-// See http://crbug.com/82038.
-template <typename T>
-class SupportsAddRefAndRelease {
-  typedef char Yes[1];
-  typedef char No[2];
-
-  struct BaseMixin {
-    void AddRef();
-  };
-
-// MSVC warns when you try to use Base if T has a private destructor, the
-// common pattern for refcounted types. It does this even though no attempt to
-// instantiate Base is made.  We disable the warning for this definition.
-#if defined(OS_WIN)
-#pragma warning(push)
-#pragma warning(disable:4624)
-#endif
-  struct Base : public T, public BaseMixin {
-  };
-#if defined(OS_WIN)
-#pragma warning(pop)
-#endif
-
-  template <void(BaseMixin::*)(void)> struct Helper {};
-
-  template <typename C>
-  static No& Check(Helper<&C::AddRef>*);
-
-  template <typename >
-  static Yes& Check(...);
-
- public:
-  static const bool value = sizeof(Check<Base>(0)) == sizeof(Yes);
-};
-
-// Helpers to assert that arguments of a recounted type are bound with a
-// scoped_refptr.
-template <bool IsClasstype, typename T>
-struct UnsafeBindtoRefCountedArgHelper : base::false_type {
-};
-
-template <typename T>
-struct UnsafeBindtoRefCountedArgHelper<true, T>
-    : base::integral_constant<bool, SupportsAddRefAndRelease<T>::value> {
-};
-
-template <typename T>
-struct UnsafeBindtoRefCountedArg : base::false_type {
-};
-
-template <typename T>
-struct UnsafeBindtoRefCountedArg<T*>
-    : UnsafeBindtoRefCountedArgHelper<base::is_class<T>::value, T> {
-};
-
-template <typename T>
-class HasIsMethodTag {
-  typedef char Yes[1];
-  typedef char No[2];
-
-  template <typename U>
-  static Yes& Check(typename U::IsMethod*);
-
-  template <typename U>
-  static No& Check(...);
-
- public:
-  static const bool value = sizeof(Check<T>(0)) == sizeof(Yes);
-};
-
-template <typename T>
-class UnretainedWrapper {
- public:
-  explicit UnretainedWrapper(T* o) : ptr_(o) {}
-  T* get() const { return ptr_; }
- private:
-  T* ptr_;
-};
-
-template <typename T>
-class ConstRefWrapper {
- public:
-  explicit ConstRefWrapper(const T& o) : ptr_(&o) {}
-  const T& get() const { return *ptr_; }
- private:
-  const T* ptr_;
-};
-
-template <typename T>
-struct IgnoreResultHelper {
-  explicit IgnoreResultHelper(T functor) : functor_(functor) {}
-
-  T functor_;
-};
-
-template <typename T>
-struct IgnoreResultHelper<Callback<T> > {
-  explicit IgnoreResultHelper(const Callback<T>& functor) : functor_(functor) {}
-
-  const Callback<T>& functor_;
-};
-
-// An alternate implementation is to avoid the destructive copy, and instead
-// specialize ParamTraits<> for OwnedWrapper<> to change the StorageType to
-// a class that is essentially a scoped_ptr<>.
-//
-// The current implementation has the benefit though of leaving ParamTraits<>
-// fully in callback_internal.h as well as avoiding type conversions during
-// storage.
-template <typename T>
-class OwnedWrapper {
- public:
-  explicit OwnedWrapper(T* o) : ptr_(o) {}
-  ~OwnedWrapper() { delete ptr_; }
-  T* get() const { return ptr_; }
-  OwnedWrapper(const OwnedWrapper& other) {
-    ptr_ = other.ptr_;
-    other.ptr_ = NULL;
-  }
-
- private:
-  mutable T* ptr_;
-};
-
-// PassedWrapper is a copyable adapter for a scoper that ignores const.
-//
-// It is needed to get around the fact that Bind() takes a const reference to
-// all its arguments.  Because Bind() takes a const reference to avoid
-// unnecessary copies, it is incompatible with movable-but-not-copyable
-// types; doing a destructive "move" of the type into Bind() would violate
-// the const correctness.
-//
-// This conundrum cannot be solved without either C++11 rvalue references or
-// a O(2^n) blowup of Bind() templates to handle each combination of regular
-// types and movable-but-not-copyable types.  Thus we introduce a wrapper type
-// that is copyable to transmit the correct type information down into
-// BindState<>. Ignoring const in this type makes sense because it is only
-// created when we are explicitly trying to do a destructive move.
-//
-// Two notes:
-//  1) PassedWrapper supports any type that has a "Pass()" function.
-//     This is intentional. The whitelisting of which specific types we
-//     support is maintained by CallbackParamTraits<>.
-//  2) is_valid_ is distinct from NULL because it is valid to bind a "NULL"
-//     scoper to a Callback and allow the Callback to execute once.
-template <typename T>
-class PassedWrapper {
- public:
-  explicit PassedWrapper(T scoper) : is_valid_(true), scoper_(scoper.Pass()) {}
-  PassedWrapper(const PassedWrapper& other)
-      : is_valid_(other.is_valid_), scoper_(other.scoper_.Pass()) {
-  }
-  T Pass() const {
-    assert(is_valid_);
-    is_valid_ = false;
-    return scoper_.Pass();
-  }
-
- private:
-  mutable bool is_valid_;
-  mutable T scoper_;
-};
-
-// Unwrap the stored parameters for the wrappers above.
-template <typename T>
-struct UnwrapTraits {
-  typedef const T& ForwardType;
-  static ForwardType Unwrap(const T& o) { return o; }
-};
-
-template <typename T>
-struct UnwrapTraits<UnretainedWrapper<T> > {
-  typedef T* ForwardType;
-  static ForwardType Unwrap(UnretainedWrapper<T> unretained) {
-    return unretained.get();
-  }
-};
-
-template <typename T>
-struct UnwrapTraits<ConstRefWrapper<T> > {
-  typedef const T& ForwardType;
-  static ForwardType Unwrap(ConstRefWrapper<T> const_ref) {
-    return const_ref.get();
-  }
-};
-
-template <typename T>
-struct UnwrapTraits<scoped_refptr<T> > {
-  typedef T* ForwardType;
-  static ForwardType Unwrap(const scoped_refptr<T>& o) { return o.get(); }
-};
-
-// We didn't import WeakPtr from Chromium.
-//
-//template <typename T>
-//struct UnwrapTraits<WeakPtr<T> > {
-//  typedef const WeakPtr<T>& ForwardType;
-//  static ForwardType Unwrap(const WeakPtr<T>& o) { return o; }
-//};
-
-template <typename T>
-struct UnwrapTraits<OwnedWrapper<T> > {
-  typedef T* ForwardType;
-  static ForwardType Unwrap(const OwnedWrapper<T>& o) {
-    return o.get();
-  }
-};
-
-template <typename T>
-struct UnwrapTraits<PassedWrapper<T> > {
-  typedef T ForwardType;
-  static T Unwrap(PassedWrapper<T>& o) {
-    return o.Pass();
-  }
-};
-
-// Utility for handling different refcounting semantics in the Bind()
-// function.
-template <bool is_method, typename T>
-struct MaybeRefcount;
-
-template <typename T>
-struct MaybeRefcount<false, T> {
-  static void AddRef(const T&) {}
-  static void Release(const T&) {}
-};
-
-template <typename T, size_t n>
-struct MaybeRefcount<false, T[n]> {
-  static void AddRef(const T*) {}
-  static void Release(const T*) {}
-};
-
-template <typename T>
-struct MaybeRefcount<true, T> {
-  static void AddRef(const T&) {}
-  static void Release(const T&) {}
-};
-
-template <typename T>
-struct MaybeRefcount<true, T*> {
-  static void AddRef(T* o) { o->AddRef(); }
-  static void Release(T* o) { o->Release(); }
-};
-
-// No need to additionally AddRef() and Release() since we are storing a
-// scoped_refptr<> inside the storage object already.
-template <typename T>
-struct MaybeRefcount<true, scoped_refptr<T> > {
-  static void AddRef(const scoped_refptr<T>& o) {}
-  static void Release(const scoped_refptr<T>& o) {}
-};
-
-template <typename T>
-struct MaybeRefcount<true, const T*> {
-  static void AddRef(const T* o) { o->AddRef(); }
-  static void Release(const T* o) { o->Release(); }
-};
-
-// We didn't import WeakPtr from Chromium.
-//
-//// IsWeakMethod is a helper that determine if we are binding a WeakPtr<> to a
-//// method.  It is used internally by Bind() to select the correct
-//// InvokeHelper that will no-op itself in the event the WeakPtr<> for
-//// the target object is invalidated.
-////
-//// P1 should be the type of the object that will be received of the method.
-//template <bool IsMethod, typename P1>
-//struct IsWeakMethod : public false_type {};
-//
-//template <typename T>
-//struct IsWeakMethod<true, WeakPtr<T> > : public true_type {};
-//
-//template <typename T>
-//struct IsWeakMethod<true, ConstRefWrapper<WeakPtr<T> > > : public true_type {};
-
-}  // namespace internal
-
-template <typename T>
-static inline internal::UnretainedWrapper<T> Unretained(T* o) {
-  return internal::UnretainedWrapper<T>(o);
-}
-
-template <typename T>
-static inline internal::ConstRefWrapper<T> ConstRef(const T& o) {
-  return internal::ConstRefWrapper<T>(o);
-}
-
-template <typename T>
-static inline internal::OwnedWrapper<T> Owned(T* o) {
-  return internal::OwnedWrapper<T>(o);
-}
-
-// We offer 2 syntaxes for calling Passed().  The first takes a temporary and
-// is best suited for use with the return value of a function. The second
-// takes a pointer to the scoper and is just syntactic sugar to avoid having
-// to write Passed(scoper.Pass()).
-template <typename T>
-static inline internal::PassedWrapper<T> Passed(T scoper) {
-  return internal::PassedWrapper<T>(scoper.Pass());
-}
-template <typename T>
-static inline internal::PassedWrapper<T> Passed(T* scoper) {
-  return internal::PassedWrapper<T>(scoper->Pass());
-}
-
-template <typename T>
-static inline internal::IgnoreResultHelper<T> IgnoreResult(T data) {
-  return internal::IgnoreResultHelper<T>(data);
-}
-
-template <typename T>
-static inline internal::IgnoreResultHelper<Callback<T> >
-IgnoreResult(const Callback<T>& data) {
-  return internal::IgnoreResultHelper<Callback<T> >(data);
-}
-
-template<typename T>
-void DeletePointer(T* obj) {
-  delete obj;
-}
-
-}  // namespace kudu
-
-#endif  // BASE_BIND_HELPERS_H_
diff --git a/src/kudu/gutil/bind_internal.h b/src/kudu/gutil/bind_internal.h
deleted file mode 100644
index 84d7363..0000000
--- a/src/kudu/gutil/bind_internal.h
+++ /dev/null
@@ -1,2697 +0,0 @@
-// This file was GENERATED by command:
-//     pump.py bind_internal.h.pump
-// DO NOT EDIT BY HAND!!!
-
-
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_BIND_INTERNAL_H_
-#define KUDU_GUTIL_BIND_INTERNAL_H_
-
-#include <utility>
-
-#include "kudu/gutil/bind_helpers.h"
-#include "kudu/gutil/callback_internal.h"
-#include "kudu/gutil/raw_scoped_refptr_mismatch_checker.h"
-#include "kudu/gutil/template_util.h"
-
-#if defined(OS_WIN)
-#include "kudu/gutil/bind_internal_win.h"
-#endif
-
-// During Chromium import, WeakPtr-related code was removed.
-
-namespace kudu {
-namespace internal {
-
-// See kudu/gutil/callback.h for user documentation.
-//
-//
-// CONCEPTS:
-//  Runnable -- A type (really a type class) that has a single Run() method
-//              and a RunType typedef that corresponds to the type of Run().
-//              A Runnable can declare that it should treated like a method
-//              call by including a typedef named IsMethod.  The value of
-//              this typedef is NOT inspected, only the existence.  When a
-//              Runnable declares itself a method, Bind() will enforce special
-//              refcounting + WeakPtr handling semantics for the first
-//              parameter which is expected to be an object.
-//  Functor -- A copyable type representing something that should be called.
-//             All function pointers, Callback<>, and Runnables are functors
-//             even if the invocation syntax differs.
-//  RunType -- A function type (as opposed to function _pointer_ type) for
-//             a Run() function.  Usually just a convenience typedef.
-//  (Bound)ArgsType -- A function type that is being (ab)used to store the
-//                     types of set of arguments.  The "return" type is always
-//                     void here.  We use this hack so that we do not need
-//                     a new type name for each arity of type. (eg.,
-//                     BindState1, BindState2).  This makes forward
-//                     declarations and friending much much easier.
-//
-// Types:
-//  RunnableAdapter<> -- Wraps the various "function" pointer types into an
-//                       object that adheres to the Runnable interface.
-//                       There are |3*ARITY| RunnableAdapter types.
-//  FunctionTraits<> -- Type traits that unwrap a function signature into a
-//                      a set of easier to use typedefs.  Used mainly for
-//                      compile time asserts.
-//                      There are |ARITY| FunctionTraits types.
-//  ForceVoidReturn<> -- Helper class for translating function signatures to
-//                       equivalent forms with a "void" return type.
-//                    There are |ARITY| ForceVoidReturn types.
-//  FunctorTraits<> -- Type traits used determine the correct RunType and
-//                     RunnableType for a Functor.  This is where function
-//                     signature adapters are applied.
-//                    There are |ARITY| ForceVoidReturn types.
-//  MakeRunnable<> -- Takes a Functor and returns an object in the Runnable
-//                    type class that represents the underlying Functor.
-//                    There are |O(1)| MakeRunnable types.
-//  InvokeHelper<> -- Take a Runnable + arguments and actully invokes it.
-// Handle the differing syntaxes needed for WeakPtr<> support,
-//                    and for ignoring return values.  This is separate from
-//                    Invoker to avoid creating multiple version of Invoker<>
-//                    which grows at O(n^2) with the arity.
-//                    There are |k*ARITY| InvokeHelper types.
-//  Invoker<> -- Unwraps the curried parameters and executes the Runnable.
-//               There are |(ARITY^2 + ARITY)/2| Invoketypes.
-//  BindState<> -- Stores the curried parameters, and is the main entry point
-//                 into the Bind() system, doing most of the type resolution.
-//                 There are ARITY BindState types.
-
-// RunnableAdapter<>
-//
-// The RunnableAdapter<> templates provide a uniform interface for invoking
-// a function pointer, method pointer, or const method pointer. The adapter
-// exposes a Run() method with an appropriate signature. Using this wrapper
-// allows for writing code that supports all three pointer types without
-// undue repetition.  Without it, a lot of code would need to be repeated 3
-// times.
-//
-// For method pointers and const method pointers the first argument to Run()
-// is considered to be the received of the method.  This is similar to STL's
-// mem_fun().
-//
-// This class also exposes a RunType typedef that is the function type of the
-// Run() function.
-//
-// If and only if the wrapper contains a method or const method pointer, an
-// IsMethod typedef is exposed.  The existence of this typedef (NOT the value)
-// marks that the wrapper should be considered a method wrapper.
-
-template <typename Functor>
-class RunnableAdapter;
-
-// Function: Arity 0.
-template <typename R>
-class RunnableAdapter<R(*)()> {
- public:
-  typedef R (RunType)();
-
-  explicit RunnableAdapter(R(*function)())
-      : function_(function) {
-  }
-
-  R Run() {
-    return function_();
-  }
-
- private:
-  R (*function_)();
-};
-
-// Method: Arity 0.
-template <typename R, typename T>
-class RunnableAdapter<R(T::*)()> {
- public:
-  typedef R (RunType)(T*);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)())
-      : method_(method) {
-  }
-
-  R Run(T* object) {
-    return (object->*method_)();
-  }
-
- private:
-  R (T::*method_)();
-};
-
-// Const Method: Arity 0.
-template <typename R, typename T>
-class RunnableAdapter<R(T::*)() const> {
- public:
-  typedef R (RunType)(const T*);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)() const)
-      : method_(method) {
-  }
-
-  R Run(const T* object) {
-    return (object->*method_)();
-  }
-
- private:
-  R (T::*method_)() const;
-};
-
-// Function: Arity 1.
-template <typename R, typename A1>
-class RunnableAdapter<R(*)(A1)> {
- public:
-  typedef R (RunType)(A1);
-
-  explicit RunnableAdapter(R(*function)(A1))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1) {
-    return function_(CallbackForward(a1));
-  }
-
- private:
-  R (*function_)(A1);
-};
-
-// Method: Arity 1.
-template <typename R, typename T, typename A1>
-class RunnableAdapter<R(T::*)(A1)> {
- public:
-  typedef R (RunType)(T*, A1);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1) {
-    return (object->*method_)(CallbackForward(a1));
-  }
-
- private:
-  R (T::*method_)(A1);
-};
-
-// Const Method: Arity 1.
-template <typename R, typename T, typename A1>
-class RunnableAdapter<R(T::*)(A1) const> {
- public:
-  typedef R (RunType)(const T*, A1);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1) {
-    return (object->*method_)(CallbackForward(a1));
-  }
-
- private:
-  R (T::*method_)(A1) const;
-};
-
-// Function: Arity 2.
-template <typename R, typename A1, typename A2>
-class RunnableAdapter<R(*)(A1, A2)> {
- public:
-  typedef R (RunType)(A1, A2);
-
-  explicit RunnableAdapter(R(*function)(A1, A2))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2) {
-    return function_(CallbackForward(a1), CallbackForward(a2));
-  }
-
- private:
-  R (*function_)(A1, A2);
-};
-
-// Method: Arity 2.
-template <typename R, typename T, typename A1, typename A2>
-class RunnableAdapter<R(T::*)(A1, A2)> {
- public:
-  typedef R (RunType)(T*, A1, A2);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2));
-  }
-
- private:
-  R (T::*method_)(A1, A2);
-};
-
-// Const Method: Arity 2.
-template <typename R, typename T, typename A1, typename A2>
-class RunnableAdapter<R(T::*)(A1, A2) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2));
-  }
-
- private:
-  R (T::*method_)(A1, A2) const;
-};
-
-// Function: Arity 3.
-template <typename R, typename A1, typename A2, typename A3>
-class RunnableAdapter<R(*)(A1, A2, A3)> {
- public:
-  typedef R (RunType)(A1, A2, A3);
-
-  explicit RunnableAdapter(R(*function)(A1, A2, A3))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3) {
-    return function_(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3));
-  }
-
- private:
-  R (*function_)(A1, A2, A3);
-};
-
-// Method: Arity 3.
-template <typename R, typename T, typename A1, typename A2, typename A3>
-class RunnableAdapter<R(T::*)(A1, A2, A3)> {
- public:
-  typedef R (RunType)(T*, A1, A2, A3);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3);
-};
-
-// Const Method: Arity 3.
-template <typename R, typename T, typename A1, typename A2, typename A3>
-class RunnableAdapter<R(T::*)(A1, A2, A3) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2, A3);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3) const;
-};
-
-// Function: Arity 4.
-template <typename R, typename A1, typename A2, typename A3, typename A4>
-class RunnableAdapter<R(*)(A1, A2, A3, A4)> {
- public:
-  typedef R (RunType)(A1, A2, A3, A4);
-
-  explicit RunnableAdapter(R(*function)(A1, A2, A3, A4))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4) {
-    return function_(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4));
-  }
-
- private:
-  R (*function_)(A1, A2, A3, A4);
-};
-
-// Method: Arity 4.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4)> {
- public:
-  typedef R (RunType)(T*, A1, A2, A3, A4);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4);
-};
-
-// Const Method: Arity 4.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2, A3, A4);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4) const;
-};
-
-// Function: Arity 5.
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5>
-class RunnableAdapter<R(*)(A1, A2, A3, A4, A5)> {
- public:
-  typedef R (RunType)(A1, A2, A3, A4, A5);
-
-  explicit RunnableAdapter(R(*function)(A1, A2, A3, A4, A5))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5) {
-    return function_(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5));
-  }
-
- private:
-  R (*function_)(A1, A2, A3, A4, A5);
-};
-
-// Method: Arity 5.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5)> {
- public:
-  typedef R (RunType)(T*, A1, A2, A3, A4, A5);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5);
-};
-
-// Const Method: Arity 5.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2, A3, A4, A5);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5) const;
-};
-
-// Function: Arity 6.
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6>
-class RunnableAdapter<R(*)(A1, A2, A3, A4, A5, A6)> {
- public:
-  typedef R (RunType)(A1, A2, A3, A4, A5, A6);
-
-  explicit RunnableAdapter(R(*function)(A1, A2, A3, A4, A5, A6))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6) {
-    return function_(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6));
-  }
-
- private:
-  R (*function_)(A1, A2, A3, A4, A5, A6);
-};
-
-// Method: Arity 6.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5, typename A6>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5, A6)> {
- public:
-  typedef R (RunType)(T*, A1, A2, A3, A4, A5, A6);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5, A6))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5, A6);
-};
-
-// Const Method: Arity 6.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5, typename A6>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5, A6) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2, A3, A4, A5, A6);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5, A6) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5, A6) const;
-};
-
-// Function: Arity 7.
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6, typename A7>
-class RunnableAdapter<R(*)(A1, A2, A3, A4, A5, A6, A7)> {
- public:
-  typedef R (RunType)(A1, A2, A3, A4, A5, A6, A7);
-
-  explicit RunnableAdapter(R(*function)(A1, A2, A3, A4, A5, A6, A7))
-      : function_(function) {
-  }
-
-  R Run(typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6,
-      typename CallbackParamTraits<A7>::ForwardType a7) {
-    return function_(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6), CallbackForward(a7));
-  }
-
- private:
-  R (*function_)(A1, A2, A3, A4, A5, A6, A7);
-};
-
-// Method: Arity 7.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5, typename A6, typename A7>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5, A6, A7)> {
- public:
-  typedef R (RunType)(T*, A1, A2, A3, A4, A5, A6, A7);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5, A6, A7))
-      : method_(method) {
-  }
-
-  R Run(T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6,
-      typename CallbackParamTraits<A7>::ForwardType a7) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6), CallbackForward(a7));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5, A6, A7);
-};
-
-// Const Method: Arity 7.
-template <typename R, typename T, typename A1, typename A2, typename A3,
-    typename A4, typename A5, typename A6, typename A7>
-class RunnableAdapter<R(T::*)(A1, A2, A3, A4, A5, A6, A7) const> {
- public:
-  typedef R (RunType)(const T*, A1, A2, A3, A4, A5, A6, A7);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)(A1, A2, A3, A4, A5, A6, A7) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object, typename CallbackParamTraits<A1>::ForwardType a1,
-      typename CallbackParamTraits<A2>::ForwardType a2,
-      typename CallbackParamTraits<A3>::ForwardType a3,
-      typename CallbackParamTraits<A4>::ForwardType a4,
-      typename CallbackParamTraits<A5>::ForwardType a5,
-      typename CallbackParamTraits<A6>::ForwardType a6,
-      typename CallbackParamTraits<A7>::ForwardType a7) {
-    return (object->*method_)(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6), CallbackForward(a7));
-  }
-
- private:
-  R (T::*method_)(A1, A2, A3, A4, A5, A6, A7) const;
-};
-
-
-// FunctionTraits<>
-//
-// Breaks a function signature apart into typedefs for easier introspection.
-template <typename Sig>
-struct FunctionTraits;
-
-template <typename R>
-struct FunctionTraits<R()> {
-  typedef R ReturnType;
-};
-
-template <typename R, typename A1>
-struct FunctionTraits<R(A1)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-};
-
-template <typename R, typename A1, typename A2>
-struct FunctionTraits<R(A1, A2)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-};
-
-template <typename R, typename A1, typename A2, typename A3>
-struct FunctionTraits<R(A1, A2, A3)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-  typedef A3 A3Type;
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4>
-struct FunctionTraits<R(A1, A2, A3, A4)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-  typedef A3 A3Type;
-  typedef A4 A4Type;
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5>
-struct FunctionTraits<R(A1, A2, A3, A4, A5)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-  typedef A3 A3Type;
-  typedef A4 A4Type;
-  typedef A5 A5Type;
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6>
-struct FunctionTraits<R(A1, A2, A3, A4, A5, A6)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-  typedef A3 A3Type;
-  typedef A4 A4Type;
-  typedef A5 A5Type;
-  typedef A6 A6Type;
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6, typename A7>
-struct FunctionTraits<R(A1, A2, A3, A4, A5, A6, A7)> {
-  typedef R ReturnType;
-  typedef A1 A1Type;
-  typedef A2 A2Type;
-  typedef A3 A3Type;
-  typedef A4 A4Type;
-  typedef A5 A5Type;
-  typedef A6 A6Type;
-  typedef A7 A7Type;
-};
-
-
-// ForceVoidReturn<>
-//
-// Set of templates that support forcing the function return type to void.
-template <typename Sig>
-struct ForceVoidReturn;
-
-template <typename R>
-struct ForceVoidReturn<R()> {
-  typedef void(RunType)();
-};
-
-template <typename R, typename A1>
-struct ForceVoidReturn<R(A1)> {
-  typedef void(RunType)(A1);
-};
-
-template <typename R, typename A1, typename A2>
-struct ForceVoidReturn<R(A1, A2)> {
-  typedef void(RunType)(A1, A2);
-};
-
-template <typename R, typename A1, typename A2, typename A3>
-struct ForceVoidReturn<R(A1, A2, A3)> {
-  typedef void(RunType)(A1, A2, A3);
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4>
-struct ForceVoidReturn<R(A1, A2, A3, A4)> {
-  typedef void(RunType)(A1, A2, A3, A4);
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5>
-struct ForceVoidReturn<R(A1, A2, A3, A4, A5)> {
-  typedef void(RunType)(A1, A2, A3, A4, A5);
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6>
-struct ForceVoidReturn<R(A1, A2, A3, A4, A5, A6)> {
-  typedef void(RunType)(A1, A2, A3, A4, A5, A6);
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6, typename A7>
-struct ForceVoidReturn<R(A1, A2, A3, A4, A5, A6, A7)> {
-  typedef void(RunType)(A1, A2, A3, A4, A5, A6, A7);
-};
-
-
-// FunctorTraits<>
-//
-// See description at top of file.
-template <typename T>
-struct FunctorTraits {
-  typedef RunnableAdapter<T> RunnableType;
-  typedef typename RunnableType::RunType RunType;
-};
-
-template <typename T>
-struct FunctorTraits<IgnoreResultHelper<T> > {
-  typedef typename FunctorTraits<T>::RunnableType RunnableType;
-  typedef typename ForceVoidReturn<
-      typename RunnableType::RunType>::RunType RunType;
-};
-
-template <typename T>
-struct FunctorTraits<Callback<T> > {
-  typedef Callback<T> RunnableType;
-  typedef typename Callback<T>::RunType RunType;
-};
-
-
-// MakeRunnable<>
-//
-// Converts a passed in functor to a RunnableType using type inference.
-
-template <typename T>
-typename FunctorTraits<T>::RunnableType MakeRunnable(const T& t) {
-  return RunnableAdapter<T>(t);
-}
-
-template <typename T>
-typename FunctorTraits<T>::RunnableType
-MakeRunnable(const IgnoreResultHelper<T>& t) {
-  return MakeRunnable(t.functor_);
-}
-
-template <typename T>
-const typename FunctorTraits<Callback<T> >::RunnableType&
-MakeRunnable(const Callback<T>& t) {
-  DCHECK(!t.is_null());
-  return t;
-}
-
-
-// InvokeHelper<>
-//
-// There are 3 logical InvokeHelper<> specializations: normal, void-return,
-// WeakCalls.
-//
-// The normal type just calls the underlying runnable.
-//
-// We need a InvokeHelper to handle void return types in order to support
-// IgnoreResult().  Normally, if the Runnable's RunType had a void return,
-// the template system would just accept "return functor.Run()" ignoring
-// the fact that a void function is being used with return. This piece of
-// sugar breaks though when the Runnable's RunType is not void.  Thus, we
-// need a partial specialization to change the syntax to drop the "return"
-// from the invocation call.
-//
-// WeakCalls similarly need special syntax that is applied to the first
-// argument to check if they should no-op themselves.
-template <bool IsWeakCall, typename ReturnType, typename Runnable,
-          typename ArgsType>
-struct InvokeHelper;
-
-template <typename ReturnType, typename Runnable>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void()>  {
-  static ReturnType MakeItSo(Runnable runnable) {
-    return runnable.Run();
-  }
-};
-
-template <typename Runnable>
-struct InvokeHelper<false, void, Runnable,
-    void()>  {
-  static void MakeItSo(Runnable runnable) {
-    runnable.Run();
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1) {
-    return runnable.Run(CallbackForward(a1));
-  }
-};
-
-template <typename Runnable,typename A1>
-struct InvokeHelper<false, void, Runnable,
-    void(A1)>  {
-  static void MakeItSo(Runnable runnable, A1 a1) {
-    runnable.Run(CallbackForward(a1));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2,
-    typename A3>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2, A3)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2, typename A3>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2, A3)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2), CallbackForward(a3));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2,
-    typename A3, typename A4>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2, A3, A4)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2, typename A3, typename A4>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2, A3, A4)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2), CallbackForward(a3),
-        CallbackForward(a4));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2,
-    typename A3, typename A4, typename A5>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2, A3, A4, A5)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4,
-      A5 a5) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2, typename A3, typename A4,
-    typename A5>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2, A3, A4, A5)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4, A5 a5) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2), CallbackForward(a3),
-        CallbackForward(a4), CallbackForward(a5));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2,
-    typename A3, typename A4, typename A5, typename A6>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2, A3, A4, A5, A6)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4,
-      A5 a5, A6 a6) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2, A3, A4, A5, A6)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4, A5 a5,
-      A6 a6) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2), CallbackForward(a3),
-        CallbackForward(a4), CallbackForward(a5), CallbackForward(a6));
-  }
-};
-
-template <typename ReturnType, typename Runnable,typename A1, typename A2,
-    typename A3, typename A4, typename A5, typename A6, typename A7>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void(A1, A2, A3, A4, A5, A6, A7)>  {
-  static ReturnType MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4,
-      A5 a5, A6 a6, A7 a7) {
-    return runnable.Run(CallbackForward(a1), CallbackForward(a2),
-        CallbackForward(a3), CallbackForward(a4), CallbackForward(a5),
-        CallbackForward(a6), CallbackForward(a7));
-  }
-};
-
-template <typename Runnable,typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6, typename A7>
-struct InvokeHelper<false, void, Runnable,
-    void(A1, A2, A3, A4, A5, A6, A7)>  {
-  static void MakeItSo(Runnable runnable, A1 a1, A2 a2, A3 a3, A4 a4, A5 a5,
-      A6 a6, A7 a7) {
-    runnable.Run(CallbackForward(a1), CallbackForward(a2), CallbackForward(a3),
-        CallbackForward(a4), CallbackForward(a5), CallbackForward(a6),
-        CallbackForward(a7));
-  }
-};
-
-// Invoker<>
-//
-// See description at the top of the file.
-template <int NumBound, typename Storage, typename RunType>
-struct Invoker;
-
-// Arity 0 -> 0.
-template <typename StorageType, typename R>
-struct Invoker<0, StorageType, R()> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void()>
-               ::MakeItSo(storage->runnable_);
-  }
-};
-
-// Arity 1 -> 1.
-template <typename StorageType, typename R,typename X1>
-struct Invoker<0, StorageType, R(X1)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType);
-
-  typedef R(UnboundRunType)(X1);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1));
-  }
-};
-
-// Arity 1 -> 0.
-template <typename StorageType, typename R,typename X1>
-struct Invoker<1, StorageType, R(X1)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1));
-  }
-};
-
-// Arity 2 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2>
-struct Invoker<0, StorageType, R(X1, X2)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2));
-  }
-};
-
-// Arity 2 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2>
-struct Invoker<1, StorageType, R(X1, X2)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType);
-
-  typedef R(UnboundRunType)(X2);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2));
-  }
-};
-
-// Arity 2 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2>
-struct Invoker<2, StorageType, R(X1, X2)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2));
-  }
-};
-
-// Arity 3 -> 3.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3>
-struct Invoker<0, StorageType, R(X1, X2, X3)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2, X3);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3));
-  }
-};
-
-// Arity 3 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3>
-struct Invoker<1, StorageType, R(X1, X2, X3)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType);
-
-  typedef R(UnboundRunType)(X2, X3);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3));
-  }
-};
-
-// Arity 3 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3>
-struct Invoker<2, StorageType, R(X1, X2, X3)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X3>::ForwardType);
-
-  typedef R(UnboundRunType)(X3);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X3>::ForwardType x3) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X3>::ForwardType x3)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3));
-  }
-};
-
-// Arity 3 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3>
-struct Invoker<3, StorageType, R(X1, X2, X3)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3));
-  }
-};
-
-// Arity 4 -> 4.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4>
-struct Invoker<0, StorageType, R(X1, X2, X3, X4)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2, X3, X4);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4));
-  }
-};
-
-// Arity 4 -> 3.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4>
-struct Invoker<1, StorageType, R(X1, X2, X3, X4)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType);
-
-  typedef R(UnboundRunType)(X2, X3, X4);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4));
-  }
-};
-
-// Arity 4 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4>
-struct Invoker<2, StorageType, R(X1, X2, X3, X4)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType);
-
-  typedef R(UnboundRunType)(X3, X4);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4));
-  }
-};
-
-// Arity 4 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4>
-struct Invoker<3, StorageType, R(X1, X2, X3, X4)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X4>::ForwardType);
-
-  typedef R(UnboundRunType)(X4);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X4>::ForwardType x4) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X4>::ForwardType x4)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4));
-  }
-};
-
-// Arity 4 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4>
-struct Invoker<4, StorageType, R(X1, X2, X3, X4)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4));
-  }
-};
-
-// Arity 5 -> 5.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<0, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2, X3, X4, X5);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 5 -> 4.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<1, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType);
-
-  typedef R(UnboundRunType)(X2, X3, X4, X5);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 5 -> 3.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<2, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType);
-
-  typedef R(UnboundRunType)(X3, X4, X5);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 5 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<3, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType);
-
-  typedef R(UnboundRunType)(X4, X5);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 5 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<4, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X5>::ForwardType);
-
-  typedef R(UnboundRunType)(X5);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X5>::ForwardType x5) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X5>::ForwardType x5)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 5 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5>
-struct Invoker<5, StorageType, R(X1, X2, X3, X4, X5)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5));
-  }
-};
-
-// Arity 6 -> 6.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<0, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2, X3, X4, X5, X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 5.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<1, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X2, X3, X4, X5, X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 4.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<2, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X3, X4, X5, X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 3.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<3, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X4, X5, X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<4, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X5, X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<5, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X6>::ForwardType);
-
-  typedef R(UnboundRunType)(X6);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X6>::ForwardType x6) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X6>::ForwardType x6)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 6 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6>
-struct Invoker<6, StorageType, R(X1, X2, X3, X4, X5, X6)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-    typedef typename StorageType::Bound6UnwrapTraits Bound6UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    typename Bound6UnwrapTraits::ForwardType x6 =
-        Bound6UnwrapTraits::Unwrap(storage->p6_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType,
-               typename Bound6UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6));
-  }
-};
-
-// Arity 7 -> 7.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<0, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X1>::ForwardType,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X1, X2, X3, X4, X5, X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X1>::ForwardType x1,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename CallbackParamTraits<X1>::ForwardType x1,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 6.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<1, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X2>::ForwardType,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X2, X3, X4, X5, X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X2>::ForwardType x2,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X2>::ForwardType x2,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 5.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<2, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X3>::ForwardType,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X3, X4, X5, X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X3>::ForwardType x3,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X3>::ForwardType x3,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 4.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<3, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X4>::ForwardType,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X4, X5, X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X4>::ForwardType x4,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X4>::ForwardType x4,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 3.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<4, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X5>::ForwardType,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X5, X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X5>::ForwardType x5,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X5>::ForwardType x5,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 2.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<5, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X6>::ForwardType,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X6, X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X6>::ForwardType x6,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X6>::ForwardType x6,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 1.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<6, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*,
-      typename CallbackParamTraits<X7>::ForwardType);
-
-  typedef R(UnboundRunType)(X7);
-
-  static R Run(BindStateBase* base,
-      typename CallbackParamTraits<X7>::ForwardType x7) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-    typedef typename StorageType::Bound6UnwrapTraits Bound6UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    typename Bound6UnwrapTraits::ForwardType x6 =
-        Bound6UnwrapTraits::Unwrap(storage->p6_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType,
-               typename Bound6UnwrapTraits::ForwardType,
-               typename CallbackParamTraits<X7>::ForwardType x7)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-// Arity 7 -> 0.
-template <typename StorageType, typename R,typename X1, typename X2,
-    typename X3, typename X4, typename X5, typename X6, typename X7>
-struct Invoker<7, StorageType, R(X1, X2, X3, X4, X5, X6, X7)> {
-  typedef R(RunType)(BindStateBase*);
-
-  typedef R(UnboundRunType)();
-
-  static R Run(BindStateBase* base) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-    typedef typename StorageType::Bound1UnwrapTraits Bound1UnwrapTraits;
-    typedef typename StorageType::Bound2UnwrapTraits Bound2UnwrapTraits;
-    typedef typename StorageType::Bound3UnwrapTraits Bound3UnwrapTraits;
-    typedef typename StorageType::Bound4UnwrapTraits Bound4UnwrapTraits;
-    typedef typename StorageType::Bound5UnwrapTraits Bound5UnwrapTraits;
-    typedef typename StorageType::Bound6UnwrapTraits Bound6UnwrapTraits;
-    typedef typename StorageType::Bound7UnwrapTraits Bound7UnwrapTraits;
-
-    typename Bound1UnwrapTraits::ForwardType x1 =
-        Bound1UnwrapTraits::Unwrap(storage->p1_);
-    typename Bound2UnwrapTraits::ForwardType x2 =
-        Bound2UnwrapTraits::Unwrap(storage->p2_);
-    typename Bound3UnwrapTraits::ForwardType x3 =
-        Bound3UnwrapTraits::Unwrap(storage->p3_);
-    typename Bound4UnwrapTraits::ForwardType x4 =
-        Bound4UnwrapTraits::Unwrap(storage->p4_);
-    typename Bound5UnwrapTraits::ForwardType x5 =
-        Bound5UnwrapTraits::Unwrap(storage->p5_);
-    typename Bound6UnwrapTraits::ForwardType x6 =
-        Bound6UnwrapTraits::Unwrap(storage->p6_);
-    typename Bound7UnwrapTraits::ForwardType x7 =
-        Bound7UnwrapTraits::Unwrap(storage->p7_);
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(typename Bound1UnwrapTraits::ForwardType,
-               typename Bound2UnwrapTraits::ForwardType,
-               typename Bound3UnwrapTraits::ForwardType,
-               typename Bound4UnwrapTraits::ForwardType,
-               typename Bound5UnwrapTraits::ForwardType,
-               typename Bound6UnwrapTraits::ForwardType,
-               typename Bound7UnwrapTraits::ForwardType)>
-               ::MakeItSo(storage->runnable_, CallbackForward(x1),
-                   CallbackForward(x2), CallbackForward(x3),
-                   CallbackForward(x4), CallbackForward(x5),
-                   CallbackForward(x6), CallbackForward(x7));
-  }
-};
-
-
-// BindState<>
-//
-// This stores all the state passed into Bind() and is also where most
-// of the template resolution magic occurs.
-//
-// Runnable is the functor we are binding arguments to.
-// RunType is type of the Run() function that the Invoker<> should use.
-// Normally, this is the same as the RunType of the Runnable, but it can
-// be different if an adapter like IgnoreResult() has been used.
-//
-// BoundArgsType contains the storage type for all the bound arguments by
-// (ab)using a function type.
-template <typename Runnable, typename RunType, typename BoundArgsType>
-struct BindState;
-
-template <typename Runnable, typename RunType>
-struct BindState<Runnable, RunType, void()> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<0, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-  explicit BindState(Runnable runnable) : runnable_(std::move(runnable)) {}
-
-  virtual ~BindState() {  }
-
-  RunnableType runnable_;
-};
-
-template <typename Runnable, typename RunType, typename P1>
-struct BindState<Runnable, RunType, void(P1)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<1, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-
-  BindState(Runnable runnable, P1 p1)
-      : runnable_(std::move(runnable)), p1_(std::move(p1)) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2>
-struct BindState<Runnable, RunType, void(P1, P2)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<2, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-
-  BindState(Runnable runnable, P1 p1, P2 p2)
-      : runnable_(std::move(runnable)), p1_(std::move(p1)), p2_(std::move(p2)) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2,
-    typename P3>
-struct BindState<Runnable, RunType, void(P1, P2, P3)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<3, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-  typedef UnwrapTraits<P3> Bound3UnwrapTraits;
-
-  BindState(Runnable runnable, P1 p1, P2 p2, P3 p3)
-      : runnable_(std::move(runnable)),
-        p1_(std::move(p1)),
-        p2_(std::move(p2)),
-        p3_(std::move(p3)) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-  P3 p3_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2,
-    typename P3, typename P4>
-struct BindState<Runnable, RunType, void(P1, P2, P3,
-    P4)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<4, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-  typedef UnwrapTraits<P3> Bound3UnwrapTraits;
-  typedef UnwrapTraits<P4> Bound4UnwrapTraits;
-
-  BindState(Runnable runnable, P1 p1, P2 p2, P3 p3, P4 p4)
-      : runnable_(std::move(runnable)),
-        p1_(std::move(p1)),
-        p2_(std::move(p2)),
-        p3_(std::move(p3)),
-        p4_(std::move(p4)) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-  P3 p3_;
-  P4 p4_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2,
-    typename P3, typename P4, typename P5>
-struct BindState<Runnable, RunType, void(P1, P2, P3, P4,
-    P5)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<5, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-  typedef UnwrapTraits<P3> Bound3UnwrapTraits;
-  typedef UnwrapTraits<P4> Bound4UnwrapTraits;
-  typedef UnwrapTraits<P5> Bound5UnwrapTraits;
-
-  BindState(Runnable runnable, P1 p1, P2 p2, P3 p3, P4 p4, P5 p5)
-      : runnable_(std::move(runnable)),
-        p1_(std::move(p1)),
-        p2_(std::move(p2)),
-        p3_(std::move(p3)),
-        p4_(std::move(p4)),
-        p5_(std::move(p5)) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-  P3 p3_;
-  P4 p4_;
-  P5 p5_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2,
-    typename P3, typename P4, typename P5, typename P6>
-struct BindState<Runnable, RunType, void(P1, P2, P3, P4, P5,
-    P6)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<6, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-  typedef UnwrapTraits<P3> Bound3UnwrapTraits;
-  typedef UnwrapTraits<P4> Bound4UnwrapTraits;
-  typedef UnwrapTraits<P5> Bound5UnwrapTraits;
-  typedef UnwrapTraits<P6> Bound6UnwrapTraits;
-
-  BindState(const Runnable& runnable, const P1& p1, const P2& p2, const P3& p3,
-      const P4& p4, const P5& p5, const P6& p6)
-      : runnable_(runnable),
-        p1_(p1),
-        p2_(p2),
-        p3_(p3),
-        p4_(p4),
-        p5_(p5),
-        p6_(p6) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-  P3 p3_;
-  P4 p4_;
-  P5 p5_;
-  P6 p6_;
-};
-
-template <typename Runnable, typename RunType, typename P1, typename P2,
-    typename P3, typename P4, typename P5, typename P6, typename P7>
-struct BindState<Runnable, RunType, void(P1, P2, P3, P4, P5, P6,
-    P7)> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<7, BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-  // Convenience typedefs for bound argument types.
-  typedef UnwrapTraits<P1> Bound1UnwrapTraits;
-  typedef UnwrapTraits<P2> Bound2UnwrapTraits;
-  typedef UnwrapTraits<P3> Bound3UnwrapTraits;
-  typedef UnwrapTraits<P4> Bound4UnwrapTraits;
-  typedef UnwrapTraits<P5> Bound5UnwrapTraits;
-  typedef UnwrapTraits<P6> Bound6UnwrapTraits;
-  typedef UnwrapTraits<P7> Bound7UnwrapTraits;
-
-  BindState(const Runnable& runnable, const P1& p1, const P2& p2, const P3& p3,
-      const P4& p4, const P5& p5, const P6& p6, const P7& p7)
-      : runnable_(runnable),
-        p1_(p1),
-        p2_(p2),
-        p3_(p3),
-        p4_(p4),
-        p5_(p5),
-        p6_(p6),
-        p7_(p7) {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-  }
-
-  virtual ~BindState() {    MaybeRefcount<HasIsMethodTag<Runnable>::value,
-      P1>::Release(p1_);  }
-
-  RunnableType runnable_;
-  P1 p1_;
-  P2 p2_;
-  P3 p3_;
-  P4 p4_;
-  P5 p5_;
-  P6 p6_;
-  P7 p7_;
-};
-
-}  // namespace internal
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_BIND_INTERNAL_H_
diff --git a/src/kudu/gutil/bind_internal.h.pump b/src/kudu/gutil/bind_internal.h.pump
deleted file mode 100644
index 8352f9b..0000000
--- a/src/kudu/gutil/bind_internal.h.pump
+++ /dev/null
@@ -1,464 +0,0 @@
-$$ This is a pump file for generating file templates.  Pump is a python
-$$ script that is part of the Google Test suite of utilities.  Description
-$$ can be found here:
-$$
-$$ http://code.google.com/p/googletest/wiki/PumpManual
-$$
-
-$$ See comment for MAX_ARITY in kudu/gutil/bind.h.pump.
-$var MAX_ARITY = 7
-$range ARITY 0..MAX_ARITY
-
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_BIND_INTERNAL_H_
-#define KUDU_GUTIL_BIND_INTERNAL_H_
-
-#include "kudu/gutil/bind_helpers.h"
-#include "kudu/gutil/callback_internal.h"
-#include "kudu/gutil/raw_scoped_refptr_mismatch_checker.h"
-#include "kudu/gutil/template_util.h"
-
-#if defined(OS_WIN)
-#include "kudu/gutil/bind_internal_win.h"
-#endif
-
-// During Chromium import, WeakPtr-related code was removed.
-
-namespace kudu {
-namespace internal {
-
-// See kudu/gutil/callback.h for user documentation.
-//
-//
-// CONCEPTS:
-//  Runnable -- A type (really a type class) that has a single Run() method
-//              and a RunType typedef that corresponds to the type of Run().
-//              A Runnable can declare that it should treated like a method
-//              call by including a typedef named IsMethod.  The value of
-//              this typedef is NOT inspected, only the existence.  When a
-//              Runnable declares itself a method, Bind() will enforce special
-//              refcounting + WeakPtr handling semantics for the first
-//              parameter which is expected to be an object.
-//  Functor -- A copyable type representing something that should be called.
-//             All function pointers, Callback<>, and Runnables are functors
-//             even if the invocation syntax differs.
-//  RunType -- A function type (as opposed to function _pointer_ type) for
-//             a Run() function.  Usually just a convenience typedef.
-//  (Bound)ArgsType -- A function type that is being (ab)used to store the
-//                     types of set of arguments.  The "return" type is always
-//                     void here.  We use this hack so that we do not need
-//                     a new type name for each arity of type. (eg.,
-//                     BindState1, BindState2).  This makes forward
-//                     declarations and friending much much easier.
-//
-// Types:
-//  RunnableAdapter<> -- Wraps the various "function" pointer types into an
-//                       object that adheres to the Runnable interface.
-//                       There are |3*ARITY| RunnableAdapter types.
-//  FunctionTraits<> -- Type traits that unwrap a function signature into a
-//                      a set of easier to use typedefs.  Used mainly for
-//                      compile time asserts.
-//                      There are |ARITY| FunctionTraits types.
-//  ForceVoidReturn<> -- Helper class for translating function signatures to
-//                       equivalent forms with a "void" return type.
-//                    There are |ARITY| ForceVoidReturn types.
-//  FunctorTraits<> -- Type traits used determine the correct RunType and
-//                     RunnableType for a Functor.  This is where function
-//                     signature adapters are applied.
-//                    There are |ARITY| ForceVoidReturn types.
-//  MakeRunnable<> -- Takes a Functor and returns an object in the Runnable
-//                    type class that represents the underlying Functor.
-//                    There are |O(1)| MakeRunnable types.
-//  InvokeHelper<> -- Take a Runnable + arguments and actully invokes it.
-//                    Handle the differing syntaxes needed for WeakPtr<> support,
-//                    and for ignoring return values.  This is separate from
-//                    Invoker to avoid creating multiple version of Invoker<>
-//                    which grows at O(n^2) with the arity.
-//                    There are |k*ARITY| InvokeHelper types.
-//  Invoker<> -- Unwraps the curried parameters and executes the Runnable.
-//               There are |(ARITY^2 + ARITY)/2| Invoketypes.
-//  BindState<> -- Stores the curried parameters, and is the main entry point
-//                 into the Bind() system, doing most of the type resolution.
-//                 There are ARITY BindState types.
-
-// RunnableAdapter<>
-//
-// The RunnableAdapter<> templates provide a uniform interface for invoking
-// a function pointer, method pointer, or const method pointer. The adapter
-// exposes a Run() method with an appropriate signature. Using this wrapper
-// allows for writing code that supports all three pointer types without
-// undue repetition.  Without it, a lot of code would need to be repeated 3
-// times.
-//
-// For method pointers and const method pointers the first argument to Run()
-// is considered to be the received of the method.  This is similar to STL's
-// mem_fun().
-//
-// This class also exposes a RunType typedef that is the function type of the
-// Run() function.
-//
-// If and only if the wrapper contains a method or const method pointer, an
-// IsMethod typedef is exposed.  The existence of this typedef (NOT the value)
-// marks that the wrapper should be considered a method wrapper.
-
-template <typename Functor>
-class RunnableAdapter;
-
-$for ARITY [[
-$range ARG 1..ARITY
-
-// Function: Arity $(ARITY).
-template <typename R[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename A$(ARG)]]>
-class RunnableAdapter<R(*)($for ARG , [[A$(ARG)]])> {
- public:
-  typedef R (RunType)($for ARG , [[A$(ARG)]]);
-
-  explicit RunnableAdapter(R(*function)($for ARG , [[A$(ARG)]]))
-      : function_(function) {
-  }
-
-  R Run($for ARG , [[typename CallbackParamTraits<A$(ARG)>::ForwardType a$(ARG)]]) {
-    return function_($for ARG , [[CallbackForward(a$(ARG))]]);
-  }
-
- private:
-  R (*function_)($for ARG , [[A$(ARG)]]);
-};
-
-// Method: Arity $(ARITY).
-template <typename R, typename T[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename A$(ARG)]]>
-class RunnableAdapter<R(T::*)($for ARG , [[A$(ARG)]])> {
- public:
-  typedef R (RunType)(T*[[]]
-$if ARITY > 0[[, ]] $for ARG , [[A$(ARG)]]);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)($for ARG , [[A$(ARG)]]))
-      : method_(method) {
-  }
-
-  R Run(T* object[[]]
-$if ARITY > 0[[, ]]  $for ARG, [[typename CallbackParamTraits<A$(ARG)>::ForwardType a$(ARG)]]) {
-    return (object->*method_)($for ARG , [[CallbackForward(a$(ARG))]]);
-  }
-
- private:
-  R (T::*method_)($for ARG , [[A$(ARG)]]);
-};
-
-// Const Method: Arity $(ARITY).
-template <typename R, typename T[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename A$(ARG)]]>
-class RunnableAdapter<R(T::*)($for ARG , [[A$(ARG)]]) const> {
- public:
-  typedef R (RunType)(const T*[[]]
-$if ARITY > 0[[, ]] $for ARG , [[A$(ARG)]]);
-  typedef base::true_type IsMethod;
-
-  explicit RunnableAdapter(R(T::*method)($for ARG , [[A$(ARG)]]) const)
-      : method_(method) {
-  }
-
-  R Run(const T* object[[]]
-$if ARITY > 0[[, ]]  $for ARG, [[typename CallbackParamTraits<A$(ARG)>::ForwardType a$(ARG)]]) {
-    return (object->*method_)($for ARG , [[CallbackForward(a$(ARG))]]);
-  }
-
- private:
-  R (T::*method_)($for ARG , [[A$(ARG)]]) const;
-};
-
-]]  $$ for ARITY
-
-
-// FunctionTraits<>
-//
-// Breaks a function signature apart into typedefs for easier introspection.
-template <typename Sig>
-struct FunctionTraits;
-
-$for ARITY [[
-$range ARG 1..ARITY
-
-template <typename R[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename A$(ARG)]]>
-struct FunctionTraits<R($for ARG , [[A$(ARG)]])> {
-  typedef R ReturnType;
-$for ARG [[
-
-  typedef A$(ARG) A$(ARG)Type;
-]]
-
-};
-
-]]
-
-
-// ForceVoidReturn<>
-//
-// Set of templates that support forcing the function return type to void.
-template <typename Sig>
-struct ForceVoidReturn;
-
-$for ARITY [[
-$range ARG 1..ARITY
-
-template <typename R[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename A$(ARG)]]>
-struct ForceVoidReturn<R($for ARG , [[A$(ARG)]])> {
-  typedef void(RunType)($for ARG , [[A$(ARG)]]);
-};
-
-]]  $$ for ARITY
-
-
-// FunctorTraits<>
-//
-// See description at top of file.
-template <typename T>
-struct FunctorTraits {
-  typedef RunnableAdapter<T> RunnableType;
-  typedef typename RunnableType::RunType RunType;
-};
-
-template <typename T>
-struct FunctorTraits<IgnoreResultHelper<T> > {
-  typedef typename FunctorTraits<T>::RunnableType RunnableType;
-  typedef typename ForceVoidReturn<
-      typename RunnableType::RunType>::RunType RunType;
-};
-
-template <typename T>
-struct FunctorTraits<Callback<T> > {
-  typedef Callback<T> RunnableType;
-  typedef typename Callback<T>::RunType RunType;
-};
-
-
-// MakeRunnable<>
-//
-// Converts a passed in functor to a RunnableType using type inference.
-
-template <typename T>
-typename FunctorTraits<T>::RunnableType MakeRunnable(const T& t) {
-  return RunnableAdapter<T>(t);
-}
-
-template <typename T>
-typename FunctorTraits<T>::RunnableType
-MakeRunnable(const IgnoreResultHelper<T>& t) {
-  return MakeRunnable(t.functor_);
-}
-
-template <typename T>
-const typename FunctorTraits<Callback<T> >::RunnableType&
-MakeRunnable(const Callback<T>& t) {
-  DCHECK(!t.is_null());
-  return t;
-}
-
-
-// InvokeHelper<>
-//
-// There are 3 logical InvokeHelper<> specializations: normal, void-return,
-// WeakCalls.
-//
-// The normal type just calls the underlying runnable.
-//
-// We need a InvokeHelper to handle void return types in order to support
-// IgnoreResult().  Normally, if the Runnable's RunType had a void return,
-// the template system would just accept "return functor.Run()" ignoring
-// the fact that a void function is being used with return. This piece of
-// sugar breaks though when the Runnable's RunType is not void.  Thus, we
-// need a partial specialization to change the syntax to drop the "return"
-// from the invocation call.
-//
-// WeakCalls similarly need special syntax that is applied to the first
-// argument to check if they should no-op themselves.
-template <bool IsWeakCall, typename ReturnType, typename Runnable,
-          typename ArgsType>
-struct InvokeHelper;
-
-$for ARITY [[
-$range ARG 1..ARITY
-$range WEAKCALL_ARG 2..ARITY
-
-template <typename ReturnType, typename Runnable[[]]
-$if ARITY > 0 [[,]] $for ARG , [[typename A$(ARG)]]>
-struct InvokeHelper<false, ReturnType, Runnable,
-    void($for ARG , [[A$(ARG)]])>  {
-  static ReturnType MakeItSo(Runnable runnable[[]]
-$if ARITY > 0[[, ]] $for ARG , [[A$(ARG) a$(ARG)]]) {
-    return runnable.Run($for ARG , [[CallbackForward(a$(ARG))]]);
-  }
-};
-
-template <typename Runnable[[]]
-$if ARITY > 0 [[,]] $for ARG , [[typename A$(ARG)]]>
-struct InvokeHelper<false, void, Runnable,
-    void($for ARG , [[A$(ARG)]])>  {
-  static void MakeItSo(Runnable runnable[[]]
-$if ARITY > 0[[, ]] $for ARG , [[A$(ARG) a$(ARG)]]) {
-    runnable.Run($for ARG , [[CallbackForward(a$(ARG))]]);
-  }
-};
-
-]] $$ for ARITY
-
-// Invoker<>
-//
-// See description at the top of the file.
-template <int NumBound, typename Storage, typename RunType>
-struct Invoker;
-
-$for ARITY [[
-
-$$ Number of bound arguments.
-$range BOUND 0..ARITY
-$for BOUND [[
-
-$var UNBOUND = ARITY - BOUND
-$range ARG 1..ARITY
-$range BOUND_ARG 1..BOUND
-$range UNBOUND_ARG (ARITY - UNBOUND + 1)..ARITY
-
-// Arity $(ARITY) -> $(UNBOUND).
-template <typename StorageType, typename R[[]]
-$if ARITY > 0 [[,]][[]]
-$for ARG , [[typename X$(ARG)]]>
-struct Invoker<$(BOUND), StorageType, R($for ARG , [[X$(ARG)]])> {
-  typedef R(RunType)(BindStateBase*[[]]
-$if UNBOUND != 0 [[, ]]
-$for UNBOUND_ARG , [[typename CallbackParamTraits<X$(UNBOUND_ARG)>::ForwardType]]);
-
-  typedef R(UnboundRunType)($for UNBOUND_ARG , [[X$(UNBOUND_ARG)]]);
-
-  static R Run(BindStateBase* base[[]]
-$if UNBOUND != 0 [[, ]][[]]
-$for UNBOUND_ARG , [[
-typename CallbackParamTraits<X$(UNBOUND_ARG)>::ForwardType x$(UNBOUND_ARG)
-]][[]]
-) {
-    StorageType* storage = static_cast<StorageType*>(base);
-
-    // Local references to make debugger stepping easier. If in a debugger,
-    // you really want to warp ahead and step through the
-    // InvokeHelper<>::MakeItSo() call below.
-$for BOUND_ARG
-[[
-
-    typedef typename StorageType::Bound$(BOUND_ARG)UnwrapTraits Bound$(BOUND_ARG)UnwrapTraits;
-]]
-
-
-$for BOUND_ARG
-[[
-
-    typename Bound$(BOUND_ARG)UnwrapTraits::ForwardType x$(BOUND_ARG) =
-        Bound$(BOUND_ARG)UnwrapTraits::Unwrap(storage->p$(BOUND_ARG)_);
-]]
-
-    return InvokeHelper<StorageType::IsWeakCall::value, R,
-           typename StorageType::RunnableType,
-           void(
-$for BOUND_ARG , [[
-typename Bound$(BOUND_ARG)UnwrapTraits::ForwardType
-]]
-
-$if UNBOUND > 0 [[$if BOUND > 0 [[, ]]]][[]]
-
-$for UNBOUND_ARG , [[
-typename CallbackParamTraits<X$(UNBOUND_ARG)>::ForwardType x$(UNBOUND_ARG)
-]]
-)>
-               ::MakeItSo(storage->runnable_
-$if ARITY > 0[[, ]] $for ARG , [[CallbackForward(x$(ARG))]]);
-  }
-};
-
-]] $$ for BOUND
-]] $$ for ARITY
-
-
-// BindState<>
-//
-// This stores all the state passed into Bind() and is also where most
-// of the template resolution magic occurs.
-//
-// Runnable is the functor we are binding arguments to.
-// RunType is type of the Run() function that the Invoker<> should use.
-// Normally, this is the same as the RunType of the Runnable, but it can
-// be different if an adapter like IgnoreResult() has been used.
-//
-// BoundArgsType contains the storage type for all the bound arguments by
-// (ab)using a function type.
-template <typename Runnable, typename RunType, typename BoundArgsType>
-struct BindState;
-
-$for ARITY [[
-$range ARG 1..ARITY
-
-template <typename Runnable, typename RunType[[]]
-$if ARITY > 0[[, ]] $for ARG , [[typename P$(ARG)]]>
-struct BindState<Runnable, RunType, void($for ARG , [[P$(ARG)]])> : public BindStateBase {
-  typedef Runnable RunnableType;
-
-  typedef base::false_type IsWeakCall;
-
-  typedef Invoker<$(ARITY), BindState, RunType> InvokerType;
-  typedef typename InvokerType::UnboundRunType UnboundRunType;
-
-$if ARITY > 0 [[
-
-  // Convenience typedefs for bound argument types.
-
-$for ARG [[
-  typedef UnwrapTraits<P$(ARG)> Bound$(ARG)UnwrapTraits;
-
-]]  $$ for ARG
-
-
-]]  $$ if ARITY > 0
-
-$$ The extra [[ ]] is needed to massage spacing. Silly pump.py.
-[[  ]]$if ARITY == 0 [[explicit ]]BindState(const Runnable& runnable
-$if ARITY > 0 [[, ]] $for ARG , [[const P$(ARG)& p$(ARG)]])
-      : runnable_(runnable)[[]]
-$if ARITY == 0 [[
- {
-
-]] $else [[
-, $for ARG , [[
-
-        p$(ARG)_(p$(ARG))
-]] {
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::AddRef(p1_);
-
-]]
-  }
-
-  virtual ~BindState() {
-$if ARITY > 0 [[
-    MaybeRefcount<HasIsMethodTag<Runnable>::value, P1>::Release(p1_);
-]]
-  }
-
-  RunnableType runnable_;
-
-$for ARG [[
-  P$(ARG) p$(ARG)_;
-
-]]
-};
-
-]] $$ for ARITY
-
-}  // namespace internal
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_BIND_INTERNAL_H_
diff --git a/src/kudu/gutil/callback.h b/src/kudu/gutil/callback.h
deleted file mode 100644
index 1a41622..0000000
--- a/src/kudu/gutil/callback.h
+++ /dev/null
@@ -1,765 +0,0 @@
-// This file was GENERATED by command:
-//     pump.py callback.h.pump
-// DO NOT EDIT BY HAND!!!
-
-
-// Copyright (c) 2012 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_CALLBACK_H_
-#define KUDU_GUTIL_CALLBACK_H_
-
-#include "kudu/gutil/callback_forward.h"
-#include "kudu/gutil/callback_internal.h"
-#include "kudu/gutil/template_util.h"
-
-// NOTE: Header files that do not require the full definition of Callback or
-// Closure should #include "kudu/gutil/callback_forward.h" instead of this file.
-
-// -----------------------------------------------------------------------------
-// Introduction
-// -----------------------------------------------------------------------------
-//
-// The templated Callback class is a generalized function object. Together
-// with the Bind() function in bind.h, they provide a type-safe method for
-// performing partial application of functions.
-//
-// Partial application (or "currying") is the process of binding a subset of
-// a function's arguments to produce another function that takes fewer
-// arguments. This can be used to pass around a unit of delayed execution,
-// much like lexical closures are used in other languages. For example, it
-// is used in Chromium code to schedule tasks on different MessageLoops.
-//
-// A callback with no unbound input parameters (kudu::Callback<void(void)>)
-// is called a kudu::Closure. Note that this is NOT the same as what other
-// languages refer to as a closure -- it does not retain a reference to its
-// enclosing environment.
-//
-// MEMORY MANAGEMENT AND PASSING
-//
-// The Callback objects themselves should be passed by const-reference, and
-// stored by copy. They internally store their state via a refcounted class
-// and thus do not need to be deleted.
-//
-// The reason to pass via a const-reference is to avoid unnecessary
-// AddRef/Release pairs to the internal state.
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for basic stuff
-// -----------------------------------------------------------------------------
-//
-// BINDING A BARE FUNCTION
-//
-//   int Return5() { return 5; }
-//   kudu::Callback<int(void)> func_cb = kudu::Bind(&Return5);
-//   LOG(INFO) << func_cb.Run();  // Prints 5.
-//
-// BINDING A CLASS METHOD
-//
-//   The first argument to bind is the member function to call, the second is
-//   the object on which to call it.
-//
-//   class Ref : public kudu::RefCountedThreadSafe<Ref> {
-//    public:
-//     int Foo() { return 3; }
-//     void PrintBye() { LOG(INFO) << "bye."; }
-//   };
-//   scoped_refptr<Ref> ref = new Ref();
-//   kudu::Callback<void(void)> ref_cb = kudu::Bind(&Ref::Foo, ref);
-//   LOG(INFO) << ref_cb.Run();  // Prints out 3.
-//
-//   By default the object must support RefCounted or you will get a compiler
-//   error. If you're passing between threads, be sure it's
-//   RefCountedThreadSafe! See "Advanced binding of member functions" below if
-//   you don't want to use reference counting.
-//
-// RUNNING A CALLBACK
-//
-//   Callbacks can be run with their "Run" method, which has the same
-//   signature as the template argument to the callback.
-//
-//   void DoSomething(const kudu::Callback<void(int, std::string)>& callback) {
-//     callback.Run(5, "hello");
-//   }
-//
-//   Callbacks can be run more than once (they don't get deleted or marked when
-//   run). However, this precludes using kudu::Passed (see below).
-//
-//   void DoSomething(const kudu::Callback<double(double)>& callback) {
-//     double myresult = callback.Run(3.14159);
-//     myresult += callback.Run(2.71828);
-//   }
-//
-// PASSING UNBOUND INPUT PARAMETERS
-//
-//   Unbound parameters are specified at the time a callback is Run(). They are
-//   specified in the Callback template type:
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(int, const std::string&)> cb = kudu::Bind(&MyFunc);
-//   cb.Run(23, "hello, world");
-//
-// PASSING BOUND INPUT PARAMETERS
-//
-//   Bound parameters are specified when you create the callback as arguments
-//   to Bind(). They will be passed to the function and the Run()ner of the
-//   callback doesn't see those values or even know that the function it's
-//   calling.
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(void)> cb = kudu::Bind(&MyFunc, 23, "hello world");
-//   cb.Run();
-//
-//   A callback with no unbound input parameters (kudu::Callback<void(void)>)
-//   is called a kudu::Closure. So we could have also written:
-//
-//   kudu::Closure cb = kudu::Bind(&MyFunc, 23, "hello world");
-//
-//   When calling member functions, bound parameters just go after the object
-//   pointer.
-//
-//   kudu::Closure cb = kudu::Bind(&MyClass::MyFunc, this, 23, "hello world");
-//
-// PARTIAL BINDING OF PARAMETERS
-//
-//   You can specify some parameters when you create the callback, and specify
-//   the rest when you execute the callback.
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(const std::string&)> cb = kudu::Bind(&MyFunc, 23);
-//   cb.Run("hello world");
-//
-//   When calling a function bound parameters are first, followed by unbound
-//   parameters.
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for advanced binding
-// -----------------------------------------------------------------------------
-//
-// BINDING A CLASS METHOD WITH WEAK POINTERS
-//
-//   kudu::Bind(&MyClass::Foo, GetWeakPtr());
-//
-//   The callback will not be issued if the object is destroyed at the time
-//   it's issued. DANGER: weak pointers are not threadsafe, so don't use this
-//   when passing between threads!
-//
-// BINDING A CLASS METHOD WITH MANUAL LIFETIME MANAGEMENT
-//
-//   kudu::Bind(&MyClass::Foo, kudu::Unretained(this));
-//
-//   This disables all lifetime management on the object. You're responsible
-//   for making sure the object is alive at the time of the call. You break it,
-//   you own it!
-//
-// BINDING A CLASS METHOD AND HAVING THE CALLBACK OWN THE CLASS
-//
-//   MyClass* myclass = new MyClass;
-//   kudu::Bind(&MyClass::Foo, kudu::Owned(myclass));
-//
-//   The object will be deleted when the callback is destroyed, even if it's
-//   not run (like if you post a task during shutdown). Potentially useful for
-//   "fire and forget" cases.
-//
-// IGNORING RETURN VALUES
-//
-//   Sometimes you want to call a function that returns a value in a callback
-//   that doesn't expect a return value.
-//
-//   int DoSomething(int arg) { cout << arg << endl; }
-//   kudu::Callback<void<int>) cb =
-//       kudu::Bind(kudu::IgnoreResult(&DoSomething));
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for binding parameters to Bind()
-// -----------------------------------------------------------------------------
-//
-// Bound parameters are specified as arguments to Bind() and are passed to the
-// function. A callback with no parameters or no unbound parameters is called a
-// Closure (kudu::Callback<void(void)> and kudu::Closure are the same thing).
-//
-// PASSING PARAMETERS OWNED BY THE CALLBACK
-//
-//   void Foo(int* arg) { cout << *arg << endl; }
-//   int* pn = new int(1);
-//   kudu::Closure foo_callback = kudu::Bind(&foo, kudu::Owned(pn));
-//
-//   The parameter will be deleted when the callback is destroyed, even if it's
-//   not run (like if you post a task during shutdown).
-//
-// PASSING PARAMETERS AS A scoped_ptr
-//
-//   void TakesOwnership(scoped_ptr<Foo> arg) {}
-//   scoped_ptr<Foo> f(new Foo);
-//   // f becomes null during the following call.
-//   kudu::Closure cb = kudu::Bind(&TakesOwnership, kudu::Passed(&f));
-//
-//   Ownership of the parameter will be with the callback until the it is run,
-//   when ownership is passed to the callback function. This means the callback
-//   can only be run once. If the callback is never run, it will delete the
-//   object when it's destroyed.
-//
-// PASSING PARAMETERS AS A scoped_refptr
-//
-//   void TakesOneRef(scoped_refptr<Foo> arg) {}
-//   scoped_refptr<Foo> f(new Foo)
-//   kudu::Closure cb = kudu::Bind(&TakesOneRef, f);
-//
-//   This should "just work." The closure will take a reference as long as it
-//   is alive, and another reference will be taken for the called function.
-//
-// PASSING PARAMETERS BY REFERENCE
-//
-//   void foo(int arg) { cout << arg << endl }
-//   int n = 1;
-//   kudu::Closure has_ref = kudu::Bind(&foo, kudu::ConstRef(n));
-//   n = 2;
-//   has_ref.Run();  // Prints "2"
-//
-//   Normally parameters are copied in the closure. DANGER: ConstRef stores a
-//   const reference instead, referencing the original parameter. This means
-//   that you must ensure the object outlives the callback!
-//
-//
-// -----------------------------------------------------------------------------
-// Implementation notes
-// -----------------------------------------------------------------------------
-//
-// WHERE IS THIS DESIGN FROM:
-//
-// The design Callback and Bind is heavily influenced by C++'s
-// tr1::function/tr1::bind, and by the "Google Callback" system used inside
-// Google.
-//
-//
-// HOW THE IMPLEMENTATION WORKS:
-//
-// There are three main components to the system:
-//   1) The Callback classes.
-//   2) The Bind() functions.
-//   3) The arguments wrappers (e.g., Unretained() and ConstRef()).
-//
-// The Callback classes represent a generic function pointer. Internally,
-// it stores a refcounted piece of state that represents the target function
-// and all its bound parameters.  Each Callback specialization has a templated
-// constructor that takes an BindState<>*.  In the context of the constructor,
-// the static type of this BindState<> pointer uniquely identifies the
-// function it is representing, all its bound parameters, and a Run() method
-// that is capable of invoking the target.
-//
-// Callback's constructor takes the BindState<>* that has the full static type
-// and erases the target function type as well as the types of the bound
-// parameters.  It does this by storing a pointer to the specific Run()
-// function, and upcasting the state of BindState<>* to a
-// BindStateBase*. This is safe as long as this BindStateBase pointer
-// is only used with the stored Run() pointer.
-//
-// To BindState<> objects are created inside the Bind() functions.
-// These functions, along with a set of internal templates, are responsible for
-//
-//  - Unwrapping the function signature into return type, and parameters
-//  - Determining the number of parameters that are bound
-//  - Creating the BindState storing the bound parameters
-//  - Performing compile-time asserts to avoid error-prone behavior
-//  - Returning an Callback<> with an arity matching the number of unbound
-//    parameters and that knows the correct refcounting semantics for the
-//    target object if we are binding a method.
-//
-// The Bind functions do the above using type-inference, and template
-// specializations.
-//
-// By default Bind() will store copies of all bound parameters, and attempt
-// to refcount a target object if the function being bound is a class method.
-// These copies are created even if the function takes parameters as const
-// references. (Binding to non-const references is forbidden, see bind.h.)
-//
-// To change this behavior, we introduce a set of argument wrappers
-// (e.g., Unretained(), and ConstRef()).  These are simple container templates
-// that are passed by value, and wrap a pointer to argument.  See the
-// file-level comment in kudu/gutil/bind_helpers.h for more info.
-//
-// These types are passed to the Unwrap() functions, and the MaybeRefcount()
-// functions respectively to modify the behavior of Bind().  The Unwrap()
-// and MaybeRefcount() functions change behavior by doing partial
-// specialization based on whether or not a parameter is a wrapper type.
-//
-// ConstRef() is similar to tr1::cref.  Unretained() is specific to Chromium.
-//
-//
-// WHY NOT TR1 FUNCTION/BIND?
-//
-// Direct use of tr1::function and tr1::bind was considered, but ultimately
-// rejected because of the number of copy constructors invocations involved
-// in the binding of arguments during construction, and the forwarding of
-// arguments during invocation.  These copies will no longer be an issue in
-// C++0x because C++0x will support rvalue reference allowing for the compiler
-// to avoid these copies.  However, waiting for C++0x is not an option.
-//
-// Measured with valgrind on gcc version 4.4.3 (Ubuntu 4.4.3-4ubuntu5), the
-// tr1::bind call itself will invoke a non-trivial copy constructor three times
-// for each bound parameter.  Also, each when passing a tr1::function, each
-// bound argument will be copied again.
-//
-// In addition to the copies taken at binding and invocation, copying a
-// tr1::function causes a copy to be made of all the bound parameters and
-// state.
-//
-// Furthermore, in Chromium, it is desirable for the Callback to take a
-// reference on a target object when representing a class method call.  This
-// is not supported by tr1.
-//
-// Lastly, tr1::function and tr1::bind has a more general and flexible API.
-// This includes things like argument reordering by use of
-// tr1::bind::placeholder, support for non-const reference parameters, and some
-// limited amount of subtyping of the tr1::function object (e.g.,
-// tr1::function<int(int)> is convertible to tr1::function<void(int)>).
-//
-// These are not features that are required in Chromium. Some of them, such as
-// allowing for reference parameters, and subtyping of functions, may actually
-// become a source of errors. Removing support for these features actually
-// allows for a simpler implementation, and a terser Currying API.
-//
-//
-// WHY NOT GOOGLE CALLBACKS?
-//
-// The Google callback system also does not support refcounting.  Furthermore,
-// its implementation has a number of strange edge cases with respect to type
-// conversion of its arguments.  In particular, the argument's constness must
-// at times match exactly the function signature, or the type-inference might
-// break.  Given the above, writing a custom solution was easier.
-//
-//
-// MISSING FUNCTIONALITY
-//  - Invoking the return of Bind.  Bind(&foo).Run() does not work;
-//  - Binding arrays to functions that take a non-const pointer.
-//    Example:
-//      void Foo(const char* ptr);
-//      void Bar(char* ptr);
-//      Bind(&Foo, "test");
-//      Bind(&Bar, "test");  // This fails because ptr is not const.
-
-namespace kudu {
-
-// First, we forward declare the Callback class template. This informs the
-// compiler that the template only has 1 type parameter which is the function
-// signature that the Callback is representing.
-//
-// After this, create template specializations for 0-7 parameters. Note that
-// even though the template typelist grows, the specialization still
-// only has one type: the function signature.
-//
-// If you are thinking of forward declaring Callback in your own header file,
-// please include "base/callback_forward.h" instead.
-template <typename Sig>
-class Callback;
-
-namespace internal {
-template <typename Runnable, typename RunType, typename BoundArgsType>
-struct BindState;
-}  // namespace internal
-
-template <typename R>
-class Callback<R(void)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)();
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run() const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get());
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*);
-
-};
-
-template <typename R, typename A1>
-class Callback<R(A1)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2>
-class Callback<R(A1, A2)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2, typename A3>
-class Callback<R(A1, A2, A3)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2, A3);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2,
-        typename internal::CallbackParamTraits<A3>::ForwardType a3) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2),
-             internal::CallbackForward(a3));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType,
-          typename internal::CallbackParamTraits<A3>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4>
-class Callback<R(A1, A2, A3, A4)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2, A3, A4);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2,
-        typename internal::CallbackParamTraits<A3>::ForwardType a3,
-        typename internal::CallbackParamTraits<A4>::ForwardType a4) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2),
-             internal::CallbackForward(a3),
-             internal::CallbackForward(a4));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType,
-          typename internal::CallbackParamTraits<A3>::ForwardType,
-          typename internal::CallbackParamTraits<A4>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5>
-class Callback<R(A1, A2, A3, A4, A5)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2, A3, A4, A5);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2,
-        typename internal::CallbackParamTraits<A3>::ForwardType a3,
-        typename internal::CallbackParamTraits<A4>::ForwardType a4,
-        typename internal::CallbackParamTraits<A5>::ForwardType a5) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2),
-             internal::CallbackForward(a3),
-             internal::CallbackForward(a4),
-             internal::CallbackForward(a5));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType,
-          typename internal::CallbackParamTraits<A3>::ForwardType,
-          typename internal::CallbackParamTraits<A4>::ForwardType,
-          typename internal::CallbackParamTraits<A5>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6>
-class Callback<R(A1, A2, A3, A4, A5, A6)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2, A3, A4, A5, A6);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2,
-        typename internal::CallbackParamTraits<A3>::ForwardType a3,
-        typename internal::CallbackParamTraits<A4>::ForwardType a4,
-        typename internal::CallbackParamTraits<A5>::ForwardType a5,
-        typename internal::CallbackParamTraits<A6>::ForwardType a6) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2),
-             internal::CallbackForward(a3),
-             internal::CallbackForward(a4),
-             internal::CallbackForward(a5),
-             internal::CallbackForward(a6));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType,
-          typename internal::CallbackParamTraits<A3>::ForwardType,
-          typename internal::CallbackParamTraits<A4>::ForwardType,
-          typename internal::CallbackParamTraits<A5>::ForwardType,
-          typename internal::CallbackParamTraits<A6>::ForwardType);
-
-};
-
-template <typename R, typename A1, typename A2, typename A3, typename A4,
-    typename A5, typename A6, typename A7>
-class Callback<R(A1, A2, A3, A4, A5, A6, A7)> : public internal::CallbackBase {
- public:
-  typedef R(RunType)(A1, A2, A3, A4, A5, A6, A7);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run(typename internal::CallbackParamTraits<A1>::ForwardType a1,
-        typename internal::CallbackParamTraits<A2>::ForwardType a2,
-        typename internal::CallbackParamTraits<A3>::ForwardType a3,
-        typename internal::CallbackParamTraits<A4>::ForwardType a4,
-        typename internal::CallbackParamTraits<A5>::ForwardType a5,
-        typename internal::CallbackParamTraits<A6>::ForwardType a6,
-        typename internal::CallbackParamTraits<A7>::ForwardType a7) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get(), internal::CallbackForward(a1),
-             internal::CallbackForward(a2),
-             internal::CallbackForward(a3),
-             internal::CallbackForward(a4),
-             internal::CallbackForward(a5),
-             internal::CallbackForward(a6),
-             internal::CallbackForward(a7));
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*,
-          typename internal::CallbackParamTraits<A1>::ForwardType,
-          typename internal::CallbackParamTraits<A2>::ForwardType,
-          typename internal::CallbackParamTraits<A3>::ForwardType,
-          typename internal::CallbackParamTraits<A4>::ForwardType,
-          typename internal::CallbackParamTraits<A5>::ForwardType,
-          typename internal::CallbackParamTraits<A6>::ForwardType,
-          typename internal::CallbackParamTraits<A7>::ForwardType);
-
-};
-
-
-// Syntactic sugar to make Callbacks<void(void)> easier to declare since it
-// will be used in a lot of APIs with delayed execution.
-typedef Callback<void(void)> Closure;
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_CALLBACK_H
diff --git a/src/kudu/gutil/callback.h.pump b/src/kudu/gutil/callback.h.pump
deleted file mode 100644
index a2864d4..0000000
--- a/src/kudu/gutil/callback.h.pump
+++ /dev/null
@@ -1,436 +0,0 @@
-$$ This is a pump file for generating file templates.  Pump is a python
-$$ script that is part of the Google Test suite of utilities.  Description
-$$ can be found here:
-$$
-$$ http://code.google.com/p/googletest/wiki/PumpManual
-$$
-
-$$ See comment for MAX_ARITY in kudu/gutil/bind.h.pump.
-$var MAX_ARITY = 7
-
-// Copyright (c) 2012 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_CALLBACK_H_
-#define KUDU_GUTIL_CALLBACK_H_
-
-#include "kudu/gutil/callback_forward.h"
-#include "kudu/gutil/callback_internal.h"
-#include "kudu/gutil/template_util.h"
-
-// NOTE: Header files that do not require the full definition of Callback or
-// Closure should #include "kudu/gutil/callback_forward.h" instead of this file.
-
-// -----------------------------------------------------------------------------
-// Introduction
-// -----------------------------------------------------------------------------
-//
-// The templated Callback class is a generalized function object. Together
-// with the Bind() function in bind.h, they provide a type-safe method for
-// performing partial application of functions.
-//
-// Partial application (or "currying") is the process of binding a subset of
-// a function's arguments to produce another function that takes fewer
-// arguments. This can be used to pass around a unit of delayed execution,
-// much like lexical closures are used in other languages. For example, it
-// is used in Chromium code to schedule tasks on different MessageLoops.
-//
-// A callback with no unbound input parameters (kudu::Callback<void(void)>)
-// is called a kudu::Closure. Note that this is NOT the same as what other
-// languages refer to as a closure -- it does not retain a reference to its
-// enclosing environment.
-//
-// MEMORY MANAGEMENT AND PASSING
-//
-// The Callback objects themselves should be passed by const-reference, and
-// stored by copy. They internally store their state via a refcounted class
-// and thus do not need to be deleted.
-//
-// The reason to pass via a const-reference is to avoid unnecessary
-// AddRef/Release pairs to the internal state.
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for basic stuff
-// -----------------------------------------------------------------------------
-//
-// BINDING A BARE FUNCTION
-//
-//   int Return5() { return 5; }
-//   kudu::Callback<int(void)> func_cb = kudu::Bind(&Return5);
-//   LOG(INFO) << func_cb.Run();  // Prints 5.
-//
-// BINDING A CLASS METHOD
-//
-//   The first argument to bind is the member function to call, the second is
-//   the object on which to call it.
-//
-//   class Ref : public kudu::RefCountedThreadSafe<Ref> {
-//    public:
-//     int Foo() { return 3; }
-//     void PrintBye() { LOG(INFO) << "bye."; }
-//   };
-//   scoped_refptr<Ref> ref = new Ref();
-//   kudu::Callback<void(void)> ref_cb = kudu::Bind(&Ref::Foo, ref);
-//   LOG(INFO) << ref_cb.Run();  // Prints out 3.
-//
-//   By default the object must support RefCounted or you will get a compiler
-//   error. If you're passing between threads, be sure it's
-//   RefCountedThreadSafe! See "Advanced binding of member functions" below if
-//   you don't want to use reference counting.
-//
-// RUNNING A CALLBACK
-//
-//   Callbacks can be run with their "Run" method, which has the same
-//   signature as the template argument to the callback.
-//
-//   void DoSomething(const kudu::Callback<void(int, std::string)>& callback) {
-//     callback.Run(5, "hello");
-//   }
-//
-//   Callbacks can be run more than once (they don't get deleted or marked when
-//   run). However, this precludes using kudu::Passed (see below).
-//
-//   void DoSomething(const kudu::Callback<double(double)>& callback) {
-//     double myresult = callback.Run(3.14159);
-//     myresult += callback.Run(2.71828);
-//   }
-//
-// PASSING UNBOUND INPUT PARAMETERS
-//
-//   Unbound parameters are specified at the time a callback is Run(). They are
-//   specified in the Callback template type:
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(int, const std::string&)> cb = kudu::Bind(&MyFunc);
-//   cb.Run(23, "hello, world");
-//
-// PASSING BOUND INPUT PARAMETERS
-//
-//   Bound parameters are specified when you create the callback as arguments
-//   to Bind(). They will be passed to the function and the Run()ner of the
-//   callback doesn't see those values or even know that the function it's
-//   calling.
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(void)> cb = kudu::Bind(&MyFunc, 23, "hello world");
-//   cb.Run();
-//
-//   A callback with no unbound input parameters (kudu::Callback<void(void)>)
-//   is called a kudu::Closure. So we could have also written:
-//
-//   kudu::Closure cb = kudu::Bind(&MyFunc, 23, "hello world");
-//
-//   When calling member functions, bound parameters just go after the object
-//   pointer.
-//
-//   kudu::Closure cb = kudu::Bind(&MyClass::MyFunc, this, 23, "hello world");
-//
-// PARTIAL BINDING OF PARAMETERS
-//
-//   You can specify some parameters when you create the callback, and specify
-//   the rest when you execute the callback.
-//
-//   void MyFunc(int i, const std::string& str) {}
-//   kudu::Callback<void(const std::string&)> cb = kudu::Bind(&MyFunc, 23);
-//   cb.Run("hello world");
-//
-//   When calling a function bound parameters are first, followed by unbound
-//   parameters.
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for advanced binding
-// -----------------------------------------------------------------------------
-//
-// BINDING A CLASS METHOD WITH WEAK POINTERS
-//
-//   kudu::Bind(&MyClass::Foo, GetWeakPtr());
-//
-//   The callback will not be issued if the object is destroyed at the time
-//   it's issued. DANGER: weak pointers are not threadsafe, so don't use this
-//   when passing between threads!
-//
-// BINDING A CLASS METHOD WITH MANUAL LIFETIME MANAGEMENT
-//
-//   kudu::Bind(&MyClass::Foo, kudu::Unretained(this));
-//
-//   This disables all lifetime management on the object. You're responsible
-//   for making sure the object is alive at the time of the call. You break it,
-//   you own it!
-//
-// BINDING A CLASS METHOD AND HAVING THE CALLBACK OWN THE CLASS
-//
-//   MyClass* myclass = new MyClass;
-//   kudu::Bind(&MyClass::Foo, kudu::Owned(myclass));
-//
-//   The object will be deleted when the callback is destroyed, even if it's
-//   not run (like if you post a task during shutdown). Potentially useful for
-//   "fire and forget" cases.
-//
-// IGNORING RETURN VALUES
-//
-//   Sometimes you want to call a function that returns a value in a callback
-//   that doesn't expect a return value.
-//
-//   int DoSomething(int arg) { cout << arg << endl; }
-//   kudu::Callback<void<int>) cb =
-//       kudu::Bind(kudu::IgnoreResult(&DoSomething));
-//
-//
-// -----------------------------------------------------------------------------
-// Quick reference for binding parameters to Bind()
-// -----------------------------------------------------------------------------
-//
-// Bound parameters are specified as arguments to Bind() and are passed to the
-// function. A callback with no parameters or no unbound parameters is called a
-// Closure (kudu::Callback<void(void)> and kudu::Closure are the same thing).
-//
-// PASSING PARAMETERS OWNED BY THE CALLBACK
-//
-//   void Foo(int* arg) { cout << *arg << endl; }
-//   int* pn = new int(1);
-//   kudu::Closure foo_callback = kudu::Bind(&foo, kudu::Owned(pn));
-//
-//   The parameter will be deleted when the callback is destroyed, even if it's
-//   not run (like if you post a task during shutdown).
-//
-// PASSING PARAMETERS AS A scoped_ptr
-//
-//   void TakesOwnership(scoped_ptr<Foo> arg) {}
-//   scoped_ptr<Foo> f(new Foo);
-//   // f becomes null during the following call.
-//   kudu::Closure cb = kudu::Bind(&TakesOwnership, kudu::Passed(&f));
-//
-//   Ownership of the parameter will be with the callback until the it is run,
-//   when ownership is passed to the callback function. This means the callback
-//   can only be run once. If the callback is never run, it will delete the
-//   object when it's destroyed.
-//
-// PASSING PARAMETERS AS A scoped_refptr
-//
-//   void TakesOneRef(scoped_refptr<Foo> arg) {}
-//   scoped_refptr<Foo> f(new Foo)
-//   kudu::Closure cb = kudu::Bind(&TakesOneRef, f);
-//
-//   This should "just work." The closure will take a reference as long as it
-//   is alive, and another reference will be taken for the called function.
-//
-// PASSING PARAMETERS BY REFERENCE
-//
-//   void foo(int arg) { cout << arg << endl }
-//   int n = 1;
-//   kudu::Closure has_ref = kudu::Bind(&foo, kudu::ConstRef(n));
-//   n = 2;
-//   has_ref.Run();  // Prints "2"
-//
-//   Normally parameters are copied in the closure. DANGER: ConstRef stores a
-//   const reference instead, referencing the original parameter. This means
-//   that you must ensure the object outlives the callback!
-//
-//
-// -----------------------------------------------------------------------------
-// Implementation notes
-// -----------------------------------------------------------------------------
-//
-// WHERE IS THIS DESIGN FROM:
-//
-// The design Callback and Bind is heavily influenced by C++'s
-// tr1::function/tr1::bind, and by the "Google Callback" system used inside
-// Google.
-//
-//
-// HOW THE IMPLEMENTATION WORKS:
-//
-// There are three main components to the system:
-//   1) The Callback classes.
-//   2) The Bind() functions.
-//   3) The arguments wrappers (e.g., Unretained() and ConstRef()).
-//
-// The Callback classes represent a generic function pointer. Internally,
-// it stores a refcounted piece of state that represents the target function
-// and all its bound parameters.  Each Callback specialization has a templated
-// constructor that takes an BindState<>*.  In the context of the constructor,
-// the static type of this BindState<> pointer uniquely identifies the
-// function it is representing, all its bound parameters, and a Run() method
-// that is capable of invoking the target.
-//
-// Callback's constructor takes the BindState<>* that has the full static type
-// and erases the target function type as well as the types of the bound
-// parameters.  It does this by storing a pointer to the specific Run()
-// function, and upcasting the state of BindState<>* to a
-// BindStateBase*. This is safe as long as this BindStateBase pointer
-// is only used with the stored Run() pointer.
-//
-// To BindState<> objects are created inside the Bind() functions.
-// These functions, along with a set of internal templates, are responsible for
-//
-//  - Unwrapping the function signature into return type, and parameters
-//  - Determining the number of parameters that are bound
-//  - Creating the BindState storing the bound parameters
-//  - Performing compile-time asserts to avoid error-prone behavior
-//  - Returning an Callback<> with an arity matching the number of unbound
-//    parameters and that knows the correct refcounting semantics for the
-//    target object if we are binding a method.
-//
-// The Bind functions do the above using type-inference, and template
-// specializations.
-//
-// By default Bind() will store copies of all bound parameters, and attempt
-// to refcount a target object if the function being bound is a class method.
-// These copies are created even if the function takes parameters as const
-// references. (Binding to non-const references is forbidden, see bind.h.)
-//
-// To change this behavior, we introduce a set of argument wrappers
-// (e.g., Unretained(), and ConstRef()).  These are simple container templates
-// that are passed by value, and wrap a pointer to argument.  See the
-// file-level comment in kudu/gutil/bind_helpers.h for more info.
-//
-// These types are passed to the Unwrap() functions, and the MaybeRefcount()
-// functions respectively to modify the behavior of Bind().  The Unwrap()
-// and MaybeRefcount() functions change behavior by doing partial
-// specialization based on whether or not a parameter is a wrapper type.
-//
-// ConstRef() is similar to tr1::cref.  Unretained() is specific to Chromium.
-//
-//
-// WHY NOT TR1 FUNCTION/BIND?
-//
-// Direct use of tr1::function and tr1::bind was considered, but ultimately
-// rejected because of the number of copy constructors invocations involved
-// in the binding of arguments during construction, and the forwarding of
-// arguments during invocation.  These copies will no longer be an issue in
-// C++0x because C++0x will support rvalue reference allowing for the compiler
-// to avoid these copies.  However, waiting for C++0x is not an option.
-//
-// Measured with valgrind on gcc version 4.4.3 (Ubuntu 4.4.3-4ubuntu5), the
-// tr1::bind call itself will invoke a non-trivial copy constructor three times
-// for each bound parameter.  Also, each when passing a tr1::function, each
-// bound argument will be copied again.
-//
-// In addition to the copies taken at binding and invocation, copying a
-// tr1::function causes a copy to be made of all the bound parameters and
-// state.
-//
-// Furthermore, in Chromium, it is desirable for the Callback to take a
-// reference on a target object when representing a class method call.  This
-// is not supported by tr1.
-//
-// Lastly, tr1::function and tr1::bind has a more general and flexible API.
-// This includes things like argument reordering by use of
-// tr1::bind::placeholder, support for non-const reference parameters, and some
-// limited amount of subtyping of the tr1::function object (e.g.,
-// tr1::function<int(int)> is convertible to tr1::function<void(int)>).
-//
-// These are not features that are required in Chromium. Some of them, such as
-// allowing for reference parameters, and subtyping of functions, may actually
-// become a source of errors. Removing support for these features actually
-// allows for a simpler implementation, and a terser Currying API.
-//
-//
-// WHY NOT GOOGLE CALLBACKS?
-//
-// The Google callback system also does not support refcounting.  Furthermore,
-// its implementation has a number of strange edge cases with respect to type
-// conversion of its arguments.  In particular, the argument's constness must
-// at times match exactly the function signature, or the type-inference might
-// break.  Given the above, writing a custom solution was easier.
-//
-//
-// MISSING FUNCTIONALITY
-//  - Invoking the return of Bind.  Bind(&foo).Run() does not work;
-//  - Binding arrays to functions that take a non-const pointer.
-//    Example:
-//      void Foo(const char* ptr);
-//      void Bar(char* ptr);
-//      Bind(&Foo, "test");
-//      Bind(&Bar, "test");  // This fails because ptr is not const.
-
-namespace kudu {
-
-// First, we forward declare the Callback class template. This informs the
-// compiler that the template only has 1 type parameter which is the function
-// signature that the Callback is representing.
-//
-// After this, create template specializations for 0-$(MAX_ARITY) parameters. Note that
-// even though the template typelist grows, the specialization still
-// only has one type: the function signature.
-//
-// If you are thinking of forward declaring Callback in your own header file,
-// please include "base/callback_forward.h" instead.
-template <typename Sig>
-class Callback;
-
-namespace internal {
-template <typename Runnable, typename RunType, typename BoundArgsType>
-struct BindState;
-}  // namespace internal
-
-
-$range ARITY 0..MAX_ARITY
-$for ARITY [[
-$range ARG 1..ARITY
-
-$if ARITY == 0 [[
-template <typename R>
-class Callback<R(void)> : public internal::CallbackBase {
-]] $else [[
-template <typename R, $for ARG , [[typename A$(ARG)]]>
-class Callback<R($for ARG , [[A$(ARG)]])> : public internal::CallbackBase {
-]]
-
- public:
-  typedef R(RunType)($for ARG , [[A$(ARG)]]);
-
-  Callback() : CallbackBase(NULL) { }
-
-  // Note that this constructor CANNOT be explicit, and that Bind() CANNOT
-  // return the exact Callback<> type.  See base/bind.h for details.
-  template <typename Runnable, typename BindRunType, typename BoundArgsType>
-  Callback(internal::BindState<Runnable, BindRunType,
-           BoundArgsType>* bind_state)
-      : CallbackBase(bind_state) {
-
-    // Force the assignment to a local variable of PolymorphicInvoke
-    // so the compiler will typecheck that the passed in Run() method has
-    // the correct type.
-    PolymorphicInvoke invoke_func =
-        &internal::BindState<Runnable, BindRunType, BoundArgsType>
-            ::InvokerType::Run;
-    polymorphic_invoke_ = reinterpret_cast<InvokeFuncStorage>(invoke_func);
-  }
-
-  bool Equals(const Callback& other) const {
-    return CallbackBase::Equals(other);
-  }
-
-  R Run($for ARG ,
-        [[typename internal::CallbackParamTraits<A$(ARG)>::ForwardType a$(ARG)]]) const {
-    PolymorphicInvoke f =
-        reinterpret_cast<PolymorphicInvoke>(polymorphic_invoke_);
-
-    return f(bind_state_.get()[[]]
-$if ARITY != 0 [[, ]]
-$for ARG ,
-             [[internal::CallbackForward(a$(ARG))]]);
-  }
-
- private:
-  typedef R(*PolymorphicInvoke)(
-      internal::BindStateBase*[[]]
-$if ARITY != 0 [[, ]]
-$for ARG , [[typename internal::CallbackParamTraits<A$(ARG)>::ForwardType]]);
-
-};
-
-
-]]  $$ for ARITY
-
-// Syntactic sugar to make Callbacks<void(void)> easier to declare since it
-// will be used in a lot of APIs with delayed execution.
-typedef Callback<void(void)> Closure;
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_CALLBACK_H
diff --git a/src/kudu/gutil/callback_forward.h b/src/kudu/gutil/callback_forward.h
deleted file mode 100644
index 956ff73..0000000
--- a/src/kudu/gutil/callback_forward.h
+++ /dev/null
@@ -1,17 +0,0 @@
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_CALLBACK_FORWARD_H_
-#define KUDU_GUTIL_CALLBACK_FORWARD_H_
-
-namespace kudu {
-
-template <typename Sig>
-class Callback;
-
-typedef Callback<void(void)> Closure;
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_CALLBACK_FORWARD_H
diff --git a/src/kudu/gutil/callback_internal.cc b/src/kudu/gutil/callback_internal.cc
deleted file mode 100644
index 1ed1af3..0000000
--- a/src/kudu/gutil/callback_internal.cc
+++ /dev/null
@@ -1,38 +0,0 @@
-// Copyright (c) 2012 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#include "kudu/gutil/callback_internal.h"
-
-#include <glog/logging.h>
-
-namespace kudu {
-namespace internal {
-
-bool CallbackBase::is_null() const {
-  return bind_state_.get() == nullptr;
-}
-
-void CallbackBase::Reset() {
-  polymorphic_invoke_ = nullptr;
-  // NULL the bind_state_ last, since it may be holding the last ref to whatever
-  // object owns us, and we may be deleted after that.
-  bind_state_ = nullptr;
-}
-
-bool CallbackBase::Equals(const CallbackBase& other) const {
-  return bind_state_.get() == other.bind_state_.get() &&
-         polymorphic_invoke_ == other.polymorphic_invoke_;
-}
-
-CallbackBase::CallbackBase(BindStateBase* bind_state)
-    : bind_state_(bind_state),
-      polymorphic_invoke_(nullptr) {
-  DCHECK(!bind_state_.get() || bind_state_->HasOneRef());
-}
-
-CallbackBase::~CallbackBase() {
-}
-
-}  // namespace internal
-}  // namespace kudu
diff --git a/src/kudu/gutil/callback_internal.h b/src/kudu/gutil/callback_internal.h
deleted file mode 100644
index f409c87..0000000
--- a/src/kudu/gutil/callback_internal.h
+++ /dev/null
@@ -1,175 +0,0 @@
-// Copyright (c) 2012 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-// This file contains utility functions and classes that help the
-// implementation, and management of the Callback objects.
-
-#ifndef KUDU_GUTIL_CALLBACK_INTERNAL_H_
-#define KUDU_GUTIL_CALLBACK_INTERNAL_H_
-
-#include <cstddef>
-
-#include "kudu/gutil/ref_counted.h"
-#include "kudu/gutil/template_util.h"
-#include "kudu/gutil/type_traits.h"
-
-namespace kudu {
-namespace internal {
-
-// BindStateBase is used to provide an opaque handle that the Callback
-// class can use to represent a function object with bound arguments.  It
-// behaves as an existential type that is used by a corresponding
-// DoInvoke function to perform the function execution.  This allows
-// us to shield the Callback class from the types of the bound argument via
-// "type erasure."
-class BindStateBase : public RefCountedThreadSafe<BindStateBase> {
- protected:
-  friend class RefCountedThreadSafe<BindStateBase>;
-  virtual ~BindStateBase() {}
-};
-
-// Holds the Callback methods that don't require specialization to reduce
-// template bloat.
-class CallbackBase {
- public:
-  // Returns true if Callback is null (doesn't refer to anything).
-  bool is_null() const;
-
-  // Returns the Callback into an uninitialized state.
-  void Reset();
-
- protected:
-  // In C++, it is safe to cast function pointers to function pointers of
-  // another type. It is not okay to use void*. We create a InvokeFuncStorage
-  // that that can store our function pointer, and then cast it back to
-  // the original type on usage.
-  typedef void(*InvokeFuncStorage)(void);
-
-  // Returns true if this callback equals |other|. |other| may be null.
-  bool Equals(const CallbackBase& other) const;
-
-  // Allow initializing of |bind_state_| via the constructor to avoid default
-  // initialization of the scoped_refptr.  We do not also initialize
-  // |polymorphic_invoke_| here because doing a normal assignment in the
-  // derived Callback templates makes for much nicer compiler errors.
-  explicit CallbackBase(BindStateBase* bind_state);
-
-  // Force the destructor to be instantiated inside this translation unit so
-  // that our subclasses will not get inlined versions.  Avoids more template
-  // bloat.
-  ~CallbackBase();
-
-  scoped_refptr<BindStateBase> bind_state_;
-  InvokeFuncStorage polymorphic_invoke_;
-};
-
-// A helper template to determine if given type is non-const move-only-type,
-// i.e. if a value of the given type should be passed via .Pass() in a
-// destructive way.
-template <typename T> struct IsMoveOnlyType {
-  template <typename U>
-  static base::YesType Test(const typename U::MoveOnlyTypeForCPP03*);
-
-  template <typename U>
-  static base::NoType Test(...);
-
-  static const bool value = sizeof(Test<T>(0)) == sizeof(base::YesType) &&
-                            !base::is_const<T>::value;
-};
-
-// This is a typetraits object that's used to take an argument type, and
-// extract a suitable type for storing and forwarding arguments.
-//
-// In particular, it strips off references, and converts arrays to
-// pointers for storage; and it avoids accidentally trying to create a
-// "reference of a reference" if the argument is a reference type.
-//
-// This array type becomes an issue for storage because we are passing bound
-// parameters by const reference. In this case, we end up passing an actual
-// array type in the initializer list which C++ does not allow.  This will
-// break passing of C-string literals.
-template <typename T, bool is_move_only = IsMoveOnlyType<T>::value>
-struct CallbackParamTraits {
-  typedef const T& ForwardType;
-  typedef T StorageType;
-};
-
-// The Storage should almost be impossible to trigger unless someone manually
-// specifies type of the bind parameters.  However, in case they do,
-// this will guard against us accidentally storing a reference parameter.
-//
-// The ForwardType should only be used for unbound arguments.
-template <typename T>
-struct CallbackParamTraits<T&, false> {
-  typedef T& ForwardType;
-  typedef T StorageType;
-};
-
-// Note that for array types, we implicitly add a const in the conversion. This
-// means that it is not possible to bind array arguments to functions that take
-// a non-const pointer. Trying to specialize the template based on a "const
-// T[n]" does not seem to match correctly, so we are stuck with this
-// restriction.
-template <typename T, size_t n>
-struct CallbackParamTraits<T[n], false> {
-  typedef const T* ForwardType;
-  typedef const T* StorageType;
-};
-
-// See comment for CallbackParamTraits<T[n]>.
-template <typename T>
-struct CallbackParamTraits<T[], false> {
-  typedef const T* ForwardType;
-  typedef const T* StorageType;
-};
-
-// Parameter traits for movable-but-not-copyable scopers.
-//
-// Callback<>/Bind() understands movable-but-not-copyable semantics where
-// the type cannot be copied but can still have its state destructively
-// transferred (aka. moved) to another instance of the same type by calling a
-// helper function.  When used with Bind(), this signifies transferal of the
-// object's state to the target function.
-//
-// For these types, the ForwardType must not be a const reference, or a
-// reference.  A const reference is inappropriate, and would break const
-// correctness, because we are implementing a destructive move.  A non-const
-// reference cannot be used with temporaries which means the result of a
-// function or a cast would not be usable with Callback<> or Bind().
-template <typename T>
-struct CallbackParamTraits<T, true> {
-  typedef T ForwardType;
-  typedef T StorageType;
-};
-
-// CallbackForward() is a very limited simulation of C++11's std::forward()
-// used by the Callback/Bind system for a set of movable-but-not-copyable
-// types.  It is needed because forwarding a movable-but-not-copyable
-// argument to another function requires us to invoke the proper move
-// operator to create a rvalue version of the type.  The supported types are
-// whitelisted below as overloads of the CallbackForward() function. The
-// default template compiles out to be a no-op.
-//
-// In C++11, std::forward would replace all uses of this function.  However, it
-// is impossible to implement a general std::forward with C++11 due to a lack
-// of rvalue references.
-//
-// In addition to Callback/Bind, this is used by PostTaskAndReplyWithResult to
-// simulate std::forward() and forward the result of one Callback as a
-// parameter to another callback. This is to support Callbacks that return
-// the movable-but-not-copyable types whitelisted above.
-template <typename T>
-typename base::enable_if<!IsMoveOnlyType<T>::value, T>::type& CallbackForward(T& t) {
-  return t;
-}
-
-template <typename T>
-typename base::enable_if<IsMoveOnlyType<T>::value, T>::type CallbackForward(T& t) {
-  return t.Pass();
-}
-
-}  // namespace internal
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_CALLBACK_INTERNAL_H_
diff --git a/src/kudu/gutil/raw_scoped_refptr_mismatch_checker.h b/src/kudu/gutil/raw_scoped_refptr_mismatch_checker.h
deleted file mode 100644
index 63e4e71..0000000
--- a/src/kudu/gutil/raw_scoped_refptr_mismatch_checker.h
+++ /dev/null
@@ -1,63 +0,0 @@
-// Copyright (c) 2011 The Chromium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file.
-
-#ifndef KUDU_GUTIL_RAW_SCOPED_REFPTR_MISMATCH_CHECKER_H_
-#define KUDU_GUTIL_RAW_SCOPED_REFPTR_MISMATCH_CHECKER_H_
-
-#include <type_traits>
-
-#include "kudu/gutil/ref_counted.h"
-
-// It is dangerous to post a task with a T* argument where T is a subtype of
-// RefCounted(Base|ThreadSafeBase), since by the time the parameter is used, the
-// object may already have been deleted since it was not held with a
-// scoped_refptr. Example: http://crbug.com/27191
-// The following set of traits are designed to generate a compile error
-// whenever this antipattern is attempted.
-
-namespace kudu {
-
-// This is a base internal implementation file used by task.h and callback.h.
-// Not for public consumption, so we wrap it in namespace internal.
-namespace internal {
-
-template <typename T>
-struct NeedsScopedRefptrButGetsRawPtr {
-#if defined(OS_WIN)
-  enum {
-    value = base::false_type::value
-  };
-#else
-  enum {
-    // Human readable translation: you needed to be a scoped_refptr if you are a
-    // raw pointer type and are convertible to a RefCounted(Base|ThreadSafeBase)
-    // type.
-    value = (std::is_pointer<T>::value &&
-             (std::is_convertible<T, subtle::RefCountedBase*>::value ||
-              std::is_convertible<T, subtle::RefCountedThreadSafeBase*>::value))
-  };
-#endif
-};
-
-template <typename Params>
-struct ParamsUseScopedRefptrCorrectly {
-  enum { value = 0 };
-};
-
-template <>
-struct ParamsUseScopedRefptrCorrectly<std::tuple<>> {
-  enum { value = 1 };
-};
-
-template <typename Head, typename... Tail>
-struct ParamsUseScopedRefptrCorrectly<std::tuple<Head, Tail...>> {
-  enum { value = !NeedsScopedRefptrButGetsRawPtr<Head>::value &&
-                  ParamsUseScopedRefptrCorrectly<std::tuple<Tail...>>::value };
-};
-
-}  // namespace internal
-
-}  // namespace kudu
-
-#endif  // KUDU_GUTIL_RAW_SCOPED_REFPTR_MISMATCH_CHECKER_H_
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index b185434..b340f04 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -436,7 +436,6 @@ ADD_KUDU_TEST(block_bloom_filter-test)
 ADD_KUDU_TEST(bloom_filter-test)
 ADD_KUDU_TEST(cache-bench RUN_SERIAL true)
 ADD_KUDU_TEST(cache-test)
-ADD_KUDU_TEST(callback_bind-test)
 ADD_KUDU_TEST(char_util-test
   DATA_FILES testdata/char_truncate_utf8.txt testdata/char_truncate_ascii.txt)
 ADD_KUDU_TEST(countdown_latch-test)
diff --git a/src/kudu/util/callback_bind-test.cc b/src/kudu/util/callback_bind-test.cc
deleted file mode 100644
index 392f496..0000000
--- a/src/kudu/util/callback_bind-test.cc
+++ /dev/null
@@ -1,119 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#include <ostream>
-#include <string>
-#include <type_traits>
-#include <utility>
-
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include "kudu/gutil/bind.h"
-#include "kudu/gutil/bind_helpers.h"
-#include "kudu/gutil/callback.h"
-#include "kudu/gutil/gscoped_ptr.h"
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/ref_counted.h"
-
-namespace kudu {
-
-using std::string;
-
-static int Return5() {
-  return 5;
-}
-
-TEST(CallbackBindTest, TestFreeFunction) {
-  Callback<int(void)> func_cb = Bind(&Return5);
-  ASSERT_EQ(5, func_cb.Run());
-}
-
-class Ref : public RefCountedThreadSafe<Ref> {
- public:
-  int Foo() { return 3; }
-};
-
-// Simple class that helps with verifying ref counting.
-// Not thread-safe.
-struct RefCountable {
-  RefCountable()
-      : refs(0) {
-  }
-  void AddRef() const {
-    refs++;
-  }
-  void Release() const {
-    refs--;
-  }
-  void Print() const {
-    LOG(INFO) << "Hello. Refs: " << refs;
-  }
-
-  mutable int refs;
-  DISALLOW_COPY_AND_ASSIGN(RefCountable);
-};
-
-TEST(CallbackBindTest, TestClassMethod) {
-  scoped_refptr<Ref> ref = new Ref();
-  Callback<int(void)> ref_cb = Bind(&Ref::Foo, ref);
-  ref = nullptr;
-  ASSERT_EQ(3, ref_cb.Run());
-}
-
-int ReturnI(int i, const char* str) {
-  return i;
-}
-
-TEST(CallbackBindTest, TestPartialBind) {
-  Callback<int(const char*)> cb = Bind(&ReturnI, 23);
-  ASSERT_EQ(23, cb.Run("hello world"));
-}
-
-char IncrementChar(gscoped_ptr<char> in) {
-  return *in + 1;
-}
-
-TEST(CallbackBindTest, TestCallScopedPtrArg) {
-  // Calling a function with a gscoped_ptr argument is just like any other
-  // function which takes gscoped_ptr:
-  gscoped_ptr<char> foo(new char('x'));
-  Callback<char(gscoped_ptr<char>)> cb = Bind(&IncrementChar);
-  ASSERT_EQ('y', cb.Run(std::move(foo)));
-}
-
-TEST(CallbackBindTest, TestBindScopedPtrArg) {
-  // Binding a function with a gscoped_ptr argument requires using Passed()
-  gscoped_ptr<char> foo(new char('x'));
-  Callback<char(void)> cb = Bind(&IncrementChar, Passed(&foo));
-  ASSERT_EQ('y', cb.Run());
-}
-
-// Test that the ref counting functionality works.
-TEST(CallbackBindTest, TestRefCounting) {
-  RefCountable countable;
-  {
-    ASSERT_EQ(0, countable.refs);
-    Closure cb = Bind(&RefCountable::Print, &countable);
-    ASSERT_EQ(1, countable.refs);
-    cb.Run();
-    ASSERT_EQ(1, countable.refs);
-  }
-  ASSERT_EQ(0, countable.refs);
-}
-
-} // namespace kudu


[kudu] 01/07: Add functionality to serialize a RowBlock into columnar format

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1d28104cac0866c83b3022e3ddf3b7237962c1da
Author: Todd Lipcon <to...@apache.org>
AuthorDate: Wed Mar 25 11:38:56 2020 -0700

    Add functionality to serialize a RowBlock into columnar format
    
    This adds the core functionality to take a RowBlock and convert it into
    a set of buffers for each column: the cell data, the null bitmap, and
    the indirect (string) data.
    
    This also updates wire_protocol-test to add coverage for nulls and
    unselected rows, and adds a comparison benchmark. The columnar code path
    is 4-5x faster in the best case, and in the worst case only 30% slower
    that the existing row-wise code path.
    
    Some follow-up commits will add further optimizations.
    
    Change-Id: I287a8aa6736f19816b0edbe16409c01f35c0319e
    Reviewed-on: http://gerrit.cloudera.org:8080/15560
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/common/CMakeLists.txt            |   2 +-
 src/kudu/common/columnar_serialization.cc | 134 ++++++++-
 src/kudu/common/columnar_serialization.h  |  38 ++-
 src/kudu/common/wire_protocol-test.cc     | 442 ++++++++++++++++++++++--------
 src/kudu/util/faststring.h                |  10 +
 5 files changed, 510 insertions(+), 116 deletions(-)

diff --git a/src/kudu/common/CMakeLists.txt b/src/kudu/common/CMakeLists.txt
index 0a1814e..4ffe9c4 100644
--- a/src/kudu/common/CMakeLists.txt
+++ b/src/kudu/common/CMakeLists.txt
@@ -99,4 +99,4 @@ ADD_KUDU_TEST(scan_spec-test)
 ADD_KUDU_TEST(schema-test)
 ADD_KUDU_TEST(table_util-test)
 ADD_KUDU_TEST(types-test)
-ADD_KUDU_TEST(wire_protocol-test NUM_SHARDS 4)
+ADD_KUDU_TEST(wire_protocol-test NUM_SHARDS 10)
diff --git a/src/kudu/common/columnar_serialization.cc b/src/kudu/common/columnar_serialization.cc
index 69e8da9..5e647d6 100644
--- a/src/kudu/common/columnar_serialization.cc
+++ b/src/kudu/common/columnar_serialization.cc
@@ -19,7 +19,6 @@
 
 #include <immintrin.h>
 
-#include <cstdint>
 #include <cstring>
 #include <ostream>
 #include <string>
@@ -27,11 +26,17 @@
 
 #include <glog/logging.h>
 
+#include "kudu/common/columnblock.h"
+#include "kudu/common/common.pb.h"
+#include "kudu/common/rowblock.h"
+#include "kudu/common/schema.h"
+#include "kudu/common/types.h"
 #include "kudu/common/zp7.h"
 #include "kudu/gutil/cpu.h"
 #include "kudu/gutil/port.h"
 #include "kudu/util/alignment.h"
 #include "kudu/util/bitmap.h"
+#include "kudu/util/slice.h"
 
 using std::vector;
 
@@ -360,6 +365,133 @@ void CopySelectedRows(const vector<uint16_t>& sel_rows,
   }
 }
 
+namespace {
+// For each of the Slices in 'cells_buf', copy the pointed-to data into 'indirect' and
+// modify the Slice so that its 'pointer' field is not an actual memory pointer, but
+// rather an offset within the indirect data buffer.
+void RelocateSlicesToIndirect(uint8_t* __restrict__ cells_buf, int n_rows,
+                              faststring* indirect) {
+  Slice* cell_slices = reinterpret_cast<Slice*>(cells_buf);
+  size_t total_size = 0;
+  for (int i = 0; i < n_rows; i++) {
+    total_size += cell_slices[i].size();
+  }
+
+  int old_size = indirect->size();
+  indirect->resize_with_extra_capacity(old_size + total_size);
+
+  uint8_t* dst_base = indirect->data();
+  uint8_t* dst = dst_base + old_size;
+
+  for (int i = 0; i < n_rows; i++) {
+    Slice* s = &cell_slices[i];
+    if (!s->empty()) {
+      memcpy(dst, s->data(), s->size());
+    }
+    *s = Slice(reinterpret_cast<const uint8_t*>(dst - dst_base), s->size());
+    dst += s->size();
+  }
+}
+
+// Specialized division for the known type sizes. Despite having some branching here,
+// this is faster than a 'div' instruction which has a 20+ cycle latency.
+size_t div_type_size(size_t s, size_t divisor) {
+  switch (divisor) {
+    case 1: return s;
+    case 2: return s/2;
+    case 4: return s/4;
+    case 8: return s/8;
+    case 16: return s/16;
+    default: return s/divisor;
+  }
+}
+
+
+// Copy the selected cells (and non-null-bitmap bits) from 'cblock' into 'dst' according to
+// the given 'sel_rows'.
+void CopySelectedCellsFromColumn(const ColumnBlock& cblock,
+                                 const SelectedRows& sel_rows,
+                                 ColumnarSerializedBatch::Column* dst) {
+  size_t type_size = cblock.type_info()->size();
+  int n_sel = sel_rows.num_selected();
+
+  // Number of initial rows in the dst values and null_bitmap.
+  DCHECK_EQ(dst->data.size() % type_size, 0);
+  size_t initial_rows = div_type_size(dst->data.size(), type_size);
+  size_t new_num_rows = initial_rows + n_sel;
+
+  dst->data.resize_with_extra_capacity(type_size * new_num_rows);
+  uint8_t* dst_buf = dst->data.data() + type_size * initial_rows;
+  const uint8_t* src_buf = cblock.cell_ptr(0);
+
+  if (sel_rows.all_selected()) {
+    memcpy(dst_buf, src_buf, type_size * n_sel);
+  } else {
+    CopySelectedRows(sel_rows.indexes(), type_size, src_buf, dst_buf);
+  }
+
+  if (cblock.is_nullable()) {
+    DCHECK_EQ(dst->non_null_bitmap->size(), BitmapSize(initial_rows));
+    dst->non_null_bitmap->resize_with_extra_capacity(BitmapSize(new_num_rows));
+    CopyNonNullBitmap(cblock.non_null_bitmap(),
+                      sel_rows.bitmap(),
+                      initial_rows, cblock.nrows(),
+                      dst->non_null_bitmap->data());
+    ZeroNullValues(type_size, initial_rows, n_sel, dst->data.data(), dst->non_null_bitmap->data());
+  }
+
+  if (cblock.type_info()->physical_type() == BINARY) {
+    RelocateSlicesToIndirect(dst_buf, n_sel, boost::get_pointer(dst->indirect_data));
+  }
+}
+} // anonymous namespace
 } // namespace internal
 
+int SerializeRowBlockColumnar(
+    const RowBlock& block,
+    const Schema* projection_schema,
+    ColumnarSerializedBatch* out) {
+  DCHECK_GT(block.nrows(), 0);
+  const Schema* tablet_schema = block.schema();
+
+  if (projection_schema == nullptr) {
+    projection_schema = tablet_schema;
+  }
+
+  // Initialize buffers for the columns.
+  // TODO(todd) don't pre-size these to 1MB per column -- quite
+  // expensive if there are a lot of columns!
+  if (out->columns.size() != projection_schema->num_columns()) {
+    CHECK_EQ(out->columns.size(), 0);
+    out->columns.reserve(projection_schema->num_columns());
+    for (const auto& col : projection_schema->columns()) {
+      out->columns.emplace_back();
+      out->columns.back().data.reserve(1024 * 1024);
+      if (col.type_info()->physical_type() == BINARY) {
+        out->columns.back().indirect_data.emplace();
+      }
+      if (col.is_nullable()) {
+        out->columns.back().non_null_bitmap.emplace();
+      }
+    }
+  }
+
+  SelectedRows sel = block.selection_vector()->GetSelectedRows();
+  int col_idx = 0;
+  for (const auto& col : projection_schema->columns()) {
+    int t_schema_idx = tablet_schema->find_column(col.name());
+    CHECK_NE(t_schema_idx, -1);
+    const ColumnBlock& column_block = block.column_block(t_schema_idx);
+
+    internal::CopySelectedCellsFromColumn(
+        column_block,
+        sel,
+        &out->columns[col_idx]);
+    col_idx++;
+  }
+
+  return sel.num_selected();
+}
+
+
 } // namespace kudu
diff --git a/src/kudu/common/columnar_serialization.h b/src/kudu/common/columnar_serialization.h
index 04025b7..d7ad494 100644
--- a/src/kudu/common/columnar_serialization.h
+++ b/src/kudu/common/columnar_serialization.h
@@ -19,15 +19,48 @@
 #include <cstdint>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
+
+#include "kudu/util/faststring.h"
+
 namespace kudu {
 
+class RowBlock;
+class Schema;
+
+// A pending batch of serialized rows, suitable for easy conversion
+// into the protobuf representation and a set of sidecars.
+struct ColumnarSerializedBatch {
+  struct Column {
+    // Underlying column data.
+    faststring data;
+
+    // Data for varlen columns (BINARY)
+    boost::optional<faststring> indirect_data;
+
+    // Each bit is set when a value is non-null
+    boost::optional<faststring> non_null_bitmap;
+  };
+  std::vector<Column> columns;
+};
+
+// Serialize the data in 'block' into the columnar batch 'out', appending to
+// any data already serialized to the same batch.
+//
+// Returns the number of selected rows serialized.
+int SerializeRowBlockColumnar(
+    const RowBlock& block,
+    const Schema* projection_schema,
+    ColumnarSerializedBatch* out);
+
+
 ////////////////////////////////////////////////////////////
 // Expose these internal functions for unit testing.
 // Do not call them outside of tests!
 // See .cc file for docs.
 ////////////////////////////////////////////////////////////
 namespace internal {
-void ZeroNullValues(int type_size,
+void ZeroNullValues(int sizeof_type,
                     int dst_idx,
                     int n_rows,
                     uint8_t* dst_values_buf,
@@ -40,7 +73,7 @@ void CopyNonNullBitmap(const uint8_t* non_null_bitmap,
                        uint8_t* dst_non_null_bitmap);
 
 void CopySelectedRows(const std::vector<uint16_t>& sel_rows,
-                      int type_size,
+                      int sizeof_type,
                       const uint8_t* __restrict__ src_buf,
                       uint8_t* __restrict__ dst_buf);
 
@@ -52,6 +85,7 @@ enum class PextMethod {
 #endif
   kSimple
 };
+
 extern PextMethod g_pext_method;
 
 std::vector<PextMethod> GetAvailablePextMethods();
diff --git a/src/kudu/common/wire_protocol-test.cc b/src/kudu/common/wire_protocol-test.cc
index 4c9ac0d..ae18f1e 100644
--- a/src/kudu/common/wire_protocol-test.cc
+++ b/src/kudu/common/wire_protocol-test.cc
@@ -20,6 +20,7 @@
 #include <algorithm>
 #include <cstdint>
 #include <cstring>
+#include <list>
 #include <numeric>
 #include <ostream>
 #include <string>
@@ -30,12 +31,14 @@
 #include <gtest/gtest.h>
 
 #include "kudu/common/column_predicate.h"
+#include "kudu/common/columnar_serialization.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/row.h"
 #include "kudu/common/rowblock.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/types.h"
 #include "kudu/common/wire_protocol.pb.h"
+#include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/bitmap.h"
@@ -43,7 +46,9 @@
 #include "kudu/util/faststring.h"
 #include "kudu/util/hash.pb.h"
 #include "kudu/util/hexdump.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/memory/arena.h"
+#include "kudu/util/random.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/stopwatch.h"  // IWYU pragma: keep
@@ -57,121 +62,58 @@ using strings::Substitute;
 
 namespace kudu {
 
-class WireProtocolTest : public KuduTest,
-                         // Used for benchmark, int corresponds to the number of columns,
-                         // double corresponds to the selection rate.
-                         public testing::WithParamInterface<tuple<int, double>> {
+class WireProtocolTest : public KuduTest {
  public:
   WireProtocolTest()
-      : schema_({ ColumnSchema("col1", STRING),
-              ColumnSchema("col2", STRING),
-              ColumnSchema("col3", UINT32, true /* nullable */) },
+      : schema_({ ColumnSchema("string", STRING),
+                  ColumnSchema("nullable_string", STRING, /* is_nullable=*/true),
+                  ColumnSchema("int", INT32),
+                  ColumnSchema("nullable_int", INT32, /* is_nullable=*/true),
+                  ColumnSchema("int64", INT64) },
         1),
         test_data_arena_(4096) {
   }
 
-  void FillRowBlockWithTestRows(RowBlock* block) {
-    test_data_arena_.Reset();
+  static void FillRowBlockWithTestRows(RowBlock* block) {
+    Random rng(SeedRandom());
+
     block->selection_vector()->SetAllTrue();
 
     for (int i = 0; i < block->nrows(); i++) {
+      if (rng.OneIn(10)) {
+        block->selection_vector()->SetRowUnselected(i);
+        continue;
+      }
+
       RowBlockRow row = block->row(i);
 
       // We make new copies of these strings into the Arena for each row so that
       // the workload is more realistic. If we just re-use the same Slice object
       // for each row, the memory accesses fit entirely into a smaller number of
       // cache lines and we may micro-optimize for the wrong thing.
-      Slice col1, col2;
-      CHECK(test_data_arena_.RelocateSlice("hello world col1", &col1));
-      CHECK(test_data_arena_.RelocateSlice("hello world col2", &col2));
-      *reinterpret_cast<Slice*>(row.mutable_cell_ptr(0)) = col1;
-      *reinterpret_cast<Slice*>(row.mutable_cell_ptr(1)) = col2;
-      *reinterpret_cast<uint32_t*>(row.mutable_cell_ptr(2)) = i;
-      row.cell(2).set_null(false);
-    }
-  }
-
-  void ResetBenchmarkSchema(int num_columns) {
-    vector<ColumnSchema> column_schemas;
-    column_schemas.reserve(num_columns);
-    for (int i = 0; i < num_columns; i++) {
-      column_schemas.emplace_back(Substitute("col$0", i), i % 2 ? STRING : INT32);
-    }
-    benchmark_schema_.Reset(column_schemas, 1);
-  }
-
-  void FillRowBlockForBenchmark(RowBlock* block) {
-    test_data_arena_.Reset();
-    for (int i = 0; i < block->nrows(); i++) {
-      RowBlockRow row = block->row(i);
-      for (int j = 0; j < benchmark_schema_.num_columns(); j++) {
-        const ColumnSchema& column_schema = benchmark_schema_.column(j);
-        DataType type = column_schema.type_info()->type();
-        if (type == STRING) {
-          Slice col;
-          CHECK(test_data_arena_.RelocateSlice(Substitute("hello world $0",
-                                               column_schema.name()), &col));
-          memcpy(row.mutable_cell_ptr(j), &col, sizeof(Slice));
-        } else if (type == INT32) {
-          memcpy(row.mutable_cell_ptr(j), &i, sizeof(int32_t));
-        } else {
-          LOG(FATAL) << "Unexpected type.";
-        }
+      CHECK(block->arena()->RelocateSlice(
+          "hello world col0",
+          reinterpret_cast<Slice*>(row.mutable_cell_ptr(0))));
+
+      if (rng.OneIn(3)) {
+        row.cell(1).set_null(true);
+      } else {
+        row.cell(1).set_null(false);
+        CHECK(block->arena()->RelocateSlice(
+            "hello world col1",
+            reinterpret_cast<Slice*>(row.mutable_cell_ptr(1))));
       }
-    }
-  }
-
-  void SelectRandomRowsWithRate(RowBlock* block, double rate) {
-    CHECK_LE(rate, 1.0);
-    CHECK_GE(rate, 0.0);
-    int select_count = block->nrows() * rate;
-    SelectionVector* select_vector = block->selection_vector();
-    if (rate == 1.0) {
-      select_vector->SetAllTrue();
-    } else if (rate == 0.0) {
-      select_vector->SetAllFalse();
-    } else {
-      vector<int> indexes(block->nrows());
-      std::iota(indexes.begin(), indexes.end(), 0);
-      std::random_shuffle(indexes.begin(), indexes.end());
-      indexes.resize(select_count);
-      select_vector->SetAllFalse();
-      for (auto index : indexes) {
-        select_vector->SetRowSelected(index);
-      }
-    }
-    CHECK_EQ(select_vector->CountSelected(), select_count);
-  }
 
-  // Use column_count to control the schema scale.
-  // Use select_rate to control the number of selected rows.
-  void RunBenchmark(int column_count, double select_rate) {
-    ResetBenchmarkSchema(column_count);
-    Arena arena(1024);
-    RowBlock block(&benchmark_schema_, 1000, &arena);
-    // Regardless of the config, use a constant number of cells for the test by
-    // looping the conversion an appropriate number of times.
-    const int64_t kNumCellsToConvert = AllowSlowTests() ? 100000000 : 1000000;
-    const int kNumTrials = kNumCellsToConvert / select_rate / column_count / block.nrows();
-    FillRowBlockForBenchmark(&block);
-    SelectRandomRowsWithRate(&block, select_rate);
+      *reinterpret_cast<int32_t*>(row.mutable_cell_ptr(2)) = i;
+      *reinterpret_cast<int32_t*>(row.mutable_cell_ptr(3)) = i;
+      row.cell(3).set_null(rng.OneIn(7));
 
-    faststring direct, indirect;
-    int64_t cycle_start = CycleClock::Now();
-    for (int i = 0; i < kNumTrials; ++i) {
-      direct.clear();
-      indirect.clear();
-      SerializeRowBlock(block, nullptr, &direct, &indirect);
+      *reinterpret_cast<int64_t*>(row.mutable_cell_ptr(4)) = i;
     }
-    int64_t cycle_end = CycleClock::Now();
-    LOG(INFO) << Substitute(
-        "Converting to PB with column count $0 and row select rate $1: $2 cycles/cell",
-        column_count, select_rate,
-        static_cast<double>(cycle_end - cycle_start) / kNumCellsToConvert);
   }
+
  protected:
   Schema schema_;
-  Schema benchmark_schema_;
   Arena test_data_arena_;
 };
 
@@ -221,25 +163,38 @@ TEST_F(WireProtocolTest, TestSchemaRoundTrip) {
   google::protobuf::RepeatedPtrField<ColumnSchemaPB> pbs;
 
   ASSERT_OK(SchemaToColumnPBs(schema_, &pbs));
-  ASSERT_EQ(3, pbs.size());
+  ASSERT_EQ(5, pbs.size());
 
   // Column 0.
   EXPECT_TRUE(pbs.Get(0).is_key());
-  EXPECT_EQ("col1", pbs.Get(0).name());
+  EXPECT_EQ("string", pbs.Get(0).name());
   EXPECT_EQ(STRING, pbs.Get(0).type());
   EXPECT_FALSE(pbs.Get(0).is_nullable());
 
   // Column 1.
   EXPECT_FALSE(pbs.Get(1).is_key());
-  EXPECT_EQ("col2", pbs.Get(1).name());
+  EXPECT_EQ("nullable_string", pbs.Get(1).name());
   EXPECT_EQ(STRING, pbs.Get(1).type());
-  EXPECT_FALSE(pbs.Get(1).is_nullable());
+  EXPECT_TRUE(pbs.Get(1).is_nullable());
 
   // Column 2.
   EXPECT_FALSE(pbs.Get(2).is_key());
-  EXPECT_EQ("col3", pbs.Get(2).name());
-  EXPECT_EQ(UINT32, pbs.Get(2).type());
-  EXPECT_TRUE(pbs.Get(2).is_nullable());
+  EXPECT_EQ("int", pbs.Get(2).name());
+  EXPECT_EQ(INT32, pbs.Get(2).type());
+  EXPECT_FALSE(pbs.Get(2).is_nullable());
+
+
+  // Column 3.
+  EXPECT_FALSE(pbs.Get(3).is_key());
+  EXPECT_EQ("nullable_int", pbs.Get(3).name());
+  EXPECT_EQ(INT32, pbs.Get(3).type());
+  EXPECT_TRUE(pbs.Get(3).is_nullable());
+
+  // Column 4.
+  EXPECT_FALSE(pbs.Get(4).is_key());
+  EXPECT_EQ("int64", pbs.Get(4).name());
+  EXPECT_EQ(INT64, pbs.Get(4).type());
+  EXPECT_FALSE(pbs.Get(4).is_nullable());
 
   // Convert back to a Schema object and verify they're identical.
   Schema schema2;
@@ -304,11 +259,10 @@ TEST_F(WireProtocolTest, TestBadSchema_DuplicateColumnName) {
   ASSERT_EQ("Invalid argument: Duplicate column name: c0", s.ToString());
 }
 
-// Create a block of rows in columnar layout and ensure that it can be
-// converted to and from protobuf.
-TEST_F(WireProtocolTest, TestColumnarRowBlockToPB) {
+// Create a block of rows and ensure that it can be converted to and from protobuf.
+TEST_F(WireProtocolTest, TestRowBlockToRowwisePB) {
   Arena arena(1024);
-  RowBlock block(&schema_, 10, &arena);
+  RowBlock block(&schema_, 30, &arena);
   FillRowBlockWithTestRows(&block);
 
   // Convert to PB.
@@ -326,14 +280,82 @@ TEST_F(WireProtocolTest, TestColumnarRowBlockToPB) {
   Slice direct_sidecar = direct;
   ASSERT_OK(ExtractRowsFromRowBlockPB(schema_, pb, indirect,
                                       &direct_sidecar, &row_ptrs));
-  ASSERT_EQ(block.nrows(), row_ptrs.size());
+  ASSERT_EQ(block.selection_vector()->CountSelected(), row_ptrs.size());
+  int dst_row_idx = 0;
   for (int i = 0; i < block.nrows(); ++i) {
-    ConstContiguousRow row_roundtripped(&schema_, row_ptrs[i]);
+    if (!block.selection_vector()->IsRowSelected(i)) {
+      continue;
+    }
+    ConstContiguousRow row_roundtripped(&schema_, row_ptrs[dst_row_idx]);
     EXPECT_EQ(schema_.DebugRow(block.row(i)),
               schema_.DebugRow(row_roundtripped));
+    dst_row_idx++;
+  }
+}
+
+// Create blocks of rows and ensure that they can be converted to the columnar serialized
+// layout.
+TEST_F(WireProtocolTest, TestRowBlockToColumnarPB) {
+  // Generate several blocks of random data.
+  static constexpr int kNumBlocks = 3;
+  Arena arena(1024);
+  std::list<RowBlock> blocks;
+  for (int i = 0; i < kNumBlocks; i++) {
+    blocks.emplace_back(&schema_, 30, &arena);
+    FillRowBlockWithTestRows(&blocks.back());
+  }
+
+  // Convert all of the RowBlocks to a single serialized (concatenated) columnar format.
+  ColumnarSerializedBatch batch;
+  for (const auto& block : blocks) {
+    SerializeRowBlockColumnar(block, nullptr, &batch);
+  }
+
+  // Verify that the resulting serialized data matches the concatenated original data blocks.
+  ASSERT_EQ(5, batch.columns.size());
+  int dst_row_idx = 0;
+  for (const auto& block : blocks) {
+    for (int src_row_idx = 0; src_row_idx < block.nrows(); src_row_idx++) {
+      if (!block.selection_vector()->IsRowSelected(src_row_idx)) {
+        continue;
+      }
+      SCOPED_TRACE(src_row_idx);
+      SCOPED_TRACE(dst_row_idx);
+      const auto& row = block.row(src_row_idx);
+      for (int c = 0; c < schema_.num_columns(); c++) {
+        SCOPED_TRACE(c);
+        const auto& col = schema_.column(c);
+        const auto& serialized_col = batch.columns[c];
+        if (col.is_nullable()) {
+          bool expect_null = row.is_null(c);;
+          EXPECT_EQ(!BitmapTest(serialized_col.non_null_bitmap->data(), dst_row_idx),
+                    expect_null);
+          if (expect_null) {
+            continue;
+          }
+        }
+        int type_size = col.type_info()->size();
+        Slice serialized_val(serialized_col.data.data() + type_size * dst_row_idx,
+                             type_size);
+        Slice orig_val(row.cell_ptr(c), type_size);
+
+        if (col.type_info()->physical_type() == BINARY) {
+          orig_val = *reinterpret_cast<const Slice*>(orig_val.data());
+          serialized_val = *reinterpret_cast<const Slice*>(serialized_val.data());
+
+          uintptr_t indirect_offset = reinterpret_cast<uintptr_t>(serialized_val.data());
+          serialized_val = Slice(serialized_col.indirect_data->data() + indirect_offset,
+                                 serialized_val.size());
+        }
+
+        EXPECT_EQ(orig_val, serialized_val);
+      }
+      dst_row_idx++;
+    }
   }
 }
 
+
 // Create a block of rows in columnar layout and ensure that it can be
 // converted to and from protobuf.
 TEST_F(WireProtocolTest, TestColumnarRowBlockToPBWithPadding) {
@@ -427,15 +449,211 @@ TEST_F(WireProtocolTest, TestColumnarRowBlockToPBWithPadding) {
   }
 }
 
-TEST_P(WireProtocolTest, TestColumnarRowBlockToPBBenchmark) {
-  int column_count = std::get<0>(GetParam());
+struct RowwiseConverter {
+  static void Run(const RowBlock& block) {
+    faststring direct;
+    faststring indirect;
+    SerializeRowBlock(block, nullptr, &direct, &indirect);
+  }
+
+  static constexpr const char* kName = "row-wise";
+};
+
+
+struct ColumnarConverter {
+  static void Run(const RowBlock& block) {
+    ColumnarSerializedBatch batch;
+    SerializeRowBlockColumnar(block, nullptr, &batch);
+  }
+
+  static constexpr const char* kName = "columnar";
+};
+
+struct BenchmarkColumnsSpec {
+  struct Col {
+    DataType type;
+    double null_fraction; // negative for non-null
+  };
+  vector<Col> columns;
+  string name;
+};
+
+class WireProtocolBenchmark :
+      public WireProtocolTest,
+      public testing::WithParamInterface<tuple<BenchmarkColumnsSpec, double>> {
+ public:
+
+  void ResetBenchmarkSchema(const BenchmarkColumnsSpec& spec) {
+    vector<ColumnSchema> column_schemas;
+    int i = 0;
+    for (const auto& c : spec.columns) {
+      column_schemas.emplace_back(Substitute("col$0", i++),
+                                  c.type,
+                                  /*nullable=*/c.null_fraction >= 0);
+    }
+    CHECK_OK(benchmark_schema_.Reset(std::move(column_schemas), 0));
+  }
+
+  void FillRowBlockForBenchmark(const BenchmarkColumnsSpec& spec,
+                                RowBlock* block) {
+    Random rng(SeedRandom());
+
+    test_data_arena_.Reset();
+    for (int i = 0; i < block->nrows(); i++) {
+      RowBlockRow row = block->row(i);
+      for (int j = 0; j < benchmark_schema_.num_columns(); j++) {
+        const ColumnSchema& column_schema = benchmark_schema_.column(j);
+        DataType type = spec.columns[j].type;
+        bool is_null = rng.NextDoubleFraction() <= spec.columns[j].null_fraction;
+        if (column_schema.is_nullable()) {
+          row.cell(j).set_null(is_null);
+        }
+        if (!is_null) {
+          switch (type) {
+            case STRING: {
+              Slice col;
+              CHECK(test_data_arena_.RelocateSlice(Substitute("hello world $0",
+                                                              column_schema.name()), &col));
+              memcpy(row.mutable_cell_ptr(j), &col, sizeof(Slice));
+              break;
+            }
+            case INT128:
+              UnalignedStore<int128_t>(row.mutable_cell_ptr(j), i);
+              break;
+            case INT64:
+              UnalignedStore<int64_t>(row.mutable_cell_ptr(j), i);
+              break;
+            case INT32:
+              UnalignedStore<int32_t>(row.mutable_cell_ptr(j), i);
+              break;
+            case INT16:
+              UnalignedStore<int16_t>(row.mutable_cell_ptr(j), i);
+              break;
+            case INT8:
+              UnalignedStore<int8_t>(row.mutable_cell_ptr(j), i);
+              break;
+            default:
+              LOG(FATAL) << "Unexpected type: " << type;
+          }
+        }
+      }
+    }
+  }
+
+  static void SelectRandomRowsWithRate(RowBlock* block, double rate) {
+    CHECK_LE(rate, 1.0);
+    CHECK_GE(rate, 0.0);
+    auto select_count = block->nrows() * rate;
+    SelectionVector* select_vector = block->selection_vector();
+    if (rate == 1.0) {
+      select_vector->SetAllTrue();
+    } else if (rate == 0.0) {
+      select_vector->SetAllFalse();
+    } else {
+      vector<int> indexes(block->nrows());
+      std::iota(indexes.begin(), indexes.end(), 0);
+      std::random_shuffle(indexes.begin(), indexes.end());
+      indexes.resize(select_count);
+      select_vector->SetAllFalse();
+      for (auto index : indexes) {
+        select_vector->SetRowSelected(index);
+      }
+    }
+    CHECK_EQ(select_vector->CountSelected(), select_count);
+  }
+
+
+  // Use column_count to control the schema scale.
+  // Use select_rate to control the number of selected rows.
+  template<class Converter>
+  double RunBenchmark(const BenchmarkColumnsSpec& spec,
+                    double select_rate) {
+    ResetBenchmarkSchema(spec);
+    Arena arena(1024);
+    RowBlock block(&benchmark_schema_, 1000, &arena);
+    // Regardless of the config, use a constant number of selected cells for the test by
+    // looping the conversion an appropriate number of times.
+    const int64_t kNumCellsToConvert = AllowSlowTests() ? 100000000 : 1000000;
+    const int64_t kCellsPerBlock = block.nrows() * spec.columns.size();
+    const double kSelectedCellsPerBlock = kCellsPerBlock * select_rate;
+    const int kNumTrials = static_cast<int>(kNumCellsToConvert /  kSelectedCellsPerBlock);
+    FillRowBlockForBenchmark(spec, &block);
+    SelectRandomRowsWithRate(&block, select_rate);
+
+    int64_t cycle_start = CycleClock::Now();
+    for (int i = 0; i < kNumTrials; ++i) {
+      Converter::Run(block);
+    }
+    int64_t cycle_end = CycleClock::Now();
+    double cycles_per_cell = static_cast<double>(cycle_end - cycle_start) / kNumCellsToConvert;
+    LOG(INFO) << Substitute(
+        "Converting $0 to PB (method $3) row select rate $1: $2 cycles/cell",
+        spec.name, select_rate, cycles_per_cell,
+        Converter::kName);
+    return cycles_per_cell;
+  }
+
+ protected:
+  Schema benchmark_schema_;
+};
+
+TEST_P(WireProtocolBenchmark, TestRowBlockToPBBenchmark) {
+  const auto& spec = std::get<0>(GetParam());
   double select_rate = std::get<1>(GetParam());
-  RunBenchmark(column_count, select_rate);
+  double cycles_per_cell_rowwise = RunBenchmark<RowwiseConverter>(spec, select_rate);
+  double cycles_per_cell_columnar = RunBenchmark<ColumnarConverter>(spec, select_rate);
+  double ratio = cycles_per_cell_rowwise / cycles_per_cell_columnar;
+  LOG(INFO) << Substitute(
+      "Converting $0 to PB row select rate $1: columnar/rowwise throughput ratio: $2x",
+      spec.name, select_rate, ratio);
 }
 
-INSTANTIATE_TEST_CASE_P(ColumnarRowBlockToPBBenchmarkParams, WireProtocolTest,
-                        testing::Combine(testing::Values(3, 30, 300),
-                                         testing::Values(1.0, 0.8, 0.5, 0.2)));
+BenchmarkColumnsSpec UniformColumns(int n_cols, DataType type, double null_fraction) {
+  vector<BenchmarkColumnsSpec::Col> cols(n_cols);
+  for (int i = 0; i < n_cols; i++) {
+    cols[i] = {type, null_fraction};
+  }
+  string null_str;
+  if (null_fraction >= 0) {
+    null_str = Substitute("$0pct_null", static_cast<int>(null_fraction * 100));
+  } else {
+    null_str = "non_null";
+  }
+  return {cols, Substitute("$0_$1_$2",
+                           n_cols,
+                           GetTypeInfo(type)->name(),
+                           null_str) };
+}
+
+INSTANTIATE_TEST_CASE_P(
+    ColumnarRowBlockToPBBenchmarkParams, WireProtocolBenchmark,
+    testing::Combine(
+        testing::Values(
+            UniformColumns(10, INT64, -1),
+            UniformColumns(10, INT32, -1),
+            UniformColumns(10, STRING, -1),
+
+            UniformColumns(10, INT128, 0),
+            UniformColumns(10, INT64, 0),
+            UniformColumns(10, INT32, 0),
+            UniformColumns(10, INT16, 0),
+            UniformColumns(10, INT8, 0),
+            UniformColumns(10, STRING, 0),
+
+            UniformColumns(10, INT128, 0.1),
+            UniformColumns(10, INT64, 0.1),
+            UniformColumns(10, INT32, 0.1),
+            UniformColumns(10, INT16, 0.1),
+            UniformColumns(10, INT8, 0.1),
+            UniformColumns(10, STRING, 0.1)),
+        // Selection rates.
+        testing::Values(1.0, 0.8, 0.5, 0.2)),
+    [](const testing::TestParamInfo<WireProtocolBenchmark::ParamType>& info) {
+      return Substitute("$0_sel_$1pct",
+                        std::get<0>(info.param).name,
+                        static_cast<int>(std::get<1>(info.param)*100));
+    });
+
 
 // Test that trying to extract rows from an invalid block correctly returns
 // Corruption statuses.
diff --git a/src/kudu/util/faststring.h b/src/kudu/util/faststring.h
index 1357eab..c7215f7 100644
--- a/src/kudu/util/faststring.h
+++ b/src/kudu/util/faststring.h
@@ -106,6 +106,16 @@ class faststring {
     ASAN_UNPOISON_MEMORY_REGION(data_, len_);
   }
 
+  // Resize to 'newsize'. In contrast to 'resize()', if this requires allocating a new
+  // backing array, the new capacity is rounded up in the same manner as if data had been
+  // appended to the buffer.
+  void resize_with_extra_capacity(size_t newsize) {
+    if (newsize > capacity_) {
+      GrowToAtLeast(newsize);
+    }
+    resize(newsize);
+  }
+
   // Releases the underlying array; after this, the buffer is left empty.
   //
   // NOTE: the data pointer returned by release() always points to dynamically


[kudu] 07/07: KUDU-3093: another band-aid for this DebugUtilTest.TestSignalStackTrace

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 263c3aa894c087691ef2c4463d46a52a94f12c2b
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Mar 30 15:10:18 2020 -0700

    KUDU-3093: another band-aid for this DebugUtilTest.TestSignalStackTrace
    
    A TSAN build yielded a stack trace like:
    
    @           0x444a88  __tsan::ProcessPendingSignals()
    @           0x4541c1  __interceptor_pthread_mutex_trylock
    @     0x7fbca26124e1  kudu::Mutex::TryAcquire()
    @     0x7fbca2612893  kudu::Mutex::Acquire()
    @           0x4fe036  kudu::MutexLock::MutexLock()
    @           0x504abd  kudu::CountDownLatch::WaitUntil()
    @           0x504a5f  kudu::CountDownLatch::WaitFor()
    @           0x4f04a9  kudu::(anonymous namespace)::SleeperThread()
    ...
    
    Rather than find the synchronization primitive frame least likely to be
    inlined, let's take a more comprehensive approach and search for multiple
    candidate frames, including SleeperThread.
    
    I tested this locally in DEBUG, RELEASE, ASAN, and TSAN modes.
    
    Change-Id: Ia4ca0f48ba1d7ad4cea40b70af271d7948f78a57
    Reviewed-on: http://gerrit.cloudera.org:8080/15605
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/util/debug-util-test.cc | 17 ++++++++++++-----
 1 file changed, 12 insertions(+), 5 deletions(-)

diff --git a/src/kudu/util/debug-util-test.cc b/src/kudu/util/debug-util-test.cc
index 2227a19..c69dbbc 100644
--- a/src/kudu/util/debug-util-test.cc
+++ b/src/kudu/util/debug-util-test.cc
@@ -40,6 +40,7 @@
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/array_view.h"
 #include "kudu/util/countdown_latch.h"
@@ -53,6 +54,7 @@
 
 using std::string;
 using std::vector;
+using strings::Substitute;
 
 DECLARE_int32(test_timeout_after);
 DECLARE_int32(stress_cpu_threads);
@@ -121,11 +123,16 @@ TEST_F(DebugUtilTest, TestSignalStackTrace) {
   // to start up and actually call our function.
   //
   // Note: due to RELEASE build inlining, we need to make sure to pick a stack
-  // frame that isn't optimized away.
-  static constexpr const char* kTestThreadStackFrame =
+  // frame that isn't optimized away. Different compilers behave differently, so
+  // we'll use a regular expression to get maximal coverage.
+  static constexpr const char* kTestThreadOneStackFrame =
+      "SleeperThread";
+  static constexpr const char* kTestThreadAnotherStackFrame =
       "kudu::ConditionVariable::WaitUntil()";
+  static const string kStackFrameRegExp = Substitute(
+      "$0|$1", kTestThreadOneStackFrame, kTestThreadAnotherStackFrame);
   ASSERT_EVENTUALLY([&]() {
-    ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), kTestThreadStackFrame);
+    ASSERT_STR_MATCHES(DumpThreadStack(t->tid()), kStackFrameRegExp);
   });
 
   // Test that we can change the signal and that the stack traces still work,
@@ -140,7 +147,7 @@ TEST_F(DebugUtilTest, TestSignalStackTrace) {
   ASSERT_FALSE(IsSignalHandlerRegistered(SIGUSR2));
 
   // Stack traces should work using the new handler.
-  ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), kTestThreadStackFrame);
+  ASSERT_STR_MATCHES(DumpThreadStack(t->tid()), kStackFrameRegExp);
 
   // Switch back to SIGUSR2 and ensure it changes back.
   ASSERT_OK(SetStackTraceSignal(SIGUSR2));
@@ -148,7 +155,7 @@ TEST_F(DebugUtilTest, TestSignalStackTrace) {
   ASSERT_FALSE(IsSignalHandlerRegistered(SIGHUP));
 
   // Stack traces should work using the new handler.
-  ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), kTestThreadStackFrame);
+  ASSERT_STR_MATCHES(DumpThreadStack(t->tid()), kStackFrameRegExp);
 
   // Register our own signal handler on SIGHUP, and ensure that
   // we get a bad Status if we try to use it.