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 2018/10/30 21:00:39 UTC

[1/2] kudu git commit: Stop using client-test-util in tools

Repository: kudu
Updated Branches:
  refs/heads/master 94e484da8 -> a2decad12


Stop using client-test-util in tools

The `kudu hms` and `kudu table` tools both had actions which used
functions from the client-test-util library. This meant test code was
being linked into production code, which is bad. The functions used by
the tool code were

ScanToStrings: Used in one place in tool code, so I just wrote the
KuduScanner iteration loop explicitly instead.

KuduSchemaFromSchema and SchemaFromKuduSchema: Replaced (both in tests
and tool code) by two static functions KuduSchema::FromSchema and
KuduSchema::ToSchema. These functions are not exported.

Change-Id: Ide54a0adce191a572960de678f0f7ee69eec8c2d
Reviewed-on: http://gerrit.cloudera.org:8080/11821
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: b022887306ca3815e12815f9d65ef0e2a7a13b1e
Parents: 94e484d
Author: Will Berkeley <wd...@gmail.org>
Authored: Mon Oct 29 15:00:52 2018 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Tue Oct 30 18:23:30 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test-util.cc             | 12 +-------
 src/kudu/client/client-test-util.h              | 11 +-------
 src/kudu/client/client-unittest.cc              |  3 +-
 src/kudu/client/client.cc                       |  2 +-
 src/kudu/client/scan_configuration.cc           |  4 +--
 src/kudu/client/schema.cc                       |  8 ++++++
 src/kudu/client/schema.h                        | 29 ++++++++++++++++----
 .../authn_token_expire-itest.cc                 |  3 +-
 .../catalog_manager_tsk-itest.cc                |  2 +-
 .../client-negotiation-failover-itest.cc        |  6 ++--
 .../integration-tests/create-table-itest.cc     | 10 +++----
 .../integration-tests/delete_table-itest.cc     |  6 ++--
 src/kudu/integration-tests/fuzz-itest.cc        |  2 +-
 .../integration-tests/security-faults-itest.cc  |  2 +-
 src/kudu/integration-tests/security-itest.cc    |  2 +-
 .../security-unknown-tsk-itest.cc               |  2 +-
 src/kudu/integration-tests/tablet_copy-itest.cc |  4 +--
 src/kudu/integration-tests/test_workload.cc     |  4 +--
 src/kudu/integration-tests/ts_itest-base.cc     |  3 +-
 src/kudu/tools/CMakeLists.txt                   |  1 -
 src/kudu/tools/kudu-admin-test.cc               |  3 +-
 src/kudu/tools/kudu-tool-test.cc                | 14 +++++-----
 src/kudu/tools/rebalancer_tool-test.cc          |  7 ++---
 src/kudu/tools/tool_action_hms.cc               | 11 ++++----
 src/kudu/tools/tool_action_table.cc             | 16 ++++++++---
 25 files changed, 84 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/client-test-util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test-util.cc b/src/kudu/client/client-test-util.cc
index 847a682..aa03f8a 100644
--- a/src/kudu/client/client-test-util.cc
+++ b/src/kudu/client/client-test-util.cc
@@ -17,8 +17,8 @@
 
 #include "kudu/client/client-test-util.h"
 
-#include <string>
 #include <ostream>
+#include <string>
 #include <vector>
 
 #include <glog/logging.h>
@@ -26,9 +26,7 @@
 
 #include "kudu/client/row_result.h"
 #include "kudu/client/scan_batch.h"
-#include "kudu/client/schema.h"
 #include "kudu/client/write_op.h"
-#include "kudu/common/schema.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
@@ -133,13 +131,5 @@ Status ScanToStrings(KuduScanner* scanner, vector<string>* row_strings) {
   return Status::OK();
 }
 
-KuduSchema KuduSchemaFromSchema(const Schema& schema) {
-  return KuduSchema(schema);
-}
-
-Schema SchemaFromKuduSchema(const KuduSchema& schema) {
-  return Schema(*schema.schema_);
-}
-
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/client-test-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test-util.h b/src/kudu/client/client-test-util.h
index 1939785..0d28481 100644
--- a/src/kudu/client/client-test-util.h
+++ b/src/kudu/client/client-test-util.h
@@ -22,16 +22,13 @@
 #include <string>
 #include <vector>
 
-#include "kudu/gutil/port.h"
 #include "kudu/client/client.h"
 #include "kudu/client/shared_ptr.h"
