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 2022/04/24 23:06:00 UTC

[GitHub] [arrow] sanjibansg opened a new pull request, #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

sanjibansg opened a new pull request, #12977:
URL: https://github.com/apache/arrow/pull/12977

   This PR fixes the issue of the partitioning field having null values while using FilenamePartitioning. 
   For FilenamePartitioning, we should only remove the prefix and thus should not use `StripPrefixAndFilename()`, which will remove the filename too along with the prefix.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] westonpace commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r882058122


##########
cpp/src/arrow/dataset/partition.cc:
##########
@@ -395,18 +396,18 @@ Result<std::vector<KeyValuePartitioning::Key>> FilenamePartitioning::ParseKeys(
   return ParsePartitionSegments(segments);

Review Comment:
   Shouldn't the line above be...
   
   ```
   std::vector<std::string> segments =
         fs::internal::SplitAbstractPath(StripNonPrefix(fs::internal::GetAbstractPathParent(path).second)path), kFilenamePartitionSep);
   ```
   In other words, if the path is `foo/bar/x=3_y=5_chunk0.parquet`:
   
   ```
   StripNonPrefix(path) // foo/bar/x=3_y=5
   fs::internal::GetAbstractPathParent(path).second // x=3_y=5_chunk0.parquet
   StripNonPrefix(fs::internal::GetAbstractPathParent(path).second) // x=3_y=5
   ```



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862231668


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   The `Inspect()` methods now accept a `PartitionPathFormat` object as an argument with the latest commit. I have modified the tests and `SplitFilenameAndPrefix()` methods to return forms of PartitionPathFormat object as required. The `Inspect()` methods then extracts the directory or prefix accordingly whichever required.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r859144782


##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -38,6 +38,10 @@ namespace dataset {
 
 constexpr char kFilenamePartitionSep = '_';
 
+struct PartitionPathFormat {

Review Comment:
   ```suggestion
   struct ARROW_DS_EXPORT PartitionPathFormat {
   ```



##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -76,11 +80,8 @@ class ARROW_DS_EXPORT Partitioning {
       const std::shared_ptr<RecordBatch>& batch) const = 0;
 
   /// \brief Parse a path into a partition expression
-  virtual Result<compute::Expression> Parse(const std::string& path) const = 0;
-
-  struct PartitionPathFormat {
-    std::string directory, prefix;
-  };
+  virtual Result<compute::Expression> Parse(const std::string& directory = "",
+                                            const std::string& prefix = "") const = 0;

Review Comment:
   Why do we need the default parameter values?



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -847,7 +847,8 @@ ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning)
     auto row_groups = Iota(metadata_subset->num_row_groups());
 
     auto partition_expression =
-        partitioning.Parse(StripPrefixAndFilename(path, options_.partition_base_dir))
+        partitioning
+            .Parse(StripPrefixAndFilename(path, options_.partition_base_dir).directory)

Review Comment:
   Why aren't we passing both components?



##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -76,11 +80,8 @@ class ARROW_DS_EXPORT Partitioning {
       const std::shared_ptr<RecordBatch>& batch) const = 0;
 
   /// \brief Parse a path into a partition expression
-  virtual Result<compute::Expression> Parse(const std::string& path) const = 0;
-
-  struct PartitionPathFormat {
-    std::string directory, prefix;
-  };
+  virtual Result<compute::Expression> Parse(const std::string& directory = "",
+                                            const std::string& prefix = "") const = 0;

Review Comment:
   Though yes, it would be better if we could pass `const PartitionPathFormat&` instead. FWIW I don't think we have to expose it to Python. Or we can just make a namedtuple on the Python side, it doesn't have to be a C++ class wrapper.



##########
cpp/src/arrow/dataset/partition.cc:
##########
@@ -805,19 +806,20 @@ std::shared_ptr<PartitioningFactory> HivePartitioning::MakeFactory(
   return std::shared_ptr<PartitioningFactory>(new HivePartitioningFactory(options));
 }
 
