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 2016/03/05 02:59:30 UTC

arrow git commit: ARROW-24: C++: Implement a logical Table container type

Repository: arrow
Updated Branches:
  refs/heads/master 9c2b95446 -> 612fbc74e


ARROW-24: C++: Implement a logical Table container type

A table enables us to interpret a collection of Arrow arrays as a logical table or "data frame"-like structure.  Each column may consist of one or more "primitive" Arrow memory containers.

Note that this currently has the limitation that the table column names must be strings. At least, this is consistent with most storage media and up-stack table implementations (e.g. R's data.frame).

Currently this is somewhat limited in the arrangement of data (a vector of chunked columns -- the columns may contain only one data chunk) -- since a Table might be assembled from a vector of row batches (coming across the wire), "pivoting" the row batches might have performance implications that we can examine further on down the road.

Author: Wes McKinney <we...@apache.org>

Closes #16 from wesm/ARROW-24 and squashes the following commits:

b701c76 [Wes McKinney] Test case for wrong number of columns passed
5faa5ac [Wes McKinney] cpplint
9a651cb [Wes McKinney] Basic table prototype. Move Schema code under arrow/table


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

Branch: refs/heads/master
Commit: 612fbc74ece160a52edbd260de8391aa07ad00ca
Parents: 9c2b954
Author: Wes McKinney <we...@apache.org>
Authored: Fri Mar 4 17:59:58 2016 -0800
Committer: Wes McKinney <we...@apache.org>
Committed: Fri Mar 4 17:59:58 2016 -0800

----------------------------------------------------------------------
 cpp/CMakeLists.txt                 |   1 -
 cpp/src/arrow/CMakeLists.txt       |   1 -
 cpp/src/arrow/schema-test.cc       | 109 ----------------------------
 cpp/src/arrow/schema.cc            |  58 ---------------
 cpp/src/arrow/schema.h             |  56 --------------
 cpp/src/arrow/table/CMakeLists.txt |   4 +
 cpp/src/arrow/table/column-test.cc |  37 +++-------
 cpp/src/arrow/table/column.cc      |   6 ++
 cpp/src/arrow/table/column.h       |   2 +
 cpp/src/arrow/table/schema-test.cc | 109 ++++++++++++++++++++++++++++
 cpp/src/arrow/table/schema.cc      |  58 +++++++++++++++
 cpp/src/arrow/table/schema.h       |  56 ++++++++++++++
 cpp/src/arrow/table/table-test.cc  | 125 ++++++++++++++++++++++++++++++++
 cpp/src/arrow/table/table.cc       |  73 +++++++++++++++++++
 cpp/src/arrow/table/table.h        |  82 +++++++++++++++++++++
 cpp/src/arrow/table/test-common.h  |  55 ++++++++++++++
 16 files changed, 579 insertions(+), 253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index 15afb1a..8042661 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -470,7 +470,6 @@ set(ARROW_SRCS
   src/arrow/array.cc
   src/arrow/builder.cc
   src/arrow/field.cc
-  src/arrow/schema.cc
   src/arrow/type.cc
 )
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index 04f8dd1..77326ce 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -30,4 +30,3 @@ install(FILES
 set(ARROW_TEST_LINK_LIBS arrow_test_util ${ARROW_MIN_TEST_LIBS})
 
 ADD_ARROW_TEST(array-test)
-ADD_ARROW_TEST(schema-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/schema-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/schema-test.cc b/cpp/src/arrow/schema-test.cc
deleted file mode 100644
index 7c190d0..0000000
--- a/cpp/src/arrow/schema-test.cc
+++ /dev/null
@@ -1,109 +0,0 @@
-// 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 <gtest/gtest.h>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "arrow/field.h"
-#include "arrow/schema.h"
-#include "arrow/type.h"
-#include "arrow/types/string.h"
-
-using std::shared_ptr;
-using std::vector;
-
-namespace arrow {
-
-TEST(TestField, Basics) {
-  shared_ptr<DataType> ftype = INT32;
-  shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
-  Field f0("f0", ftype);
-  Field f0_nn("f0", ftype_nn);
-
-  ASSERT_EQ(f0.name, "f0");
-  ASSERT_EQ(f0.type->ToString(), ftype->ToString());
-
-  ASSERT_TRUE(f0.nullable());
-  ASSERT_FALSE(f0_nn.nullable());
-}
-
-TEST(TestField, Equals) {
-  shared_ptr<DataType> ftype = INT32;
-  shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
-
-  Field f0("f0", ftype);
-  Field f0_nn("f0", ftype_nn);
-  Field f0_other("f0", ftype);
-
-  ASSERT_EQ(f0, f0_other);
-  ASSERT_NE(f0, f0_nn);
-}
-
-class TestSchema : public ::testing::Test {
- public:
-  void SetUp() {}
-};
-
-TEST_F(TestSchema, Basics) {
-  auto f0 = std::make_shared<Field>("f0", INT32);
-  auto f1 = std::make_shared<Field>("f1", std::make_shared<UInt8Type>(false));
-  auto f1_optional = std::make_shared<Field>("f1", std::make_shared<UInt8Type>());
-
-  auto f2 = std::make_shared<Field>("f2", std::make_shared<StringType>());
-
-  vector<shared_ptr<Field> > fields = {f0, f1, f2};
-  auto schema = std::make_shared<Schema>(fields);
-
-  ASSERT_EQ(3, schema->num_fields());
-  ASSERT_EQ(f0, schema->field(0));
-  ASSERT_EQ(f1, schema->field(1));
-  ASSERT_EQ(f2, schema->field(2));
-
-  auto schema2 = std::make_shared<Schema>(fields);
-
-  vector<shared_ptr<Field> > fields3 = {f0, f1_optional, f2};
-  auto schema3 = std::make_shared<Schema>(fields3);
-  ASSERT_TRUE(schema->Equals(schema2));
-  ASSERT_FALSE(schema->Equals(schema3));
-
-  ASSERT_TRUE(schema->Equals(*schema2.get()));
-  ASSERT_FALSE(schema->Equals(*schema3.get()));
-}
-
-TEST_F(TestSchema, ToString) {
-  auto f0 = std::make_shared<Field>("f0", std::make_shared<Int32Type>());
-  auto f1 = std::make_shared<Field>("f1", std::make_shared<UInt8Type>(false));
-  auto f2 = std::make_shared<Field>("f2", std::make_shared<StringType>());
-  auto f3 = std::make_shared<Field>("f3",
-      std::make_shared<ListType>(std::make_shared<Int16Type>()));
-
-  vector<shared_ptr<Field> > fields = {f0, f1, f2, f3};
-  auto schema = std::make_shared<Schema>(fields);
-
-  std::string result = schema->ToString();
-  std::string expected = R"(f0 ?int32
-f1 uint8
-f2 ?string
-f3 ?list<?int16>
-)";
-
-  ASSERT_EQ(expected, result);
-}
-
-} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/schema.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/schema.cc b/cpp/src/arrow/schema.cc
deleted file mode 100644
index a735fd3..0000000
--- a/cpp/src/arrow/schema.cc
+++ /dev/null
@@ -1,58 +0,0 @@
-// 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/schema.h"
-
-#include <memory>
-#include <string>
-#include <sstream>
-#include <vector>
-
-#include "arrow/field.h"
-
-namespace arrow {
-
-Schema::Schema(const std::vector<std::shared_ptr<Field> >& fields) :
-    fields_(fields) {}
-
-bool Schema::Equals(const Schema& other) const {
-  if (this == &other) return true;
-  if (num_fields() != other.num_fields()) {
-    return false;
-  }
-  for (int i = 0; i < num_fields(); ++i) {
-    if (!field(i)->Equals(*other.field(i).get())) {
-      return false;
-    }
-  }
-  return true;
-}
-
-bool Schema::Equals(const std::shared_ptr<Schema>& other) const {
-  return Equals(*other.get());
-}
-
-std::string Schema::ToString() const {
-  std::stringstream buffer;
-
-  for (auto field : fields_) {
-    buffer << field->ToString() << std::endl;
-  }
-  return buffer.str();
-}
-
-} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/schema.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/schema.h b/cpp/src/arrow/schema.h
deleted file mode 100644
index d04e3f6..0000000
--- a/cpp/src/arrow/schema.h
+++ /dev/null
@@ -1,56 +0,0 @@
-// 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.
-
-#ifndef ARROW_SCHEMA_H
-#define ARROW_SCHEMA_H
-
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "arrow/field.h"
-#include "arrow/type.h"
-
-namespace arrow {
-
-class Schema {
- public:
-  explicit Schema(const std::vector<std::shared_ptr<Field> >& fields);
-
-  // Returns true if all of the schema fields are equal
-  bool Equals(const Schema& other) const;
-  bool Equals(const std::shared_ptr<Schema>& other) const;
-
-  // Return the ith schema element. Does not boundscheck
-  const std::shared_ptr<Field>& field(int i) const {
-    return fields_[i];
-  }
-
-  // Render a string representation of the schema suitable for debugging
-  std::string ToString() const;
-
-  int num_fields() const {
-    return fields_.size();
-  }
-
- private:
-  std::vector<std::shared_ptr<Field> > fields_;
-};
-
-} // namespace arrow
-
-#endif  // ARROW_FIELD_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/CMakeLists.txt b/cpp/src/arrow/table/CMakeLists.txt
index a401622..b51258f 100644
--- a/cpp/src/arrow/table/CMakeLists.txt
+++ b/cpp/src/arrow/table/CMakeLists.txt
@@ -21,6 +21,8 @@
 
 set(TABLE_SRCS
   column.cc
+  schema.cc
+  table.cc
 )
 
 set(TABLE_LIBS
@@ -37,3 +39,5 @@ install(FILES
   DESTINATION include/arrow/table)
 
 ADD_ARROW_TEST(column-test)
+ADD_ARROW_TEST(schema-test)
+ADD_ARROW_TEST(table-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/column-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column-test.cc b/cpp/src/arrow/table/column-test.cc
index 15f554f..4959b82 100644
--- a/cpp/src/arrow/table/column-test.cc
+++ b/cpp/src/arrow/table/column-test.cc
@@ -22,48 +22,29 @@
 #include <vector>
 
 #include "arrow/field.h"
-#include "arrow/schema.h"
 #include "arrow/table/column.h"
+#include "arrow/table/schema.h"
+#include "arrow/table/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 #include "arrow/types/integer.h"
-#include "arrow/util/bit-util.h"
-#include "arrow/util/buffer.h"
-#include "arrow/util/memory-pool.h"
-#include "arrow/util/status.h"
 
 using std::shared_ptr;
 using std::vector;
 
 namespace arrow {
 
-class TestColumn : public ::testing::Test {
- public:
-  void SetUp() {
-    pool_ = GetDefaultMemoryPool();
-  }
-
-  template <typename ArrayType>
-  std::shared_ptr<Array> MakeArray(int32_t length, int32_t null_count = 0) {
-    auto data = std::make_shared<PoolBuffer>(pool_);
-    auto nulls = std::make_shared<PoolBuffer>(pool_);
-    data->Resize(length * sizeof(typename ArrayType::value_type));
-    nulls->Resize(util::bytes_for_bits(length));
-    return std::make_shared<ArrayType>(length, data, 10, nulls);
-  }
-
+class TestColumn : public TestBase {
  protected:
-  MemoryPool* pool_;
-
   std::shared_ptr<ChunkedArray> data_;
   std::unique_ptr<Column> column_;
 };
 
 TEST_F(TestColumn, BasicAPI) {
   ArrayVector arrays;
-  arrays.push_back(MakeArray<Int32Array>(100));
-  arrays.push_back(MakeArray<Int32Array>(100, 10));
-  arrays.push_back(MakeArray<Int32Array>(100, 20));
+  arrays.push_back(MakePrimitive<Int32Array>(100));
+  arrays.push_back(MakePrimitive<Int32Array>(100, 10));
+  arrays.push_back(MakePrimitive<Int32Array>(100, 20));
 
   auto field = std::make_shared<Field>("c0", INT32);
   column_.reset(new Column(field, arrays));
@@ -77,15 +58,15 @@ TEST_F(TestColumn, BasicAPI) {
 
 TEST_F(TestColumn, ChunksInhomogeneous) {
   ArrayVector arrays;
-  arrays.push_back(MakeArray<Int32Array>(100));
-  arrays.push_back(MakeArray<Int32Array>(100, 10));
+  arrays.push_back(MakePrimitive<Int32Array>(100));
+  arrays.push_back(MakePrimitive<Int32Array>(100, 10));
 
   auto field = std::make_shared<Field>("c0", INT32);
   column_.reset(new Column(field, arrays));
 
   ASSERT_OK(column_->ValidateData());
 
-  arrays.push_back(MakeArray<Int16Array>(100, 10));
+  arrays.push_back(MakePrimitive<Int16Array>(100, 10));
   column_.reset(new Column(field, arrays));
   ASSERT_RAISES(Invalid, column_->ValidateData());
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/column.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column.cc b/cpp/src/arrow/table/column.cc
index 82750cf..d68b491 100644
--- a/cpp/src/arrow/table/column.cc
+++ b/cpp/src/arrow/table/column.cc
@@ -40,6 +40,12 @@ Column::Column(const std::shared_ptr<Field>& field, const ArrayVector& chunks) :
 }
 
 Column::Column(const std::shared_ptr<Field>& field,
+    const std::shared_ptr<Array>& data) :
+    field_(field) {
+  data_ = std::make_shared<ChunkedArray>(ArrayVector({data}));
+}
+
+Column::Column(const std::shared_ptr<Field>& field,
     const std::shared_ptr<ChunkedArray>& data) :
     field_(field),
     data_(data) {}

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/column.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column.h b/cpp/src/arrow/table/column.h
index 9e9064e..64423bf 100644
--- a/cpp/src/arrow/table/column.h
+++ b/cpp/src/arrow/table/column.h
@@ -67,6 +67,8 @@ class Column {
   Column(const std::shared_ptr<Field>& field,
       const std::shared_ptr<ChunkedArray>& data);
 
+  Column(const std::shared_ptr<Field>& field, const std::shared_ptr<Array>& data);
+
   int64_t length() const {
     return data_->length();
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/schema-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/schema-test.cc b/cpp/src/arrow/table/schema-test.cc
new file mode 100644
index 0000000..0cf1b3c
--- /dev/null
+++ b/cpp/src/arrow/table/schema-test.cc
@@ -0,0 +1,109 @@
+// 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 <gtest/gtest.h>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/field.h"
+#include "arrow/table/schema.h"
+#include "arrow/type.h"
+#include "arrow/types/string.h"
+
+using std::shared_ptr;
+using std::vector;
+
+namespace arrow {
+
+TEST(TestField, Basics) {
+  shared_ptr<DataType> ftype = INT32;
+  shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
+  Field f0("f0", ftype);
+  Field f0_nn("f0", ftype_nn);
+
+  ASSERT_EQ(f0.name, "f0");
+  ASSERT_EQ(f0.type->ToString(), ftype->ToString());
+
+  ASSERT_TRUE(f0.nullable());
+  ASSERT_FALSE(f0_nn.nullable());
+}
+
+TEST(TestField, Equals) {
+  shared_ptr<DataType> ftype = INT32;
+  shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
+
+  Field f0("f0", ftype);
+  Field f0_nn("f0", ftype_nn);
+  Field f0_other("f0", ftype);
+
+  ASSERT_EQ(f0, f0_other);
+  ASSERT_NE(f0, f0_nn);
+}
+
+class TestSchema : public ::testing::Test {
+ public:
+  void SetUp() {}
+};
+
+TEST_F(TestSchema, Basics) {
+  auto f0 = std::make_shared<Field>("f0", INT32);
+  auto f1 = std::make_shared<Field>("f1", std::make_shared<UInt8Type>(false));
+  auto f1_optional = std::make_shared<Field>("f1", std::make_shared<UInt8Type>());
+
+  auto f2 = std::make_shared<Field>("f2", std::make_shared<StringType>());
+
+  vector<shared_ptr<Field> > fields = {f0, f1, f2};
+  auto schema = std::make_shared<Schema>(fields);
+
+  ASSERT_EQ(3, schema->num_fields());
+  ASSERT_EQ(f0, schema->field(0));
+  ASSERT_EQ(f1, schema->field(1));
+  ASSERT_EQ(f2, schema->field(2));
+
+  auto schema2 = std::make_shared<Schema>(fields);
+
+  vector<shared_ptr<Field> > fields3 = {f0, f1_optional, f2};
+  auto schema3 = std::make_shared<Schema>(fields3);
+  ASSERT_TRUE(schema->Equals(schema2));
+  ASSERT_FALSE(schema->Equals(schema3));
+
+  ASSERT_TRUE(schema->Equals(*schema2.get()));
+  ASSERT_FALSE(schema->Equals(*schema3.get()));
+}
+
+TEST_F(TestSchema, ToString) {
+  auto f0 = std::make_shared<Field>("f0", std::make_shared<Int32Type>());
+  auto f1 = std::make_shared<Field>("f1", std::make_shared<UInt8Type>(false));
+  auto f2 = std::make_shared<Field>("f2", std::make_shared<StringType>());
+  auto f3 = std::make_shared<Field>("f3",
+      std::make_shared<ListType>(std::make_shared<Int16Type>()));
+
+  vector<shared_ptr<Field> > fields = {f0, f1, f2, f3};
+  auto schema = std::make_shared<Schema>(fields);
+
+  std::string result = schema->ToString();
+  std::string expected = R"(f0 ?int32
+f1 uint8
+f2 ?string
+f3 ?list<?int16>
+)";
+
+  ASSERT_EQ(expected, result);
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/schema.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/schema.cc b/cpp/src/arrow/table/schema.cc
new file mode 100644
index 0000000..fb3b4d6
--- /dev/null
+++ b/cpp/src/arrow/table/schema.cc
@@ -0,0 +1,58 @@
+// 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/table/schema.h"
+
+#include <memory>
+#include <string>
+#include <sstream>
+#include <vector>
+
+#include "arrow/field.h"
+
+namespace arrow {
+
+Schema::Schema(const std::vector<std::shared_ptr<Field> >& fields) :
+    fields_(fields) {}
+
+bool Schema::Equals(const Schema& other) const {
+  if (this == &other) return true;
+  if (num_fields() != other.num_fields()) {
+    return false;
+  }
+  for (int i = 0; i < num_fields(); ++i) {
+    if (!field(i)->Equals(*other.field(i).get())) {
+      return false;
+    }
+  }
+  return true;
+}
+
+bool Schema::Equals(const std::shared_ptr<Schema>& other) const {
+  return Equals(*other.get());
+}
+
+std::string Schema::ToString() const {
+  std::stringstream buffer;
+
+  for (auto field : fields_) {
+    buffer << field->ToString() << std::endl;
+  }
+  return buffer.str();
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/schema.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/schema.h b/cpp/src/arrow/table/schema.h
new file mode 100644
index 0000000..d04e3f6
--- /dev/null
+++ b/cpp/src/arrow/table/schema.h
@@ -0,0 +1,56 @@
+// 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.
+
+#ifndef ARROW_SCHEMA_H
+#define ARROW_SCHEMA_H
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/field.h"
+#include "arrow/type.h"
+
+namespace arrow {
+
+class Schema {
+ public:
+  explicit Schema(const std::vector<std::shared_ptr<Field> >& fields);
+
+  // Returns true if all of the schema fields are equal
+  bool Equals(const Schema& other) const;
+  bool Equals(const std::shared_ptr<Schema>& other) const;
+
+  // Return the ith schema element. Does not boundscheck
+  const std::shared_ptr<Field>& field(int i) const {
+    return fields_[i];
+  }
+
+  // Render a string representation of the schema suitable for debugging
+  std::string ToString() const;
+
+  int num_fields() const {
+    return fields_.size();
+  }
+
+ private:
+  std::vector<std::shared_ptr<Field> > fields_;
+};
+
+} // namespace arrow
+
+#endif  // ARROW_FIELD_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/table-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/table-test.cc b/cpp/src/arrow/table/table-test.cc
new file mode 100644
index 0000000..dd4f74c
--- /dev/null
+++ b/cpp/src/arrow/table/table-test.cc
@@ -0,0 +1,125 @@
+// 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 <gtest/gtest.h>
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/field.h"
+#include "arrow/table/column.h"
+#include "arrow/table/schema.h"
+#include "arrow/table/table.h"
+#include "arrow/table/test-common.h"
+#include "arrow/test-util.h"
+#include "arrow/type.h"
+#include "arrow/types/integer.h"
+
+using std::shared_ptr;
+using std::vector;
+
+namespace arrow {
+
+class TestTable : public TestBase {
+ public:
+  void MakeExample1(int length) {
+    auto f0 = std::make_shared<Field>("f0", INT32);
+    auto f1 = std::make_shared<Field>("f1", UINT8);
+    auto f2 = std::make_shared<Field>("f2", INT16);
+
+    vector<shared_ptr<Field> > fields = {f0, f1, f2};
+    schema_ = std::make_shared<Schema>(fields);
+
+    columns_ = {
+      std::make_shared<Column>(schema_->field(0), MakePrimitive<Int32Array>(length)),
+      std::make_shared<Column>(schema_->field(1), MakePrimitive<UInt8Array>(length)),
+      std::make_shared<Column>(schema_->field(2), MakePrimitive<Int16Array>(length))
+    };
+  }
+
+ protected:
+  std::unique_ptr<Table> table_;
+  shared_ptr<Schema> schema_;
+  vector<std::shared_ptr<Column> > columns_;
+};
+
+TEST_F(TestTable, EmptySchema) {
+  auto empty_schema = shared_ptr<Schema>(new Schema({}));
+  table_.reset(new Table("data", empty_schema, columns_));
+  ASSERT_OK(table_->ValidateColumns());
+  ASSERT_EQ(0, table_->num_rows());
+  ASSERT_EQ(0, table_->num_columns());
+}
+
+TEST_F(TestTable, Ctors) {
+  int length = 100;
+  MakeExample1(length);
+
+  std::string name = "data";
+
+  table_.reset(new Table(name, schema_, columns_));
+  ASSERT_OK(table_->ValidateColumns());
+  ASSERT_EQ(name, table_->name());
+  ASSERT_EQ(length, table_->num_rows());
+  ASSERT_EQ(3, table_->num_columns());
+
+  table_.reset(new Table(name, schema_, columns_, length));
+  ASSERT_OK(table_->ValidateColumns());
+  ASSERT_EQ(name, table_->name());
+  ASSERT_EQ(length, table_->num_rows());
+}
+
+TEST_F(TestTable, Metadata) {
+  int length = 100;
+  MakeExample1(length);
+
+  std::string name = "data";
+  table_.reset(new Table(name, schema_, columns_));
+
+  ASSERT_TRUE(table_->schema()->Equals(schema_));
+
+  auto col = table_->column(0);
+  ASSERT_EQ(schema_->field(0)->name, col->name());
+  ASSERT_EQ(schema_->field(0)->type, col->type());
+}
+
+TEST_F(TestTable, InvalidColumns) {
+  // Check that columns are all the same length
+  int length = 100;
+  MakeExample1(length);
+
+  table_.reset(new Table("data", schema_, columns_, length - 1));
+  ASSERT_RAISES(Invalid, table_->ValidateColumns());
+
+  columns_.clear();
+
+  // Wrong number of columns
+  table_.reset(new Table("data", schema_, columns_, length));
+  ASSERT_RAISES(Invalid, table_->ValidateColumns());
+
+  columns_ = {
+    std::make_shared<Column>(schema_->field(0), MakePrimitive<Int32Array>(length)),
+    std::make_shared<Column>(schema_->field(1), MakePrimitive<UInt8Array>(length)),
+    std::make_shared<Column>(schema_->field(2), MakePrimitive<Int16Array>(length - 1))
+  };
+
+  table_.reset(new Table("data", schema_, columns_, length));
+  ASSERT_RAISES(Invalid, table_->ValidateColumns());
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/table.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/table.cc b/cpp/src/arrow/table/table.cc
new file mode 100644
index 0000000..4cefc92
--- /dev/null
+++ b/cpp/src/arrow/table/table.cc
@@ -0,0 +1,73 @@
+// 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/table/table.h"
+
+#include <memory>
+#include <sstream>
+
+#include "arrow/field.h"
+#include "arrow/table/column.h"
+#include "arrow/table/schema.h"
+#include "arrow/util/status.h"
+
+namespace arrow {
+
+Table::Table(const std::string& name, const std::shared_ptr<Schema>& schema,
+    const std::vector<std::shared_ptr<Column> >& columns) :
+    name_(name),
+    schema_(schema),
+    columns_(columns) {
+  if (columns.size() == 0) {
+    num_rows_ = 0;
+  } else {
+    num_rows_ = columns[0]->length();
+  }
+}
+
+Table::Table(const std::string& name, const std::shared_ptr<Schema>& schema,
+    const std::vector<std::shared_ptr<Column> >& columns, int64_t num_rows) :
+    name_(name),
+    schema_(schema),
+    columns_(columns),
+    num_rows_(num_rows) {}
+
+Status Table::ValidateColumns() const {
+  if (num_columns() != schema_->num_fields()) {
+    return Status::Invalid("Number of columns did not match schema");
+  }
+
+  if (columns_.size() == 0) {
+    return Status::OK();
+  }
+
+  // Make sure columns are all the same length
+  for (size_t i = 0; i < columns_.size(); ++i) {
+    const Column* col = columns_[i].get();
+    if (col->length() != num_rows_) {
+      std::stringstream ss;
+      ss << "Column " << i << " expected length "
+         << num_rows_
+         << " but got length "
+         << col->length();
+      return Status::Invalid(ss.str());
+    }
+  }
+  return Status::OK();
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/table.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/table.h b/cpp/src/arrow/table/table.h
new file mode 100644
index 0000000..b012938
--- /dev/null
+++ b/cpp/src/arrow/table/table.h
@@ -0,0 +1,82 @@
+// 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.
+
+#ifndef ARROW_TABLE_TABLE_H
+#define ARROW_TABLE_TABLE_H
+
+#include <memory>
+#include <string>
+#include <vector>
+
+namespace arrow {
+
+class Column;
+class Schema;
+class Status;
+
+// Immutable container of fixed-length columns conforming to a particular schema
+class Table {
+ public:
+  // If columns is zero-length, the table's number of rows is zero
+  Table(const std::string& name, const std::shared_ptr<Schema>& schema,
+      const std::vector<std::shared_ptr<Column> >& columns);
+
+  Table(const std::string& name, const std::shared_ptr<Schema>& schema,
+      const std::vector<std::shared_ptr<Column> >& columns, int64_t num_rows);
+
+  // @returns: the table's name, if any (may be length 0)
+  const std::string& name() const {
+    return name_;
+  }
+
+  // @returns: the table's schema
+  const std::shared_ptr<Schema>& schema() const {
+    return schema_;
+  }
+
+  // Note: Does not boundscheck
+  // @returns: the i-th column
+  const std::shared_ptr<Column>& column(int i) const {
+    return columns_[i];
+  }
+
+  // @returns: the number of columns in the table
+  int num_columns() const {
+    return columns_.size();
+  }
+
+  // @returns: the number of rows (the corresponding length of each column)
+  int64_t num_rows() const {
+    return num_rows_;
+  }
+
+  // After construction, perform any checks to validate the input arguments
+  Status ValidateColumns() const;
+
+ private:
+  // The table's name, optional
+  std::string name_;
+
+  std::shared_ptr<Schema> schema_;
+  std::vector<std::shared_ptr<Column> > columns_;
+
+  int64_t num_rows_;
+};
+
+} // namespace arrow
+
+#endif  // ARROW_TABLE_TABLE_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/612fbc74/cpp/src/arrow/table/test-common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/test-common.h b/cpp/src/arrow/table/test-common.h
new file mode 100644
index 0000000..efe2f22
--- /dev/null
+++ b/cpp/src/arrow/table/test-common.h
@@ -0,0 +1,55 @@
+// 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 <gtest/gtest.h>
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/field.h"
+#include "arrow/table/column.h"
+#include "arrow/table/schema.h"
+#include "arrow/table/table.h"
+#include "arrow/test-util.h"
+#include "arrow/type.h"
+#include "arrow/util/bit-util.h"
+#include "arrow/util/buffer.h"
+#include "arrow/util/memory-pool.h"
+
+namespace arrow {
+
+class TestBase : public ::testing::Test {
+ public:
+  void SetUp() {
+    pool_ = GetDefaultMemoryPool();
+  }
+
+  template <typename ArrayType>
+  std::shared_ptr<Array> MakePrimitive(int32_t length, int32_t null_count = 0) {
+    auto data = std::make_shared<PoolBuffer>(pool_);
+    auto nulls = std::make_shared<PoolBuffer>(pool_);
+    EXPECT_OK(data->Resize(length * sizeof(typename ArrayType::value_type)));
+    EXPECT_OK(nulls->Resize(util::bytes_for_bits(length)));
+    return std::make_shared<ArrayType>(length, data, 10, nulls);
+  }
+
+ protected:
+  MemoryPool* pool_;
+};
+
+} // namespace arrow