You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by fs...@apache.org on 2019/06/18 02:53:11 UTC

[arrow] branch master updated: ARROW-4912: [C++] add method for easy renaming of a Table's columns

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5c562e8  ARROW-4912: [C++] add method for easy renaming of a Table's columns
5c562e8 is described below

commit 5c562e832f16ce881d0cae9237307e4290ac3dd2
Author: Benjamin Kietzman <be...@gmail.com>
AuthorDate: Mon Jun 17 22:52:59 2019 -0400

    ARROW-4912: [C++] add method for easy renaming of a Table's columns
    
    This will allow users to produce a table with custom column names from a csv file without specifying a schema (and requiring type inference)
    
    Author: Benjamin Kietzman <be...@gmail.com>
    
    Closes #4557 from bkietz/4912-Allow-specifying-column-names-to-CSV-rea and squashes the following commits:
    
    213c19383 <Benjamin Kietzman> set correct schema in Table::RenameColumns, tests in C++ and py
    1341a80e1 <Benjamin Kietzman> fix careless copypasta
    a6eb3f0ae <Benjamin Kietzman> use gmock's improved assert syntax
    9ab8de811 <Benjamin Kietzman> add autopep8 note to developers/python.rst
    5fdbf1a80 <Benjamin Kietzman> autopep8, test f/incorrect number of args
    81e5adc8e <Benjamin Kietzman> add cython bindings
    e46752673 <Benjamin Kietzman> Table::RenameCols can fail if an incorrect number of columns are provided
    be18cfc20 <Benjamin Kietzman> add method for easy renaming of a Table's columns
---
 cpp/src/arrow/table-test.cc          | 14 ++++++++++++++
 cpp/src/arrow/table.cc               | 24 ++++++++++++++++++++++++
 cpp/src/arrow/table.h                |  7 +++++++
 cpp/src/arrow/type-test.cc           | 20 ++++++++++++++++++++
 cpp/src/arrow/type.cc                |  4 ++++
 cpp/src/arrow/type.h                 |  3 +++
 docs/source/developers/python.rst    |  8 ++++++++
 python/pyarrow/includes/libarrow.pxd |  3 +++
 python/pyarrow/table.pxi             | 24 ++++++++++++++++++++++++
 python/pyarrow/tests/test_table.py   | 17 +++++++++++++++++
 10 files changed, 124 insertions(+)

diff --git a/cpp/src/arrow/table-test.cc b/cpp/src/arrow/table-test.cc
index b545e3b..3b2ed57 100644
--- a/cpp/src/arrow/table-test.cc
+++ b/cpp/src/arrow/table-test.cc
@@ -19,6 +19,7 @@
 #include <memory>
 #include <vector>
 
+#include <gmock/gmock.h>
 #include <gtest/gtest.h>
 
 #include "arrow/array.h"
@@ -498,6 +499,19 @@ TEST_F(TestTable, SetColumn) {
   ASSERT_TRUE(result->Equals(*expected));
 }
 
