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/07/17 20:16:39 UTC

[GitHub] [arrow] wesm opened a new pull request, #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

   This completes the porting to use ExecSpan everywhere. I also changed the ExecBatchIterator benchmarks to use ExecSpan to show the performance improvement in input splitting that we've talked about in the past:
   
   Splitting inputs into small ExecSpan:
   
   ```
   ------------------------------------------------------------------------------------
   Benchmark                          Time             CPU   Iterations UserCounters...
   ------------------------------------------------------------------------------------
   BM_ExecSpanIterator/1024      205671 ns       205667 ns         3395 items_per_second=4.86223k/s
   BM_ExecSpanIterator/4096       54749 ns        54750 ns        13121 items_per_second=18.265k/s
   BM_ExecSpanIterator/16384      15979 ns        15979 ns        42628 items_per_second=62.5824k/s
   BM_ExecSpanIterator/65536       5597 ns         5597 ns       125099 items_per_second=178.668k/s
   ```
   
   Splitting inputs into small ExecBatch:
   
   ```
   -------------------------------------------------------------------------------------
   Benchmark                           Time             CPU   Iterations UserCounters...
   -------------------------------------------------------------------------------------
   BM_ExecBatchIterator/1024    17163432 ns     17163171 ns           41 items_per_second=58.2643/s
   BM_ExecBatchIterator/4096     4243467 ns      4243316 ns          163 items_per_second=235.665/s
   BM_ExecBatchIterator/16384    1093680 ns      1093638 ns          620 items_per_second=914.38/s
   BM_ExecBatchIterator/65536     272451 ns       272435 ns         2584 items_per_second=3.6706k/s
   ```
   
   Because the input in this benchmark has 1M elements, this shows that splitting into 1024 chunks of size 1024 adds only 0.2ms of overhead with ExecSpanIterator versus 17.16ms of overhead with ExecBatchIterator (> 80x improvement). 
   
   This won't by itself do much to impact performance in Acero but things for the community to explore in the future are the following (this work that I've been doing has been a precondition to consider this):
   
   * A leaner ExecuteScalarExpression implementation that reuses temporary allocations (ARROW-16758)
   * Parallel expression evaluation
   * Better defining morsel (~1M elements) versus task (~1K elements) granularity in execution 
   * Work stealing so that we don't "hog" the thread pools, and we keep the work pinned to a particular CPU core if there are other things going on at the same time


-- 
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] wesm commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -202,14 +202,16 @@ Result<Datum> GroupByUsingExecPlan(const std::vector<Datum>& arguments,
   inputs.reserve(inputs.size() + keys.size());
   inputs.insert(inputs.end(), keys.begin(), keys.end());
 
-  ARROW_ASSIGN_OR_RAISE(auto batch_iterator,
-                        ExecBatchIterator::Make(inputs, ctx->exec_chunksize()));
+  ExecSpanIterator span_iterator;
+  ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make(inputs));
+  RETURN_NOT_OK(span_iterator.Init(batch, ctx->exec_chunksize()));
   BatchesWithSchema input;
   input.schema = schema(std::move(scan_fields));
-  ExecBatch batch;
-  while (batch_iterator->Next(&batch)) {
-    if (batch.length == 0) continue;
-    input.batches.push_back(std::move(batch));
+  ExecSpan span;
+  while (span_iterator.Next(&span)) {
+    if (span.length == 0) continue;
+    // TODO(wesm): investigate possibility of ExecPlans using ExecSpan

Review Comment:
   Removing this comment -- this is something we can tackle in more general refactoring of the execution engine (to introduce a distinction between tasks and morsels) in the future



-- 
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] wesm commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/exec_internal.h:
##########
@@ -110,6 +80,7 @@ class ARROW_EXPORT ExecSpanIterator {
   bool initialized_ = false;
   bool have_chunked_arrays_ = false;
   bool have_all_scalars_ = false;
+  bool promote_if_all_scalars_ = false;

Review Comment:
   +1



-- 
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] wesm commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/exec_internal.h:
##########
@@ -84,7 +51,10 @@ class ARROW_EXPORT ExecSpanIterator {
   /// \param[in] batch the input ExecBatch
   /// \param[in] max_chunksize the maximum length of each ExecSpan. Depending
   /// on the chunk layout of ChunkedArray.
-  Status Init(const ExecBatch& batch, int64_t max_chunksize = kDefaultMaxChunksize);
+  /// \param[in] promote_if_all_scalars if all of the values are scalars,
+  /// return them in each ExecSpan as ArraySpan of length 1

Review Comment:
   I'm adding a comment that it should be set to true for scalar/vector executors and false for scalar/hash aggregation execution



-- 
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] wesm commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -294,20 +282,21 @@ struct GrouperFastImpl : Grouper {
                                     ctx_->memory_pool()));
           }
         }
