You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/05/10 17:55:06 UTC

[GitHub] [arrow] westonpace opened a new pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

westonpace opened a new pull request #10289:
URL: https://github.com/apache/arrow/pull/10289


   Questions:
   
   - This is my first PR in the parquet namespace, I'm not sure of all the special rules.
   - The field ID generation doesn't happen on the `parquet::schema` -> `arrow::schema` phase but on the `parquet::format::schema` -> `parquet::schema` phase.  So in order to test I had to add `#include "generated/parquet_types.h"` to `arrow_schema_test.cc` and I wasn't sure if I was allowed to reference the `generated/*` files like that.
   - This PR simply allows user specified field id's to be persisted.  Is that sufficient for PARQUET-1798 (the title is rather general) or should I open up a dedicated JIRA?


-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640045424



##########
File path: cpp/src/parquet/schema.h
##########
@@ -268,7 +268,7 @@ class PARQUET_EXPORT GroupNode : public Node {
  public:
   // The field_id here is the default to use if it is not set in the SchemaElement
   static std::unique_ptr<Node> FromParquet(const void* opaque_element,
-                                           NodeVector fields = {}, int field_id = -1);
+                                           NodeVector fields = {});

Review comment:
       I don't believe so.  There is no reason they can't create the node and then modify the `field_id` after the fact.  The purpose of `FromParquet` is to take the information in the `opaque_element` and put it into the `Node`.  Why allow the `field_id` to be overridden in this way but not allow it for `name` or `repetition`?




-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640051142



##########
File path: cpp/src/parquet/schema_test.cc
##########
@@ -171,17 +167,16 @@ TEST_F(TestPrimitiveNode, Attrs) {
 }
 
 TEST_F(TestPrimitiveNode, FromParquet) {
-  SchemaElement elt =
-      NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32, field_id_);
+  SchemaElement elt = NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32);

Review comment:
       No, the user does not need this capability.  See comment below.




-- 
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



[GitHub] [arrow] pitrou commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640374939



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }
+  arrow_fields.push_back(::arrow::field(
+      "nested",
+      ::arrow::struct_({::arrow::field("outer", ::arrow::struct_({::arrow::field(
+                                                    "inner", ::arrow::utf8())})),
+                        ::arrow::field("sibling", ::arrow::date32())}),
+      false));
+
+  ASSERT_OK(ConvertSchema(arrow_fields));
+  auto field_ids = GetFieldIdsDfs(result_schema_->fields());
+  for (int actual_id : field_ids) {
+    ASSERT_EQ(actual_id, -1);
+  }
+}
+
+std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) {
+  return ::arrow::key_value_metadata({"PARQUET:field_id"}, {std::to_string(field_id)});
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdPreserveExisting) {

Review comment:
       It doesn't seem to test that the `PARQUET:field_id` Arrow annotation ends up in the Parquet `field_id` member, does it?




-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640624863



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }

Review comment:
       Fixed.




-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-847214875


   I've rebased (and verified again that the build failures are unrelated).  Gentle ping for review @emkornfield / @pitrou 


-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640624255



##########
File path: python/pyarrow/tests/parquet/test_metadata.py
##########
@@ -303,28 +320,30 @@ def test_field_id_metadata():
     pf = pq.ParquetFile(pa.BufferReader(contents))
     schema = pf.schema_arrow
 
-    # Expected Parquet schema for reference
-    #
-    # required group field_id=0 schema {
-    #   optional int32 field_id=1 f0;
-    #   optional group field_id=2 f1 (List) {
-    #     repeated group field_id=3 list {
-    #       optional int32 field_id=4 item;
-    #     }
-    #   }
-    #   optional binary field_id=5 f2;
-    # }
-
     field_name = b'PARQUET:field_id'

Review comment:
       Thanks, fixed to use the existing variable.




-- 
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



[GitHub] [arrow] github-actions[bot] commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-837049354


   https://issues.apache.org/jira/browse/PARQUET-1798


-- 
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



[GitHub] [arrow] pitrou commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-849705643


   AppVeyor build at https://ci.appveyor.com/project/pitrou/arrow/builds/39351310


-- 
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



[GitHub] [arrow] pitrou commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-839634828


   Based on my understanding, it seems that we should:
   * when reading from Parquet, reflect Parquet field_ids (if any) under the `PARQUET:field_id` metadata key
   * when writing to Parquet, generate Parquet field_ids from the `PARQUET:field_id` metadata key (if present)
   * not attempt to auto-generate any field_ids if they are not present in metadata


