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

[hbase] 50/133: HBASE-17283 [C++] Result class

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 8d90a78b149886327f792e6b886c728e28f30adb
Author: Sudeep Sunthankar <su...@hashmapinc.com>
AuthorDate: Fri Dec 30 10:27:29 2016 +1100

    HBASE-17283 [C++] Result class
    
    Signed-off-by: Enis Soztutar <en...@apache.org>
---
 hbase-native-client/core/BUCK           |   7 +
 hbase-native-client/core/result-test.cc | 301 ++++++++++++++++++++++++++++++++
 hbase-native-client/core/result.cc      | 118 +++++++++++++
 hbase-native-client/core/result.h       | 131 ++++++++++++++
 4 files changed, 557 insertions(+)

diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 20e4736..a5ea5c0 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -32,6 +32,7 @@ cxx_library(
         "configuration.h",
         "hbase_configuration_loader.h",
         "scan.h",
+        "result.h",
     ],
     srcs=[
         "cell.cc",
@@ -43,6 +44,7 @@ cxx_library(
         "configuration.cc",
         "hbase_configuration_loader.cc",
         "scan.cc",
+        "result.cc",
     ],
     deps=[
         "//connection:connection",
@@ -87,6 +89,11 @@ cxx_test(
     srcs=["scan-test.cc",],
     deps=[":core",],
     run_test_separately=True,)
+cxx_test(
+    name="result-test",
+    srcs=["result-test.cc",],
+    deps=[":core",],
+    run_test_separately=True,)
 cxx_binary(
     name="simple-client",
     srcs=["simple-client.cc",],
diff --git a/hbase-native-client/core/result-test.cc b/hbase-native-client/core/result-test.cc
new file mode 100644
index 0000000..134e80f
--- /dev/null
+++ b/hbase-native-client/core/result-test.cc
@@ -0,0 +1,301 @@
+/*
+ * 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/result.h"
+
+#include <limits>
+#include <memory>
+#include <vector>
+#include <string>
+#include <gtest/gtest.h>
+
+#include "core/cell.h"
+using namespace hbase;
+
+void PopulateCells(std::vector<std::shared_ptr<Cell> > &cells) {
+  // Populate some Results
+  // We assume that for a single Cell, the corresponding row, families and
+  // qualifiers are present.
+  // We have also considered different versions in the test for the same row.
+  std::string row = "row";
+  for (int i = 0; i < 10; i++) {
+    std::string family = "family-" + std::to_string(i);
+    std::string column = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+
+    switch (i) {
+      case 5: {
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482113040506, "value-5", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482111803856, "value-X", CellType::PUT));
+        break;
+      }
+      case 8: {
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482113040506, "value-8", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482111803856, "value-X", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482110969958, "value-Y", CellType::PUT));
+        break;
+      }
+      case 9: {
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482113040506, "value-9", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482111803856, "value-X", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482110969958, "value-Y", CellType::PUT));
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, 1482110876075, "value-Z", CellType::PUT));
+        break;
+      }
+      default: {
+        cells.push_back(std::make_shared<Cell>(row, family, column,
+                                               std::numeric_limits<long>::max(),
+                                               value, CellType::PUT));
+      }
+    }
+  }
+  return;
+}
+
+TEST(Result, EmptyResult) {
+  std::vector<std::shared_ptr<Cell> > cells;
+  Result result(cells, true, false, false);
+  EXPECT_EQ(true, result.IsEmpty());
+  EXPECT_EQ(0, result.Size());
+}
+
+TEST(Result, FilledResult) {
+  std::vector<std::shared_ptr<Cell> > cells;
+  PopulateCells(cells);
+
+  Result result(cells, true, false, false);
+
+  EXPECT_EQ(false, result.IsEmpty());
+  EXPECT_EQ(16, result.Size());
+
+  // Get Latest Cell for the given family and qualifier.
+  auto latest_cell(result.ColumnLatestCell("family", "column"));
+  // Nothing of the above family/qualifier combo is present so it should be
+  // nullptr
+  ASSERT_FALSE(latest_cell.get());
+
+  // Try to get the latest cell for the given family and qualifier.
+  latest_cell = result.ColumnLatestCell("family-4", "column-4");
+  // Now shouldn't be a nullptr
+  ASSERT_TRUE(latest_cell.get());
+  // And Value must match too
+  EXPECT_EQ("value-4", latest_cell->Value());
+
+  // Value will be nullptr as no such family and qualifier is present
+  ASSERT_FALSE(result.Value("family-4", "qualifier"));
+  // Value will be present as family and qualifier is present
+  ASSERT_TRUE(result.Value("family-4", "column-4"));
+  // Value should be present and match.
+  EXPECT_EQ(latest_cell->Value(),
+            (*result.ColumnLatestCell("family-4", "column-4")).Value());
+  EXPECT_EQ("value-5",
+            (*result.ColumnLatestCell("family-5", "column-5")).Value());
+  EXPECT_EQ("value-8",
+            (*result.ColumnLatestCell("family-8", "column-8")).Value());
+  EXPECT_EQ("value-7", *result.Value("family-7", "column-7"));
+
+  // Get cells for the given family and qualifier
+  auto column_cells = result.ColumnCells("family", "column");
+  // Size should be 0
+  EXPECT_EQ(0, column_cells.size());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-0", "column-0");
+  EXPECT_EQ(1, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-5", "column-5");
+  EXPECT_EQ(2, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("value-5", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-8", "column-8");
+  EXPECT_EQ(3, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("row", column_cells[2]->Row());
+  EXPECT_EQ("value-8", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("value-Y", column_cells[2]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-9", "column-9");
+  EXPECT_EQ(4, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("row", column_cells[2]->Row());
+  EXPECT_EQ("row", column_cells[3]->Row());
+  EXPECT_EQ("value-9", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("value-Y", column_cells[2]->Value());
+  EXPECT_EQ("value-Z", column_cells[3]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Test all the Cell values
+  const auto &result_cells = result.Cells();
+  int i = 0, j = 0;
+  for (const auto &cell : result_cells) {
+    std::string row = "row";
+    std::string family = "family-" + std::to_string(i);
+    std::string column = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+    switch (j) {
+      case 6:
+      case 10:
+      case 13: {
+        EXPECT_EQ("value-X", cell->Value());
+        ++j;
+        continue;
+      }
+      case 11:
+      case 14: {
+        EXPECT_EQ("value-Y", cell->Value());
+        ++j;
+        continue;
+      }
+      case 15: {
+        EXPECT_EQ("value-Z", cell->Value());
+        ++j;
+        continue;
+      }
+    }
+    EXPECT_EQ(row, cell->Row());
+    EXPECT_EQ(family, cell->Family());
+    EXPECT_EQ(column, cell->Qualifier());
+    EXPECT_EQ(value, cell->Value());
+    ++i;
+    ++j;
+  }
+
+  auto result_map_tmp = result.Map();
+  result_map_tmp["testf"]["testq"][1] = "value";
+  EXPECT_EQ(11, result_map_tmp.size());
+
+  auto result_map = result.Map();
+  EXPECT_EQ(10, result_map.size());
+
+  i = 0;
+  for (auto family_map : result_map) {
+    std::string family = "family-" + std::to_string(i);
+    std::string qualifier = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+    EXPECT_EQ(family, family_map.first);
+    for (auto qualifier_map : family_map.second) {
+      EXPECT_EQ(qualifier, qualifier_map.first);
+      j = 0;
+      for (auto version_map : qualifier_map.second) {
+        switch (i) {
+          case 5: {
+            if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-5", version_map.second);
+            }
+            break;
+          }
+          case 8: {
+            if (2 == j) {
+              EXPECT_EQ(1482110969958, version_map.first);
+              EXPECT_EQ("value-Y", version_map.second);
+            } else if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-8", version_map.second);
+            }
+            break;
+          }
+          case 9: {
+            if (3 == j) {
+              EXPECT_EQ(1482110876075, version_map.first);
+              EXPECT_EQ("value-Z", version_map.second);
+            } else if (2 == j) {
+              EXPECT_EQ(1482110969958, version_map.first);
+              EXPECT_EQ("value-Y", version_map.second);
+            } else if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-9", version_map.second);
+            }
+            break;
+          }
+          default: {
+            EXPECT_EQ(std::numeric_limits<long>::max(), version_map.first);
+            EXPECT_EQ(value, version_map.second);
+          }
+        }
+        ++j;
+      }
+    }
+    ++i;
+  }
+
+  auto family_map = result.FamilyMap("family-0");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-0", qual_val_map.first);
+    EXPECT_EQ("value-0", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-1");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-1", qual_val_map.first);
+    EXPECT_EQ("value-1", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-5");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-5", qual_val_map.first);
+    EXPECT_EQ("value-5", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-9");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-9", qual_val_map.first);
+    EXPECT_EQ("value-9", qual_val_map.second);
+  }
+}
diff --git a/hbase-native-client/core/result.cc b/hbase-native-client/core/result.cc
new file mode 100644
index 0000000..9842915
--- /dev/null
+++ b/hbase-native-client/core/result.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/result.h"
+
+namespace hbase {
+
+Result::~Result() {}
+
+Result::Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists,
+               bool stale, bool partial)
+    : exists_(exists), stale_(stale), partial_(partial) {
+  for (const auto &cell : cells) {
+    cells_.push_back(cell);
+    // We create the map when cells are added. unlike java where map is created
+    // when result.getMap() is called
+    result_map_[cell->Family()][cell->Qualifier()][cell->Timestamp()] =
+        cell->Value();
+  }
+  row_ = (cells_.size() == 0 ? "" : cells_[0]->Row());
+}
+
+Result::Result(const Result &result) {
+  exists_ = result.exists_;
+  stale_ = result.stale_;
+  partial_ = result.partial_;
+  row_ = result.row_;
+  if (!result.cells_.empty()) {
+    for (const auto &cell : result.cells_) {
+      cells_.push_back(cell);
+      result_map_[cell->Family()][cell->Qualifier()][cell->Timestamp()] =
+          cell->Value();
+    }
+  }
+}
+const std::vector<std::shared_ptr<Cell> > &Result::Cells() const {
+  return cells_;
+}
+
+std::vector<std::shared_ptr<Cell> > Result::ColumnCells(
+    const std::string &family, const std::string &qualifier) const {
+  std::vector<std::shared_ptr<Cell> > column_cells;
+  // TODO implement a BinarySearch here ?
+  for (const auto &cell : cells_) {
+    if (cell->Family() == family && cell->Qualifier() == qualifier) {
+      column_cells.push_back(cell);
+    }
+  }
+  return column_cells;
+}
+
+const std::shared_ptr<Cell> Result::ColumnLatestCell(
+    const std::string &family, const std::string &qualifier) const {
+  // TODO implement a BinarySearch here ?
+  for (const auto &cell : cells_) {
+    // We find the latest(first) occurrence of the Cell for a given column and
+    // qualifier and break
+    if (cell->Family() == family && cell->Qualifier() == qualifier) {
+      return cell;
+    }
+  }
+  return nullptr;
+}
+
+std::shared_ptr<std::string> Result::Value(const std::string &family,
+                                           const std::string &qualifier) const {
+  std::shared_ptr<Cell> latest_cell(ColumnLatestCell(family, qualifier));
+  if (latest_cell.get()) {
+    return std::make_shared<std::string>(latest_cell->Value());
+  }
+  return nullptr;
+}
+
+bool Result::IsEmpty() const { return cells_.empty(); }
+
+const std::string &Result::Row() const { return row_; }
+
+const int Result::Size() const { return cells_.size(); }
+
+const ResultMap &Result::Map() const { return result_map_; }
+
+const ResultFamilyMap Result::FamilyMap(const std::string &family) const {
+  ResultFamilyMap family_map;
+  if (!IsEmpty()) {
+    for (auto itr = result_map_.begin(); itr != result_map_.end(); ++itr) {
+      if (family == itr->first) {
+        for (auto qitr = itr->second.begin(); qitr != itr->second.end();
+             ++qitr) {
+          for (auto vitr = qitr->second.begin(); vitr != qitr->second.end();
+               ++vitr) {
+            // We break after inserting the first value. Result.java takes only
+            // the first value
+            family_map[qitr->first] = vitr->second;
+            break;
+          }
+        }
+      }
+    }
+  }
+  return family_map;
+}
+} /* namespace hbase */
diff --git a/hbase-native-client/core/result.h b/hbase-native-client/core/result.h
new file mode 100644
index 0000000..ecfdd55
--- /dev/null
+++ b/hbase-native-client/core/result.h
@@ -0,0 +1,131 @@
+/*
+ * 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 <functional>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "core/cell.h"
+
+namespace hbase {
+
+/**
+ * @brief Map of families to all versions of its qualifiers and values
+ * We need to have a reverse ordered map, when storing TS -> value, so that the
+ * most recent value is stored first
+ */
+using ResultMap = std::map<
+    std::string, std::map<std::string, std::map<int64_t, std::string,
+                                                std::greater<int64_t> > > >;
+
+/**
+ * @brief Map of qualifiers to values.
+ */
+using ResultFamilyMap = std::map<std::string, std::string>;
+
+class Result {
+ public:
+  /**
+   * Constructors
+   */
+  Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists,
+         bool stale, bool partial);
+  Result(const Result &result);
+  ~Result();
+
+  /**
+   * @brief Return the vector of Cells backing this Result instance. This vector
+   * will be ordered in the same manner
+   * as the one which was passed while creation of the Result instance.
+   */
+  const std::vector<std::shared_ptr<Cell> > &Cells() const;
+
+  /**
+   * @brief  Return a vector of Cells for the family and qualifier or empty list
+   * if the column
+   * did not exist in the result.
+   * @param family - column family
+   * @param qualifier - column qualifier
+   */
+  std::vector<std::shared_ptr<Cell> > ColumnCells(
+      const std::string &family, const std::string &qualifier) const;
+
+  /**
+   * @brief Returns the Cell for the most recent timestamp for a given family
+   * and qualifier.
+   * Returns map of qualifiers to values, only includes latest values
+   * @param family - column family.
+   * @param qualifier - column qualifier
+   */
+  const std::shared_ptr<Cell> ColumnLatestCell(
+      const std::string &family, const std::string &qualifier) const;
+
+  /**
+   * @brief Get the latest version of the specified family and qualifier.
+   * @param family - column family
+   * @param qualifier - column qualifier
+   */
+  std::shared_ptr<std::string> Value(const std::string &family,
+                                     const std::string &qualifier) const;
+
+  /**
+   * @brief Returns if the underlying Cell vector is empty or not
+   */
+  bool IsEmpty() const;
+
+  /**
+   * @brief Retrieves the row key that corresponds to the row from which this
+   * Result was created.
+   */
+  const std::string &Row() const;
+
+  /**
+   * @brief Returns the size of the underlying Cell vector
+   */
+  const int Size() const;
+
+  /**
+   * @brief Map of families to all versions of its qualifiers and values.
+   * Returns a three level Map of the form:
+   * Map<family,Map<qualifier,Map<timestamp,value>>>>
+   * All other map returning methods make use of this map internally
+   * The Map is created when the Result instance is created
+   */
+  const ResultMap &Map() const;
+
+  /**
+   * @brief Map of qualifiers to values.
+   * Returns a Map of the form: Map<qualifier,value>
+   * @param family - column family to get
+   */
+  const ResultFamilyMap FamilyMap(const std::string &family) const;
+
+ private:
+  bool exists_ = false;
+  bool stale_ = false;
+  bool partial_ = false;
+  std::string row_ = "";
+  std::vector<std::shared_ptr<Cell> > cells_;
+  ResultMap result_map_;
+};
+} /* namespace hbase */