-        return ConsumeImpl(expanded);
+        return ConsumeImpl(ExecSpan(expanded));
       }
     }
     return ConsumeImpl(batch);
   }
 
-  Result<Datum> ConsumeImpl(const ExecBatch& batch) {
+  Result<Datum> ConsumeImpl(const ExecSpan& batch) {
     int64_t num_rows = batch.length;
     int num_columns = batch.num_values();
     // Process dictionaries
     for (int icol = 0; icol < num_columns; ++icol) {
       if (key_types_[icol].id() == Type::DICTIONARY) {
-        auto data = batch[icol].array();
-        auto dict = MakeArray(data->dictionary);
+        const ArraySpan& data = batch[icol].array;
+        // TODO(wesm): do not require ToArrayData here

Review Comment:
   Removed the comment for now, each of the usages of `ToArrayData` can be given consideration in the future



-- 
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] wesm commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/kernels/aggregate_basic_internal.h:
##########
@@ -448,9 +449,12 @@ struct MinMaxImpl : public ScalarAggregator {
     return Status::OK();
   }
 
-  Status ConsumeArray(const ArrayType& arr) {
+  Status ConsumeArray(const ArraySpan& arr_span) {
     StateType local;
 
+    // TODO(wesm): do not use ToArrayData
+    ArrayType arr(arr_span.ToArrayData());

Review Comment:
   It can be done later, non-urgently. The span has to be "boxed" in order to use `ArrayType::GetView`, but we could make that unnecessary in a subsequent PR. I'm removing the TODO comment since usages of `ToArrayData` can be found easily



-- 
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] wesm commented on pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

   Merging. The CI issues look like assorted flakiness to me
   


-- 
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] wesm commented on pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

   The only CI failures are the linkage issues related to protobuf/otel


-- 
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] wesm merged pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

Posted by GitBox <gi...@apache.org>.
wesm merged PR #13630:
URL: https://github.com/apache/arrow/pull/13630


-- 
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] pitrou commented on a diff in pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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


##########
cpp/src/arrow/compute/exec/aggregate.cc:
##########
@@ -193,8 +193,11 @@ Result<Datum> GroupBy(const std::vector<Datum>& arguments, const std::vector<Dat
 
   // Merge if necessary
   for (size_t thread_index = 1; thread_index < thread_ids.size(); ++thread_index) {
+    // TODO: Return ExecSpan from GetUniques; but need to figure out memory
+    // management strategy

Review Comment:
   Same here?



##########
cpp/src/arrow/compute/exec/aggregate.cc:
##########
@@ -174,15 +171,18 @@ Result<Datum> GroupBy(const std::vector<Datum>& arguments, const std::vector<Dat
       auto grouper = groupers[thread_index].get();
 
       // compute a batch of group ids
+      // TODO(wesm): refactor Grouper::Consume to write into preallocated
+      // memory

Review Comment:
   Rather than adding TODOs in the code, this would better be tracked and remembered using a JIRA, IMHO.



##########
cpp/src/arrow/compute/exec_internal.h:
##########
@@ -84,7 +51,10 @@ class ARROW_EXPORT ExecSpanIterator {
   /// \param[in] batch the input ExecBatch
   /// \param[in] max_chunksize the maximum length of each ExecSpan. Depending
   /// on the chunk layout of ChunkedArray.
-  Status Init(const ExecBatch& batch, int64_t max_chunksize = kDefaultMaxChunksize);
+  /// \param[in] promote_if_all_scalars if all of the values are scalars,
+  /// return them in each ExecSpan as ArraySpan of length 1

Review Comment:
   What happens otherwise? (i.e. if all values are scalars but `promote_if_all_scalars` is false)



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -294,20 +282,21 @@ struct GrouperFastImpl : Grouper {
                                     ctx_->memory_pool()));
           }
         }
-        return ConsumeImpl(expanded);
+        return ConsumeImpl(ExecSpan(expanded));
       }
     }
     return ConsumeImpl(batch);
   }
 