-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640049553



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }
+  arrow_fields.push_back(::arrow::field(
+      "nested",
+      ::arrow::struct_({::arrow::field("outer", ::arrow::struct_({::arrow::field(
+                                                    "inner", ::arrow::utf8())})),
+                        ::arrow::field("sibling", ::arrow::date32())}),
+      false));
+
+  ASSERT_OK(ConvertSchema(arrow_fields));
+  auto field_ids = GetFieldIdsDfs(result_schema_->fields());
+  for (int actual_id : field_ids) {
+    ASSERT_EQ(actual_id, -1);
+  }
+}
+
+std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) {
+  return ::arrow::key_value_metadata({"PARQUET:field_id"}, {std::to_string(field_id)});
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdPreserveExisting) {

Review comment:
       Perhaps `TestConvertRoundTrip::ConvertSchema` should be renamed to `RoundTripSchema`.  It does the following transformations...
   
   * `vector<Field> -> arrow::Schema`
   * `arrow::Schema` -> `parquet::SchemaDescriptor`
   * `parquet::SchemaDescriptor` -> `vector<parquet::format::SchemaElement>`
   * `vector<parquet::format::SchemaElement>` -> `parquet::SchemaDescriptor`
   * `parquet::SchemaDescriptor` -> `arrow::Schema`
   
   So I believe it does indeed test the Parquet schema node.




-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640049553



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }
+  arrow_fields.push_back(::arrow::field(
+      "nested",
+      ::arrow::struct_({::arrow::field("outer", ::arrow::struct_({::arrow::field(
+                                                    "inner", ::arrow::utf8())})),
+                        ::arrow::field("sibling", ::arrow::date32())}),
+      false));
+
+  ASSERT_OK(ConvertSchema(arrow_fields));
+  auto field_ids = GetFieldIdsDfs(result_schema_->fields());
+  for (int actual_id : field_ids) {
+    ASSERT_EQ(actual_id, -1);
+  }
+}
+
+std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) {
+  return ::arrow::key_value_metadata({"PARQUET:field_id"}, {std::to_string(field_id)});
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdPreserveExisting) {

Review comment:
       Perhaps `TestConvertRoundTrip::ConvertSchema` should be renamed to `RoundTripSchema`.  It does the following transformations...
   
   * `vector<Field>` -> `arrow::Schema`
   * `arrow::Schema` -> `parquet::SchemaDescriptor`
   * `parquet::SchemaDescriptor` -> `vector<parquet::format::SchemaElement>`
   * `vector<parquet::format::SchemaElement>` -> `parquet::SchemaDescriptor`
   * `parquet::SchemaDescriptor` -> `arrow::Schema`
   
   So I believe it does indeed test the Parquet schema node.




-- 
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



[GitHub] [arrow] pitrou commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r639561783



##########
File path: python/pyarrow/tests/parquet/test_metadata.py
##########
@@ -303,28 +320,30 @@ def test_field_id_metadata():
     pf = pq.ParquetFile(pa.BufferReader(contents))
     schema = pf.schema_arrow
 
-    # Expected Parquet schema for reference
-    #
-    # required group field_id=0 schema {
-    #   optional int32 field_id=1 f0;
-    #   optional group field_id=2 f1 (List) {
-    #     repeated group field_id=3 list {
-    #       optional int32 field_id=4 item;
-    #     }
-    #   }
-    #   optional binary field_id=5 f2;
-    # }
-
     field_name = b'PARQUET:field_id'

Review comment:
       You already have it named `field_id` above.

##########
File path: cpp/src/parquet/schema_test.cc
##########
@@ -171,17 +167,16 @@ TEST_F(TestPrimitiveNode, Attrs) {
 }
 
 TEST_F(TestPrimitiveNode, FromParquet) {
-  SchemaElement elt =
-      NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32, field_id_);
+  SchemaElement elt = NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32);

Review comment:
       I'm not sure I understand this change in the tests. The user should still be able to pass an explicit `field_id` when creating a schema node, no?

##########
File path: cpp/src/parquet/schema.h
##########
@@ -268,7 +268,7 @@ class PARQUET_EXPORT GroupNode : public Node {
  public:
   // The field_id here is the default to use if it is not set in the SchemaElement
   static std::unique_ptr<Node> FromParquet(const void* opaque_element,
-                                           NodeVector fields = {}, int field_id = -1);
+                                           NodeVector fields = {});

Review comment:
       While we won't use the optional `field_id` argument anymore in Arrow, consumers of this API (that read Parquet files without going through Arrow) may still want to use it for themselves, no?

##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }

Review comment:
       Missing closing brace here?