-std::string StripPrefixAndFilename(const std::string& path, const std::string& prefix) {
+PartitionPathFormat StripPrefixAndFilename(const std::string& path,
+                                           const std::string& prefix) {
   auto maybe_base_less = fs::internal::RemoveAncestor(prefix, path);
   auto base_less = maybe_base_less ? std::string(*maybe_base_less) : path;
   auto basename_filename = fs::internal::GetAbstractPathParent(base_less);
-  return basename_filename.first;
+  return PartitionPathFormat{basename_filename.first, basename_filename.second};

Review Comment:
   nit: `std::move` things here



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   (Can we cover this with a test?)



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -847,7 +847,8 @@ ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning)
     auto row_groups = Iota(metadata_subset->num_row_groups());
 
     auto partition_expression =
-        partitioning.Parse(StripPrefixAndFilename(path, options_.partition_base_dir))
+        partitioning
+            .Parse(StripPrefixAndFilename(path, options_.partition_base_dir).directory)

Review Comment:
   (Also, can we cover this with a test?)



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   Hmm, don't we still want the filename in the path in case the partitioning factory is a filename PartitioningFactory?



##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,9 +278,11 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;

Review Comment:
   Is this necessary? It gets shadowed below



##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -353,16 +359,17 @@ class ARROW_DS_EXPORT FilenamePartitioning : public KeyValuePartitioning {
       std::vector<std::string> field_names, PartitioningFactoryOptions = {});
 
  private:
-  Result<std::vector<Key>> ParseKeys(const std::string& path) const override;
+  Result<std::vector<Key>> ParseKeys(const std::string& directory,
+                                     const std::string& prefix) const override;
 
   Result<PartitionPathFormat> FormatValues(const ScalarVector& values) const override;
 };
 
 /// \brief Remove a prefix and the filename of a path.
 ///
 /// e.g., `StripPrefixAndFilename("/data/year=2019/c.txt", "/data") -> "year=2019"`

Review Comment:
   Example in docstring needs updating



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857857993


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   We may pass both the filename and the path(other than the prefix) to the `Parse()` method, but the filename is only required for FilenamePartitioning I believe, and the path will be required by the Directory & Hive Partitioning.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] github-actions[bot] commented on pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

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

   https://issues.apache.org/jira/browse/ARROW-16302


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r859106787


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   With the latest change, I modified the `StripPrefixAndFIlename()` method to return a `PartitionPathFormat` object which will contain both the directory and filename prefix and then passing that to the `Parse()` method which now expects both the directory and filename-prefix. 
   
   We can modify the `Parse()` method as well to accept an object of `PartitionPathFormat` that way it will be symmetrical to the `Format()` method. But then, we need to implement similar changes to PyArrow, and I believe then to use the `partitioning.parse()` method in PyArrow we have to define an object of the `PartitionPathFormat` first. 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] westonpace closed pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
westonpace closed pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning
URL: https://github.com/apache/arrow/pull/12977


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861963501


##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -353,16 +359,17 @@ class ARROW_DS_EXPORT FilenamePartitioning : public KeyValuePartitioning {
       std::vector<std::string> field_names, PartitioningFactoryOptions = {});
 
  private:
-  Result<std::vector<Key>> ParseKeys(const std::string& path) const override;
+  Result<std::vector<Key>> ParseKeys(const std::string& directory,
+                                     const std::string& prefix) const override;
 
   Result<PartitionPathFormat> FormatValues(const ScalarVector& values) const override;
 };
 
 /// \brief Remove a prefix and the filename of a path.
 ///
 /// e.g., `StripPrefixAndFilename("/data/year=2019/c.txt", "/data") -> "year=2019"`