-  Result<Datum> ConsumeImpl(const ExecBatch& batch) {
+  Result<Datum> ConsumeImpl(const ExecSpan& batch) {
     int64_t num_rows = batch.length;
     int num_columns = batch.num_values();
     // Process dictionaries
     for (int icol = 0; icol < num_columns; ++icol) {
       if (key_types_[icol].id() == Type::DICTIONARY) {
-        auto data = batch[icol].array();
-        auto dict = MakeArray(data->dictionary);
+        const ArraySpan& data = batch[icol].array;
+        // TODO(wesm): do not require ToArrayData here

Review Comment:
   Add a JIRA?



##########
cpp/src/arrow/compute/exec_internal.h:
##########
@@ -110,6 +80,7 @@ class ARROW_EXPORT ExecSpanIterator {
   bool initialized_ = false;
   bool have_chunked_arrays_ = false;
   bool have_all_scalars_ = false;
+  bool promote_if_all_scalars_ = false;

Review Comment:
   For the sake of avoiding confusion, can this have the same default value `true` as in the constructor?



##########
cpp/src/arrow/compute/kernel.h:
##########
@@ -609,20 +609,17 @@ struct VectorKernel : public Kernel {
 // ----------------------------------------------------------------------
 // ScalarAggregateKernel (for ScalarAggregateFunction)
 
-using ScalarAggregateConsume = std::function<Status(KernelContext*, const ExecBatch&)>;
-
-using ScalarAggregateMerge =
-    std::function<Status(KernelContext*, KernelState&&, KernelState*)>;
-
+typedef Status (*ScalarAggregateConsume)(KernelContext*, const ExecSpan&);
+typedef Status (*ScalarAggregateMerge)(KernelContext*, KernelState&&, KernelState*);
 // Finalize returns Datum to permit multiple return values
-using ScalarAggregateFinalize = std::function<Status(KernelContext*, Datum*)>;
+typedef Status (*ScalarAggregateFinalize)(KernelContext*, Datum*);

Review Comment:
   The style guide discourages `typedef` and recommends `using` instead



##########
cpp/src/arrow/compute/kernels/aggregate_basic_internal.h:
##########
@@ -448,9 +449,12 @@ struct MinMaxImpl : public ScalarAggregator {
     return Status::OK();
   }
 
-  Status ConsumeArray(const ArrayType& arr) {
+  Status ConsumeArray(const ArraySpan& arr_span) {
     StateType local;
 
+    // TODO(wesm): do not use ToArrayData
+    ArrayType arr(arr_span.ToArrayData());

Review Comment:
   Is this something that you plan to do in a later PR? Or can it just be done here?



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -202,14 +202,16 @@ Result<Datum> GroupByUsingExecPlan(const std::vector<Datum>& arguments,
   inputs.reserve(inputs.size() + keys.size());
   inputs.insert(inputs.end(), keys.begin(), keys.end());
 
-  ARROW_ASSIGN_OR_RAISE(auto batch_iterator,
-                        ExecBatchIterator::Make(inputs, ctx->exec_chunksize()));
+  ExecSpanIterator span_iterator;
+  ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make(inputs));
+  RETURN_NOT_OK(span_iterator.Init(batch, ctx->exec_chunksize()));
   BatchesWithSchema input;
   input.schema = schema(std::move(scan_fields));
-  ExecBatch batch;
-  while (batch_iterator->Next(&batch)) {
-    if (batch.length == 0) continue;
-    input.batches.push_back(std::move(batch));
+  ExecSpan span;
+  while (span_iterator.Next(&span)) {
+    if (span.length == 0) continue;
+    // TODO(wesm): investigate possibility of ExecPlans using ExecSpan

Review Comment:
   Should this be a JIRA instead? TODOs sprinkled in code are often ignored.



##########
cpp/src/arrow/compute/kernel.h:
##########
@@ -659,23 +656,20 @@ struct ScalarAggregateKernel : public Kernel {
 // ----------------------------------------------------------------------
 // HashAggregateKernel (for HashAggregateFunction)
 
-using HashAggregateResize = std::function<Status(KernelContext*, int64_t)>;
-
-using HashAggregateConsume = std::function<Status(KernelContext*, const ExecBatch&)>;
-
-using HashAggregateMerge =
-    std::function<Status(KernelContext*, KernelState&&, const ArrayData&)>;
+typedef Status (*HashAggregateResize)(KernelContext*, int64_t);
+typedef Status (*HashAggregateConsume)(KernelContext*, const ExecSpan&);
+typedef Status (*HashAggregateMerge)(KernelContext*, KernelState&&, const ArrayData&);
 
 // Finalize returns Datum to permit multiple return values
-using HashAggregateFinalize = std::function<Status(KernelContext*, Datum*)>;
+typedef Status (*HashAggregateFinalize)(KernelContext*, Datum*);

Review Comment:
   Same here.



##########
cpp/src/arrow/compute/kernels/aggregate_quantile.cc:
##########
@@ -471,45 +471,6 @@ struct ExactQuantiler<InType, enable_if_t<is_decimal_type<InType>::value>> {
   SortQuantiler<InType> impl;
 };
 
-template <typename T>
-Status ScalarQuantile(KernelContext* ctx, const Scalar& scalar, ExecResult* out) {
-  const QuantileOptions& options = QuantileState::Get(ctx);
-  using CType = typename TypeTraits<T>::CType;
-  ArrayData* output = out->array_data().get();
-  output->length = options.q.size();
-  auto out_type = IsDataPoint(options) ? scalar.type : float64();
-  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
-                        ctx->Allocate(output->length * out_type->byte_width()));
-
-  if (!scalar.is_valid || options.min_count > 1) {
-    output->null_count = output->length;
-    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(output->length));
-    bit_util::SetBitsTo(output->buffers[0]->mutable_data(), /*offset=*/0, output->length,
-                        false);
-    if (IsDataPoint(options)) {
-      CType* out_buffer = output->template GetMutableValues<CType>(1);
-      std::fill(out_buffer, out_buffer + output->length, CType(0));
-    } else {
-      double* out_buffer = output->template GetMutableValues<double>(1);
-      std::fill(out_buffer, out_buffer + output->length, 0.0);
-    }
-    return Status::OK();
-  }
-  output->null_count = 0;
-  if (IsDataPoint(options)) {
-    CType* out_buffer = output->template GetMutableValues<CType>(1);
-    for (int64_t i = 0; i < output->length; i++) {
-      out_buffer[i] = UnboxScalar<T>::Unbox(scalar);
-    }
-  } else {
-    double* out_buffer = output->template GetMutableValues<double>(1);
-    for (int64_t i = 0; i < output->length; i++) {
-      out_buffer[i] = DataPointToDouble(UnboxScalar<T>::Unbox(scalar), *scalar.type);
-    }
-  }
-  return Status::OK();
-}
-

Review Comment:
   Neat removal :-)



-- 
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 #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

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


-- 
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] wesm commented on pull request #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

   Thanks for the review. Will merge when CI green


-- 
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 #13630: ARROW-16852: [C++] Migrate remaining kernels to use ExecSpan, remove ExecBatchIterator

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

   Benchmark runs are scheduled for baseline = c445243a1455a7e06c7bed95596499e348056bac and contender = 4d931ff1c0f5661a9b134c514555c8d769001759. 4d931ff1c0f5661a9b134c514555c8d769001759 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/e236dc0baca944c39aba0b10373ee78d...416d0f20e0a7464aa2298b095ec0e24e/)
   [Failed :arrow_down:2.19% :arrow_up:0.21%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/6ca53824b9bb4138b82381c91d3f5287...df1a70fbca9a4fd699797283645f1dcf/)
   [Failed] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/78939a04be7e4e6b9c0047ad92848c03...2cbbc47be7b348beb9e905a8f589c4bb/)
   [Finished :arrow_down:1.03% :arrow_up:0.43%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/e39359f410b843cf967af5fa572cbc08...342374c3c2a04d398df9e01cf22cde4a/)
   Buildkite builds:
   [Failed] [`4d931ff1` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1151)
   [Failed] [`4d931ff1` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1156)
   [Failed] [`4d931ff1` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1143)
   [Finished] [`4d931ff1` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1160)
   [Failed] [`c445243a` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1150)
   [Failed] [`c445243a` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1155)
   [Failed] [`c445243a` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1142)
   [Finished] [`c445243a` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1159)
   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