##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }
+  arrow_fields.push_back(::arrow::field(
+      "nested",
+      ::arrow::struct_({::arrow::field("outer", ::arrow::struct_({::arrow::field(
+                                                    "inner", ::arrow::utf8())})),
+                        ::arrow::field("sibling", ::arrow::date32())}),
+      false));
+
+  ASSERT_OK(ConvertSchema(arrow_fields));
+  auto field_ids = GetFieldIdsDfs(result_schema_->fields());
+  for (int actual_id : field_ids) {
+    ASSERT_EQ(actual_id, -1);
+  }
+}
+
+std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) {
+  return ::arrow::key_value_metadata({"PARQUET:field_id"}, {std::to_string(field_id)});
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdPreserveExisting) {

Review comment:
       Well... this test only checks that Arrow metadata is preserved, right? It doesn't test that the metadata is actually converted into a field_id on the Parquet schema node.
   

##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {

Review comment:
       Why "GroupId"? Should this be "FieldId"?




-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-840008403


   > * not attempt to auto-generate any field_ids if they are not present in metadata
   @pitrou 
   
   That should simplify things.  Just to clarify, this will be a bit of a regression as we currently auto-generate field IDs today.
   
   > https://issues.apache.org/jira/browse/PARQUET-951 informs field IDs a little bit better. It is from other systems, in this case protobuf (and I imagine thrift might also have something similar) has each field in a message annotated with a unique ID. Based on this I agree with Antoine's assessment, haven't actually looked at the code (is this not what is done?).
   @emkornfield 
   
   Correct.  We already pulled the field id out of thrift and into Arrow metadata.  The only problem was that the logic to do the reverse was missing.  This PR is only adding that.
   
   There could be some follow-up work for integrating with other parts of the Arrow ecosystem.  I will send some questions to the ML.


-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-849644236


   As an extra level of sanity-checking I created a parquet file with Arrow and then read it in with fastparquet and verified the field_id is correct (both for a missing field_id and a valid field_id).


-- 
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



[GitHub] [arrow] emkornfield commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-837090190


   I'll try to look tonight or tomorrow morning. Otherwise, Antoine is likely the best person.


-- 
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



[GitHub] [arrow] emkornfield edited a comment on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
emkornfield edited a comment on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-839865510


   https://issues.apache.org/jira/browse/PARQUET-951 informs field IDs a little bit better.  It is from other systems, in this case protobuf (and I imagine thrift might also have something similar) has each field in a message annotated with a unique ID.  Based on this I agree with Antoine's assessment, haven't actually looked at the code (is this not what is done?).


-- 
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



[GitHub] [arrow] pitrou commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640374369



##########
File path: cpp/src/parquet/schema_test.cc
##########
@@ -171,17 +167,16 @@ TEST_F(TestPrimitiveNode, Attrs) {
 }
 
 TEST_F(TestPrimitiveNode, FromParquet) {
-  SchemaElement elt =
-      NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32, field_id_);
+  SchemaElement elt = NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32);

Review comment:
       Well, let's say I want to create a `Node` with a given field id as was done in this test. Would you I do that?




-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-840847789


   Per @pitrou 's suggestion I have removed the logic auto-generating field_id entirely.  I also added a python test to ensure things are working full path.
   
   This is ready for review again.