Review Comment:
   Updated the docstring



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857860950


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   Sure. From the point of view of datasets however it doesn't matter that some implementations only need some of the info.
   
   CC @westonpace for thoughts



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r859106787


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   With the latest change, I modified the `StripPrefixAndFilename()` method to return a `PartitionPathFormat` object which will contain both the directory and filename prefix and then passing that to the `Parse()` method which now expects both the directory and filename-prefix. 
   
   We can modify the `Parse()` method as well to accept an object of `PartitionPathFormat` that way it will be symmetrical to the `Format()` method. But then, we need to implement similar changes to PyArrow, and I believe then to use the `partitioning.parse()` method in PyArrow we have to define an object of the `PartitionPathFormat` first. 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857740111


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   So, it's like passing the entire `info.path()` in the `Parse()` method and then various partitioning implementations will do it in the way they want?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857748824


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   Yeah, I think we can still strip the prefix, but we can let the partitioning handle the rest. 
   
   And in that case it might make sense to split the remainder of the path and the filename for the partitioning implementation too and just pass both as arguments. 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862060024


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -847,7 +847,8 @@ ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning)
     auto row_groups = Iota(metadata_subset->num_row_groups());
 
     auto partition_expression =
-        partitioning.Parse(StripPrefixAndFilename(path, options_.partition_base_dir))
+        partitioning
+            .Parse(StripPrefixAndFilename(path, options_.partition_base_dir).directory)

Review Comment:
   Making sure that a filename partitioning works properly in this path, basically, since before it seems like it would have failed since the filename was being omitted.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862058051


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -847,7 +847,8 @@ ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning)
     auto row_groups = Iota(metadata_subset->num_row_groups());
 
     auto partition_expression =
-        partitioning.Parse(StripPrefixAndFilename(path, options_.partition_base_dir))
+        partitioning
+            .Parse(StripPrefixAndFilename(path, options_.partition_base_dir).directory)

Review Comment:
   Now that we are using the `PartitionPathFormat` as the argument, the `Parse()` method will work accordingly as per the partitioning mode. Any particular test you want here?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861964090


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,9 +278,11 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;

Review Comment:
   Yes, corrected that, thanks for pointing that out!



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861978402


##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -310,8 +310,8 @@ class ARROW_DS_EXPORT FunctionPartitioning : public Partitioning {
 
   std::string type_name() const override { return name_; }
 
