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:41 UTC

[hbase] 53/133: HBASE-17451 [C++] HBase Request and Response Converter

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 094637299e9b53d669980a795989a6d8c2fad186
Author: Sudeep Sunthankar <su...@hashmapinc.com>
AuthorDate: Wed Jan 11 20:44:39 2017 +1100

    HBASE-17451 [C++] HBase Request and Response Converter
    
    Signed-off-by: Enis Soztutar <en...@apache.org>
---
 hbase-native-client/core/BUCK                      |  13 +++
 hbase-native-client/core/request_converter-test.cc | 127 +++++++++++++++++++++
 hbase-native-client/core/request_converter.cc      | 118 +++++++++++++++++++
 hbase-native-client/core/request_converter.h       |  69 +++++++++++
 hbase-native-client/core/response_converter.cc     |  50 ++++++++
 hbase-native-client/core/response_converter.h      |  47 ++++++++
 6 files changed, 424 insertions(+)

diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index a5ea5c0..b7db41a 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -33,6 +33,8 @@ cxx_library(
         "hbase_configuration_loader.h",
         "scan.h",
         "result.h",
+        "request_converter.h",
+        "response_converter.h",
     ],
     srcs=[
         "cell.cc",
@@ -45,6 +47,8 @@ cxx_library(
         "hbase_configuration_loader.cc",
         "scan.cc",
         "result.cc",
+        "request_converter.cc",
+        "response_converter.cc",
     ],
     deps=[
         "//connection:connection",
@@ -94,6 +98,15 @@ cxx_test(
     srcs=["result-test.cc",],
     deps=[":core",],
     run_test_separately=True,)
+cxx_test(
+    name="request_converter-test",
+    srcs=["request_converter-test.cc",],
+    deps=[
+        ":core",
+        "//connection:connection",
+        "//if:if",
+    ],
+    run_test_separately=True,)
 cxx_binary(
     name="simple-client",
     srcs=["simple-client.cc",],
diff --git a/hbase-native-client/core/request_converter-test.cc b/hbase-native-client/core/request_converter-test.cc
new file mode 100644
index 0000000..84d5e8b
--- /dev/null
+++ b/hbase-native-client/core/request_converter-test.cc
@@ -0,0 +1,127 @@
+/*
+ * 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 "core/request_converter.h"
+
+#include <gtest/gtest.h>
+#include <limits>
+#include "connection/request.h"
+#include "core/get.h"
+#include "core/scan.h"
+
+using hbase::Get;
+using hbase::Scan;
+
+using hbase::pb::GetRequest;
+using hbase::pb::RegionSpecifier;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+using hbase::pb::ScanRequest;
+
+TEST(RequestConverter, ToGet) {
+  std::string row_str = "row-test";
+  Get get(row_str);
+  get.AddFamily("family-1");
+  get.AddFamily("family-2");
+  get.AddFamily("family-3");
+  get.AddColumn("family-2", "qualifier-1");
+  get.AddColumn("family-2", "qualifier-2");
+  get.AddColumn("family-2", "qualifier-3");
+  get.SetCacheBlocks(false);
+  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  get.SetMaxVersions(2);
+  get.SetTimeRange(10000, 20000);
+  std::string region_name("RegionName");
+
+  auto req = hbase::RequestConverter::ToGetRequest(get, region_name);
+  auto msg = std::static_pointer_cast<GetRequest>(req->req_msg());
+
+  // Tests whether the PB object is properly set or not.
+  ASSERT_TRUE(msg->has_region());
+  ASSERT_TRUE(msg->region().has_value());
+  EXPECT_EQ(msg->region().value(), region_name);
+
+  ASSERT_TRUE(msg->has_get());
+  EXPECT_EQ(msg->get().row(), row_str);
+  EXPECT_FALSE(msg->get().cache_blocks());
+  EXPECT_EQ(msg->get().consistency(), hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(msg->get().max_versions(), 2);
+  EXPECT_EQ(msg->get().column_size(), 3);
+  for (int i = 0; i < msg->get().column_size(); ++i) {
+    EXPECT_EQ(msg->get().column(i).family(), "family-" + std::to_string(i + 1));
+    for (int j = 0; j < msg->get().column(i).qualifier_size(); ++j) {
+      EXPECT_EQ(msg->get().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
+    }
+  }
+}
+
+TEST(RequestConverter, ToScan) {
+  std::string start_row("start-row");
+  std::string stop_row("stop-row");
+  hbase::Scan scan;
+  scan.AddFamily("family-1");
+  scan.AddFamily("family-2");
+  scan.AddFamily("family-3");
+  scan.AddColumn("family-2", "qualifier-1");
+  scan.AddColumn("family-2", "qualifier-2");
+  scan.AddColumn("family-2", "qualifier-3");
+  scan.SetReversed(true);
+  scan.SetStartRow(start_row);
+  scan.SetStopRow(stop_row);
+  scan.SetSmall(true);
+  scan.SetCaching(3);
+  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  scan.SetCacheBlocks(true);
+  scan.SetAllowPartialResults(true);
+  scan.SetLoadColumnFamiliesOnDemand(true);
+  scan.SetMaxVersions(5);
+  scan.SetTimeRange(10000, 20000);
+  std::string region_name("RegionName");
+
+  auto req = hbase::RequestConverter::ToScanRequest(scan, region_name);
+  auto msg = std::static_pointer_cast<ScanRequest>(req->req_msg());
+
+  // Tests whether the PB object is properly set or not.
+  ASSERT_TRUE(msg->has_region());
+  ASSERT_TRUE(msg->region().has_value());
+  EXPECT_EQ(msg->region().value(), region_name);
+
+  ASSERT_TRUE(msg->has_scan());
+  EXPECT_TRUE(msg->scan().reversed());
+  EXPECT_EQ(msg->scan().start_row(), start_row);
+  EXPECT_EQ(msg->scan().stop_row(), stop_row);
+  EXPECT_TRUE(msg->scan().small());
+  EXPECT_EQ(msg->scan().caching(), 3);
+  EXPECT_EQ(msg->scan().consistency(), hbase::pb::Consistency::TIMELINE);
+  EXPECT_TRUE(msg->scan().cache_blocks());
+  EXPECT_TRUE(msg->scan().allow_partial_results());
+  EXPECT_TRUE(msg->scan().load_column_families_on_demand());
+  EXPECT_EQ(msg->scan().max_versions(), 5);
+  EXPECT_EQ(msg->scan().max_result_size(), std::numeric_limits<uint64_t>::max());
+
+  EXPECT_EQ(msg->scan().column_size(), 3);
+  for (int i = 0; i < msg->scan().column_size(); ++i) {
+    EXPECT_EQ(msg->scan().column(i).family(), "family-" + std::to_string(i + 1));
+    for (int j = 0; j < msg->scan().column(i).qualifier_size(); ++j) {
+      EXPECT_EQ(msg->scan().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
+    }
+  }
+  ASSERT_FALSE(msg->client_handles_partials());
+  ASSERT_FALSE(msg->client_handles_heartbeats());
+  ASSERT_FALSE(msg->track_scan_metrics());
+}
diff --git a/hbase-native-client/core/request_converter.cc b/hbase-native-client/core/request_converter.cc
new file mode 100644
index 0000000..eba07df
--- /dev/null
+++ b/hbase-native-client/core/request_converter.cc
@@ -0,0 +1,118 @@
+/*
+ * 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 "core/request_converter.h"
+#include <utility>
+#include "if/Client.pb.h"
+
+using hbase::Request;
+using hbase::pb::GetRequest;
+using hbase::pb::RegionSpecifier;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+using hbase::pb::ScanRequest;
+
+namespace hbase {
+
+RequestConverter::~RequestConverter() {}
+
+RequestConverter::RequestConverter() {}
+
+void RequestConverter::SetRegion(const std::string &region_name,
+                                 RegionSpecifier *region_specifier) {
+  region_specifier->set_type(
+      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_REGION_NAME);
+  region_specifier->set_value(region_name);
+}
+
+std::unique_ptr<Request> RequestConverter::ToGetRequest(const Get &get,
+                                                        const std::string &region_name) {
+  auto pb_req = Request::get();
+
+  auto pb_msg = std::static_pointer_cast<GetRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  auto pb_get = pb_msg->mutable_get();
+  pb_get->set_max_versions(get.MaxVersions());
+  pb_get->set_cache_blocks(get.CacheBlocks());
+  pb_get->set_consistency(get.Consistency());
+
+  if (!get.Timerange().IsAllTime()) {
+    hbase::pb::TimeRange *pb_time_range = pb_get->mutable_time_range();
+    pb_time_range->set_from(get.Timerange().MinTimeStamp());
+    pb_time_range->set_to(get.Timerange().MaxTimeStamp());
+  }
+  pb_get->set_row(get.Row());
+  if (get.HasFamilies()) {
+    for (const auto &family : get.Family()) {
+      auto column = pb_get->add_column();
+      column->set_family(family.first);
+      for (const auto &qualifier : family.second) {
+        column->add_qualifier(qualifier);
+      }
+    }
+  }
+
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
+                                                         const std::string &region_name) {
+  auto pb_req = Request::scan();
+
+  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
+
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  auto pb_scan = pb_msg->mutable_scan();
+  pb_scan->set_max_versions(scan.MaxVersions());
+  pb_scan->set_cache_blocks(scan.CacheBlocks());
+  pb_scan->set_reversed(scan.IsReversed());
+  pb_scan->set_small(scan.IsSmall());
+  pb_scan->set_caching(scan.Caching());
+  pb_scan->set_start_row(scan.StartRow());
+  pb_scan->set_stop_row(scan.StopRow());
+  pb_scan->set_consistency(scan.Consistency());
+  pb_scan->set_max_result_size(scan.MaxResultSize());
+  pb_scan->set_allow_partial_results(scan.AllowPartialResults());
+  pb_scan->set_load_column_families_on_demand(scan.LoadColumnFamiliesOnDemand());
+
+  if (!scan.Timerange().IsAllTime()) {
+    hbase::pb::TimeRange *pb_time_range = pb_scan->mutable_time_range();
+    pb_time_range->set_from(scan.Timerange().MinTimeStamp());
+    pb_time_range->set_to(scan.Timerange().MaxTimeStamp());
+  }
+
+  if (scan.HasFamilies()) {
+    for (const auto &family : scan.Family()) {
+      auto column = pb_scan->add_column();
+      column->set_family(family.first);
+      for (const auto &qualifier : family.second) {
+        column->add_qualifier(qualifier);
+      }
+    }
+  }
+
+  // TODO We will change this later.
+  pb_msg->set_client_handles_partials(false);
+  pb_msg->set_client_handles_heartbeats(false);
+  pb_msg->set_track_scan_metrics(false);
+
+  return pb_req;
+}
+} /* namespace hbase */
diff --git a/hbase-native-client/core/request_converter.h b/hbase-native-client/core/request_converter.h
new file mode 100644
index 0000000..57f08cc
--- /dev/null
+++ b/hbase-native-client/core/request_converter.h
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include "connection/request.h"
+#include "core/get.h"
+#include "core/scan.h"
+#include "if/HBase.pb.h"
+
+using hbase::pb::RegionSpecifier;
+namespace hbase {
+
+/**
+ * RequestConverter class
+ * This class converts a Client side Get, Scan, Mutate operation to corresponding PB message.
+ */
+class RequestConverter {
+ public:
+  ~RequestConverter();
+
+  /**
+   * @brief Returns a Request object comprising of PB GetRequest created using
+   * passed 'get'
+   * @param get - Get object used for creating GetRequest
+   * @param region_name - table region
+   */
+  static std::unique_ptr<Request> ToGetRequest(const Get &get, const std::string &region_name);
+
+  /**
+   * @brief Returns a Request object comprising of PB ScanRequest created using
+   * passed 'scan'
+   * @param scan - Scan object used for creating ScanRequest
+   * @param region_name - table region
+   */
+  static std::unique_ptr<Request> ToScanRequest(const Scan &scan, const std::string &region_name);
+
+ private:
+  // Constructor not required. We have all static methods to create PB requests.
+  RequestConverter();
+
+  /**
+   * @brief fills region_specifier with region values.
+   * @param region_name - table region
+   * @param region_specifier - RegionSpecifier to be filled and passed in PB
+   * Request.
+   */
+  static void SetRegion(const std::string &region_name, RegionSpecifier *region_specifier);
+};
+
+} /* namespace hbase */
diff --git a/hbase-native-client/core/response_converter.cc b/hbase-native-client/core/response_converter.cc
new file mode 100644
index 0000000..3fe2ba9
--- /dev/null
+++ b/hbase-native-client/core/response_converter.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 "core/response_converter.h"
+
+#include <vector>
+
+#include "core/cell.h"
+#include "if/Client.pb.h"
+
+using hbase::pb::GetResponse;
+
+namespace hbase {
+
+ResponseConverter::ResponseConverter() {}
+
+ResponseConverter::~ResponseConverter() {}
+
+std::unique_ptr<hbase::Result> ResponseConverter::FromGetResponse(const Response& resp) {
+  auto get_resp = std::static_pointer_cast<GetResponse>(resp.resp_msg());
+
+  std::vector<std::shared_ptr<Cell>> vcells;
+  for (auto cell : get_resp->result().cell()) {
+    std::shared_ptr<Cell> pcell =
+        std::make_shared<Cell>(cell.row(), cell.family(), cell.qualifier(), cell.timestamp(),
+                               cell.value(), static_cast<hbase::CellType>(cell.cell_type()));
+    vcells.push_back(pcell);
+  }
+
+  return std::make_unique<hbase::Result>(vcells, get_resp->result().exists(),
+                                         get_resp->result().stale(), get_resp->result().partial());
+}
+
+} /* namespace hbase */
diff --git a/hbase-native-client/core/response_converter.h b/hbase-native-client/core/response_converter.h
new file mode 100644
index 0000000..7cbde81
--- /dev/null
+++ b/hbase-native-client/core/response_converter.h
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include "core/result.h"
+#include "connection/response.h"
+
+namespace hbase {
+
+/**
+ * ResponseConverter class
+ * This class converts a PB Response to corresponding Result or other objects.
+ */
+class ResponseConverter {
+ public:
+  ~ResponseConverter();
+
+  /**
+   * @brief Returns a Result object created by PB Message in passed Response object.
+   * @param resp - Response object having the PB message.
+   */
+  static std::unique_ptr<hbase::Result> FromGetResponse(const Response &resp);
+
+ private:
+  // Constructor not required. We have all static methods to extract response from PB messages.
+  ResponseConverter();
+};
+
+} /* namespace hbase */