You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/04/09 18:32:02 UTC

arrow git commit: ARROW-773: [CPP] Add Table::AddColumn API

Repository: arrow
Updated Branches:
  refs/heads/master b0863cb63 -> 449f99162


ARROW-773: [CPP] Add Table::AddColumn API

Author: Nong Li <no...@gmail.com>

Closes #513 from nongli/arrow-773 and squashes the following commits:

e6f5846 [Nong Li] ARROW-773: [CPP] Add Table::AddColumn API


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

Branch: refs/heads/master
Commit: 449f99162abab52378e2d6b2ca18099df567dc29
Parents: b0863cb
Author: Nong Li <no...@gmail.com>
Authored: Sun Apr 9 14:31:57 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Sun Apr 9 14:31:57 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/table-test.cc         | 73 ++++++++++++++++++++++++++++++++
 cpp/src/arrow/table.cc              | 24 +++++++++++
 cpp/src/arrow/table.h               |  4 ++
 cpp/src/arrow/type.cc               |  9 ++++
 cpp/src/arrow/type.h                |  2 +
 cpp/src/arrow/util/CMakeLists.txt   |  1 +
 cpp/src/arrow/util/stl-util-test.cc | 60 ++++++++++++++++++++++++++
 cpp/src/arrow/util/stl.h            | 20 +++++++++
 8 files changed, 193 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/table-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table-test.cc b/cpp/src/arrow/table-test.cc
index cd32f4a..156c3d1 100644
--- a/cpp/src/arrow/table-test.cc
+++ b/cpp/src/arrow/table-test.cc
@@ -384,6 +384,79 @@ TEST_F(TestTable, RemoveColumn) {
   ASSERT_TRUE(result->Equals(Table(ex_schema, ex_columns)));
 }
 
+TEST_F(TestTable, AddColumn) {
+  const int64_t length = 10;
+  MakeExample1(length);
+
+  Table table(schema_, columns_);
+
+  std::shared_ptr<Table> result;
+  // Some negative tests with invalid index
+  Status status = table.AddColumn(10, columns_[0], &result);
+  ASSERT_TRUE(status.IsInvalid());
+  status = table.AddColumn(-1, columns_[0], &result);
+  ASSERT_TRUE(status.IsInvalid());
+
+  // Add column with wrong length
+  auto longer_col = std::make_shared<Column>(
+      schema_->field(0), MakePrimitive<Int32Array>(length + 1));
+  status = table.AddColumn(0, longer_col, &result);
+  ASSERT_TRUE(status.IsInvalid());
+
+  // Add column 0 in different places
+  ASSERT_OK(table.AddColumn(0, columns_[0], &result));
+  auto ex_schema = std::shared_ptr<Schema>(new Schema({
+      schema_->field(0),
+      schema_->field(0),
+      schema_->field(1),
+      schema_->field(2)}));
+  std::vector<std::shared_ptr<Column>> ex_columns = {
+      table.column(0),
+      table.column(0),
+      table.column(1),
+      table.column(2)};
+  ASSERT_TRUE(result->Equals(Table(ex_schema, ex_columns)));
+
+  ASSERT_OK(table.AddColumn(1, columns_[0], &result));
+  ex_schema = std::shared_ptr<Schema>(new Schema({
+      schema_->field(0),
+      schema_->field(0),
+      schema_->field(1),
+      schema_->field(2)}));
+  ex_columns = {
+      table.column(0),
+      table.column(0),
+      table.column(1),
+      table.column(2)};
+  ASSERT_TRUE(result->Equals(Table(ex_schema, ex_columns)));
+
+  ASSERT_OK(table.AddColumn(2, columns_[0], &result));
+  ex_schema = std::shared_ptr<Schema>(new Schema({
+      schema_->field(0),
+      schema_->field(1),
+      schema_->field(0),
+      schema_->field(2)}));
+  ex_columns = {
+      table.column(0),
+      table.column(1),
+      table.column(0),
+      table.column(2)};
+  ASSERT_TRUE(result->Equals(Table(ex_schema, ex_columns)));
+
+  ASSERT_OK(table.AddColumn(3, columns_[0], &result));
+  ex_schema = std::shared_ptr<Schema>(new Schema({
+      schema_->field(0),
+      schema_->field(1),
+      schema_->field(2),
+      schema_->field(0)}));
+  ex_columns = {
+      table.column(0),
+      table.column(1),
+      table.column(2),
+      table.column(0)};
+  ASSERT_TRUE(result->Equals(Table(ex_schema, ex_columns)));
+}
+
 class TestRecordBatch : public TestBase {};
 
 TEST_F(TestRecordBatch, Equals) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/table.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index da61fbb..9b39f77 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -321,6 +321,30 @@ Status Table::RemoveColumn(int i, std::shared_ptr<Table>* out) const {
   return Status::OK();
 }
 