-  Result<compute::Expression> Parse(const std::string& path) const override {
-    return parse_impl_(path);
+  Result<compute::Expression> Parse(const PartitionPathFormat& path) const override {
+    return parse_impl_(path.directory);

Review Comment:
   Hmm, not for this PR but I would expect FunctionPartitioning to be "as powerful as" any other partitioning. But I don't think it's used much anyways.



##########
python/pyarrow/_dataset.pyx:
##########
@@ -1313,9 +1313,12 @@ cdef class Partitioning(_Weakrefable):
     cdef inline shared_ptr[CPartitioning] unwrap(self):
         return self.wrapped
 
-    def parse(self, path):
+    def parse(self, directory="", prefix=""):

Review Comment:
   nit but do both of these need a default? I can see prefix having a default because it's a new argument



##########
python/pyarrow/_dataset.pyx:
##########
@@ -1313,9 +1313,12 @@ cdef class Partitioning(_Weakrefable):
     cdef inline shared_ptr[CPartitioning] unwrap(self):
         return self.wrapped
 
-    def parse(self, path):
+    def parse(self, directory="", prefix=""):

Review Comment:
   Also IMO the parameter should be named "filename", and/or we should have a docstring



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   This still seems off, but I can't figure out how to hit this case.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862234806


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -847,7 +847,8 @@ ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning)
     auto row_groups = Iota(metadata_subset->num_row_groups());
 
     auto partition_expression =
-        partitioning.Parse(StripPrefixAndFilename(path, options_.partition_base_dir))
+        partitioning
+            .Parse(StripPrefixAndFilename(path, options_.partition_base_dir).directory)

Review Comment:
   I have added a round-trip test in PyArrow to check whether the partitions are read correctly. Do we need any tests other than 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857740111


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   So, it's like passing the entire `info.path()` in the `Parse()` method and then various partitioning implementations will do it in the way they want? I think we can then strip the prefix/filename in the `ParseKeys` method of each of the partitioning modes.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862080350


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   I think we can do the same changes in the `Inspect()` method which currently accepts a path. Instead of passing a vector of strings, we can then pass a vector of `PartitionPathFormat` object, and then the `Inspect` methods of individual partitioning modes will use either the directory or the filename accordingly?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12977:
URL: https://github.com/apache/arrow/pull/12977#issuecomment-1113616149

   Thanks.
   
   @westonpace any comments here?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857616730


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   I wonder if we should make it the responsibility of the partitioning implementation to strip the filename, instead of hardcoding an exception. (After all, what if the user wants to define a custom filename-based partitioning scheme?) Or we could pass both the path and the filename separately to `partitioning->Parse`.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r882424734


##########
cpp/src/arrow/dataset/partition.cc:
##########
@@ -395,18 +396,18 @@ Result<std::vector<KeyValuePartitioning::Key>> FilenamePartitioning::ParseKeys(
   return ParsePartitionSegments(segments);

Review Comment:
   Yes, corrected that, thanks!



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861960853


##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -38,6 +38,10 @@ namespace dataset {
 
 constexpr char kFilenamePartitionSep = '_';
 
+struct PartitionPathFormat {

Review Comment:
   Made the change, thanks!



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861963368


##########
cpp/src/arrow/dataset/partition.h:
##########
@@ -76,11 +80,8 @@ class ARROW_DS_EXPORT Partitioning {
       const std::shared_ptr<RecordBatch>& batch) const = 0;
 
   /// \brief Parse a path into a partition expression
-  virtual Result<compute::Expression> Parse(const std::string& path) const = 0;
-
-  struct PartitionPathFormat {
-    std::string directory, prefix;
-  };
+  virtual Result<compute::Expression> Parse(const std::string& directory = "",
+                                            const std::string& prefix = "") const = 0;

Review Comment:
   - Removed the default parameter
   - Modified the Parse method to use a `PartitionPathFormat` object as an argument. As for the PyArrow interface, modified the `parse()` method to accept just the two strings (directory & prefix) and then uses a cppclass object to form the PartitionPathFormat object which is then passed into the internal `Parse()` method. Is this a good approach? 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862327749


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   With the changes in the `Inspect()` method, I think the R build is failing, I am trying to investigate on fixing it, but not very sure about the R implementation.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r859106787


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   With the latest change, I modified the `StripPrefixAndFilename()` method to return a `PartitionPathFormat` object which will contain both the directory and filename prefix and then passing that to the `Parse()` method which now expects both the directory and filename-prefix. 
   
   We can modify the `Parse()` method as well to accept an object of `PartitionPathFormat` that way it will be symmetrical to the `Format()` method. But then, we need to implement similar changes to PyArrow, and I believe then we have to define an object of `PartitionPathFormat` first to use the `partitioning.parse()` method in PyArrow.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r857749287


##########
cpp/src/arrow/dataset/discovery.cc:
##########
@@ -278,8 +278,13 @@ Result<std::shared_ptr<Dataset>> FileSystemDatasetFactory::Finish(FinishOptions
   }
 
   std::vector<std::shared_ptr<FileFragment>> fragments;
+  std::string fixed_path;
   for (const auto& info : files_) {
-    auto fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    if (partitioning->type_name() == "filename") {
+      fixed_path = StripPrefix(info.path(), options_.partition_base_dir);
+    } else {
+      fixed_path = StripPrefixAndFilename(info.path(), options_.partition_base_dir);
+    }

Review Comment:
   (We should still strip the prefix since presumably we don't want partitioning to depend on the prefix itself.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r861963687


##########
cpp/src/arrow/dataset/partition.cc:
##########
@@ -805,19 +806,20 @@ std::shared_ptr<PartitioningFactory> HivePartitioning::MakeFactory(
   return std::shared_ptr<PartitioningFactory>(new HivePartitioningFactory(options));
 }
 
-std::string StripPrefixAndFilename(const std::string& path, const std::string& prefix) {
+PartitionPathFormat StripPrefixAndFilename(const std::string& path,
+                                           const std::string& prefix) {
   auto maybe_base_less = fs::internal::RemoveAncestor(prefix, path);
   auto base_less = maybe_base_less ? std::string(*maybe_base_less) : path;
   auto basename_filename = fs::internal::GetAbstractPathParent(base_less);
-  return basename_filename.first;
+  return PartitionPathFormat{basename_filename.first, basename_filename.second};

Review Comment:
   Made the change, thanks!



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862046122


##########
python/pyarrow/_dataset.pyx:
##########
@@ -1313,9 +1313,12 @@ cdef class Partitioning(_Weakrefable):
     cdef inline shared_ptr[CPartitioning] unwrap(self):
         return self.wrapped
 
-    def parse(self, path):
+    def parse(self, directory="", prefix=""):

Review Comment:
   Previously, when the `parse()` method only accepted a path, we used to pass the filename there, as FilenamePartitioning only needs the filename and not the directory.  But, now that it is expecting both a directory and filename prefix, so I believe a directory will not be required for FilenamePartitioning, thus using an empty string as default.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] westonpace commented on pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

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

   @github-actions autotune


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862046361


##########
python/pyarrow/_dataset.pyx:
##########
@@ -1313,9 +1313,12 @@ cdef class Partitioning(_Weakrefable):
     cdef inline shared_ptr[CPartitioning] unwrap(self):
         return self.wrapped
 
-    def parse(self, path):
+    def parse(self, directory="", prefix=""):

Review Comment:
   Renamed `prefix` to `filename` in `PartitionPathFormat`



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862081385


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   That probably makes the most sense, but we might want to split out that refactoring separately, and also make sure that we can hit this path in a unit test in the first place (I was trying this morning but couldn't) as I don't want to expand the scope of this PR too much.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] sanjibansg commented on a diff in pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
sanjibansg commented on code in PR #12977:
URL: https://github.com/apache/arrow/pull/12977#discussion_r862231668


##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -873,7 +874,8 @@ Result<std::vector<std::shared_ptr<Schema>>> ParquetDatasetFactory::InspectSchem
 
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
-      stripped[i++] = StripPrefixAndFilename(e.first, options_.partition_base_dir);
+      stripped[i++] =
+          StripPrefixAndFilename(e.first, options_.partition_base_dir).directory;

Review Comment:
   The `Inspect()` methods now accept a `PartitionPathFormat` object as an argument with the latest commit. I have modified the tests and `SplitFilenameAndPrefix()` methods to return forms of PartitionPathFormat object as required. The `Inspect()` methods then extracts the directory or prefix accordingly whichever is required.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] ursabot commented on pull request #12977: ARROW-16302: [C++] Null values in partitioning field for FilenamePartitioning

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #12977:
URL: https://github.com/apache/arrow/pull/12977#issuecomment-1139252523

   Benchmark runs are scheduled for baseline = f3e09b9b6e4ae9398844354ce0578903e857772a and contender = adb5b00023bf57122c6de540746008eff2d1ad92. adb5b00023bf57122c6de540746008eff2d1ad92 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/4ae562d2a8814a56a7c6acccd1d8db64...ce60238782bb46848bcf1ee9dbc5fd68/)
   [Failed :arrow_down:0.08% :arrow_up:0.0%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/b2d0eaed7e24408c8c396bc97d5ec8ae...0ca1f21e72e840ea89966c860470ba4e/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/05f7d05deec44ee2817847e4d6c62df5...40a875162ee74f6c80e5b8c01269ddfa/)
   [Finished :arrow_down:0.08% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/c7441dbb1b1e4264971887a6c13965c1...9313e03c5abe4ab085944e1902e67b48/)
   Buildkite builds:
   [Finished] [`adb5b000` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/834)
   [Failed] [`adb5b000` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/833)
   [Finished] [`adb5b000` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/823)
   [Finished] [`adb5b000` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/837)
   [Finished] [`f3e09b9b` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/833)
   [Finished] [`f3e09b9b` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/832)
   [Finished] [`f3e09b9b` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/822)
   [Finished] [`f3e09b9b` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/836)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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