You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by uw...@apache.org on 2018/01/28 15:59:47 UTC

[arrow] branch master updated: ARROW-1835: [C++] Create Arrow schema from std::tuple types

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

uwe 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 450bf47  ARROW-1835: [C++] Create Arrow schema from std::tuple types
450bf47 is described below

commit 450bf474f5add5f0ab09008a3057d1b57811ad6b
Author: Korn, Uwe <Uw...@blue-yonder.com>
AuthorDate: Sun Jan 28 16:59:41 2018 +0100

    ARROW-1835: [C++] Create Arrow schema from std::tuple types
    
    Author: Korn, Uwe <Uw...@blue-yonder.com>
    
    Closes #1478 from xhochy/ARROW-1835 and squashes the following commits:
    
    9728740 [Korn, Uwe] Remove documentation tag that conflicts with gcc
    a690248 [Korn, Uwe] ARROW-1835: [C++] Create Arrow schema from std::tuple types
---
 cpp/src/arrow/CMakeLists.txt |   2 +
 cpp/src/arrow/stl-test.cc    |  78 ++++++++++++++++++++++
 cpp/src/arrow/stl.h          | 153 +++++++++++++++++++++++++++++++++++++++++++
 cpp/src/arrow/type.h         |  13 ++++
 4 files changed, 246 insertions(+)

diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index ad86256..74674be 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -153,6 +153,7 @@ install(FILES
   pretty_print.h
   record_batch.h
   status.h
+  stl.h
   table.h
   table_builder.h
   tensor.h
@@ -183,6 +184,7 @@ ADD_ARROW_TEST(memory_pool-test)
 ADD_ARROW_TEST(pretty_print-test)
 ADD_ARROW_TEST(public-api-test)
 ADD_ARROW_TEST(status-test)
+ADD_ARROW_TEST(stl-test)
 ADD_ARROW_TEST(type-test)
 ADD_ARROW_TEST(table-test)
 ADD_ARROW_TEST(table_builder-test)
diff --git a/cpp/src/arrow/stl-test.cc b/cpp/src/arrow/stl-test.cc
new file mode 100644
index 0000000..c85baa3
--- /dev/null
+++ b/cpp/src/arrow/stl-test.cc
@@ -0,0 +1,78 @@
+// 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 "arrow/stl.h"
+
+namespace arrow {
+namespace stl {
+
+TEST(TestSchemaFromTuple, PrimitiveTypesVector) {
+  Schema expected_schema(
+      {field("column1", int8(), false), field("column2", int16(), false),
+       field("column3", int32(), false), field("column4", int64(), false),
+       field("column5", uint8(), false), field("column6", uint16(), false),
+       field("column7", uint32(), false), field("column8", uint64(), false),
+       field("column9", boolean(), false), field("column10", utf8(), false)});
+
+  std::shared_ptr<Schema> schema =
+      SchemaFromTuple<std::tuple<int8_t, int16_t, int32_t, int64_t, uint8_t, uint16_t,
+                                 uint32_t, uint64_t, bool, std::string>>::
+          MakeSchema(std::vector<std::string>({"column1", "column2", "column3", "column4",
+                                               "column5", "column6", "column7", "column8",
+                                               "column9", "column10"}));
+  ASSERT_TRUE(expected_schema.Equals(*schema));
+}
+
+TEST(TestSchemaFromTuple, PrimitiveTypesTuple) {
+  Schema expected_schema(
+      {field("column1", int8(), false), field("column2", int16(), false),
+       field("column3", int32(), false), field("column4", int64(), false),
+       field("column5", uint8(), false), field("column6", uint16(), false),
+       field("column7", uint32(), false), field("column8", uint64(), false),
+       field("column9", boolean(), false), field("column10", utf8(), false)});
+
+  std::shared_ptr<Schema> schema = SchemaFromTuple<
+      std::tuple<int8_t, int16_t, int32_t, int64_t, uint8_t, uint16_t, uint32_t, uint64_t,
+                 bool, std::string>>::MakeSchema(std::make_tuple("column1", "column2",
+                                                                 "column3", "column4",
+                                                                 "column5", "column6",
+                                                                 "column7", "column8",
+                                                                 "column9", "column10"));
+  ASSERT_TRUE(expected_schema.Equals(*schema));
+}
+
+TEST(TestSchemaFromTuple, SimpleList) {
+  Schema expected_schema({field("column1", list(utf8()), false)});
+  std::shared_ptr<Schema> schema =
+      SchemaFromTuple<std::tuple<std::vector<std::string>>>::MakeSchema({"column1"});
+
+  ASSERT_TRUE(expected_schema.Equals(*schema));
+}
+
+TEST(TestSchemaFromTuple, NestedList) {
+  Schema expected_schema({field("column1", list(list(boolean())), false)});
+  std::shared_ptr<Schema> schema =
+      SchemaFromTuple<std::tuple<std::vector<std::vector<bool>>>>::MakeSchema(
+          {"column1"});
+
+  ASSERT_TRUE(expected_schema.Equals(*schema));
+}
+
+}  // namespace stl
+}  // namespace arrow
diff --git a/cpp/src/arrow/stl.h b/cpp/src/arrow/stl.h
new file mode 100644
index 0000000..3250b5a
--- /dev/null
+++ b/cpp/src/arrow/stl.h
@@ -0,0 +1,153 @@
+// 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_STL_H
+#define ARROW_STL_H
+
+#include <string>
+#include <tuple>
+#include <vector>
+
+#include "arrow/type.h"
+
+namespace arrow {
+
+class Schema;
+
+namespace stl {
+
+/// Traits meta class to map standard C/C++ types to equivalent Arrow types.
+template <typename T>
+struct ConversionTraits {};
+
+#define ARROW_STL_CONVERSION(c_type, ArrowType_) \
+  template <>                                    \
+  struct ConversionTraits<c_type> {              \
+    using ArrowType = ArrowType_;                \
+    constexpr static bool nullable = false;      \
+  };
+
+ARROW_STL_CONVERSION(bool, BooleanType)
+ARROW_STL_CONVERSION(int8_t, Int8Type)
+ARROW_STL_CONVERSION(int16_t, Int16Type)
+ARROW_STL_CONVERSION(int32_t, Int32Type)
+ARROW_STL_CONVERSION(int64_t, Int64Type)
+ARROW_STL_CONVERSION(uint8_t, UInt8Type)
+ARROW_STL_CONVERSION(uint16_t, UInt16Type)
+ARROW_STL_CONVERSION(uint32_t, UInt32Type)
+ARROW_STL_CONVERSION(uint64_t, UInt64Type)
+ARROW_STL_CONVERSION(float, FloatType)
+ARROW_STL_CONVERSION(double, DoubleType)
+ARROW_STL_CONVERSION(std::string, StringType)
+
+template <typename value_c_type>
+struct ConversionTraits<std::vector<value_c_type>> {
+  using ArrowType = meta::ListType<typename ConversionTraits<value_c_type>::ArrowType>;
+  constexpr static bool nullable = false;
+};
+
+/// Build an arrow::Schema based upon the types defined in a std::tuple-like structure.
+///
+/// While the type information is available at compile-time, we still need to add the
+/// column names at runtime, thus these methods are not constexpr.
+template <typename Tuple, std::size_t N = std::tuple_size<Tuple>::value>
+struct SchemaFromTuple {
+  using Element = typename std::tuple_element<N - 1, Tuple>::type;
+  using ArrowType = typename ConversionTraits<Element>::ArrowType;
+
+  // Implementations that take a vector-like object for the column names.
+
+  /// Recursively build a vector of arrow::Field from the defined types.
+  ///
+  /// In most cases MakeSchema is the better entrypoint for the Schema creation.
+  static std::vector<std::shared_ptr<Field>> MakeSchemaRecursion(
+      const std::vector<std::string>& names) {
+    std::vector<std::shared_ptr<Field>> ret =
+        SchemaFromTuple<Tuple, N - 1>::MakeSchemaRecursion(names);
+    ret.push_back(field(names[N - 1], std::make_shared<ArrowType>(),
+                        ConversionTraits<Element>::nullable));
+    return ret;
+  }
+
+  /// Build a Schema from the types of the tuple-like structure passed in as template
+  /// parameter assign the column names at runtime.
+  ///
+  /// An example usage of this API can look like the following:
+  ///
+  /// \code{.cpp}
+  /// using TupleType = std::tuple<int, std::vector<std::string>>;
+  /// std::shared_ptr<Schema> schema =
+  ///   SchemaFromTuple<TupleType>::MakeSchema({"int_column", "list_of_strings_column"});
+  /// \endcode
+  static std::shared_ptr<Schema> MakeSchema(const std::vector<std::string>& names) {
+    return std::make_shared<Schema>(MakeSchemaRecursion(names));
+  }
+
+  // Implementations that take a tuple-like object for the column names.
+
+  /// Recursively build a vector of arrow::Field from the defined types.
+  ///
+  /// In most cases MakeSchema is the better entrypoint for the Schema creation.
+  template <typename NamesTuple>
+  static std::vector<std::shared_ptr<Field>> MakeSchemaRecursionT(
+      const NamesTuple& names) {
+    std::vector<std::shared_ptr<Field>> ret =
+        SchemaFromTuple<Tuple, N - 1>::MakeSchemaRecursionT(names);
+    ret.push_back(field(std::get<N - 1>(names), std::make_shared<ArrowType>(),
+                        ConversionTraits<Element>::nullable));
+    return ret;
+  }
+
+  /// Build a Schema from the types of the tuple-like structure passed in as template
+  /// parameter assign the column names at runtime.
+  ///
+  /// An example usage of this API can look like the following:
+  ///
+  /// \code{.cpp}
+  /// using TupleType = std::tuple<int, std::vector<std::string>>;
+  /// std::shared_ptr<Schema> schema =
+  ///   SchemaFromTuple<TupleType>::MakeSchema({"int_column", "list_of_strings_column"});
+  /// \endcode
+  template <typename NamesTuple>
+  static std::shared_ptr<Schema> MakeSchema(const NamesTuple& names) {
+    return std::make_shared<Schema>(MakeSchemaRecursionT<NamesTuple>(names));
+  }
+};
+
+template <typename Tuple>
+struct SchemaFromTuple<Tuple, 0> {
+  static std::vector<std::shared_ptr<Field>> MakeSchemaRecursion(
+      const std::vector<std::string>& names) {
+    std::vector<std::shared_ptr<Field>> ret;
+    ret.reserve(names.size());
+    return ret;
+  }
+
+  template <typename NamesTuple>
+  static std::vector<std::shared_ptr<Field>> MakeSchemaRecursionT(
+      const NamesTuple& names) {
+    std::vector<std::shared_ptr<Field>> ret;
+    ret.reserve(std::tuple_size<NamesTuple>::value);
+    return ret;
+  }
+};
+/// @endcond
+
+}  // namespace stl
+}  // namespace arrow
+
+#endif  // ARROW_STL_H
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 009e07d..cfee6fd 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -407,6 +407,19 @@ class ARROW_EXPORT ListType : public NestedType {
   std::string name() const override { return "list"; }
 };
 
+namespace meta {
+
+/// Additional ListType class that can be instantiated with only compile-time arguments.
+template <typename T>
+class ARROW_EXPORT ListType : public ::arrow::ListType {
+ public:
+  using ValueType = T;
+
+  ListType() : ::arrow::ListType(std::make_shared<T>()) {}
+};
+
+}  // namespace meta
+
 // BinaryType type is represents lists of 1-byte values.
 class ARROW_EXPORT BinaryType : public DataType, public NoExtraMeta {
  public:

-- 
To stop receiving notification emails like this one, please contact
uwe@apache.org.