You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2019/03/12 12:45:02 UTC

[hbase] 14/133: HBASE-15718 Add on TableName implementation and tests

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

zghao pushed a commit to branch HBASE-14850
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 937c5774e61e88d2bba3278aaea03d8e0390d17a
Author: Elliott Clark <ec...@apache.org>
AuthorDate: Tue Apr 26 14:51:06 2016 -0700

    HBASE-15718 Add on TableName implementation and tests
    
    Summary:
    Table name will be needed to look up a row in meta. So this
    patch adds on the implementation for creating that lookup
    row. It also adds tests for TableName
    
    Test Plan: Added on a unit test
    
    Differential Revision: https://reviews.facebook.net/D57285
---
 hbase-native-client/connection/client-handler.cc   |  1 -
 hbase-native-client/core/BUCK                      | 69 ++++++++++++----------
 .../core/{table-name.cc => meta-utils.cc}          | 12 ++++
 .../core/{table-name.h => meta-utils.h}            |  8 +--
 hbase-native-client/core/simple-client.cc          |  4 +-
 hbase-native-client/core/table-name-test.cc        | 50 ++++++++++++++++
 hbase-native-client/core/table-name.cc             | 15 +++++
 hbase-native-client/core/table-name.h              | 25 +++++++-
 hbase-native-client/if/Master.proto                | 13 ++++
 hbase-native-client/if/ZooKeeper.proto             | 10 +++-
 10 files changed, 165 insertions(+), 42 deletions(-)

diff --git a/hbase-native-client/connection/client-handler.cc b/hbase-native-client/connection/client-handler.cc
index 7b7794d..205993a7 100644
--- a/hbase-native-client/connection/client-handler.cc
+++ b/hbase-native-client/connection/client-handler.cc
@@ -60,7 +60,6 @@ void ClientHandler::read(Context *ctx, std::unique_ptr<IOBuf> buf) {
   }
 }
 
