You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by la...@apache.org on 2019/07/31 02:58:25 UTC

[kudu] branch master updated: [common] add FindColumn for Schema

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

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


The following commit(s) were added to refs/heads/master by this push:
     new de49f8e  [common] add FindColumn for Schema
de49f8e is described below

commit de49f8e20638da4fc016609e9324445a5ce528c6
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Fri Jul 26 15:05:04 2019 +0800

    [common] add FindColumn for Schema
    
    There are several implementations of FindColumn,
    now add it as a member of Schema.
    
    Change-Id: Iede12b95b774754f914295cecd2f6797008fed46
    Reviewed-on: http://gerrit.cloudera.org:8080/13936
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/client/scan_batch.cc          | 16 +++-------------
 src/kudu/common/partial_row.cc         | 29 ++++++++++-------------------
 src/kudu/common/row_operations-test.cc | 21 +++++----------------
 src/kudu/common/schema-test.cc         | 15 +++++++++++++++
 src/kudu/common/schema.cc              | 10 ++++++++++
 src/kudu/common/schema.h               |  4 ++++
 src/kudu/master/master_service.cc      |  4 ++--
 7 files changed, 49 insertions(+), 50 deletions(-)

diff --git a/src/kudu/client/scan_batch.cc b/src/kudu/client/scan_batch.cc
index 8ecd347..8e59cee 100644
--- a/src/kudu/client/scan_batch.cc
+++ b/src/kudu/client/scan_batch.cc
@@ -27,7 +27,6 @@
 #include "kudu/common/common.pb.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/types.h"
-#include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/bitmap.h"
 #include "kudu/util/int128.h"