-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640625060



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {

Review comment:
       Yes, I'm not sure where Group came from.  Fixed.




-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640625783



##########
File path: cpp/src/parquet/arrow/arrow_schema_test.cc
##########
@@ -1157,6 +1160,94 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
   ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
+class TestConvertRoundTrip : public ::testing::Test {
+ public:
+  ::arrow::Status ConvertSchema(
+      const std::vector<std::shared_ptr<Field>>& fields,
+      std::shared_ptr<::parquet::ArrowWriterProperties> arrow_properties =
+          ::parquet::default_arrow_writer_properties()) {
+    arrow_schema_ = ::arrow::schema(fields);
+    std::shared_ptr<::parquet::WriterProperties> properties =
+        ::parquet::default_writer_properties();
+    RETURN_NOT_OK(ToParquetSchema(arrow_schema_.get(), *properties.get(),
+                                  *arrow_properties, &parquet_schema_));
+    ::parquet::schema::ToParquet(parquet_schema_->group_node(), &parquet_format_schema_);
+    auto parquet_schema = ::parquet::schema::FromParquet(parquet_format_schema_);
+    return FromParquetSchema(parquet_schema.get(), &result_schema_);
+  }
+
+ protected:
+  std::shared_ptr<::arrow::Schema> arrow_schema_;
+  std::shared_ptr<SchemaDescriptor> parquet_schema_;
+  std::vector<SchemaElement> parquet_format_schema_;
+  std::shared_ptr<::arrow::Schema> result_schema_;
+};
+
+int GetFieldId(const ::arrow::Field& field) {
+  if (field.metadata() == nullptr) {
+    return -1;
+  }
+  auto maybe_field = field.metadata()->Get("PARQUET:field_id");
+  if (!maybe_field.ok()) {
+    return -1;
+  }
+  return std::stoi(maybe_field.ValueOrDie());
+}
+
+void GetFieldIdsDfs(const ::arrow::FieldVector& fields, std::vector<int>* field_ids) {
+  for (const auto& field : fields) {
+    field_ids->push_back(GetFieldId(*field));
+    GetFieldIdsDfs(field->type()->fields(), field_ids);
+  }
+}
+
+std::vector<int> GetFieldIdsDfs(const ::arrow::FieldVector& fields) {
+  std::vector<int> field_ids;
+  GetFieldIdsDfs(fields, &field_ids);
+  return field_ids;
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdMissingIfNotSpecified) {
+  std::vector<std::shared_ptr<Field>> arrow_fields;
+  arrow_fields.push_back(::arrow::field("simple", ::arrow::int32(), false));
+  /// { "nested": { "outer": { "inner" }, "sibling" }
+  arrow_fields.push_back(::arrow::field(
+      "nested",
+      ::arrow::struct_({::arrow::field("outer", ::arrow::struct_({::arrow::field(
+                                                    "inner", ::arrow::utf8())})),
+                        ::arrow::field("sibling", ::arrow::date32())}),
+      false));
+
+  ASSERT_OK(ConvertSchema(arrow_fields));
+  auto field_ids = GetFieldIdsDfs(result_schema_->fields());
+  for (int actual_id : field_ids) {
+    ASSERT_EQ(actual_id, -1);
+  }
+}
+
+std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) {
+  return ::arrow::key_value_metadata({"PARQUET:field_id"}, {std::to_string(field_id)});
+}
+
+TEST_F(TestConvertRoundTrip, GroupIdPreserveExisting) {

Review comment:
       I now verify the field IDs at all three levels (round-tripped arrow, parquet, and thrift).




-- 
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



[GitHub] [arrow] westonpace commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r640624676



##########
File path: cpp/src/parquet/schema_test.cc
##########
@@ -171,17 +167,16 @@ TEST_F(TestPrimitiveNode, Attrs) {
 }
 
 TEST_F(TestPrimitiveNode, FromParquet) {
-  SchemaElement elt =
-      NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32, field_id_);
+  SchemaElement elt = NewPrimitive(name_, FieldRepetitionType::OPTIONAL, Type::INT32);

Review comment:
       Ah, I see your point.  These tests were not testing `FromParquet` so the field id setting was still valid.  I have restored them.




-- 
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



[GitHub] [arrow] pitrou commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-839631978


   I'm not even sure what field_ids are supposed to be for. The parquet spec only has this to say:
   ```thrift
     /** When the original schema supports field ids, this will save the
      * original field id in the parquet schema
      */
     9: optional i32 field_id;
   ```
   
   I suppose "original schema" means something non-Parquet, but what? Is it just some kind of arbitrary application-defined id?


-- 
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



[GitHub] [arrow] pitrou closed pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #10289:
URL: https://github.com/apache/arrow/pull/10289


   


-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-842819891


   Ok, looks like I jumped the gun with the last comment.  Removing the old auto-generation behavior broke some tests I wasn't looking at.  They are fixed now.   I believe the CI failures are unrelated at this point.  I may force-push tomorrow just for good measure.
   
   Review is welcome.


-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-837050570


   CC @emkornfield Are you able to review this?  Not sure who I should ping for a parquet change.


-- 
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



[GitHub] [arrow] emkornfield commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-839865510


   https://issues.apache.org/jira/browse/PARQUET-951 informs field IDs a little bit better.  It is from other systems, in this case protobuf (and I imagine thrift might also have something similar).  Based on this I agree with Antoine's assessment, haven't actually looked at the code (is this not what is done?).


-- 
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



[GitHub] [arrow] wesm commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
wesm commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-837106370


   cc @tgooch44


-- 
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



[GitHub] [arrow] westonpace commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-849644976


   Provided this passes CI (I'll check in the morning) I believe I have addressed all concerns.


-- 
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



[GitHub] [arrow] pitrou commented on a change in pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#discussion_r630890862



##########
File path: cpp/src/parquet/schema.cc
##########
@@ -43,6 +44,39 @@ void ThrowInvalidLogicalType(const LogicalType& logical_type) {
   throw ParquetException(ss.str());
 }
 
+class PARQUET_EXPORT FieldIdCheckout {

Review comment:
       Can you explain what this is for?




-- 
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



[GitHub] [arrow] pitrou commented on pull request #10289: PARQUET-1798: [C++] Review logic around automatic assignment of field_id's

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10289:
URL: https://github.com/apache/arrow/pull/10289#issuecomment-849645174


   Thanks a lot for checking!


-- 
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