-
 // TODO(eclark): Figure out how to handle the
 // network errors that are going to come.
 Future<Unit> ClientHandler::write(Context *ctx, Request r) {
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 195fc5c..9db6fda 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -16,44 +16,49 @@
 # limitations under the License.
 
 # This is the main library.
-cxx_library(name="core",
-            exported_headers=[
-                "client.h",
-                "connection.h",
-                "get-request.h",
-                "get-result.h",
-                "hbase_macros.h",
-                "location-cache.h",
-                "table-name.h",
-            ],
-            srcs=[
-                "client.cc",
-                "get-request.cc",
-                "get-result.cc",
-                "location-cache.cc",
-                "table-name.cc",
-            ],
-            deps=[
-                "//connection:connection",
-                "//if:if",
-                "//serde:serde",
-                "//third-party:folly",
-                "//third-party:wangle",
-                "//third-party:zookeeper_mt",
-            ],
-            visibility=[
-                'PUBLIC',
-            ], )
-
+cxx_library(
+    name="core",
+    exported_headers=[
+        "client.h",
+        "connection.h",
+        "get-request.h",
+        "get-result.h",
+        "hbase_macros.h",
+        "location-cache.h",
+        "table-name.h",
+        # TODO: move this out of exported
+        # Once meta lookup works
+        "meta-utils.h",
+    ],
+    srcs=[
+        "client.cc",
+        "get-request.cc",
+        "get-result.cc",
+        "location-cache.cc",
+        "meta-utils.cc",
+        "table-name.cc",
+    ],
+    deps=[
+        "//connection:connection",
+        "//if:if",
+        "//serde:serde",
+        "//third-party:folly",
+        "//third-party:wangle",
+        "//third-party:zookeeper_mt",
+    ],
+    visibility=[
+        'PUBLIC',
+    ], )
 cxx_test(name="location-cache-test",
          srcs=[
              "test-env.cc",
              "location-cache-test.cc",
          ],
-         deps=[
-             ":core",
-         ],
+         deps=[":core", ],
          run_test_separately=True, )
+cxx_test(name="table-name-test",
+         srcs=["table-name-test.cc", ],
+         deps=[":core", ], )
 cxx_binary(name="simple-client",
            srcs=["simple-client.cc", ],
            deps=[":core", "//connection:connection"], )
diff --git a/hbase-native-client/core/table-name.cc b/hbase-native-client/core/meta-utils.cc
similarity index 74%
copy from hbase-native-client/core/table-name.cc
copy to hbase-native-client/core/meta-utils.cc
index ffaaed0..7c7d45f 100644
--- a/hbase-native-client/core/table-name.cc
+++ b/hbase-native-client/core/meta-utils.cc
@@ -16,4 +16,16 @@
  * limitations under the License.
  *
  */
+
+#include "core/meta-utils.h"
+
+#include <folly/Conv.h>
+
 #include "core/table-name.h"
+
+using namespace hbase;
+
+std::string MetaUtil::region_lookup_rowkey(const TableName &tn,
+                                           const std::string &row) {
+  return folly::to<std::string>(tn, ",", row, ",", "999999999999999999");
+}
diff --git a/hbase-native-client/core/table-name.h b/hbase-native-client/core/meta-utils.h
similarity index 81%
copy from hbase-native-client/core/table-name.h
copy to hbase-native-client/core/meta-utils.h
index 37c3461..391d0ea 100644
--- a/hbase-native-client/core/table-name.h
+++ b/hbase-native-client/core/meta-utils.h
@@ -18,15 +18,13 @@
  */
 #pragma once
 
-#include <memory>
 #include <string>
 
 namespace hbase {
+class TableName;
 
-// This is the core class of a HBase client.
-class TableName {
+class MetaUtil {
 public:
-  explicit TableName(std::string tableName);
-  explicit TableName(std::string namespaceName, std::string tableName);
+  std::string region_lookup_rowkey(const TableName &tn, const std::string &row);
 };
 } // namespace hbase
diff --git a/hbase-native-client/core/simple-client.cc b/hbase-native-client/core/simple-client.cc
index 5e6f82e..8b2fae5 100644
--- a/hbase-native-client/core/simple-client.cc
+++ b/hbase-native-client/core/simple-client.cc
@@ -69,10 +69,12 @@ int main(int argc, char *argv[]) {
   msg->mutable_region()->set_type(
       RegionSpecifier_RegionSpecifierType::
           RegionSpecifier_RegionSpecifierType_ENCODED_REGION_NAME);
+
   // What row.
   msg->mutable_get()->set_row(FLAGS_row);
+
   // Send it.
-  r.set_msg(std::move(msg));
+  r.set_msg(msg);
   auto resp = (*conn)(r).get(milliseconds(5000));
 
   auto get_resp = std::static_pointer_cast<GetResponse>(resp.response());
diff --git a/hbase-native-client/core/table-name-test.cc b/hbase-native-client/core/table-name-test.cc
new file mode 100644
index 0000000..8aff375
--- /dev/null
+++ b/hbase-native-client/core/table-name-test.cc
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#include <folly/Conv.h>
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include "core/table-name.h"
+
+using namespace hbase;
+
+TEST(TestTableName, TestToStringNoDefault) {
+  TableName tn{"TestTableName"};
+  std::string result = folly::to<std::string>(tn);
+  ASSERT_EQ(result.find("default"), std::string::npos);
+  ASSERT_EQ("TestTableName", result);
+}
+TEST(TestTableName, TestToStringIncludeNS) {
+  TableName tn{"hbase", "acl"};
+  std::string result = folly::to<std::string>(tn);
+  ASSERT_EQ(result.find("hbase"), 0);
+  ASSERT_EQ("hbase:acl", result);
+}
+TEST(TestTableName, TestIsDefault) {
+  TableName default_t1{"in_default"};
+  ASSERT_TRUE(default_t1.is_default_name_space());
+
+  TableName default_t2{"default", "in_also"};
+  ASSERT_TRUE(default_t2.is_default_name_space());
+
+  TableName non_default{"testing", "hmm"};
+  ASSERT_FALSE(non_default.is_default_name_space());
+}
diff --git a/hbase-native-client/core/table-name.cc b/hbase-native-client/core/table-name.cc
index ffaaed0..33087fd 100644
--- a/hbase-native-client/core/table-name.cc
+++ b/hbase-native-client/core/table-name.cc
@@ -17,3 +17,18 @@
  *
  */
 #include "core/table-name.h"
+
+#include <folly/Conv.h>
+
+using namespace hbase;
+
+TableName::TableName(std::string table_name)
+    : name_space_("default"), table_(table_name) {}
+TableName::TableName(std::string name_space, std::string table_name)
+    : name_space_(name_space), table_(table_name) {}
+bool TableName::is_default_name_space() const {
+  return name_space_.length() == 0 || name_space_ == "default";
+}
+bool TableName::operator==(const TableName &other) const {
+  return name_space_ == other.name_space_ && table_ == other.table_;
+}
diff --git a/hbase-native-client/core/table-name.h b/hbase-native-client/core/table-name.h
index 37c3461..816f7da 100644
--- a/hbase-native-client/core/table-name.h
+++ b/hbase-native-client/core/table-name.h
@@ -21,12 +21,33 @@
 #include <memory>
 #include <string>
 
+#include <folly/Conv.h>
+
 namespace hbase {
 
 // This is the core class of a HBase client.
 class TableName {
 public:
-  explicit TableName(std::string tableName);
-  explicit TableName(std::string namespaceName, std::string tableName);
+  explicit TableName(std::string table_name);
+  explicit TableName(std::string name_space, std::string table_name);
+
+  std::string name_space() const { return name_space_; };
+  std::string table() const { return table_; };
+  bool is_default_name_space() const;
+  bool operator==(const TableName &other) const;
+
+private:
+  std::string name_space_;
+  std::string table_;
 };
+
+// Provide folly::to<std::string>(TableName);
+template <class String> void toAppend(const TableName &in, String *result) {
+  if (in.is_default_name_space()) {
+    folly::toAppend(in.table(), result);
+  } else {
+    folly::toAppend(in.name_space(), ':', in.table(), result);
+  }
+}
+
 } // namespace hbase
diff --git a/hbase-native-client/if/Master.proto b/hbase-native-client/if/Master.proto
index 1f7a3b7..ad8111e 100644
--- a/hbase-native-client/if/Master.proto
+++ b/hbase-native-client/if/Master.proto
@@ -288,6 +288,7 @@ message SetSplitOrMergeEnabledRequest {
   required bool enabled = 1;
   optional bool synchronous = 2;
   repeated MasterSwitchType switch_types = 3;
+  optional bool skip_lock = 4;
 }
 
 message SetSplitOrMergeEnabledResponse {
@@ -302,6 +303,12 @@ message IsSplitOrMergeEnabledResponse {
   required bool enabled = 1;
 }
 
+message ReleaseSplitOrMergeLockAndRollbackRequest {
+}
+
+message ReleaseSplitOrMergeLockAndRollbackResponse {
+}
+
 message NormalizeRequest {
 }
 
@@ -672,6 +679,12 @@ service MasterService {
     returns(IsSplitOrMergeEnabledResponse);
 
   /**
+   * Release lock and rollback state.
+   */
+  rpc ReleaseSplitOrMergeLockAndRollback(ReleaseSplitOrMergeLockAndRollbackRequest)
+    returns(ReleaseSplitOrMergeLockAndRollbackResponse);
+
+  /**
    * Run region normalizer. Can NOT run for various reasons. Check logs.
    */
   rpc Normalize(NormalizeRequest)
diff --git a/hbase-native-client/if/ZooKeeper.proto b/hbase-native-client/if/ZooKeeper.proto
index 186d183..41c0e0e 100644
--- a/hbase-native-client/if/ZooKeeper.proto
+++ b/hbase-native-client/if/ZooKeeper.proto
@@ -165,4 +165,12 @@ message TableLock {
  */
 message SwitchState {
   optional bool enabled = 1;
-}
\ No newline at end of file
+}
+
+/**
+ * State for split and merge, used in hbck
+ */
+message SplitAndMergeState {
+  optional bool split_enabled = 1;
+  optional bool merge_enabled = 2;
+}