+Status Table::AddColumn(int i, const std::shared_ptr<Column>& col,
+    std::shared_ptr<Table>* out) const {
+  if (i < 0 || i > num_columns() + 1) {
+    return Status::Invalid("Invalid column index.");
+  }
+  if (col == nullptr) {
+    std::stringstream ss;
+    ss << "Column " << i << " was null";
+    return Status::Invalid(ss.str());
+  }
+  if (col->length() != num_rows_) {
+    std::stringstream ss;
+    ss << "Added column's length must match table's length. Expected length " << num_rows_
+        << " but got length " << col->length();
+    return Status::Invalid(ss.str());
+  }
+
+  std::shared_ptr<Schema> new_schema;
+  RETURN_NOT_OK(schema_->AddField(i, col->field(), &new_schema));
+
+  *out = std::make_shared<Table>(new_schema, AddVectorElement(columns_, i, col));
+  return Status::OK();
+}
+
 Status Table::ValidateColumns() const {
   if (num_columns() != schema_->num_fields()) {
     return Status::Invalid("Number of columns did not match schema");

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/table.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index 0f35dd8..dcea53d 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -181,6 +181,10 @@ class ARROW_EXPORT Table {
   /// schemas are immutable)
   Status RemoveColumn(int i, std::shared_ptr<Table>* out) const;
 
+  /// Add column to the table, producing a new Table
+  Status AddColumn(int i, const std::shared_ptr<Column>& column,
+      std::shared_ptr<Table>* out) const;
+
   // @returns: the number of columns in the table
   int num_columns() const { return static_cast<int>(columns_.size()); }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index d99551d..abbb626 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -257,6 +257,15 @@ std::shared_ptr<Field> Schema::GetFieldByName(const std::string& name) {
   }
 }
 
+Status Schema::AddField(int i, const std::shared_ptr<Field>& field,
+    std::shared_ptr<Schema>* out) const {
+  DCHECK_GE(i, 0);
+  DCHECK_LE(i, this->num_fields());
+
+  *out = std::make_shared<Schema>(AddVectorElement(fields_, i, field));
+  return Status::OK();
+}
+
 Status Schema::RemoveField(int i, std::shared_ptr<Schema>* out) const {
   DCHECK_GE(i, 0);
   DCHECK_LT(i, this->num_fields());

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 0e69133..36ab9d8 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -619,6 +619,8 @@ class ARROW_EXPORT Schema {
   // Render a string representation of the schema suitable for debugging
   std::string ToString() const;
 
+  Status AddField(int i, const std::shared_ptr<Field>& field,
+      std::shared_ptr<Schema>* out) const;
   Status RemoveField(int i, std::shared_ptr<Schema>* out) const;
 
   int num_fields() const { return static_cast<int>(fields_.size()); }

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/CMakeLists.txt b/cpp/src/arrow/util/CMakeLists.txt
index 8d9afcc..c1b6877 100644
--- a/cpp/src/arrow/util/CMakeLists.txt
+++ b/cpp/src/arrow/util/CMakeLists.txt
@@ -69,3 +69,4 @@ if (ARROW_BUILD_BENCHMARKS)
 endif()
 
 ADD_ARROW_TEST(bit-util-test)
+ADD_ARROW_TEST(stl-util-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/util/stl-util-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/stl-util-test.cc b/cpp/src/arrow/util/stl-util-test.cc
new file mode 100644
index 0000000..526520e
--- /dev/null
+++ b/cpp/src/arrow/util/stl-util-test.cc
@@ -0,0 +1,60 @@
+// 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 "arrow/util/stl.h"
+
+#include <cstdint>
+#include <vector>
+
+#include "gtest/gtest.h"
+
+#include "arrow/test-util.h"
+
+namespace arrow {
+
+TEST(StlUtilTest, VectorAddRemoveTest) {
+  std::vector<int> values;
+  std::vector<int> result = AddVectorElement(values, 0, 100);
+  EXPECT_EQ(values.size(), 0);
+  EXPECT_EQ(result.size(), 1);
+  EXPECT_EQ(result[0], 100);
+
+  // Add 200 at index 0 and 300 at the end.
+  std::vector<int> result2 = AddVectorElement(result, 0, 200);
+  result2 = AddVectorElement(result2, result2.size(), 300);
+  EXPECT_EQ(result.size(), 1);
+  EXPECT_EQ(result2.size(), 3);
+  EXPECT_EQ(result2[0], 200);
+  EXPECT_EQ(result2[1], 100);
+  EXPECT_EQ(result2[2], 300);
+
+  // Remove 100, 300, 200
+  std::vector<int> result3 = DeleteVectorElement(result2, 1);
+  EXPECT_EQ(result2.size(), 3);
+  EXPECT_EQ(result3.size(), 2);
+  EXPECT_EQ(result3[0], 200);
+  EXPECT_EQ(result3[1], 300);
+
+  result3 = DeleteVectorElement(result3, 1);
+  EXPECT_EQ(result3.size(), 1);
+  EXPECT_EQ(result3[0], 200);
+
+  result3 = DeleteVectorElement(result3, 0);
+  EXPECT_TRUE(result3.empty());
+}
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/449f9916/cpp/src/arrow/util/stl.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/stl.h b/cpp/src/arrow/util/stl.h
index 3ec535d..bd25053 100644
--- a/cpp/src/arrow/util/stl.h
+++ b/cpp/src/arrow/util/stl.h
@@ -20,10 +20,14 @@
 
 #include <vector>
 
+#include <arrow/util/logging.h>
+
 namespace arrow {
 
 template <typename T>
 inline std::vector<T> DeleteVectorElement(const std::vector<T>& values, size_t index) {
+  DCHECK(!values.empty());
+  DCHECK_LT(index, values.size());
   std::vector<T> out;
   out.reserve(values.size() - 1);
   for (size_t i = 0; i < index; ++i) {
@@ -35,6 +39,22 @@ inline std::vector<T> DeleteVectorElement(const std::vector<T>& values, size_t i
   return out;
 }
 
+template <typename T>
+inline std::vector<T> AddVectorElement(const std::vector<T>& values, size_t index,
+    const T& new_element) {
+  DCHECK_LE(index, values.size());
+  std::vector<T> out;
+  out.reserve(values.size() + 1);
+  for (size_t i = 0; i < index; ++i) {
+    out.push_back(values[i]);
+  }
+  out.push_back(new_element);
+  for (size_t i = index; i < values.size(); ++i) {
+    out.push_back(values[i]);
+  }
+  return out;
+}
+
 }  // namespace arrow
 
 #endif  // ARROW_UTIL_STL_H