+#include "kudu/gutil/port.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
-class Schema;
-
 namespace client {
-class KuduSchema;
 
 // Log any pending errors in the given session, and then crash the current
 // process.
@@ -60,11 +57,5 @@ Status CountRowsWithRetries(KuduScanner* scanner, size_t* row_count);
 Status ScanToStrings(KuduScanner* scanner,
                      std::vector<std::string>* row_strings) WARN_UNUSED_RESULT;
 
-// Convert a kudu::Schema to a kudu::client::KuduSchema.
-KuduSchema KuduSchemaFromSchema(const Schema& schema);
-
-// Convert a kudu::client::KuduSchema to a kudu::Schema.
-Schema SchemaFromKuduSchema(const KuduSchema& schema);
-
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/client-unittest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-unittest.cc b/src/kudu/client/client-unittest.cc
index f1b2fea..bd8126a 100644
--- a/src/kudu/client/client-unittest.cc
+++ b/src/kudu/client/client-unittest.cc
@@ -28,7 +28,6 @@
 #include <gtest/gtest.h>
 
 #include "kudu/client/client-internal.h"
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/error_collector.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/value.h"
@@ -284,7 +283,7 @@ TEST(ClientUnitTest, TestKuduSchemaToStringWithColumnIds) {
   SchemaBuilder builder;
   builder.AddKeyColumn("key", DataType::INT32);
   const auto schema = builder.Build();
-  const auto kudu_schema = KuduSchemaFromSchema(schema);
+  const auto kudu_schema = KuduSchema::FromSchema(schema);
 
   // The string version of the KuduSchema should not have column ids, even
   // though the default string version of the underlying Schema should.

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 44f6b93..c475f49 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1395,7 +1395,7 @@ Status KuduScanner::SetCacheBlocks(bool cache_blocks) {
 }
 
 KuduSchema KuduScanner::GetProjectionSchema() const {
-  return KuduSchema(*data_->configuration().projection());
+  return KuduSchema::FromSchema(*data_->configuration().projection());
 }
 
 Status KuduScanner::SetRowFormatFlags(uint64_t flags) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/scan_configuration.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/scan_configuration.cc b/src/kudu/client/scan_configuration.cc
index 13cc9a5..700e64e 100644
--- a/src/kudu/client/scan_configuration.cc
+++ b/src/kudu/client/scan_configuration.cc
@@ -45,7 +45,7 @@ const int ScanConfiguration::kHtTimestampBitsToShift = 12;
 ScanConfiguration::ScanConfiguration(KuduTable* table)
     : table_(table),
       projection_(table->schema().schema_),
-      client_projection_(*table->schema().schema_),
+      client_projection_(KuduSchema::FromSchema(*table->schema().schema_)),
       has_batch_size_bytes_(false),
       batch_size_bytes_(0),
       selection_(KuduClient::CLOSEST_REPLICA),
@@ -88,7 +88,7 @@ Status ScanConfiguration::SetProjectedColumnIndexes(const vector<int>& col_index
   unique_ptr<Schema> s(new Schema());
   RETURN_NOT_OK(s->Reset(cols, 0));
   projection_ = pool_.Add(s.release());
-  client_projection_ = KuduSchema(*projection_);
+  client_projection_ = KuduSchema::FromSchema(*projection_);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index 085e161..f09fc10 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -742,5 +742,13 @@ string KuduSchema::ToString() const {
                  : "()";
 }
 
+KuduSchema KuduSchema::FromSchema(const Schema& schema) {
+  return KuduSchema(schema);
+}
+
+Schema KuduSchema::ToSchema(const KuduSchema& kudu_schema) {
+  return Schema(*kudu_schema.schema_);
+}
+
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/client/schema.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index 9347298..a243c42 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -574,6 +574,28 @@ class KUDU_EXPORT KuduSchema {
   /// @return A string describing this schema.
   std::string ToString() const;
 
+  /// @cond PRIVATE_API
+
+  /// Convert a Schema to a KuduSchema.
+  ///
+  /// Private API.
+  ///
+  /// @param[in] schema
+  ///   The Schema to convert
+  /// @return The converted KuduSchema
+  static KuduSchema FromSchema(const Schema& schema) KUDU_NO_EXPORT;
+
+  /// Convert a KuduSchema to a Schema.
+  ///
+  /// Private API.
+  ///
+  /// @param[in] kudu_schema
+  ///   The KuduSchema to convert
+  /// @return The converted Schema
+  static Schema ToSchema(const KuduSchema& kudu_schema) KUDU_NO_EXPORT;
+
+  /// @endcond
+
  private:
   friend class ClientTest;
   friend class KuduClient;
@@ -593,12 +615,7 @@ class KUDU_EXPORT KuduSchema {
   friend class tools::RemoteKsckCluster;
   friend class tools::ReplicaDumper;
 
-  /// @cond
-  friend KuduSchema KuduSchemaFromSchema(const Schema& schema);
-  friend Schema SchemaFromKuduSchema(const KuduSchema& schema);
-  /// @endcond
-
-  // For use by kudu tests.
+  // For use by KuduSchema::FromSchema.
   explicit KuduSchema(const Schema& schema);
 
   // Private since we don't want users to rely on the first N columns

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/authn_token_expire-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/authn_token_expire-itest.cc b/src/kudu/integration-tests/authn_token_expire-itest.cc
index bdfdb3f..fdbf5cd 100644
--- a/src/kudu/integration-tests/authn_token_expire-itest.cc
+++ b/src/kudu/integration-tests/authn_token_expire-itest.cc
@@ -25,7 +25,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -107,7 +106,7 @@ class AuthnTokenExpireITestBase : public KuduTest {
       : token_validity_seconds_(token_validity_seconds),
         num_masters_(num_masters),
         num_tablet_servers_(num_tablet_servers),
-        schema_(client::KuduSchemaFromSchema(CreateKeyValueTestSchema())) {
+        schema_(KuduSchema::FromSchema(CreateKeyValueTestSchema())) {
     cluster_opts_.num_tablet_servers = num_tablet_servers_;
     cluster_opts_.num_masters = num_masters_;
     cluster_opts_.enable_kerberos = true;

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
index 12169af..d3e7dab 100644
--- a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
+++ b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
@@ -120,7 +120,7 @@ class CatalogManagerTskITest : public KuduTest {
     ASSERT_OK(cluster_->CreateClient(&builder, &client));
 
     // Create a table.
-    KuduSchema schema = client::KuduSchemaFromSchema(CreateKeyValueTestSchema());
+    auto schema = KuduSchema::FromSchema(CreateKeyValueTestSchema());
     gscoped_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
 
     ASSERT_OK(table_creator->table_name(kTableName)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/client-negotiation-failover-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/client-negotiation-failover-itest.cc b/src/kudu/integration-tests/client-negotiation-failover-itest.cc
index 4fee884..d815502 100644
--- a/src/kudu/integration-tests/client-negotiation-failover-itest.cc
+++ b/src/kudu/integration-tests/client-negotiation-failover-itest.cc
@@ -22,12 +22,10 @@
 #include <thread>
 #include <vector>
 
-#include <glog/logging.h>
-
 #include <gflags/gflags_declare.h>
+#include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -138,7 +136,7 @@ TEST_F(ClientFailoverOnNegotiationTimeoutITest, Kudu1580ConnectToTServer) {
           .default_rpc_timeout(MonoDelta::FromMilliseconds(kTimeoutMs)),
       &client));
   unique_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
-  KuduSchema schema(client::KuduSchemaFromSchema(CreateKeyValueTestSchema()));
+  KuduSchema schema(KuduSchema::FromSchema(CreateKeyValueTestSchema()));
   ASSERT_OK(table_creator->table_name(kTableName)
       .schema(&schema)
       .add_hash_partitions({ "key" }, kNumTabletServers)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/create-table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc
index b7d4da8..eff9c4a 100644
--- a/src/kudu/integration-tests/create-table-itest.cc
+++ b/src/kudu/integration-tests/create-table-itest.cc
@@ -30,7 +30,6 @@
 #include <glog/stl_logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -66,6 +65,7 @@ METRIC_DECLARE_histogram(handler_latency_kudu_tserver_TabletServerAdminService_C
 
 namespace kudu {
 
+using client::KuduSchema;
 using cluster::ClusterNodes;
 
 const char* const kTableName = "test-table";
@@ -92,7 +92,7 @@ TEST_F(CreateTableITest, TestCreateWhenMajorityOfReplicasFailCreation) {
   // This won't succeed because we can't create enough replicas to get
   // a quorum.
   gscoped_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(GetSimpleTestSchema()));
+  auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema());
   ASSERT_OK(table_creator->table_name(kTableName)
             .schema(&client_schema)
             .set_range_partition_columns({ "key" })
@@ -157,7 +157,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenly) {
   NO_FATALS(StartCluster({}, {}, kNumServers));
 
   gscoped_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(GetSimpleTestSchema()));
+  auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema());
   ASSERT_OK(table_creator->table_name(kTableName)
             .schema(&client_schema)
             .set_range_partition_columns({ "key" })
@@ -220,7 +220,7 @@ static void LookUpRandomKeysLoop(std::shared_ptr<master::MasterServiceProxy> mas
                                  const char* table_name,
                                  AtomicBool* quit) {
   Schema schema(GetSimpleTestSchema());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(schema));
+  auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema());
   gscoped_ptr<KuduPartialRow> r(client_schema.NewRow());
 
   while (!quit->Load()) {
@@ -288,7 +288,7 @@ TEST_F(CreateTableITest, TestCreateTableWithDeadTServers) {
   cluster_->ShutdownNodes(ClusterNodes::TS_ONLY);
 
   Schema schema(GetSimpleTestSchema());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(schema));
+  auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema());
   gscoped_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
 
   // Don't bother waiting for table creation to finish; it'll never happen

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/delete_table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-itest.cc b/src/kudu/integration-tests/delete_table-itest.cc
index 83b2bff..b63cb4a 100644
--- a/src/kudu/integration-tests/delete_table-itest.cc
+++ b/src/kudu/integration-tests/delete_table-itest.cc
@@ -34,7 +34,6 @@
 #include <gtest/gtest.h>
 #include <rapidjson/document.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/scan_batch.h"
 #include "kudu/client/schema.h"
@@ -82,7 +81,6 @@ using kudu::client::KuduClient;
 using kudu::client::KuduScanner;
 using kudu::client::KuduScanBatch;
 using kudu::client::KuduSchema;
-using kudu::client::KuduSchemaFromSchema;
 using kudu::client::KuduTable;
 using kudu::client::KuduTableCreator;
 using kudu::cluster::ExternalTabletServer;
@@ -1122,7 +1120,7 @@ TEST_F(DeleteTableITest, TestUnknownTabletsAreNotDeleted) {
   NO_FATALS(StartCluster(extra_ts_flags, extra_master_flags, kNumTabletServers));
 
   Schema schema(GetSimpleTestSchema());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(schema));
+  client::KuduSchema client_schema(client::KuduSchema::FromSchema(schema));
   unique_ptr<KuduTableCreator> creator(client_->NewTableCreator());
   ASSERT_OK(creator->table_name("test")
       .schema(&client_schema)
@@ -1376,7 +1374,7 @@ TEST_P(DeleteTableTombstonedParamTest, TestTabletTombstone) {
   const int kNumTablets = 2;
   vector<const KuduPartialRow*> split_rows;
   Schema schema(GetSimpleTestSchema());
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(schema));
+  client::KuduSchema client_schema(client::KuduSchema::FromSchema(schema));
   KuduPartialRow* split_row = client_schema.NewRow();
   ASSERT_OK(split_row->SetInt32(0, numeric_limits<int32_t>::max() / kNumTablets));
   split_rows.push_back(split_row);

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/fuzz-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/fuzz-itest.cc b/src/kudu/integration-tests/fuzz-itest.cc
index d393bdf..e43d990 100644
--- a/src/kudu/integration-tests/fuzz-itest.cc
+++ b/src/kudu/integration-tests/fuzz-itest.cc
@@ -198,7 +198,7 @@ class FuzzTest : public KuduTest {
   }
 
   void CreateTabletAndStartClusterWithSchema(const Schema& schema) {
-    schema_ =  client::KuduSchemaFromSchema(schema);
+    schema_ =  KuduSchema::FromSchema(schema);
     KuduTest::SetUp();
 
     InternalMiniClusterOptions opts;

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/security-faults-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-faults-itest.cc b/src/kudu/integration-tests/security-faults-itest.cc
index dec000e..03b1aa4 100644
--- a/src/kudu/integration-tests/security-faults-itest.cc
+++ b/src/kudu/integration-tests/security-faults-itest.cc
@@ -122,7 +122,7 @@ class SecurityComponentsFaultsITest : public KuduTest {
     RETURN_NOT_OK(cluster_->CreateClient(nullptr, &client));
 
     // Create a table.
-    KuduSchema schema = client::KuduSchemaFromSchema(CreateKeyValueTestSchema());
+    KuduSchema schema = KuduSchema::FromSchema(CreateKeyValueTestSchema());
     gscoped_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
 
     RETURN_NOT_OK(table_creator->table_name(kTableName)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index e3128b4..8b2134c 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -146,7 +146,7 @@ void SecurityITest::SmokeTestCluster() {
   ASSERT_OK(cluster_->CreateClient(nullptr, &client));
 
   // Create a table.
-  KuduSchema schema = client::KuduSchemaFromSchema(CreateKeyValueTestSchema());
+  KuduSchema schema = client::KuduSchema::FromSchema(CreateKeyValueTestSchema());
   gscoped_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
   ASSERT_OK(table_creator->table_name(kTableName)
             .set_range_partition_columns({ "key" })

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/security-unknown-tsk-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-unknown-tsk-itest.cc b/src/kudu/integration-tests/security-unknown-tsk-itest.cc
index 4bc7f44..4db79ad 100644
--- a/src/kudu/integration-tests/security-unknown-tsk-itest.cc
+++ b/src/kudu/integration-tests/security-unknown-tsk-itest.cc
@@ -94,7 +94,7 @@ class SecurityUnknownTskTest : public KuduTest {
   SecurityUnknownTskTest()
       : num_tablet_servers_(3),
         heartbeat_interval_ms_(100),
-        schema_(client::KuduSchemaFromSchema(CreateKeyValueTestSchema())) {
+        schema_(KuduSchema::FromSchema(CreateKeyValueTestSchema())) {
 
     // Make the ts->master heartbeat interval shorter to run the test faster.
     FLAGS_heartbeat_interval_ms = heartbeat_interval_ms_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/tablet_copy-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc
index 5b60b06..135a474 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -32,7 +32,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -93,7 +92,6 @@ DEFINE_int32(test_delete_leader_num_writer_threads, 1,
              "Number of writer threads in TestDeleteLeaderDuringTabletCopyStressTest.");
 
 using kudu::client::KuduSchema;
-using kudu::client::KuduSchemaFromSchema;
 using kudu::client::KuduTableCreator;
 using kudu::cluster::ExternalMiniClusterOptions;
 using kudu::cluster::ExternalTabletServer;
@@ -633,7 +631,7 @@ TEST_F(TabletCopyITest, TestConcurrentTabletCopys) {
   // Create a table with several tablets. These will all be simultaneously
   // copied to a single target node from the same leader host.
   const int kNumTablets = 10;
-  KuduSchema client_schema(KuduSchemaFromSchema(GetSimpleTestSchema()));
+  KuduSchema client_schema(KuduSchema::FromSchema(GetSimpleTestSchema()));
   vector<const KuduPartialRow*> splits;
   for (int i = 0; i < kNumTablets - 1; i++) {
     KuduPartialRow* row = client_schema.NewRow();

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/test_workload.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/test_workload.cc b/src/kudu/integration-tests/test_workload.cc
index 1c15db1..ff555b5 100644
--- a/src/kudu/integration-tests/test_workload.cc
+++ b/src/kudu/integration-tests/test_workload.cc
@@ -23,7 +23,6 @@
 
 #include <glog/logging.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/scan_batch.h"
 #include "kudu/client/schema.h"
@@ -48,7 +47,6 @@ using client::KuduInsert;
 using client::KuduScanBatch;
 using client::KuduScanner;
 using client::KuduSchema;
-using client::KuduSchemaFromSchema;
 using client::KuduSession;
 using client::KuduTable;
 using client::KuduTableCreator;
@@ -72,7 +70,7 @@ TestWorkload::TestWorkload(MiniCluster* cluster)
     not_found_allowed_(false),
     network_error_allowed_(false),
     remote_error_allowed_(false),
-    schema_(KuduSchemaFromSchema(GetSimpleTestSchema())),
+    schema_(KuduSchema::FromSchema(GetSimpleTestSchema())),
     num_replicas_(3),
     num_tablets_(1),
     table_name_(kDefaultTableName),

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/integration-tests/ts_itest-base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_itest-base.cc b/src/kudu/integration-tests/ts_itest-base.cc
index 09449b7..8b5c5af 100644
--- a/src/kudu/integration-tests/ts_itest-base.cc
+++ b/src/kudu/integration-tests/ts_itest-base.cc
@@ -32,7 +32,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/common/wire_protocol.pb.h"
@@ -526,7 +525,7 @@ void TabletServerIntegrationTestBase::CreateClient(shared_ptr<client::KuduClient
 void TabletServerIntegrationTestBase::CreateTable(const string& table_id) {
   // The tests here make extensive use of server schemas, but we need
   // a client schema to create the table.
-  client::KuduSchema client_schema(client::KuduSchemaFromSchema(schema_));
+  client::KuduSchema client_schema(client::KuduSchema::FromSchema(schema_));
   gscoped_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
   ASSERT_OK(table_creator->table_name(table_id)
            .schema(&client_schema)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/tools/CMakeLists.txt b/src/kudu/tools/CMakeLists.txt
index 6e3ad21..c93439a 100644
--- a/src/kudu/tools/CMakeLists.txt
+++ b/src/kudu/tools/CMakeLists.txt
@@ -50,7 +50,6 @@ target_link_libraries(kudu_tools_util
   consensus
   gutil
   kudu_client
-  kudu_client_test_util
   kudu_common
   kudu_fs
   kudu_util

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/kudu-admin-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 3293fca..fb4bfbb 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -35,7 +35,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -1649,7 +1648,7 @@ TEST_F(AdminCliTest, TestListTablesDetail) {
 
   // Add another table to test multiple tables output.
   const string kAnotherTableId = "TestAnotherTable";
-  KuduSchema client_schema(client::KuduSchemaFromSchema(schema_));
+  auto client_schema = KuduSchema::FromSchema(schema_);
   gscoped_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
   ASSERT_OK(table_creator->table_name(kAnotherTableId)
            .schema(&client_schema)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index cde6559..659c1ee 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1476,7 +1476,7 @@ void ToolTest::RunLoadgen(int num_tservers,
 
     shared_ptr<KuduClient> client;
     ASSERT_OK(cluster_->CreateClient(nullptr, &client));
-    KuduSchema client_schema(client::KuduSchemaFromSchema(kSchema));
+    auto client_schema = KuduSchema::FromSchema(kSchema);
     unique_ptr<client::KuduTableCreator> table_creator(
         client->NewTableCreator());
     ASSERT_OK(table_creator->table_name(table_name)
@@ -2480,7 +2480,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(kudu_client->OpenTable("default.control", &control));
   ASSERT_OK(hms_catalog.CreateTable(
         control->id(), control->name(), kUsername,
-        client::SchemaFromKuduSchema(control->schema())));
+        KuduSchema::ToSchema(control->schema())));
 
   // Test case: Upper-case names are handled specially in a few places.
   shared_ptr<KuduTable> test_uppercase;
@@ -2488,7 +2488,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(kudu_client->OpenTable("default.UPPERCASE", &test_uppercase));
   ASSERT_OK(hms_catalog.CreateTable(
         test_uppercase->id(), test_uppercase->name(), kUsername,
-        client::SchemaFromKuduSchema(test_uppercase->schema())));
+        KuduSchema::ToSchema(test_uppercase->schema())));
 
   // Test case: inconsistent schema.
   shared_ptr<KuduTable> inconsistent_schema;
@@ -2504,7 +2504,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(kudu_client->OpenTable("default.inconsistent_name", &inconsistent_name));
   ASSERT_OK(hms_catalog.CreateTable(
         inconsistent_name->id(), "default.inconsistent_name_hms", kUsername,
-        client::SchemaFromKuduSchema(inconsistent_name->schema())));
+        KuduSchema::ToSchema(inconsistent_name->schema())));
 
   // Test case: inconsistent master addresses.
   shared_ptr<KuduTable> inconsistent_master_addrs;
@@ -2515,7 +2515,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(invalid_hms_catalog.Start());
   ASSERT_OK(invalid_hms_catalog.CreateTable(
         inconsistent_master_addrs->id(), inconsistent_master_addrs->name(), kUsername,
-        client::SchemaFromKuduSchema(inconsistent_master_addrs->schema())));
+        KuduSchema::ToSchema(inconsistent_master_addrs->schema())));
 
   // Test cases: orphan tables in the HMS.
   ASSERT_OK(hms_catalog.CreateTable(
@@ -2752,10 +2752,10 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   ASSERT_OK(kudu_client->OpenTable("default.duplicate_hms_tables", &duplicate_hms_tables));
   ASSERT_OK(hms_catalog.CreateTable(
         duplicate_hms_tables->id(), "default.duplicate_hms_tables", kUsername,
-        client::SchemaFromKuduSchema(duplicate_hms_tables->schema())));
+        KuduSchema::ToSchema(duplicate_hms_tables->schema())));
   ASSERT_OK(hms_catalog.CreateTable(
         duplicate_hms_tables->id(), "default.duplicate_hms_tables_2", kUsername,
-        client::SchemaFromKuduSchema(duplicate_hms_tables->schema())));
+        KuduSchema::ToSchema(duplicate_hms_tables->schema())));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "duplicate_hms_tables_3",
         "default.duplicate_hms_tables",
         master_addr, HmsClient::kExternalTable, kUsername));

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/rebalancer_tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/rebalancer_tool-test.cc b/src/kudu/tools/rebalancer_tool-test.cc
index 3e6c8d1..10fd9dd 100644
--- a/src/kudu/tools/rebalancer_tool-test.cc
+++ b/src/kudu/tools/rebalancer_tool-test.cc
@@ -31,7 +31,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -224,7 +223,7 @@ static Status CreateUnbalancedTables(
 
   // Create tables with their tablet replicas landing only on the tablet servers
   // which are up and running.
-  KuduSchema client_schema(client::KuduSchemaFromSchema(table_schema));
+  auto client_schema = KuduSchema::FromSchema(table_schema);
   for (auto i = 0; i < num_tables; ++i) {
     const string table_name = Substitute(table_name_pattern, i);
     unique_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
@@ -472,7 +471,7 @@ TEST_P(DDLDuringRebalancingTest, TablesCreatedAndDeletedDuringRebalancing) {
   CountDownLatch run_latch(1);
 
   thread creator([&]() {
-    KuduSchema client_schema(client::KuduSchemaFromSchema(schema_));
+    auto client_schema = KuduSchema::FromSchema(schema_);
     for (auto idx = 0; ; ++idx) {
       if (run_latch.WaitFor(MonoDelta::FromMilliseconds(500))) {
         break;
@@ -1091,7 +1090,7 @@ TEST_P(RebalancerAndSingleReplicaTablets, SingleReplicasStayOrMove) {
 
   // Create few tables with their tablet replicas landing only on those
   // (kRepFactor + 1) running tablet servers.
-  KuduSchema client_schema(client::KuduSchemaFromSchema(schema_));
+  auto client_schema = KuduSchema::FromSchema(schema_);
   for (auto i = 0; i < kNumTables; ++i) {
     const string table_name = Substitute(table_name_pattern, i);
     unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/tool_action_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_hms.cc b/src/kudu/tools/tool_action_hms.cc
index ac963d6..be04332 100644
--- a/src/kudu/tools/tool_action_hms.cc
+++ b/src/kudu/tools/tool_action_hms.cc
@@ -29,8 +29,8 @@
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
+#include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
 #include "kudu/common/schema.h"
 #include "kudu/gutil/map-util.h"
@@ -74,6 +74,7 @@ namespace tools {
 
 using client::KuduClient;
 using client::KuduClientBuilder;
+using client::KuduSchema;
 using client::KuduTable;
 using client::KuduTableAlterer;
 using client::sp::shared_ptr;
@@ -163,7 +164,7 @@ Status HmsDowngrade(const RunnerContext& context) {
 bool IsSynced(const string& master_addresses,
               const KuduTable& kudu_table,
               const hive::Table& hms_table) {
-  Schema schema(client::SchemaFromKuduSchema(kudu_table.schema()));
+  auto schema = KuduSchema::ToSchema(kudu_table.schema());
   hive::Table hms_table_copy(hms_table);
   Status s = HmsCatalog::PopulateTable(kudu_table.id(), kudu_table.name(), boost::none,
                                        schema, master_addresses, &hms_table_copy);
@@ -506,7 +507,7 @@ Status FixHmsMetadata(const RunnerContext& context) {
     for (const auto& kudu_table : report.missing_hms_tables) {
       const string& table_id = kudu_table->id();
       const string& table_name = kudu_table->name();
-      Schema schema = client::SchemaFromKuduSchema(kudu_table->schema());
+      auto schema = KuduSchema::ToSchema(kudu_table->schema());
       string normalized_table_name(table_name.data(), table_name.size());
       CHECK_OK(hms::HmsCatalog::NormalizeTableName(&normalized_table_name));
 
@@ -568,7 +569,7 @@ Status FixHmsMetadata(const RunnerContext& context) {
       } else {
         RETURN_NOT_OK_PREPEND(hms_catalog->UpgradeLegacyImpalaTable(
                   kudu_table.id(), hms_table.dbName, hms_table.tableName,
-                  client::SchemaFromKuduSchema(kudu_table.schema())),
+                  KuduSchema::ToSchema(kudu_table.schema())),
             Substitute("failed to upgrade legacy Impala HMS metadata for table $0",
               hms_table_name));
       }
@@ -635,7 +636,7 @@ Status FixHmsMetadata(const RunnerContext& context) {
         LOG(INFO) << "[dryrun] Refreshing HMS table metadata for Kudu table "
                   << TableIdent(kudu_table);
       } else {
-        Schema schema(client::SchemaFromKuduSchema(kudu_table.schema()));
+        auto schema = KuduSchema::ToSchema(kudu_table.schema());
         RETURN_NOT_OK_PREPEND(
             hms_catalog->AlterTable(kudu_table.id(), hms_table_name, hms_table_name, schema),
             Substitute("failed to refresh HMS table metadata for Kudu table $0",

http://git-wip-us.apache.org/repos/asf/kudu/blob/b0228873/src/kudu/tools/tool_action_table.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 097b9b0..0519c88 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -27,9 +27,9 @@
 #include <gflags/gflags_declare.h>
 #include <rapidjson/document.h>
 
-#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/replica_controller-internal.h"
+#include "kudu/client/scan_batch.h"
 #include "kudu/client/scan_predicate.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
@@ -68,6 +68,7 @@ using client::KuduPredicate;
 using client::KuduScanner;
 using client::KuduScanToken;
 using client::KuduScanTokenBuilder;
+using client::KuduSchema;
 using client::KuduTable;
 using client::KuduTableAlterer;
 using client::internal::ReplicaController;
@@ -159,14 +160,14 @@ Status DescribeTable(const RunnerContext& context) {
   RETURN_NOT_OK(client->OpenTable(table_name, &table));
 
   // The schema.
-  const client::KuduSchema& schema = table->schema();
+  const KuduSchema& schema = table->schema();
   cout << "TABLE " << table_name << " " << schema.ToString() << endl;
 
   // The partition schema with current range partitions.
   vector<Partition> partitions;
   RETURN_NOT_OK_PREPEND(table->ListPartitions(&partitions),
                         "failed to retrieve current partitions");
-  const auto& schema_internal = client::SchemaFromKuduSchema(schema);
+  const auto& schema_internal = KuduSchema::ToSchema(schema);
   const auto& partition_schema = table->partition_schema();
   vector<string> partition_strs;
   for (const auto& partition : partitions) {
@@ -336,8 +337,15 @@ Status LocateRow(const RunnerContext& context) {
     KuduScanner* scanner_ptr;
     RETURN_NOT_OK(tokens[0]->IntoKuduScanner(&scanner_ptr));
     unique_ptr<KuduScanner> scanner(scanner_ptr);
+    RETURN_NOT_OK(scanner->Open());
     vector<string> row_str;
-    RETURN_NOT_OK(ScanToStrings(scanner.get(), &row_str));
+    client::KuduScanBatch batch;
+    while (scanner->HasMoreRows()) {
+      RETURN_NOT_OK(scanner->NextBatch(&batch));
+      for (const auto& row : batch) {
+        row_str.emplace_back(row.ToString());
+      }
+    }
     if (row_str.empty()) {
       return Status::NotFound("row does not exist");
     }


[2/2] kudu git commit: mt-rowset_delta_compaction-test: remove dead code

Posted by ad...@apache.org.
mt-rowset_delta_compaction-test: remove dead code

This should have been removed in commit b6306163b.

Change-Id: I9ce38fc2555f05ab5401fc56d4c5dca20258d436
Reviewed-on: http://gerrit.cloudera.org:8080/11824
Tested-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: a2decad12bf84c91caf26d1963b99b3067dbd132
Parents: b022887
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Oct 5 16:25:19 2018 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Tue Oct 30 21:00:21 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/mt-rowset_delta_compaction-test.cc | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a2decad1/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/mt-rowset_delta_compaction-test.cc b/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
index 28e827e..a9ff5f9 100644
--- a/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
+++ b/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
@@ -157,9 +157,6 @@ class TestMultiThreadedRowSetDeltaCompaction : public TestRowSet {
     for (const auto& thread : compaction_threads_) {
       ASSERT_OK(ThreadJoiner(thread.get()).Join());
     }
-    for (const auto& thread : alter_schema_threads_) {
-      ASSERT_OK(ThreadJoiner(thread.get()).Join());
-    }
   }
 
   void WriteTestRowSetWithZeros() {
@@ -197,7 +194,6 @@ class TestMultiThreadedRowSetDeltaCompaction : public TestRowSet {
   vector<scoped_refptr<kudu::Thread> > update_threads_;
   vector<scoped_refptr<kudu::Thread> > flush_threads_;
   vector<scoped_refptr<kudu::Thread> > compaction_threads_;
-  vector<scoped_refptr<kudu::Thread> > alter_schema_threads_;
 };
 
 static void SetupFlagsForSlowTests() {