You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2021/04/05 10:33:03 UTC

[GitHub] [parquet-cpp] pitrou commented on a change in pull request #158: PARQUET-712: Add library to read into Arrow memory

pitrou commented on a change in pull request #158:
URL: https://github.com/apache/parquet-cpp/pull/158#discussion_r607025397



##########
File path: src/parquet/arrow/schema.cc
##########
@@ -0,0 +1,351 @@
+// 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 "parquet/arrow/schema.h"
+
+#include <string>
+#include <vector>
+
+#include "parquet/api/schema.h"
+#include "parquet/arrow/utils.h"
+
+#include "arrow/types/decimal.h"
+#include "arrow/types/string.h"
+#include "arrow/util/status.h"
+
+using arrow::Field;
+using arrow::Status;
+using arrow::TypePtr;
+
+using ArrowType = arrow::Type;
+
+using parquet::Repetition;
+using parquet::schema::Node;
+using parquet::schema::NodePtr;
+using parquet::schema::GroupNode;
+using parquet::schema::PrimitiveNode;
+
+using ParquetType = parquet::Type;
+using parquet::LogicalType;
+
+namespace parquet {
+
+namespace arrow {
+
+const auto BOOL = std::make_shared<::arrow::BooleanType>();
+const auto UINT8 = std::make_shared<::arrow::UInt8Type>();
+const auto INT8 = std::make_shared<::arrow::Int8Type>();
+const auto UINT16 = std::make_shared<::arrow::UInt16Type>();
+const auto INT16 = std::make_shared<::arrow::Int16Type>();
+const auto UINT32 = std::make_shared<::arrow::UInt32Type>();
+const auto INT32 = std::make_shared<::arrow::Int32Type>();
+const auto UINT64 = std::make_shared<::arrow::UInt64Type>();
+const auto INT64 = std::make_shared<::arrow::Int64Type>();
+const auto FLOAT = std::make_shared<::arrow::FloatType>();
+const auto DOUBLE = std::make_shared<::arrow::DoubleType>();
+const auto UTF8 = std::make_shared<::arrow::StringType>();
+const auto TIMESTAMP_MS =
+    std::make_shared<::arrow::TimestampType>(::arrow::TimestampType::Unit::MILLI);
+const auto BINARY =
+    std::make_shared<::arrow::ListType>(std::make_shared<::arrow::Field>("", UINT8));
+
+TypePtr MakeDecimalType(const PrimitiveNode* node) {
+  int precision = node->decimal_metadata().precision;
+  int scale = node->decimal_metadata().scale;
+  return std::make_shared<::arrow::DecimalType>(precision, scale);
+}
+
+static Status FromByteArray(const PrimitiveNode* node, TypePtr* out) {
+  switch (node->logical_type()) {
+    case LogicalType::UTF8:
+      *out = UTF8;
+      break;
+    case LogicalType::DECIMAL:
+      *out = MakeDecimalType(node);
+      break;
+    default:
+      // BINARY
+      *out = BINARY;
+      break;
+  }
+  return Status::OK();
+}
+
+static Status FromFLBA(const PrimitiveNode* node, TypePtr* out) {
+  switch (node->logical_type()) {
+    case LogicalType::NONE:
+      *out = BINARY;
+      break;
+    case LogicalType::DECIMAL:
+      *out = MakeDecimalType(node);
+      break;
+    default:
+      return Status::NotImplemented("unhandled type");
+      break;
+  }
+
+  return Status::OK();
+}
+
+static Status FromInt32(const PrimitiveNode* node, TypePtr* out) {
+  switch (node->logical_type()) {
+    case LogicalType::NONE:
+      *out = INT32;
+      break;
+    case LogicalType::UINT_8:
+      *out = UINT8;
+      break;
+    case LogicalType::INT_8:
+      *out = INT8;
+      break;
+    case LogicalType::UINT_16:
+      *out = UINT16;
+      break;
+    case LogicalType::INT_16:
+      *out = INT16;
+      break;
+    case LogicalType::UINT_32:
+      *out = UINT32;
+      break;
+    case LogicalType::DECIMAL:
+      *out = MakeDecimalType(node);
+      break;
+    default:
+      return Status::NotImplemented("Unhandled logical type for int32");
+      break;
+  }
+  return Status::OK();
+}
+
+static Status FromInt64(const PrimitiveNode* node, TypePtr* out) {
+  switch (node->logical_type()) {
+    case LogicalType::NONE:
+      *out = INT64;
+      break;
+    case LogicalType::UINT_64:
+      *out = UINT64;
+      break;
+    case LogicalType::DECIMAL:
+      *out = MakeDecimalType(node);
+      break;
+    case LogicalType::TIMESTAMP_MILLIS:
+      *out = TIMESTAMP_MS;
+      break;
+    default:
+      return Status::NotImplemented("Unhandled logical type for int64");
+      break;
+  }
+  return Status::OK();
+}
+
+// TODO: Logical Type Handling
+Status NodeToField(const NodePtr& node, std::shared_ptr<Field>* out) {
+  std::shared_ptr<::arrow::DataType> type;
+
+  if (node->is_repeated()) {
+    return Status::NotImplemented("No support yet for repeated node types");
+  }
+
+  if (node->is_group()) {
+    const GroupNode* group = static_cast<const GroupNode*>(node.get());
+    std::vector<std::shared_ptr<Field>> fields(group->field_count());
+    for (int i = 0; i < group->field_count(); i++) {
+      RETURN_NOT_OK(NodeToField(group->field(i), &fields[i]));
+    }
+    type = std::make_shared<::arrow::StructType>(fields);
+  } else {
+    // Primitive (leaf) node
+    const PrimitiveNode* primitive = static_cast<const PrimitiveNode*>(node.get());
+
+    switch (primitive->physical_type()) {
+      case ParquetType::BOOLEAN:
+        type = BOOL;
+        break;
+      case ParquetType::INT32:
+        RETURN_NOT_OK(FromInt32(primitive, &type));
+        break;
+      case ParquetType::INT64:
+        RETURN_NOT_OK(FromInt64(primitive, &type));
+        break;
+      case ParquetType::INT96:
+        // TODO: Do we have that type in Arrow?
+        // type = TypePtr(new Int96Type());
+        return Status::NotImplemented("int96");
+      case ParquetType::FLOAT:
+        type = FLOAT;
+        break;
+      case ParquetType::DOUBLE:
+        type = DOUBLE;
+        break;
+      case ParquetType::BYTE_ARRAY:
+        // TODO: Do we have that type in Arrow?
+        RETURN_NOT_OK(FromByteArray(primitive, &type));
+        break;
+      case ParquetType::FIXED_LEN_BYTE_ARRAY:
+        RETURN_NOT_OK(FromFLBA(primitive, &type));
+        break;
+    }
+  }
+
+  *out = std::make_shared<Field>(node->name(), type, !node->is_required());
+  return Status::OK();
+}
+
+Status FromParquetSchema(
+    const SchemaDescriptor* parquet_schema, std::shared_ptr<::arrow::Schema>* out) {
+  // TODO(wesm): Consider adding an arrow::Schema name attribute, which comes
+  // from the root Parquet node
+  const GroupNode* schema_node = parquet_schema->group_node();
+
+  std::vector<std::shared_ptr<Field>> fields(schema_node->field_count());
+  for (int i = 0; i < schema_node->field_count(); i++) {
+    RETURN_NOT_OK(NodeToField(schema_node->field(i), &fields[i]));
+  }
+
+  *out = std::make_shared<::arrow::Schema>(fields);
+  return Status::OK();
+}
+
+Status StructToNode(const std::shared_ptr<::arrow::StructType>& type,
+    const std::string& name, bool nullable, const WriterProperties& properties,
+    NodePtr* out) {
+  Repetition::type repetition = Repetition::REQUIRED;
+  if (nullable) { repetition = Repetition::OPTIONAL; }
+
+  std::vector<NodePtr> children(type->num_children());
+  for (int i = 0; i < type->num_children(); i++) {
+    RETURN_NOT_OK(FieldToNode(type->child(i), properties, &children[i]));
+  }
+
+  *out = GroupNode::Make(name, repetition, children);
+  return Status::OK();
+}
+
+Status FieldToNode(const std::shared_ptr<Field>& field,
+    const WriterProperties& properties, NodePtr* out) {
+  LogicalType::type logical_type = LogicalType::NONE;
+  ParquetType::type type;
+  Repetition::type repetition = Repetition::REQUIRED;
+  if (field->nullable) { repetition = Repetition::OPTIONAL; }
+  int length = -1;
+
+  switch (field->type->type) {
+    // TODO:
+    // case ArrowType::NA:
+    // break;
+    case ArrowType::BOOL:
+      type = ParquetType::BOOLEAN;
+      break;
+    case ArrowType::UINT8:
+      type = ParquetType::INT32;
+      logical_type = LogicalType::UINT_8;
+      break;
+    case ArrowType::INT8:
+      type = ParquetType::INT32;
+      logical_type = LogicalType::INT_8;
+      break;
+    case ArrowType::UINT16:
+      type = ParquetType::INT32;
+      logical_type = LogicalType::UINT_16;
+      break;
+    case ArrowType::INT16:
+      type = ParquetType::INT32;
+      logical_type = LogicalType::INT_16;
+      break;
+    case ArrowType::UINT32:
+      if (properties.version() == ::parquet::ParquetVersion::PARQUET_1_0) {
+        type = ParquetType::INT64;

Review comment:
       @xhochy Can you explain why UINT_32 is not emitted for Parquet version 1.0? There's zero comment and I can't find any explanation otherwise.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org