+TEST_F(TestTable, RenameColumns) {
+  MakeExample1(10);
+  auto table = Table::Make(schema_, columns_);
+  EXPECT_THAT(table->ColumnNames(), testing::ElementsAre("f0", "f1", "f2"));
+
+  std::shared_ptr<Table> renamed;
+  ASSERT_OK(table->RenameColumns({"zero", "one", "two"}, &renamed));
+  EXPECT_THAT(renamed->ColumnNames(), testing::ElementsAre("zero", "one", "two"));
+  ASSERT_OK(renamed->Validate());
+
+  ASSERT_RAISES(Invalid, table->RenameColumns({"hello", "world"}, &renamed));
+}
+
 TEST_F(TestTable, RemoveColumnEmpty) {
   // ARROW-1865
   const int64_t length = 10;
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index b018b8b..07b0189 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -478,6 +478,30 @@ Status Table::FromChunkedStructArray(const std::shared_ptr<ChunkedArray>& array,
   return Status::OK();
 }
 
+std::vector<std::string> Table::ColumnNames() const {
+  std::vector<std::string> names(num_columns());
+  for (int i = 0; i < num_columns(); ++i) {
+    names[i] = column(i)->name();
+  }
+  return names;
+}
+
+Status Table::RenameColumns(const std::vector<std::string>& names,
+                            std::shared_ptr<Table>* out) const {
+  if (names.size() != static_cast<size_t>(num_columns())) {
+    return Status::Invalid("tried to rename a table of ", num_columns(),
+                           " columns but only ", names.size(), " names were provided");
+  }
+  std::vector<std::shared_ptr<Column>> columns(num_columns());
+  std::vector<std::shared_ptr<Field>> fields(num_columns());
+  for (int i = 0; i < num_columns(); ++i) {
+    fields[i] = column(i)->field()->WithName(names[i]);
+    columns[i] = std::make_shared<Column>(fields[i], column(i)->data());
+  }
+  *out = Table::Make(::arrow::schema(std::move(fields)), std::move(columns), num_rows());
+  return Status::OK();
+}
+
 Status ConcatenateTables(const std::vector<std::shared_ptr<Table>>& tables,
                          std::shared_ptr<Table>* table) {
   if (tables.size() == 0) {
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index 8016371..94e9f22 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -297,6 +297,13 @@ class ARROW_EXPORT Table {
   virtual Status SetColumn(int i, const std::shared_ptr<Column>& column,
                            std::shared_ptr<Table>* out) const = 0;
 
+  /// \brief Return names of all columns
+  std::vector<std::string> ColumnNames() const;
+
+  /// \brief Rename columns with provided names
+  Status RenameColumns(const std::vector<std::string>& names,
+                       std::shared_ptr<Table>* out) const;
+
   /// \brief Replace schema key-value metadata with new metadata (EXPERIMENTAL)
   /// \since 0.5.0
   ///
diff --git a/cpp/src/arrow/type-test.cc b/cpp/src/arrow/type-test.cc
index 91562ee..7ad1d8a 100644
--- a/cpp/src/arrow/type-test.cc
+++ b/cpp/src/arrow/type-test.cc
@@ -121,6 +121,26 @@ TEST(TestField, TestFlatten) {
   ASSERT_TRUE(vec[1]->Equals(*expected1));
 }
 
+TEST(TestField, TestReplacement) {
+  auto metadata = std::shared_ptr<KeyValueMetadata>(
+      new KeyValueMetadata({"foo", "bar"}, {"bizz", "buzz"}));
+  auto f0 = field("f0", int32(), true, metadata);
+  auto fzero = f0->WithType(utf8());
+  auto f1 = f0->WithName("f1");
+
+  ASSERT_FALSE(f0->Equals(fzero));
+  ASSERT_FALSE(fzero->Equals(f1));
+  ASSERT_FALSE(f1->Equals(f0));
+
+  ASSERT_EQ(fzero->name(), "f0");
+  ASSERT_TRUE(fzero->type()->Equals(utf8()));
+  ASSERT_TRUE(fzero->metadata()->Equals(*metadata));
+
+  ASSERT_EQ(f1->name(), "f1");
+  ASSERT_TRUE(f1->type()->Equals(int32()));
+  ASSERT_TRUE(f1->metadata()->Equals(*metadata));
+}
+
 class TestSchema : public ::testing::Test {
  public:
   void SetUp() {}
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 762d46d..b215331 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -56,6 +56,10 @@ std::shared_ptr<Field> Field::WithType(const std::shared_ptr<DataType>& type) co
   return std::make_shared<Field>(name_, type, nullable_, metadata_);
 }
 
+std::shared_ptr<Field> Field::WithName(const std::string& name) const {
+  return std::make_shared<Field>(name, type_, nullable_, metadata_);
+}
+
 std::vector<std::shared_ptr<Field>> Field::Flatten() const {
   std::vector<std::shared_ptr<Field>> flattened;
   if (type_->id() == Type::STRUCT) {
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index b581739..a101759 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -291,6 +291,9 @@ class ARROW_EXPORT Field {
   /// \brief Return a copy of this field with the replaced type.
   std::shared_ptr<Field> WithType(const std::shared_ptr<DataType>& type) const;
 
+  /// \brief Return a copy of this field with the replaced name.
+  std::shared_ptr<Field> WithName(const std::string& name) const;
+
   std::vector<std::shared_ptr<Field>> Flatten() const;
 
   bool Equals(const Field& other, bool check_metadata = true) const;
diff --git a/docs/source/developers/python.rst b/docs/source/developers/python.rst
index 0242714..4691d2e 100644
--- a/docs/source/developers/python.rst
+++ b/docs/source/developers/python.rst
@@ -39,6 +39,14 @@ build. Check for style errors before submitting your pull request with:
    flake8 .
    flake8 --config=.flake8.cython .
 
+The package ``autopep8`` (also available from pip or conda) can automatically
+fix many of the errors reported by ``flake8``:
+
+.. code-block:: shell
+
+   autopep8 --in-place ../integration/integration_test.py
+   autopep8 --in-place --global-config=.flake8.cython pyarrow/table.pxi
+
 Unit Testing
 ============
 
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 178a250..9f16838 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -551,6 +551,9 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         CStatus SetColumn(int i, const shared_ptr[CColumn]& column,
                           shared_ptr[CTable]* out)
 
+        vector[c_string] ColumnNames()
+        CStatus RenameColumns(const vector[c_string]&, shared_ptr[CTable]* out)
+
         CStatus Flatten(CMemoryPool* pool, shared_ptr[CTable]* out)
 
         CStatus Validate()
diff --git a/python/pyarrow/table.pxi b/python/pyarrow/table.pxi
index 9aac2d1..310bf54 100644
--- a/python/pyarrow/table.pxi
+++ b/python/pyarrow/table.pxi
@@ -1514,6 +1514,30 @@ cdef class Table(_PandasConvertible):
 
         return pyarrow_wrap_table(c_table)
 
+    @property
+    def column_names(self):
+        """
+        Names of the table's columns
+        """
+        names = self.table.ColumnNames()
+        return [frombytes(name) for name in names]
+
+    def rename_columns(self, names):
+        """
+        Create new table with columns renamed to provided names
+        """
+        cdef:
+            shared_ptr[CTable] c_table
+            vector[c_string] c_names
+
+        for name in names:
+            c_names.push_back(tobytes(name))
+
+        with nogil:
+            check_status(self.table.RenameColumns(c_names, &c_table))
+
+        return pyarrow_wrap_table(c_table)
+
     def drop(self, columns):
         """
         Drop one or more columns and return a new table.
diff --git a/python/pyarrow/tests/test_table.py b/python/pyarrow/tests/test_table.py
index 56c3592..84b512e 100644
--- a/python/pyarrow/tests/test_table.py
+++ b/python/pyarrow/tests/test_table.py
@@ -739,6 +739,23 @@ def test_table_remove_column_empty():
     assert t3.equals(table)
 
 
+def test_table_rename_columns():
+    data = [
+        pa.array(range(5)),
+        pa.array([-10, -5, 0, 5, 10]),
+        pa.array(range(5, 10))
+    ]
+    table = pa.Table.from_arrays(data, names=['a', 'b', 'c'])
+    assert table.column_names == ['a', 'b', 'c']
+
+    t2 = table.rename_columns(['eh', 'bee', 'sea'])
+    t2._validate()
+    assert t2.column_names == ['eh', 'bee', 'sea']
+
+    expected = pa.Table.from_arrays(data, names=['eh', 'bee', 'sea'])
+    assert t2.equals(expected)
+
+
 def test_table_flatten():
     ty1 = pa.struct([pa.field('x', pa.int16()),
                      pa.field('y', pa.float32())])