@@ -75,15 +74,6 @@ Slice KuduScanBatch::indirect_data() const {
 
 namespace {
 
-inline Status FindColumn(const Schema& schema, const Slice& col_name, int* idx) {
-  StringPiece sp(reinterpret_cast<const char*>(col_name.data()), col_name.size());
-  *idx = schema.find_column(sp);
-  if (PREDICT_FALSE(*idx == -1)) {
-    return Status::NotFound("No such column", col_name);
-  }
-  return Status::OK();
-}
-
 // Just enough of a "cell" to support the Schema::DebugCellAppend calls
 // made by KuduScanBatch::RowPtr::ToString.
 class RowCell {
@@ -118,7 +108,7 @@ bool KuduScanBatch::RowPtr::IsNull(int col_idx) const {
 
 bool KuduScanBatch::RowPtr::IsNull(const Slice& col_name) const {
   int col_idx;
-  CHECK_OK(FindColumn(*schema_, col_name, &col_idx));
+  CHECK_OK(schema_->FindColumn(col_name, &col_idx));
   return IsNull(col_idx);
 }
 
@@ -164,7 +154,7 @@ Status KuduScanBatch::RowPtr::GetDouble(const Slice& col_name, double* val) cons
 
 Status KuduScanBatch::RowPtr::GetUnscaledDecimal(const Slice& col_name, int128_t* val) const {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return GetUnscaledDecimal(col_idx, val);
 }
 
@@ -219,7 +209,7 @@ Status KuduScanBatch::RowPtr::GetBinary(int col_idx, Slice* val) const {
 template<typename T>
 Status KuduScanBatch::RowPtr::Get(const Slice& col_name, typename T::cpp_type* val) const {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return Get<T>(col_idx, val);
 }
 
diff --git a/src/kudu/common/partial_row.cc b/src/kudu/common/partial_row.cc
index 7209457..abb00dd 100644
--- a/src/kudu/common/partial_row.cc
+++ b/src/kudu/common/partial_row.cc
@@ -29,29 +29,20 @@
 #include "kudu/common/schema.h"
 #include "kudu/common/types.h"
 #include "kudu/gutil/port.h"
-#include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/bitmap.h"
 #include "kudu/util/decimal_util.h"
 #include "kudu/util/int128.h"
 #include "kudu/util/logging.h"
+#ifndef NDEBUG
 #include "kudu/util/memory/overwrite.h"
+#endif
 #include "kudu/util/status.h"
 
 using std::string;
 using strings::Substitute;
 
 namespace kudu {
-namespace {
-inline Status FindColumn(const Schema& schema, const Slice& col_name, int* idx) {
-  StringPiece sp(reinterpret_cast<const char*>(col_name.data()), col_name.size());
-  *idx = schema.find_column(sp);
-  if (PREDICT_FALSE(*idx == -1)) {
-    return Status::NotFound("No such column", col_name);
-  }
-  return Status::OK();
-}
-} // anonymous namespace
 
 KuduPartialRow::KuduPartialRow(const Schema* schema)
   : schema_(schema) {
@@ -118,7 +109,7 @@ Status KuduPartialRow::Set(const Slice& col_name,
                            const typename T::cpp_type& val,
                            bool owned) {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return Set<T>(col_idx, val, owned);
 }
 
@@ -274,7 +265,7 @@ Status KuduPartialRow::SetDouble(const Slice& col_name, double val) {
 }
 Status KuduPartialRow::SetUnscaledDecimal(const Slice& col_name, int128_t val) {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return SetUnscaledDecimal(col_idx, val);
 }
 Status KuduPartialRow::SetBool(int col_idx, bool val) {
@@ -398,7 +389,7 @@ Status KuduPartialRow::SetSliceCopy(int col_idx, const Slice& val) {
 
 Status KuduPartialRow::SetNull(const Slice& col_name) {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return SetNull(col_idx);
 }
 
@@ -420,7 +411,7 @@ Status KuduPartialRow::SetNull(int col_idx) {
 
 Status KuduPartialRow::Unset(const Slice& col_name) {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return Unset(col_idx);
 }
 
@@ -603,7 +594,7 @@ bool KuduPartialRow::IsColumnSet(int col_idx) const {
 
 bool KuduPartialRow::IsColumnSet(const Slice& col_name) const {
   int col_idx;
-  CHECK_OK(FindColumn(*schema_, col_name, &col_idx));
+  CHECK_OK(schema_->FindColumn(col_name, &col_idx));
   return IsColumnSet(col_idx);
 }
 
@@ -621,7 +612,7 @@ bool KuduPartialRow::IsNull(int col_idx) const {
 
 bool KuduPartialRow::IsNull(const Slice& col_name) const {
   int col_idx;
-  CHECK_OK(FindColumn(*schema_, col_name, &col_idx));
+  CHECK_OK(schema_->FindColumn(col_name, &col_idx));
   return IsNull(col_idx);
 }
 
@@ -656,7 +647,7 @@ Status KuduPartialRow::GetUnscaledDecimal(const Slice &col_name, int128_t *val)
 }
 Status KuduPartialRow::GetUnscaledDecimal(const Slice &col_name, int128_t *val) const {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return GetUnscaledDecimal(col_idx, val);
 }
 Status KuduPartialRow::GetString(const Slice& col_name, Slice* val) const {
@@ -730,7 +721,7 @@ template<typename T>
 Status KuduPartialRow::Get(const Slice& col_name,
                            typename T::cpp_type* val) const {
   int col_idx;
-  RETURN_NOT_OK(FindColumn(*schema_, col_name, &col_idx));
+  RETURN_NOT_OK(schema_->FindColumn(col_name, &col_idx));
   return Get<T>(col_idx, val);
 }
 
diff --git a/src/kudu/common/row_operations-test.cc b/src/kudu/common/row_operations-test.cc
index ccb08e6..6741fdb 100644
--- a/src/kudu/common/row_operations-test.cc
+++ b/src/kudu/common/row_operations-test.cc
@@ -36,8 +36,6 @@
 #include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/dynamic_annotations.h"
 #include "kudu/gutil/macros.h"
-#include "kudu/gutil/port.h"
-#include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/util/bitmap.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/slice.h"
@@ -373,15 +371,6 @@ string TestProjection(RowOperationsPB::Type type,
   return ops[0].ToString(server_schema);
 }
 
-Status FindColumn(const Schema& schema, const Slice& col_name, int* idx) {
-  StringPiece sp(reinterpret_cast<const char*>(col_name.data()), col_name.size());
-  *idx = schema.find_column(sp);
-  if (PREDICT_FALSE(*idx == -1)) {
-    return Status::NotFound("No such column", col_name);
-  }
-  return Status::OK();
-}
-
 } // anonymous namespace
 
 // Test decoding partial rows from a client who has a schema which matches
@@ -402,7 +391,7 @@ TEST_F(RowOperationsTest, ProjectionTestWholeSchemaSpecified) {
   // Force to set null on key column.
   {
     int col_idx;
-    ASSERT_OK(FindColumn(client_schema, "key", &col_idx));
+    ASSERT_OK(client_schema.FindColumn("key", &col_idx));
     KuduPartialRow client_row(&client_schema);
     ContiguousRow row(&client_schema, client_row.row_data_);
     row.set_null(col_idx, true);
@@ -609,7 +598,7 @@ TEST_F(RowOperationsTest, TestProjectUpdates) {
   {
     KuduPartialRow client_row(&client_schema);
     int col_idx;
-    ASSERT_OK(FindColumn(client_schema, "key", &col_idx));
+    ASSERT_OK(client_schema.FindColumn("key", &col_idx));
     ContiguousRow row(&client_schema, client_row.row_data_);
     row.set_null(col_idx, true);
     BitmapSet(client_row.isset_bitmap_, col_idx);
@@ -623,7 +612,7 @@ TEST_F(RowOperationsTest, TestProjectUpdates) {
     KuduPartialRow client_row(&client_schema);
     ASSERT_OK(client_row.SetInt32("key", 12345));
     int col_idx;
-    ASSERT_OK(FindColumn(client_schema, "int_val", &col_idx));
+    ASSERT_OK(client_schema.FindColumn("int_val", &col_idx));
     ContiguousRow row(&client_schema, client_row.row_data_);
     row.set_null(col_idx, true);
     BitmapSet(client_row.isset_bitmap_, col_idx);
@@ -728,7 +717,7 @@ TEST_F(RowOperationsTest, TestProjectDeletes) {
   {
     KuduPartialRow client_row(&client_schema);
     int col_idx;
-    ASSERT_OK(FindColumn(client_schema, "key", &col_idx));
+    ASSERT_OK(client_schema.FindColumn("key", &col_idx));
     ContiguousRow row(&client_schema, client_row.row_data_);
     row.set_null(col_idx, true);
     BitmapSet(client_row.isset_bitmap_, col_idx);
@@ -743,7 +732,7 @@ TEST_F(RowOperationsTest, TestProjectDeletes) {
     ASSERT_OK(client_row.SetInt32("key", 12345));
     ASSERT_OK(client_row.SetInt32("key_2", 12345));
     int col_idx;
-    ASSERT_OK(FindColumn(client_schema, "int_val", &col_idx));
+    ASSERT_OK(client_schema.FindColumn("int_val", &col_idx));
     ContiguousRow row(&client_schema, client_row.row_data_);
     row.set_null(col_idx, true);
     BitmapSet(client_row.isset_bitmap_, col_idx);
diff --git a/src/kudu/common/schema-test.cc b/src/kudu/common/schema-test.cc
index 16e4dcd..71bbbdc 100644
--- a/src/kudu/common/schema-test.cc
+++ b/src/kudu/common/schema-test.cc
@@ -657,6 +657,21 @@ TEST_F(TestSchema, TestCreateProjection) {
             partial_schema.ToString());
 }
 
+TEST_F(TestSchema, TestFindColumn) {
+  Schema schema({ ColumnSchema("col1", STRING),
+                  ColumnSchema("col2", INT32) },
+                1);
+
+  int col_idx;
+  ASSERT_OK(schema.FindColumn("col1", &col_idx));
+  ASSERT_EQ(0, col_idx);
+  ASSERT_OK(schema.FindColumn("col2", &col_idx));
+  ASSERT_EQ(1, col_idx);
+  Status s = schema.FindColumn("col3", &col_idx);
+  ASSERT_TRUE(s.IsNotFound());
+  ASSERT_EQ(s.ToString(), "Not found: No such column: col3");
+}
+
 #ifdef NDEBUG
 TEST(TestKeyEncoder, BenchmarkSimpleKey) {
   faststring fs;
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index 65659c3..74671bd 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -343,6 +343,16 @@ Status Schema::Reset(const vector<ColumnSchema>& cols,
   return Status::OK();
 }
 
+Status Schema::FindColumn(Slice col_name, int* idx) const {
+  DCHECK(idx);
+  StringPiece sp(reinterpret_cast<const char*>(col_name.data()), col_name.size());
+  *idx = find_column(sp);
+  if (PREDICT_FALSE(*idx == kColumnNotFound)) {
+    return Status::NotFound("No such column", col_name);
+  }
+  return Status::OK();
+}
+
 Status Schema::CreateProjectionByNames(const std::vector<StringPiece>& col_names,
                                        Schema* out) const {
   vector<ColumnId> ids;
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index 3b6c599..1916868 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -506,6 +506,10 @@ class Schema {
                const std::vector<ColumnId>& ids,
                int key_columns);
 
+  // Find the column index corresponding to the given column name,
+  // return a bad Status if not found.
+  Status FindColumn(Slice col_name, int* idx) const;
+
   // Return the number of bytes needed to represent a single row of this schema, without
   // accounting for the null bitmap if the Schema contains nullable values.
   //
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 8a03f45..7c5d916 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -252,8 +252,8 @@ void MasterServiceImpl::TSHeartbeat(const TSHeartbeatRequestPB* req,
   ts_desc->UpdateHeartbeatTime();
   ts_desc->set_num_live_replicas(req->num_live_tablets());
   ts_desc->set_num_live_replicas_by_dimension(
-      std::move(TabletNumByDimensionMap(req->num_live_tablets_by_dimension().begin(),
-                                        req->num_live_tablets_by_dimension().end())));
+      TabletNumByDimensionMap(req->num_live_tablets_by_dimension().begin(),
+                              req->num_live_tablets_by_dimension().end()));
 
   // 5. Only leaders handle tablet reports.
   if (is_leader_master && req->has_tablet_report()) {