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/06/23 13:23:51 UTC

[GitHub] [arrow] iChauster opened a new pull request, #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   **Currently a draft, please disregard asof join implementations which have yet to be merged**


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);
+    columns.push_back(rand.Float64(num_rows, -1e5, 1e5));

Review Comment:
   No particular reason. I will expose this as a property of the table.



-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > If what you're referring to multi-table joins (a left hand table joined with one or multiple right hand tables), all right hand tables share the same properties. I think it would be interesting to write a few benchmarks where the right hand tables have a bit more variation amongst themselves.
   
   I think I was mostly curious about differences in density between the left table and the right table(s).  For example, a dense left table and a sparse right table or a sparse left table and a dense right table.  The left table roughly defines the keyframes so I would expect the density of the left table to be more significant to performance than the density of the right table.


-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Some initial questions and I'll take a more detailed look soon.
   
   > Here is a very primitive version of our Asof Join Benchmarks (asof_join_benchmark.cc). Our main goal is to benchmark on four qualities: the effect of table density (the frequency of rows, e.g a row every 2s as opposed to every 1h over some time range), table width (# of columns), tids (# of keys), and multi-table joins. We also have a baseline comparison benchmark with hash joins (which is currently in this file).
   
   * Does the table density get applied uniformly over all input columns?  In other words, do you worry about cases where one input is very dense and the others are not so dense?
   * When you say multi-table joins how many tables are you testing?  Or is that a parameter?
   * # of columns and # of keys is good.  Eventually you will need to worry about data types I would think (probably more for payload columns than for key columns)
   
   > I think this needs some work before it goes into arrow. We currently run this benchmark by generating .feather files with Python via bamboo-streaming's datagen.py to represent each table, and then reading them in through cpp (see make_arrow_ipc_reader_node). We perhaps want to write a utility that allows us to do this in cpp, while varying many of the metrics I've mentioned above, or finding a way to generate those files as part of the benchmark.
   
   Another potential approach is to create a custom source node that generates data.  We do something like this for our tpc-h benchmark.  This allows us to isolate the scanning from the actual compute operations.  However, this kind of requires you to write the data generator in C++ which is probably not ideal.
   
   > There are also quite a large number of BENCHMARK_CAPTURE statements, as an immediate workaround to some limitations in Google Benchmarks. I haven't found a great non-verbose way to pass in the parameters needed (strings and vectors) while also having readable titles and details about the benchmark being written to the output file. Let me know if you have any advice about this / know some one who does.
   
   I'll think about this when I take a closer look.  Google benchmark is not necessarily the perfect tool for every situation.  But maybe there is something we can do.
   
   > While being processed, is a single source node dedicated a single thread?
   
   No.
   
   > How many threads can call InputReceived of the following node at once?
   
   That is mostly determined by the capacity of the executor which defaults to std::hardware_concurrency (e.g. one per core or two per core if you have hyperthreading) for the CPU thread pool.  At the moment it can be even greater than this number but this is an issue we are hoping to fix soon (limiting these calls to only CPU thread pool calls).
   
   > I was also wondering if you could clarify how the arrow threading engine would work for a node that has multiple inputs (an asof join / hash join ingesting from multiple source nodes, for example).
   
   Even if the asof join node had a single source node it could still be called many times.  You can think of a source node as a parallel while loop across all the batches:
   
   ```
   while (!source.empty()) {
     exec_context->executor->Spawn([this] {
       ExecBatch next_batch = ReadNext();
       output->InputReceived(next_batch);
     }
   }
   ```
   
   If there are multiple sources then they are still all submitting tasks to the same common thread pool so you won't see any more threads.  Also, in many unit tests and small use cases the source doesn't have enough data for more than one task so you often don't see the full scale of parallelism until you are testing larger datasets.
   
   There are some changes planned for the scheduler but most of what I said already will remain more or less true.  The future scheduler could potentially prioritize one source above others (for example, it often makes sense with a hash-join node to prioritize the build side input) so that is something to consider (for the as-of join node you probably want to read from all sources evenly I think).


-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Hey @westonpace, let me know if you got a chance to take a look at this!


-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =

Review Comment:
   The actual error: ValueOrDie called on an error: IOError: Cannot create temporary subdirectory in any of the platform temporary directories



-- 
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] iChauster commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Hey all,
   
   Wrote an initial version of a cpp data generation function called `MakeRandomTable` in `test_util.cc`. It is definitely a more simplified version of the python scripts, but it has everything we need to benchmark (frequency, width, key density, batch size variation). This also allowed us to simplify our benchmarking code quite a bit, since we no longer had to fight with Google Benchmarks over string parameters.
   
   I've removed the python data generation scripts from this PR and we will figure out how to get that sorted for a more comprehensive end-to-end benchmark which showcases the streaming features of the node. I've also removed the hash join benchmarks to keep in line with our other cpp microbenchmarks.
   
   Let me know what you think!


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs
+              arrow::compute::TableSourceNodeOptions(table, batch_size)),
+          rows, row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  size_t rows = 0;
+  size_t bytes = 0;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    left_table_properties.column_prefix = "lt";
+    left_table_properties.seed = 0;

Review Comment:
   Refactored as above ^



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {

Review Comment:
   I see. One alternative I can think of is reading in the files in memory through a table object, and using a `TableSourceNode` in the preparation stage. I'm assuming that would be less I/O intensive?



-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,40 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/// Specify properties of a table to be generated.
+struct TableGenerationProperties {
+  /// Indicates the amount of time between data points that lie between
+  /// the start and end parameters.
+  int time_frequency;
+  /// The number of additional random columns in the table.
+  int num_columns;
+  /// The number of unique keys in the table.
+  int num_ids;
+  /// Specifies the prefix of each randomly generated column.
+  std::string column_prefix;
+  /// Specifies the minimum value in the randomly generated column(s).
+  int min_column_value;
+  /// Specifies the maximum value in the randomly generated column(s).
+  int max_column_value;
+  /// The random seed the random array generator is given to generate the additional
+  /// columns.
+  int seed;
+  /// Specifies the beginning of 'time' recorded in the table, inclusive.
+  int start;
+  /// Specifies the end of 'time' recorded in the table, inclusive.
+  int end;
+};
+
+/// The table generated in accordance to the TableGenerationProperties has the following
+/// schema: time (int64) id (int32) [properties.column_prefix]0 (float64)

Review Comment:
   ```suggestion
   /// schema: time (int64) id (int32) [properties.column_prefix]{idx} (float64)
   ```
   How about this?



-- 
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] icexelloss commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   I see.
   
   For the purpose of this PR, we will convert some simple data generation to C++ to merge this PR and will move the end-to-end benchmarks to Python/Continuous bench marking. 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] pitrou commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   The problem is really how well it integrates into the development workflow. Right now every C++ micro-benchmark is an autonomous executable that you can launch with any additional setup.
   
   Also I think the discussion is being continued at the bottom of this PR (see Weston's comment and my answer).



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   The problem is really how well it integrates into the development workflow. Right now every C++ micro-benchmark is an autonomous executable that you can launch without any additional setup.
   
   Also I think the discussion is being continued at the bottom of this PR (see Weston's comment and my answer).



-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have these "random scripts scattered across the codebase" and my hope is what we can standardize creation of these benchmark dataset in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a baseline for all future time series benchmarks. Does that sounds acceptable?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize creation of these benchmark dataset in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a baseline for all future time series benchmarks. Does that sounds acceptable?



-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize these benchmark dataset generation in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a starting point for all future time series benchmarks. Does that sounds acceptable?



-- 
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] vibhatha commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }

Review Comment:
   May be? 
   
   ```suggestion
     std::vector<std::string> right_hand_tables;
     right_hand_tables.reserve(num_tables);
     
     for (int j = 1; j <= num_tables; j++) {
       right_hand_tables.push_back(
           generate_file_name(freq, width_table[width_index], num_ids_table[num_ids_index],
             std::to_string(j)));
     }
   ```



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs

Review Comment:
   I've removed these. We use `MakeExecNode` elsewhere without need for comments.



-- 
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] iChauster commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Hi @westonpace , friendly ping -- let me know when you get a chance to take a look at this!


-- 
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] iChauster commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Previous description: 
   
   Hi @westonpace,
   
   Here is a _very_ primitive version of our Asof Join Benchmarks (`asof_join_benchmark.cc`). Our main goal is to benchmark on four qualities: the effect of table density (the frequency of rows, e.g a row every 2s as opposed to every 1h over some time range), table width (# of columns), tids (# of keys), and multi-table joins. We also have a baseline comparison benchmark with hash joins (which is currently in this file).
   
   I think this needs some work before it goes into arrow. We currently run this benchmark by generating `.feather` files with Python via bamboo-streaming's datagen.py to represent each table, and then reading them in through cpp (see `make_arrow_ipc_reader_node`). We perhaps want to write a utility that allows us to do this in cpp, while varying many of the metrics I've mentioned above, or finding a way to generate those files as part of the benchmark.
   
   There are also quite a large number of `BENCHMARK_CAPTURE` statements, as an immediate workaround to some limitations in Google Benchmarks. I haven't found a great non-verbose way to pass in the parameters needed (strings and vectors) while also having readable titles and details about the benchmark being written to the output file. Let me know if you have any advice about this / know some one who does.


-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   :warning: Ticket **has not been started in JIRA**, please click 'Start Progress'.


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs
+              arrow::compute::TableSourceNodeOptions(table, batch_size)),
+          rows, row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  size_t rows = 0;
+  size_t bytes = 0;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    left_table_properties.column_prefix = "lt";
+    left_table_properties.seed = 0;

Review Comment:
   Hmm, although the input is all the same, the way this is written right now causes all the nodes to be tied to the lifespan of the `plan` (we would have to rebuild the nodes/plan each iteration). Maybe I'll refactor this slightly so we can at least keep the tables in memory and reuse those objects. Otherwise, we might have to go a level lower than `MakeExecNode`?



-- 
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] jonkeane commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > I see.
   > 
   > For the purpose of this PR, we will convert some simple data generation to C++ to merge this PR and will move the end-to-end benchmarks to Python/Continuous bench marking. Thanks!
   
   Sorry I was out last week — very much agree with what @pitrou and @westonpace have said here. Having macro benchmarks in our continuous benchmarking suite would be fantastic. There are a few readmes in that repo, but I'm always happy to help out if you get stuck putting something together—feel free to tag me in a PR or issue or wherever. 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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,40 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+Result<std::shared_ptr<Table>> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector;
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
+  for (int64_t time = properties.start; time <= properties.end;
+       time += properties.time_frequency) {
+    for (int32_t id = 0; id < properties.num_ids; id++) {
+      ARROW_RETURN_NOT_OK(time_column_builder.Append(time));
+      ARROW_RETURN_NOT_OK(id_column_builder.Append(id));
+    }
+  }
+
+  int64_t num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());

Review Comment:
   I apologize, sorry for missing this one!



-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Thanks for the help so far, @westonpace !
   > Does the table density get applied uniformly over all input columns? In other words, do you worry about cases where one input is very dense and the others are not so dense?
   If what you're referring to multi-table joins (a left hand table joined with one or multiple right hand tables), all right hand tables share the same properties. I think it would be interesting to write a few benchmarks where the right hand tables have a bit more variation amongst themselves.
   
   If you're referring to within the same table, I think our benchmarking approach currently doesn't test case joins with high time freq and low id density (and vice versa). We currently have set some 'baselines' and try to only vary one property at a time. For example, if we are interested in time frequency, we set the remaining properties to the baseline, and test time frequency in the symmetric case (LH table and RH table have the same time frequency for various values), and the asymmetric case.
   
   Since it seems performance varies mostly from time frequency and key density, I can see a few benchmarks written where we vary both at the same time.
   
   > When you say multi-table joins how many tables are you testing? Or is that a parameter?
   This is a parameter. We currently are testing 1 table to 51 table joins for `AsOf` (the number of inputs to the `AsOfJoin` node).
   
   > number of columns and number of keys is good. Eventually you will need to worry about data types I would think (probably more for payload columns than for key columns)
   Yes -- although I think this would require both a slight refactor for `AsOf` implementation as well as a more heavy change in the way we generate our tables. Currently, we don't have too much flexibility in data types in the bamboo streaming table generation.
   
   


-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }

Review Comment:
   Thank you, @vibhatha !



-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   I'll take a closer look at this tomorrow.  Another possibility for the python/c++ discrepancy would be to move the benchmarks themselves into python.  I don't think we have any python benchmarks in pyarrow today that get collected by conbench (there are some python benchmarks in https://github.com/ursacomputing/benchmarks) but it might be useful to have.
   
   @pitrou / @iChauster how would you feel about running these benchmarks in python?  It would require some migration but much of the logic could remain the same.
   @jonkeane might have some useful opinions here too


-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   @pitrou Any thoughts on this? Want to make sure we are on the right path for data generation 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] icexelloss commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > I would be ok with writing these benchmarks in Python, but then they won't be part of the C++ micro-benchmarks suite. Perhaps they can be part of conbench, is that ok?
   
   Running these benchmarks in Python seems OK to me (but not ideal). Another option - can we check in the benchmark data files in the repo? Apparently they cannot be very large but might work because this is for micro-benchmarking


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+// arrow::internal::TemporaryDir::Make("./benchmark_data/").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    boost::process::system("mkdir benchmark_data/");
+    std::error_code err;
+    boost::process::system("python3 -m generate_benchmark_files ./benchmark_data/", err);
+    if (err) {

Review Comment:
   Also, is there an elegant way to access the path to `benchmark_scripts/table_generation`? Thinking about injecting this into the PYTHONPATH (passing in the environment to boost::process). 



-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have these "random scripts scattered across the codebase" and my hope is what we can standardize creation of these benchmark dataset in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a baseline for all future time series benchmarks.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have these "random scripts scattered across the codebase" and my hope is what we can standardize creation of these benchmark dataset in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a baseline for all future time series benchmarks.



-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > > 
   > 
   > @icexelloss You can find a simple `write` operator example here:
   > 
   > https://github.com/apache/arrow/blob/998cca30c70a7e0bbc3f83957923f2e3019b314b/cpp/examples/arrow/execution_plan_documentation_examples.cc#L795
   > 
   > Is this helpful? Let me know I can help out with a better example if required.
   
   Thanks for this! Unfortunately I couldn't get my version to compile. I did find a workaround though -- I am basically calling `UnifySchema` on our table schemas to create our joined table schema, and reading in our batches into a table object after the plan executes. I then write the table into a CSV directly.


-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Also, because you mentioned the hash join graph file, I've been working with a Jupyter Notebook on the side to do the same (reading json benchmark files, reading into pandas and performing some simple visualizations). Is this something that would be helpful? Li mentioned I could publish it separately as well.


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), nullptr);
+
+  left_table_properties.column_prefix = "lt";
+  left_table_properties.seed = 0;
+  TableStats left_table_stats = MakeTable(left_table_properties);
+
+  size_t right_hand_rows = 0;
+  size_t right_hand_bytes = 0;
+  std::vector<TableStats> right_input_tables;
+  right_input_tables.reserve(num_right_tables);
+
+  for (int i = 0; i < num_right_tables; i++) {
+    right_table_properties.column_prefix = "rt" + std::to_string(i);
+    right_table_properties.seed = i + 1;
+    TableStats right_table_stats = MakeTable(right_table_properties);
+    right_hand_rows += right_table_stats.total_rows;
+    right_hand_bytes += right_table_stats.total_bytes;
+    right_input_tables.push_back(right_table_stats);
+  }
+
+  for (auto _ : state) {
+    state.PauseTiming();
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::vector<ExecNode*> input_nodes = {
+        MakeTableSourceNode(plan, left_table_stats.table, left_table_batch_size)};
+    input_nodes.reserve(right_input_tables.size() + 1);
+    for (TableStats table_stats : right_input_tables) {
+      input_nodes.push_back(
+          MakeTableSourceNode(plan, table_stats.table, right_table_batch_size));
+    }
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * join_node,
+                         MakeExecNode(factory_name, plan.get(), input_nodes, options));
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+  }
+
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_rows + right_hand_rows)),
+      benchmark::Counter::kIsRate);

Review Comment:
   Changed to just input_rows_per_second, remove total.



-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   <!--
     Licensed to the Apache Software Foundation (ASF) under one
     or more contributor license agreements.  See the NOTICE file
     distributed with this work for additional information
     regarding copyright ownership.  The ASF licenses this file
     to you under the Apache License, Version 2.0 (the
     "License"); you may not use this file except in compliance
     with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
     Unless required by applicable law or agreed to in writing,
     software distributed under the License is distributed on an
     "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
     KIND, either express or implied.  See the License for the
     specific language governing permissions and limitations
     under the License.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   Yes, I agree -- I created a larger set just for exploratory purposes and have been refining it down. We found the most interesting results varying key density and time frequency (in short, we observe some interesting piecewise (linear to constant) relationships with respect to real time and properties of our right hand tables, that is vastly different than the strictly linear relationship we observe for hash join). 
   
   The rest of the properties like column width, multi-table joins, and batch size seem more or less straightforward / linear -- so we can certainly reduce the amount of datapoints here which should alleviate the code bloating...
   
   Keep in mind we are also benchmarking hashjoin here as well for comparisons, which @icexelloss mentioned might not be suitable for this file here. Do you have any advice if those hashjoin comparison benchmarks are worth keeping in arrow?



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }

Review Comment:
   Left another comment about getting this to work locally!



-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =

Review Comment:
   `TemporaryDir::Make` takes a string prefix, not a full path. So it should not contain any directory separators such as `/`.



-- 
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] icexelloss commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   @iChauster What's needed for the "Draft" state to be removed? 


-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++ (could be biased because I am more familiar with Python). I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize these benchmark dataset generation in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a starting point for all future time series benchmarks. Does that sounds acceptable?



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);

Review Comment:
   After some testing, it seems these numbers are identical.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs
+              arrow::compute::TableSourceNodeOptions(table, batch_size)),
+          rows, row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  size_t rows = 0;
+  size_t bytes = 0;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    left_table_properties.column_prefix = "lt";
+    left_table_properties.seed = 0;

Review Comment:
   Hmm, although the input is all the same, the way this is written right now causes all the nodes to be tied to the lifespan of the `plan`. Maybe I'll refactor this slightly so we can at least keep the tables in memory and reuse those objects. Otherwise, we might have to go a level lower than `MakeExecNode`?



-- 
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 merged pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


-- 
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] icexelloss commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Side topic: Is there an example to use a "write node" in Acero? We had some code a few month ago but it seems it no longer works (probably due to some API changes):
   https://github.com/twosigma/bamboo-streaming/blob/master/experiments/msw_fulltest/writernode.cpp#L26
   
   And I couldn't find an example/test in Acero that uses a write node - if there is, can you perhaps point me to one?


-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Can you update the description?  This becomes the commit message when things merge and it still claims to be a draft.


-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++ (could be biased because I am more familiar with Python, Ivan is an intern working for me in the summer but I will be maintaining this code going forward). I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize these benchmark dataset generation in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a starting point for all future time series benchmarks. Does that sounds acceptable?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++ (could be biased because I am more familiar with Python, Ivan is an intern working for me in the summer and I will be maintaining this code going forward). I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize these benchmark dataset generation in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a starting point for all future time series benchmarks. Does that sounds acceptable?



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   Yeah, I think we would prefer to move some of the python bamboo streaming data generation scripts over into arrow. Do you have any advice on where to put them? (it seems slightly odd to be putting python in a folder under cpp)...



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   Yeah, I think we would prefer to move some of the python bamboo streaming data generation scripts over into arrow than writing a table generator in cpp. Do you have any advice on where to put them? (it seems slightly odd to be putting python in a folder under cpp)...



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   Yes, I agree -- I created a larger set just for exploratory purposes and have been refining it down. We found the most interesting results varying key density and time frequency (we observe some piecewise linear and constant relationships between real_time and the respective property, for example). 
   
   The rest of the properties like column width, multi-table joins, and batch size seem more or less straightforward / linear -- so we can certainly reduce the amount of datapoints here.
   
   Keep in mind we are also benchmarking hashjoin here as well for comparisons, which @icexelloss mentioned might not be suitable for this file here. Do you have any advice if those hashjoin comparison benchmarks are worth keeping in arrow?



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =

Review Comment:
   @westonpace, this doesn't work for me locally, is my usage correct?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+// arrow::internal::TemporaryDir::Make("./benchmark_data/").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    boost::process::system("mkdir benchmark_data/");
+    std::error_code err;
+    boost::process::system("python3 -m generate_benchmark_files ./benchmark_data/", err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+std::shared_ptr<arrow::Schema> get_resultant_join_schema(

Review Comment:
   I have a few pieces of code (this function and `output_results_to_csv`) that are related to reading the output of tables from plans / `StartAndCollect` that I used to verify correctness of outputs. I haven't really seen examples like this around the codebase but I'm not sure they fit here. Should I move this into some utility?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+// arrow::internal::TemporaryDir::Make("./benchmark_data/").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    boost::process::system("mkdir benchmark_data/");
+    std::error_code err;
+    boost::process::system("python3 -m generate_benchmark_files ./benchmark_data/", err);
+    if (err) {

Review Comment:
   I think you mentioned handling errors gracefully, is this sufficient? I suppose we should try and exit the program as well.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -460,42 +460,39 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
-std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+std::shared_ptr<Table> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
   int total_columns = properties.num_columns + 2;
   std::vector<std::shared_ptr<Array>> columns;
   columns.reserve(total_columns);
-  arrow::FieldVector field_vector = arrow::FieldVector();
+  arrow::FieldVector field_vector;
   field_vector.reserve(total_columns);
 
-  field_vector.push_back(std::make_shared<Field>("time", int64()));
-  field_vector.push_back(std::make_shared<Field>("id", int32()));
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
 
-  int num_rows = 0;
-  std::vector<int64_t> time_column;
-  std::vector<int32_t> id_column;
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
   for (int time = properties.start; time <= properties.end;
-         time += properties.time_frequency) {
+       time += properties.time_frequency) {
     for (int id = 0; id < properties.num_ids; id++) {
-      time_column.push_back(time);
-      id_column.push_back(id);
-      num_rows += 1;
+      time_column_builder.Append(time);
+      id_column_builder.Append(id);
     }
   }
-  std::shared_ptr<Array> time_array;
-  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
-  columns.push_back(time_array);
-  std::shared_ptr<Array> id_array;
-  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
-  columns.push_back(id_array);
+
+  int num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());

Review Comment:
   Is using `ValueOrDie()` here okay? I tried using `CHECK_OK_AND_ASSIGN` but I don't think that works in a non-void function.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);
+    columns.push_back(rand.Float64(num_rows, -1e5, 1e5));
+  }
+  std::shared_ptr<arrow::Schema> schema = std::make_shared<arrow::Schema>(field_vector);

Review Comment:
   A bit of a cpp clarification question, why does `field_vector` need to be moved 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] westonpace commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {

Review Comment:
   Can you rename this table so it is obvious it has something to do with asof join / time series data?  We have a couple of different random generation utilities already and so it will help to distinguish.



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));

Review Comment:
   ```suggestion
     field_vector.push_back(field("time", int64()));
     field_vector.push_back(field("id", int32()));
   ```
   
   What you have is fine, this is just a slightly more compact shortcut



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();

Review Comment:
   ```suggestion
     arrow::FieldVector field_vector;
   ```



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));

Review Comment:
   ```suggestion
       field_vector.push_back(field(properties.column_prefix + std::to_string(i), float64()));
   ```



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;

Review Comment:
   ```suggestion
         num_rows++;
   ```



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;

Review Comment:
   You could also just do `int num_rows = time_column.size()` after the for-loop.  Or you could precompute this (`((properties.end - properties.start) / properties.time_frequency) * properties.num_ids`) but it works how you have it.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";

Review Comment:
   ```suggestion
   static const char* kTimeCol = "time";
   static const char* kKeyCol = "id";
   ```
   
   It's a bit weird but the Google style guide suggests kConstantCase for constants.



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {

Review Comment:
   ```suggestion
   std::shared_ptr<Table> MakeRandomTable(const TableGenerationProperties& properties) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs

Review Comment:
   If a parameter name is not obvious it should be commented as `/*factory_name=*/"table_source"`.  Also, we should use the same argument names as the method declares (e.g. `factory_name` and not `registered type`.



##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,38 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/*

Review Comment:
   Can you use the `/// \brief`... style of commenting found elsewhere in the code base? 



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);
+    columns.push_back(rand.Float64(num_rows, -1e5, 1e5));

Review Comment:
   Is there a particular reason for bounding the range of values to `-1e5, 1e5`?



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);

Review Comment:
   It would be slightly more efficient to use `Int64Builder` and `Int32Builder` instead of creating a vector first and then copying the vector into an array.



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);
+    columns.push_back(rand.Float64(num_rows, -1e5, 1e5));
+  }
+  std::shared_ptr<arrow::Schema> schema = std::make_shared<arrow::Schema>(field_vector);

Review Comment:
   ```suggestion
     std::shared_ptr<arrow::Schema> schema = schema(std::move(field_vector));
   ```
   another shortcut method :)



##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,38 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/*
+  Specify properties of a table to be generated.
+    - num_ids is the number of unique keys in the table
+    - time_frequency indicates the amount of time between data points that lie between
+  start and end (inclusive)
+    - num_columns indicates the amount of random columns in the table
+    - column_prefix specifies the prefix each randomly generated column should have
+    - seed is the random seed the random array generator is given to generate the random
+  columns
+    - start specifies the beginning of 'time' recorded in the table
+    - end specifies the end of 'time' recorded in the table
+*/
+struct TableGenerationProperties {
+  int time_frequency;
+  int num_columns;
+  int num_ids;
+  std::string column_prefix;
+  uint seed;
+  int start;
+  int end;
+};
+
+/*

Review Comment:
   Same suggestion as above regarding commenting style.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;

Review Comment:
   ```suggestion
   constexpr int kDefaultStart = 0;
   constexpr int kDefaultEnd = 500;
   ```



##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,38 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/*
+  Specify properties of a table to be generated.
+    - num_ids is the number of unique keys in the table
+    - time_frequency indicates the amount of time between data points that lie between
+  start and end (inclusive)
+    - num_columns indicates the amount of random columns in the table
+    - column_prefix specifies the prefix each randomly generated column should have
+    - seed is the random seed the random array generator is given to generate the random

Review Comment:
   These can be comments on the fields themselves.  See `AsofJoinNodeOptions` in `arrow/compute/exec/options.h`



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs
+              arrow::compute::TableSourceNodeOptions(table, batch_size)),
+          rows, row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  size_t rows = 0;
+  size_t bytes = 0;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    left_table_properties.column_prefix = "lt";
+    left_table_properties.seed = 0;

Review Comment:
   Since the seed is constant you should be able to generate your inputs outside this loop, once.  Then you can run many iterations on the same inputs.  This should speed up the runtime of this benchmark.



##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,38 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/*
+  Specify properties of a table to be generated.
+    - num_ids is the number of unique keys in the table
+    - time_frequency indicates the amount of time between data points that lie between
+  start and end (inclusive)
+    - num_columns indicates the amount of random columns in the table
+    - column_prefix specifies the prefix each randomly generated column should have
+    - seed is the random seed the random array generator is given to generate the random
+  columns
+    - start specifies the beginning of 'time' recorded in the table
+    - end specifies the end of 'time' recorded in the table
+*/
+struct TableGenerationProperties {
+  int time_frequency;
+  int num_columns;
+  int num_ids;
+  std::string column_prefix;
+  uint seed;
+  int start;
+  int end;
+};
+
+/*
+  The table generated in accordance to the TableGenerationProperties has the following
+  schema: time (int64) id (int32) [properties.column_prefix]0 (float64)
+      [properties.column_prefix]1 (float64)
+      ...
+      [properties.column_prefix][properties.num_columns] (float64)
+  Each id has rows corresponding to a singular data point in the time range (start, end,
+  time_frequency). The table is sorted by time.
+*/
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties);

Review Comment:
   ```suggestion
   std::shared_ptr<Table> MakeRandomTable(const TableGenerationProperties& properties);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+const int default_start = 0;
+const int default_end = 500;
+
+struct TableSourceNodeStats {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableSourceNodeStats MakeTableSourceNode(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan, TableGenerationProperties properties,
+    int batch_size) {
+  std::shared_ptr<Table> table = MakeRandomTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {*arrow::compute::MakeExecNode(
+              "table_source",  // registered type
+              plan.get(),      // execution plan
+              {},              // inputs
+              arrow::compute::TableSourceNodeOptions(table, batch_size)),
+          rows, row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  size_t rows = 0;
+  size_t bytes = 0;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    left_table_properties.column_prefix = "lt";
+    left_table_properties.seed = 0;
+    TableSourceNodeStats left_table_stats =
+        MakeTableSourceNode(plan, left_table_properties, left_table_batch_size);
+    std::vector<ExecNode*> inputs = {left_table_stats.execNode};
+    int right_hand_rows = 0;
+    size_t right_hand_bytes = 0;
+    for (int i = 0; i < num_right_tables; i++) {
+      std::ostringstream string_stream;
+      string_stream << "rt" << i;
+      right_table_properties.column_prefix = string_stream.str();

Review Comment:
   ```suggestion
         right_table_properties.column_prefix = "rt" + std::to_string(i);
   ```



-- 
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] iChauster commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   @westonpace are these failed checks related?


-- 
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 pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   I would be ok with writing these benchmarks in Python, but then they won't be part of the C++ micro-benchmarks suite. Perhaps they can be part of conbench, is that ok?


-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   Hmm, would it be possible to do the data generation in C++? I don't think we want to come to a situation where some benchmarks launch random Python scripts scattered accross the codebase, and that even rely on PyArrow being installed.



-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,456 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+arrow::internal::TemporaryDir::Make("asof_benchmarks").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    std::error_code err;
+    std::ostringstream call_stream;
+    call_stream << "python3 -m generate_benchmark_files " << temp_dir.get()->path().ToString();
+    boost::process::system(call_stream.str(), err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+    return arrow::Status::OK();
+  }
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static ReaderNodeTableProperties
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,

Review Comment:
   ```suggestion
   MakeArrowIpcReaderNode(std::shared_ptr<arrow::compute::ExecPlan>& plan,
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,456 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+arrow::internal::TemporaryDir::Make("asof_benchmarks").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    std::error_code err;
+    std::ostringstream call_stream;
+    call_stream << "python3 -m generate_benchmark_files " << temp_dir.get()->path().ToString();
+    boost::process::system(call_stream.str(), err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+    return arrow::Status::OK();
+  }
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static ReaderNodeTableProperties
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename,
+                           int64_t batch_size) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);

Review Comment:
   ```suggestion
         sizeof(double) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,456 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+arrow::internal::TemporaryDir::Make("asof_benchmarks").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    std::error_code err;
+    std::ostringstream call_stream;
+    call_stream << "python3 -m generate_benchmark_files " << temp_dir.get()->path().ToString();
+    boost::process::system(call_stream.str(), err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+    return arrow::Status::OK();
+  }
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static ReaderNodeTableProperties
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename,
+                           int64_t batch_size) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), plan->exec_context()->executor());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, row_size * rows};
+}
+
+static ReaderNodeTableProperties make_table_source_node(

Review Comment:
   ```suggestion
   static ReaderNodeTableProperties MakeTableSourceNode(
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+// arrow::internal::TemporaryDir::Make("./benchmark_data/").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    boost::process::system("mkdir benchmark_data/");
+    std::error_code err;
+    boost::process::system("python3 -m generate_benchmark_files ./benchmark_data/", err);
+    if (err) {

Review Comment:
   An abort (or `FAIL` call, which is just an abort + print when outside of gtest) would be good for exiting.  I think something like `FAIL() << "Could not generate python files: " << err.message()` would be fine.
   
   Not sure about the path to `benchmark_scripts/table_generation`.  People running tests will generally be in some kind of copy of the GH repo.  So you could just start at CWD and walk up the file tree looking for `cpp/src/benchmark_scripts/table_generation`.  Then allow that to be overridden by an environment variable.  I'm guessing that would suffice in most cases.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,456 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+arrow::internal::TemporaryDir::Make("asof_benchmarks").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    std::error_code err;
+    std::ostringstream call_stream;
+    call_stream << "python3 -m generate_benchmark_files " << temp_dir.get()->path().ToString();
+    boost::process::system(call_stream.str(), err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;

Review Comment:
   ```suggestion
     std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader_;
     int next_;
   ```
   
   Perhaps `next_row_group_` or `next_batch_index_`?  But `next_` is ok too.



##########
cpp/src/arrow/compute/exec/benchmark_scripts/table_generation/batch_process.py:
##########
@@ -0,0 +1,104 @@
+import json
+from datagen import *
+
+from pyarrow import ipc
+from pyarrow.ipc import RecordBatchStreamWriter
+import pyarrow as pa
+import pyarrow.parquet as pq
+
+
+class BatchStreamStats:
+    def __init__(self, filename):
+        self.filename = filename
+        self.num_batches = 0
+        self.num_rows = 0
+        self.num_bytes = 0
+
+    def close(self):
+        if filename:
+            with open(filename, "w") as f:
+                f.write(f"num_batches = {self.num_batches}\n"
+                        f"num_rows = {self.num_rows}\n"
+                        f"num_bytes = {self.num_bytes}\n"
+                )
+
+    def process(self, batch_source):
+        for batch in batch_source:
+            self.num_batches += 1
+            self.num_rows += batch.num_rows
+            self.num_bytes += batch.nbytes
+
+class BatchStreamParquetFileWriter:
+    def __init__(self, filename):
+        self.filename = filename
+        self.pqwriter = None
+
+    def close(self):
+        if self.pqwriter is not None:
+            self.pqwriter.close()
+
+    def process(self, batch_source):
+        for batch in batch_source:
+            table = pa.Table.from_batches([batch], schema=batch.schema)
+            if self.pqwriter is None:
+                self.pqwriter = pq.ParquetWriter(self.filename, table.schema)
+            self.pqwriter.write_table(table)
+
+class BatchStreamFeatherFileWriter:
+    def __init__(self, filename):
+        self.filename = filename
+        self.file = None
+        self.ipcwriter = None
+
+    def close(self):
+        if self.ipcwriter is not None:
+            self.ipcwriter.close()
+        if self.file is not None:
+            self.file.close()
+
+    def process(self, batch_source):
+        if self.file is None:
+            self.file = open(self.filename, "wb")
+        for batch in batch_source:
+            if self.ipcwriter is None:
+                self.ipcwriter = ipc.new_file(self.file, batch.schema)
+            self.ipcwriter.write_batch(batch)
+
+class BatchStreamRbsWriter:
+    def __init__(self, filename):
+        self.filename = filename
+        self.file = None
+        self.rbswriter = None
+
+    def close(self):
+        if self.rbswriter is not None:
+            self.rbswriter.close()
+        if self.file is not None:
+            self.file.close()
+
+    def process(self, batch_source):
+        if self.file is None:
+            self.file = open(self.filename, "wb")
+        for batch in batch_source:
+            if self.rbswriter is None:
+                self.rbswriter = RecordBatchStreamWriter(self.file, batch.schema)
+            self.rbswriter.write_batch(batch)

Review Comment:
   `pyarrow.dataset.write_dataset` accepts an iterable of record batches as its input.  Each of these classes could probably be replaced by a single call to `write_dataset`.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,456 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =
+arrow::internal::TemporaryDir::Make("asof_benchmarks").ValueOrDie();
+
+struct ReaderNodeTableProperties {
+  ExecNode* execNode;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+// requires export PYTHONPATH=/path/to/benchmark_scripts/table_generation
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    std::error_code err;
+    std::ostringstream call_stream;
+    call_stream << "python3 -m generate_benchmark_files " << temp_dir.get()->path().ToString();
+    boost::process::system(call_stream.str(), err);
+    if (err) {
+      std::cerr << "Could not generate python files." << std::endl;
+      std::cerr << "Error Message: " << err.message() << std::endl;
+    }
+    createdBenchmarkFiles = true;
+  }
+}
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"20_cols", "100_cols", "500_cols"};
+  std::string num_ids_table[] = {"100_ids", "5000_ids", "10000_ids"};
+
+  std::vector<std::string> right_hand_tables;
+  right_hand_tables.reserve(num_tables);
+
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(generate_file_name(
+        freq, width_table[width_index], num_ids_table[num_ids_index], std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+    return arrow::Status::OK();
+  }
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static ReaderNodeTableProperties
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename,
+                           int64_t batch_size) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), plan->exec_context()->executor());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, row_size * rows};
+}
+
+static ReaderNodeTableProperties make_table_source_node(
+    std::shared_ptr<arrow::compute::ExecPlan>& plan,
+    std::shared_ptr<arrow::fs::FileSystem>& fs, const std::string& filename,
+    int64_t batch_size) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::vector<std::shared_ptr<RecordBatch>> record_batches;
+  for (int i = 0; i < in_reader.get()->num_record_batches(); i++) {
+    record_batches.push_back(in_reader.get()->ReadRecordBatch(i).ValueOrDie());
+  }
+  std::shared_ptr<Table> table = Table::FromRecordBatches(record_batches).ValueOrDie();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size = sizeof(_Float64) * (in_reader->schema()->num_fields() - 2) +

Review Comment:
   ```suggestion
     size_t row_size = sizeof(double) * (in_reader->schema()->num_fields() - 2) +
   ```



##########
cpp/src/arrow/compute/exec/benchmark_scripts/table_generation/datagen.py:
##########
@@ -0,0 +1,868 @@
+import copy
+from datetime import date, datetime, timedelta
+from pandas.tseries.frequencies import to_offset
+import pyarrow as pa
+import numpy as np
+
+_DEFAULT_BATCH_SIZE = 100
+
+_DTYPE_IND = np.int32
+_DTYPE_ID = np.int32
+# strings in detail-arrays should be no longer than 300
+_DTYPE_STRING = np.dtype("<U300")
+
+_DEFAULT_NUM_IDS = 5000
+_DEFAULT_NUM_FEATURE_CATEGORIES = 100
+
+# the arrays below provide mock details
+# an array shorter than the number of ids is used in round-robin fashion
+_DEFAULT_DETAIL_EXCHANGE = np.array(
+    ["N", "O"], dtype=_DTYPE_STRING)
+_DEFAULT_DETAIL_NAME = np.array(
+    ["American Electric Power", "Canada Electric Power"], dtype=_DTYPE_STRING)
+_DEFAULT_DETAIL_COUNTRY_NAME = np.array(
+    ["United States", "Canada"], dtype=_DTYPE_STRING)
+_DEFAULT_DETAIL_SHORT_NAME = np.array(
+    ["A", "B"], dtype=_DTYPE_STRING)
+_DEFAULT_DETAIL_CURRENCY_ID = np.array([10203040, 50607080], dtype=_DTYPE_ID)
+_DEFAULT_DETAIL_CODE = np.array([10203040, 50607080], dtype=_DTYPE_ID)
+_DEFAULT_DETAIL_COUNTRY_ID = np.array([10203040, 50607080], dtype=_DTYPE_ID)
+_DEFAULT_DETAIL_TYPE_ID = np.array([10203040, 50607080], dtype=_DTYPE_ID)
+
+
+def _arrow_type(dtype):
+    if dtype == np.int8:
+        return pa.int8()
+    elif dtype == np.int16:
+        return pa.int16()
+    elif dtype == np.int32:
+        return pa.int32()
+    elif dtype == np.int64:
+        return pa.int64()
+    elif dtype == np.uint8:
+        return pa.uint8()
+    elif dtype == np.uint16:
+        return pa.uint16()
+    elif dtype == np.uint32:
+        return pa.uint32()
+    elif dtype == np.uint64:
+        return pa.uint64()
+    elif dtype == np.float16:
+        return pa.float16()
+    elif dtype == np.float32:
+        return pa.float32()
+    elif dtype == np.float64:
+        return pa.float64()
+    elif dtype == _DTYPE_STRING:
+        return pa.string()
+    else:
+        raise RuntimeError("unknown dtype %r" % dtype)
+
+def _doc(*args, **kwargs):
+    def _doc_wrap(f):
+        f.__doc__ = f.__doc__.format(*args, **kwargs)
+        return f
+    return _doc_wrap
+
+def _max_freq(freqs):
+    offsets = [to_offset(freq) for freq in freqs]
+    return freqs[offsets.index(max(offsets))]
+
+class SimpleMidnightCalendar:
+    """
+    A simple midnight calendar. All days are business days, and business hours
+    are midnight only, irrespective of holidays or time zone.
+    """
+
+    def is_business_day(self, curr_date):
+        """
+        Returns True
+        """
+        return True
+
+    def business_begin_ns(self, curr_date):
+        """
+        Returns midnight
+        """
+        return 0
+
+    def business_end_ns(self, curr_date):
+        """
+        Returns midnight
+        """
+        return 0
+
+class SimpleBusinessCalendar:
+    """
+    A simple business calendar. Business days are Monday through Friday, and business
+    hours are 9:30 to 14:00, irrespective of holidays or time zone.
+    """
+
+    def is_business_day(self, curr_date):
+        """
+        Returns True for a date on Monday through Friday
+        """
+        return curr_date.weekday() < 5
+
+    def business_begin_ns(self, curr_date):
+        """
+        Returns nanoseconds-since-midnight at 9:30
+        """
+        return 9 * 3600 * 1000000000
+
+    def trading_end_ns(self, curr_date):
+        """
+        Returns nanoseconds-since-midnight at 16:00
+        """
+        return 16 * 3600 * 1000000000
+
+class SimpleDailyBusinessCalendar:
+    """
+    A simple daily business calendar. Business days are Monday through Friday, and
+    business hours are midnight only, irrespective of holidays or time zone.
+    """
+
+    def is_business_day(self, curr_date):
+        """
+        Returns True for a date on Monday through Friday
+        """
+        return curr_date.weekday() < 5
+
+    def business_begin_ns(self, curr_date):
+        """
+        Returns midnight
+        """
+        return 0
+
+    def trading_end_ns(self, curr_date):
+        """
+        Returns midnight
+        """
+        return 0
+
+class ArrayDistribution:
+    def __init__(self, dtype):
+        self.dtype = dtype
+
+    def __call__(self, rng, block_size, num_blocks):
+        raise NotImplementedError
+
+class ArrayAllDistribution(ArrayDistribution):
+    """
+    Samples in order all elements of a given array.
+    """
+    def __init__(self, arr):
+        super().__init__(arr.dtype)
+        self._arr = arr
+
+    def __call__(self, rng, block_size, num_blocks):
+        ind = np.arange(block_size, dtype=_DTYPE_ID)
+        return self._arr[np.tile(ind % len(self._arr), num_blocks)]
+
+class ArrayUniformDistribution(ArrayDistribution):
+    """
+    Samples uniformly the elements of a given array.
+    """
+    def __init__(self, arr):
+        super().__init__(arr.dtype)
+        self._arr = arr
+
+    def __call__(self, rng, block_size, num_blocks):
+        ind = rng.integers(low=0, high=len(self._arr),
+                size=block_size*num_blocks, dtype=_DTYPE_IND)
+        return self._arr[ind]
+
+class DriftOptions:
+    drift_blocks: int = 1 # number of blocks between drift chances
+    prob_drift: float = 1 # probability of each drift chance
+    prob_add: float = 0.1 # probability of adding an item on drift cycle
+    prob_del: float = 0.1 # probability of deleting an item on drift cycle
+    frac_init: float = 0.8 # initial fraction of items inside
+    frac_min: float = 0.05 # minimum fraction of items inside
+    frac_max: float = 1.0 # maximum fraction of items inside
+
+class ArrayUniformDriftDistribution(ArrayDistribution):
+    """
+    Samples uniformly from a drifting set of elements of a given array.
+
+    The sampler works in units of blocks, defined by the block_size parameter
+    of the call operator, and is controlled by DriftOptions. For a fixed seed,
+    it generates a fixed sequence of blocks, regardless of how many blocks are
+    generated in each invocation of the call operator.
+
+    The sampler maintains a subset of elements of the array from which to sample
+    a block. This subset is initially a fraction DriftOptions.frac_init of the
+    elements of the array. After each block, there is a probability
+    DriftOptions.prob_drift that a next subset of elements is created by adding
+    elements with probability DriftOptions.prob_add and deleting elements with
+    probability DriftOptions.prob_del subject to this subset being a fraction
+    between DriftOptions.frac_min and DriftOptions.frac_max of the elements of
+    the array. The set of elements used for sampling is updated only each
+    DriftOptions.drift_blocks blocks to the union of these next sets of elements
+    created over these blocks. Thus, two samplers that differ only in their
+    DriftOptions.drift_blocks setting will sample from synchronized sets of
+    elements. For example, one sampler generates a block every hour using a set
+    of elements associated with one hour while another generates every 24 hours
+    using the union of the sets of elements associated with each of 24 hours.
+    """
+    def __init__(self, arr, drift_opts = None):
+        super().__init__(arr.dtype)
+        self._arr = arr
+        self._drift_opts = copy.deepcopy(drift_opts) or DriftOptions()
+        # countdown in blocks until next drift chance
+        self._driftb = self._drift_opts.drift_blocks
+        # constant range of indices
+        self._ind = np.arange(len(arr), dtype=_DTYPE_IND)
+        # permutation of indices into self._arr
+        self._ptr = np.arange(len(arr), dtype=_DTYPE_IND)
+        # temporary buffer
+        self._buf = np.empty_like(arr)
+        # selects inside-indices of self._ind
+        self._flag = np.empty_like(arr, dtype=bool)
+        # selects inside-indices of self._ind
+        self._draw = np.empty_like(arr, dtype=bool)
+        # probability of item to be added or deleted
+        self._prob = np.empty_like(arr, dtype=np.float64)
+        # inside/outside part below/at-or-above self._n
+        self._n = int(len(arr) * self._drift_opts.frac_init)
+        # minimum allowed value of n
+        self._n0 = int(len(arr) * self._drift_opts.frac_min)
+        # maximum allowed value of n
+        self._n1 = int(len(arr) * self._drift_opts.frac_max)
+        # initially, select inside-indices as range(n)
+        self._flag[:] = False
+        self._draw[self._ptr[:self._n]] = True
+        self._draw[self._ptr[self._n:]] = False
+
+    def __call__(self, rng, block_size, num_blocks):
+        # prepare result indices into self._arr
+        result_size = block_size * num_blocks
+        result = np.empty((result_size,), dtype=_DTYPE_IND)
+        # one block at a time, so two sequences of calls with fixed block_size
+        # produce the same draws from rng regardless of how num_blocks varies
+        for i in range(0, result_size, block_size):
+            result[i:i+block_size] = self._do_draw(rng, block_size)
+            # drift not yet for drawing
+            if rng.random() < self._drift_opts.prob_drift:
+                self._do_drift(rng)
+                self._flag[self._ptr[:self._n]] = True
+            # countdown to next drift chance
+            self._driftb -= 1
+            if self._driftb == 0:
+                self._driftb = self._drift_opts.drift_blocks
+                # drift for drawing
+                self._draw[:] = self._flag[:]
+                # reselect inside-indices
+                self._flag[:] = False
+        return self._arr[result]
+
+    def _do_draw(self, rng, block_size):
+        # uniform draw from the set of selected indices into current block
+        ind1 = self._ind[self._draw]
+        ind2 = rng.integers(low=0, high=len(ind1),
+                size=block_size, dtype=_DTYPE_IND)
+        return ind1[ind2]
+
+    def _do_drift(self, rng):
+        n = self._n
+        # shuffle inside and outside parts
+        rng.shuffle(self._ptr[:n])
+        rng.shuffle(self._ptr[n:])
+        # repartition
+        rng.random(size=len(self._prob), out=self._prob)
+        n_add = sum(self._prob[n:] < self._drift_opts.prob_add)
+        n_del = sum(self._prob[:n] < self._drift_opts.prob_del)
+        n_adj = n + n_add - n_del
+        n_add -= max(0, n_adj - self._n1)
+        n_del -= max(0, self._n0 - n_adj)
+        # update partition
+        na, nb, nn = n - n_del, n + n_add, n + n_add - n_del
+        self._buf[na:nb] = self._ptr[na:nb] # save del and add parts
+        self._ptr[na:nn] = self._buf[n:nb]  # place add part inside
+        self._ptr[nn:nb] = self._buf[na:n]  # place del part outside
+        self._n = nn
+
+
+class NormalDistribution(ArrayDistribution):
+    """
+    Samples from a normal distribution.
+    """
+    def __init__(self, loc=0.0, scale=1.0):
+        super().__init__(np.float64)
+        self.loc = loc
+        self.scale = scale
+
+    def __call__(self, rng, block_size, num_blocks):
+        return rng.normal(
+            loc=self.loc, scale=self.scale, size=block_size*num_blocks)
+
+class SquaredNormalDistribution(ArrayDistribution):
+    """
+    Samples from a squared normal distribution.
+    """
+    def __init__(self, loc=0.0, scale=1.0):
+        super().__init__(np.float64)
+        self.loc = loc
+        self.scale = scale
+
+    def __call__(self, rng, block_size, num_blocks):
+        x = rng.normal(
+            loc=self.loc, scale=self.scale, size=block_size*num_blocks)
+        return x * x
+
+def _get_array(ids):
+    if isinstance(ids, np.ndarray):
+        return ids
+    elif type(ids) in [type(0), type([])]:
+        return np.arange(ids, dtype=_DTYPE_ID)
+    raise ValueError("invalid ids " + str(ids))
+
+def _get_dist(dist):
+    if type(dist) == type(""):
+        if dist == "all":
+            return ArrayAllDistribution
+        elif dist == "uniform":
+            return ArrayUniformDistribution
+        elif dist == "uniform-drift":
+            return ArrayUniformDriftDistribution
+    elif isinstance(dist, Distribution):
+        return dist
+    raise ValueError("invalid dist " + str(dist))
+
+def _get_id_dist(ids=_DEFAULT_NUM_IDS, dist="all"):
+    return _get_dist(dist)(_get_array(ids))
+
+def _get_category_id_dist(ids=_DEFAULT_NUM_FEATURE_CATEGORIES, dist="all"):
+    return _get_dist(dist)(_get_array(ids))
+
+class DataGenerator:
+    """
+    A data generator. See the generate(...) method for details.
+    """
+
+    def _normalize_date(self, date_value, default_date):
+        """
+        Normalizes to a date. If the date value is a string, it is parsed in ISO
+        format and returned. If it is already a date, it is returned as is. If
+        it is None, it defaults to the given default value.
+        """
+        if isinstance(date_value, str):
+            return date.fromisoformat(date_value)
+        elif isinstance(date_value, date):
+            return date_value
+        elif date_value is None:
+            return default_date
+        else:
+            raise ValueError("invalid date: " + str(date_value))
+
+    def _date_to_midnight_ns(self, curr_date):
+        """
+        Returns a datetime at midnight on the given date.
+        """
+        midnight = datetime.combine(curr_date, datetime.min.time()).timestamp()
+        return int(midnight * 1000000000)
+
+    def generate(self, begin_date=None, end_date=None, seed=None, calendar=None,
+            freq="200ms", time_dist="constant", dists=None, num_ids=1,
+            batch_size=_DEFAULT_BATCH_SIZE):
+        """
+        Generates market data using these paremeters:
+
+            begin_date  the date on which to begin generating.
+                        Defaults to the minimum date.
+            end_date    the (inclusive) date on which to end generating.
+                        Defaults to the maximum date.
+            seed        the seed to use for pseudorandom generation.
+                        Defaults to a random seed.
+            calendar    the calendar defining trading dates and hours.
+                        Defaults to SimpleMidnightCalendar().
+            freq        the time frequency of periods to generate with.
+                        Defaults to "200ms".
+            time_dist   the distribution of times within a period.
+                        Defaults to "constant". Also accepts "uniform".
+            dists       list of (name, distribution) tuples.
+            num_ids     the number of ids to generate per period.
+            batch_size  the maximum number of time points to generate per batch.
+                        Defaults to 100. Taken as at least 1.
+
+        Returns an iterator of RecordBatch instances, each with a schema like:
+
+            "time" timestamp
+            name0 dtype0
+            ...
+
+        The BatchRecord may have fewer time points than requested in batch_size
+        when the date range or business date ends.
+
+        Example:
+
+>>> dgen = DataGenerator()
+>>> for batch in dgen.generate("2020-01-01", "2020-01-01", seed=1,
+...         freq="10h",
+...         dists=[
+...             ("id", ArrayAllDistribution(np.arange(3, dtype=np.int32))),
+...             ("value", NormalDistribution())
+...         ],
+...         num_ids=3, batch_size=2):
+...     for item in batch.to_pydict().items():
+...         print(item)
+... 
+('time', [Timestamp('2020-01-01 05:00:00+0000', tz='UTC'), Timestamp('2020-01-01 05:00:00+0000', tz='UTC'), Timestamp('2020-01-01 05:00:00+0000', tz='UTC')])
+('id', [0, 1, 2])
+('value', [0.345584192064786, 0.8216181435011584, 0.33043707618338714])
+        """
+
+        begin_date = self._normalize_date(begin_date, date.min)
+        end_date = self._normalize_date(end_date, date.max)
+        if calendar is None:
+            calendar = SimpleMidnightCalendar()
+        offset_ns = to_offset(freq).nanos
+        if time_dist not in ["constant", "uniform"]:
+            raise ValueError("invalid time_dist: " + str(time_dist))
+        if dists is None:
+            dists = []
+        batch_size = max(1, batch_size)
+
+        schema = pa.schema(
+                [("time", pa.timestamp('ns', 'UTC'))] +
+                [(name, _arrow_type(dist.dtype)) for name, dist in dists]
+        )
+
+        uniform_time_dist = (time_dist == "uniform")
+        rng = np.random.default_rng(seed=seed)
+        one_day = timedelta(days=1)
+        curr_date = begin_date
+        curr_date_ns = self._date_to_midnight_ns(curr_date)
+        while True:
+            if calendar.is_business_day(curr_date):
+                begin_time_ns = (
+                        curr_date_ns + calendar.business_begin_ns(curr_date))
+                end_time_ns = (
+                        curr_date_ns + calendar.trading_end_ns(curr_date))
+                curr_offset_ns = min(offset_ns, end_time_ns - begin_time_ns + 1)
+                while begin_time_ns <= end_time_ns:
+                    times_num = 1 + (
+                            (end_time_ns - begin_time_ns) // curr_offset_ns)
+                    batch_num = min(times_num, batch_size)
+                    curr_array_size = (num_ids * batch_num,)
+                    batch_end_time_ns = (
+                            begin_time_ns + (batch_num - 1) * curr_offset_ns)
+
+                    time_ns_value_array = np.arange(
+                            begin_time_ns, batch_end_time_ns + 1,
+                            curr_offset_ns, dtype=np.uint64)
+                    time_ns_array = np.repeat(
+                        time_ns_value_array, num_ids).astype('datetime64[ns]')
+                    if uniform_time_dist:
+                        dist_offset_ns = min(curr_offset_ns,
+                                end_time_ns - begin_time_ns)
+                        time_ns_array += rng.integers(low=0,
+                                high=dist_offset_ns, size=curr_array_size,
+                                dtype=np.uint64)
+                        time_ns_array = np.sort(time_ns_array)
+                    arrays = ([time_ns_array] +
+                        [dist(rng, num_ids, batch_num) for _, dist in dists])
+                    parrays = [pa.array(a) for a in arrays]
+
+                    batch = pa.RecordBatch.from_arrays(parrays, schema=schema)
+                    yield batch
+
+                    begin_time_ns += batch_num * curr_offset_ns
+            if curr_date == end_date:
+                break
+            curr_date += one_day
+            curr_date_ns = self._date_to_midnight_ns(curr_date)
+
+class TableMDGenerator:

Review Comment:
   Why are there so many TableXyzGenerator types?  What is common / different between them all?



##########
cpp/src/arrow/compute/exec/benchmark_scripts/table_generation/batch_process.py:
##########
@@ -0,0 +1,104 @@
+import json
+from datagen import *
+
+from pyarrow import ipc
+from pyarrow.ipc import RecordBatchStreamWriter
+import pyarrow as pa
+import pyarrow.parquet as pq
+
+
+class BatchStreamStats:

Review Comment:
   Is this used?



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,40 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/// Specify properties of a table to be generated.
+struct TableGenerationProperties {
+  /// Indicates the amount of time between data points that lie between
+  /// the start and end parameters.
+  int time_frequency;
+  /// The number of additional random columns in the table.
+  int num_columns;
+  /// The number of unique keys in the table.
+  int num_ids;
+  /// Specifies the prefix of each randomly generated column.
+  std::string column_prefix;
+  /// Specifies the minimum value in the randomly generated column(s).
+  int min_column_value;
+  /// Specifies the maximum value in the randomly generated column(s).
+  int max_column_value;
+  /// The random seed the random array generator is given to generate the additional
+  /// columns.
+  int seed;
+  /// Specifies the beginning of 'time' recorded in the table, inclusive.
+  int start;
+  /// Specifies the end of 'time' recorded in the table, inclusive.
+  int end;
+};
+
+/// The table generated in accordance to the TableGenerationProperties has the following
+/// schema: time (int64) id (int32) [properties.column_prefix]0 (float64)

Review Comment:
   That's good, added a definition for the range for idx and removed the other redundant lines.



-- 
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] icexelloss commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   @westonpace Thanks for the detailed reply. I remember when I use 
   
   ```
   ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
   ```
   
   in asof_join test, I got wrong results and see data coming in out of order and therefore, having to use
   
   ```
   ExecContext ctx(default_memory_pool(), nullptr);
   ```
   
   - I wonder what's the threading model for the latter? Is it just one thread doing everything?


-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Benchmark runs are scheduled for baseline = a963392c7cc845f05c9691b799474eea75eea926 and contender = 96679463056e9985b8ff745fb2273f71338bd864. 96679463056e9985b8ff745fb2273f71338bd864 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/cc7ed77636854740a569819b95065597...f29a8a1a683341bea99764148ca0454b/)
   [Finished :arrow_down:0.24% :arrow_up:0.1%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/465e079b71774ce7a9f48e479675f8e8...0ce4ff3cd0734ed6ba860438ac1f935e/)
   [Finished :arrow_down:0.82% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/081ec41b8a1b4ebe800d4cd41087de81...7c414c6b3c344ecd8548d58aa25d5dc4/)
   [Finished :arrow_down:0.04% :arrow_up:0.25%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/8708ed68d3f74a68a1e39f1ed2d25715...4ba33be1927c4d2284ccc960003ad3e2/)
   Buildkite builds:
   [Failed] [`96679463` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1219)
   [Finished] [`96679463` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1231)
   [Finished] [`96679463` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1213)
   [Finished] [`96679463` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1233)
   [Failed] [`a963392c` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1218)
   [Finished] [`a963392c` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1230)
   [Finished] [`a963392c` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1212)
   [Finished] [`a963392c` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1232)
   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


[GitHub] [arrow] westonpace commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   This is a lot of possible combinations.  How long does it take to run the full suite?  Are all of these combinations informative?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,

Review Comment:
   Minor nit: `join_node` instead of `asof_join_node` since this function is used in both paths?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>

Review Comment:
   I'm not a big fan of using tuple as a return type here as it is rather opaque what these ints are representing.  Could you just create a simple struct?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   A benchmark can't really depend on an external generation tool.  I think when I looked through this the first time I figured this was a python file you would be adding to the arrow repo.  Is that planned?  Or will this somehow go away?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }

Review Comment:
   This seems rather unsafe.  Can you use `arrow::internal::TemporaryDir` in `src/arrow/util/io_util.h` instead?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {

Review Comment:
   It would be good to keep I/O out of the benchmark if at all possible.  Otherwise your benchmark ends up being rather jittery and can end up simply reflecting the performance of the I/O and not the actual compute.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,

Review Comment:
   Yeah, all these cases are symmetric to limit the memory / time resources of this benchmark, so I've refactored this particular parameter.



-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   Hi @westonpace, let me know if you can take a look at a few of the questions above I had about merging this into arrow.
   
   I was also wondering if you could clarify how the arrow threading engine would work for a node that has multiple inputs (an asof join / hash join ingesting from multiple source nodes, for example).
   
   While being processed, is a single source node dedicated a single thread?
   How many threads can call `InputReceived` of the following node at once? 
   


-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > I think I was mostly curious about differences in density between the left table and the right table(s). For example, a dense left table and a sparse right table or a sparse left table and a dense right table. The left table roughly defines the keyframes so I would expect the density of the left table to be more significant to performance than the density of the right table.
   
   Ah yes, we did try this as part of our asymmetric case. The left table definitely impacts performance the most, with clear separations in time. We also observed another interesting property, which suggests that the time frequency of the right tables does not matter as long as they are >= the left hand table time frequency.
   
   That is, if the left hand table has a time frequency of 10 minutes, we see an increase in real_time when joining tables with 1d, 1h, 30m, etc, but the time it takes to join with right hand tables with 10m, 5m, 1m, is the same.


-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

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


-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   Yes, I agree -- I created a larger set just for exploratory purposes and have been refining it down. We found the most interesting results varying key density and time frequency (we observe some piecewise linear and constant relationships between real_time and the respective property, for example). 
   
   The rest of the properties like column width, multi-table joins, and batch size seem more or less straightforward / linear -- so we can certainly reduce the amount of datapoints here which should alleviate the code bloating...
   
   Keep in mind we are also benchmarking hashjoin here as well for comparisons, which @icexelloss mentioned might not be suitable for this file here. Do you have any advice if those hashjoin comparison benchmarks are worth keeping in arrow?



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,568 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <boost/process.hpp>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/csv/writer.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+// static std::shared_ptr<arrow::internal::TemporaryDir> temp_dir =

Review Comment:
   Thanks @pitrou , that was the fix!



-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   > Hmm, would it be possible to do the data generation in C++?
   
   It's possible but I feel things like data generation is much easier to write in python comparing to C++. I agree that it's not ideal to have "random scripts scattered across the codebase" and my hope is what we can standardize these benchmark dataset generation in Python and use them for both microbenchmarks and end-to-end benchmarks. Since we plan to add more time series related functionalities, I would like to use this script as a baseline for all future time series benchmarks. Does that sounds acceptable?



-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {

Review Comment:
   Yes, that should work fine.



-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   Hmm...we don't have much precedent yet but we do have a few python files scattered amongst the C++ sources.  `src/arrow/compute/exec/hash_join_graphs.py` is intended for making graphs from the hash_join benchmarks.  However, if you wanted to create a test data directory or scripts directory somewhere in the cpp tree that would probably be ok too.  Maybe @pitrou has an opinion here?
   
   Rather than using `system()` which is Linux-specific it would be nicer to use something like `boost::process`.  You can see an example of this in `s3_test_util.cc` where we launch minio.  We should gracefully handle the following:
   
    * Python not installed
    * Python installed but needed dependencies are not
    
   We should probably also create a benchmarking.md document or something like that in the `cpp/src/arrow/compute/exec` that explains the tool, what is needed to run it, and what purpose it serves.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -460,42 +460,39 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
-std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+std::shared_ptr<Table> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
   int total_columns = properties.num_columns + 2;
   std::vector<std::shared_ptr<Array>> columns;
   columns.reserve(total_columns);
-  arrow::FieldVector field_vector = arrow::FieldVector();
+  arrow::FieldVector field_vector;
   field_vector.reserve(total_columns);
 
-  field_vector.push_back(std::make_shared<Field>("time", int64()));
-  field_vector.push_back(std::make_shared<Field>("id", int32()));
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
 
-  int num_rows = 0;
-  std::vector<int64_t> time_column;
-  std::vector<int32_t> id_column;
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
   for (int time = properties.start; time <= properties.end;
-         time += properties.time_frequency) {
+       time += properties.time_frequency) {
     for (int id = 0; id < properties.num_ids; id++) {
-      time_column.push_back(time);
-      id_column.push_back(id);
-      num_rows += 1;
+      time_column_builder.Append(time);
+      id_column_builder.Append(id);
     }
   }
-  std::shared_ptr<Array> time_array;
-  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
-  columns.push_back(time_array);
-  std::shared_ptr<Array> id_array;
-  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
-  columns.push_back(id_array);
+
+  int num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());

Review Comment:
   Is this okay? I tried using `CHECK_OK_AND_ASSIGN` but I don't think that works in a non-void function.



-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), nullptr);
+
+  left_table_properties.column_prefix = "lt";
+  left_table_properties.seed = 0;
+  TableStats left_table_stats = MakeTable(left_table_properties);
+
+  size_t right_hand_rows = 0;
+  size_t right_hand_bytes = 0;
+  std::vector<TableStats> right_input_tables;
+  right_input_tables.reserve(num_right_tables);
+
+  for (int i = 0; i < num_right_tables; i++) {
+    right_table_properties.column_prefix = "rt" + std::to_string(i);
+    right_table_properties.seed = i + 1;
+    TableStats right_table_stats = MakeTable(right_table_properties);
+    right_hand_rows += right_table_stats.total_rows;
+    right_hand_bytes += right_table_stats.total_bytes;
+    right_input_tables.push_back(right_table_stats);
+  }
+
+  for (auto _ : state) {
+    state.PauseTiming();
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::vector<ExecNode*> input_nodes = {
+        MakeTableSourceNode(plan, left_table_stats.table, left_table_batch_size)};
+    input_nodes.reserve(right_input_tables.size() + 1);
+    for (TableStats table_stats : right_input_tables) {
+      input_nodes.push_back(
+          MakeTableSourceNode(plan, table_stats.table, right_table_batch_size));
+    }
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * join_node,
+                         MakeExecNode(factory_name, plan.get(), input_nodes, options));
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+  }
+
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_rows + right_hand_rows)),
+      benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_bytes + right_hand_bytes)),
+      benchmark::Counter::kIsRate);
+
+  state.counters["maximum_peak_memory"] =
+      benchmark::Counter(static_cast<double>(ctx.memory_pool()->max_memory()));
+}
+
+static void AsOfJoinOverhead(benchmark::State& state) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(kTimeCol, kKeyCol, tolerance);
+  TableJoinOverhead(
+      state,
+      TableGenerationProperties{int(state.range(0)), int(state.range(1)),
+                                int(state.range(2)), "", kDefaultMinColumnVal,
+                                kDefaultMaxColumnVal, 0, kDefaultStart, kDefaultEnd},
+      int(state.range(3)),
+      TableGenerationProperties{int(state.range(5)), int(state.range(6)),
+                                int(state.range(7)), "", kDefaultMinColumnVal,
+                                kDefaultMaxColumnVal, 0, kDefaultStart, kDefaultEnd},
+      int(state.range(8)), int(state.range(4)), "asofjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) {
+  bench
+      ->ArgNames({"left_freq", "left_cols", "left_ids", "left_batch_size",
+                  "num_right_tables", "right_freq", "right_cols", "right_ids",
+                  "right_batch_size"})
+      ->UseRealTime();
+
+  int default_freq = 5;
+  int default_cols = 20;
+  int default_ids = 500;
+  int default_num_tables = 1;
+  int default_batch_size = 100;
+
+  for (int freq : {1, 5, 10}) {
+    bench->Args({freq, default_cols, default_ids, default_batch_size, default_num_tables,
+                 freq, default_cols, default_ids, default_batch_size});
+  }
+  for (int cols : {10, 20, 100}) {
+    bench->Args({default_freq, cols, default_ids, default_batch_size, default_num_tables,
+                 default_freq, cols, default_ids, default_batch_size});
+  }
+  for (int ids : {100, 500, 1000}) {
+    bench->Args({default_freq, default_cols, ids, default_batch_size, default_num_tables,
+                 default_freq, default_cols, ids, default_batch_size});
+  }
+  for (int num_tables : {1, 10, 50}) {
+    bench->Args({default_freq, default_cols, default_ids, default_batch_size, num_tables,
+                 default_freq, default_cols, default_ids, default_batch_size});
+  }
+  for (int batch_size : {1, 500, 1000}) {

Review Comment:
   Increased batch size, also increased the data generation time range so this batch size makes (more) sense.



-- 
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 pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > On the other hand, this test seems more like an end-to-end benchmark and less like a specific node micro benchmark (and thus might make more sense fully in python) since it includes IPC reading and decoding
   
   That's a good point as well. Micro-benchmarks are supposed to run pretty quickly (less than one second per iteration ideally). If that is not the case here, this should probably be done elsewhere, such as in our [continuous benchmarking suite](https://github.com/ursacomputing/benchmarks/tree/main/benchmarks).


-- 
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] icexelloss commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}

Review Comment:
   Thanks! Will continue there.



-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,40 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+Result<std::shared_ptr<Table>> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector;
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
+  for (int64_t time = properties.start; time <= properties.end;
+       time += properties.time_frequency) {
+    for (int32_t id = 0; id < properties.num_ids; id++) {
+      ARROW_RETURN_NOT_OK(time_column_builder.Append(time));
+      ARROW_RETURN_NOT_OK(id_column_builder.Append(id));
+    }
+  }
+
+  int64_t num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());

Review Comment:
   ```suggestion
     ARROW_ASSIGN_OR_RAISE(auto time_column, time_column_builder.Finish());
     columns.push_back(std::move(time_column));
     ARROW_ASSIGN_OR_RAISE(auto id_column, id_column_builder.Finish());
     columns.push_back(std::move(id_column));
   ```



-- 
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] iChauster commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   @icexelloss @westonpace unmarking as draft. Got around to finalizing the data generation scripts and the initial version of the documentation. I also cleaned up the file and removed some unused utilities.
   
   I have left in the HashJoin benchmarks for now. I have also included implementation for two source nodes -- a TableSourceNode and a source node with a `RecordBatchFileReader.` Per our discussion on the mailing list these sort of benchmark two different things and I am not sure if we should choose one or do both for our benchmarks. Currently, I am using the table implementation (however these are quite interchangeable).
   
   Lastly, I added a memory statistic, using `memory_pool()->max_memory()`, however the results seem pretty constant between the different benchmarks.
   
   Let me know what you think!


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,44 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector = arrow::FieldVector();
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(std::make_shared<Field>("time", int64()));
+  field_vector.push_back(std::make_shared<Field>("id", int32()));
+
+  int num_rows = 0;
+  std::vector<int64_t> time_column;
+  std::vector<int32_t> id_column;
+  for (int time = properties.start; time <= properties.end;
+         time += properties.time_frequency) {
+    for (int id = 0; id < properties.num_ids; id++) {
+      time_column.push_back(time);
+      id_column.push_back(id);
+      num_rows += 1;
+    }
+  }
+  std::shared_ptr<Array> time_array;
+  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
+  columns.push_back(time_array);
+  std::shared_ptr<Array> id_array;
+  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
+  columns.push_back(id_array);
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    std::ostringstream string_stream;
+    string_stream << properties.column_prefix << i;
+    field_vector.push_back(std::make_shared<Field>(string_stream.str(), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);
+    columns.push_back(rand.Float64(num_rows, -1e5, 1e5));
+  }
+  std::shared_ptr<arrow::Schema> schema = std::make_shared<arrow::Schema>(field_vector);

Review Comment:
   Perhaps a more directed question: Is there a rule of thumb for using `std::move`?



-- 
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] vibhatha commented on pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   > 
   
   @icexelloss You can find a simple `write` operator example here: https://github.com/apache/arrow/blob/998cca30c70a7e0bbc3f83957923f2e3019b314b/cpp/examples/arrow/execution_plan_documentation_examples.cc#L795
   
   Is this helpful? Let me know I can help out with a better example if 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] westonpace commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);

Review Comment:
   There are some utilities you can use in `arrow/util/byte_size.h` too if you wanted a more accurate version of the size (e.g. will report size used by validity bitmaps).
   
   However, this is fine too I think.  It represents a more conceptual data size.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}

Review Comment:
   Is this method really needed?  It's just one line.



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));

Review Comment:
   ```suggestion
         arrow::compute::TableSourceNodeOptions(std::move(table), batch_size));
   ```



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -460,42 +460,39 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
-std::shared_ptr<Table> MakeRandomTable(TableGenerationProperties properties) {
+std::shared_ptr<Table> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
   int total_columns = properties.num_columns + 2;
   std::vector<std::shared_ptr<Array>> columns;
   columns.reserve(total_columns);
-  arrow::FieldVector field_vector = arrow::FieldVector();
+  arrow::FieldVector field_vector;
   field_vector.reserve(total_columns);
 
-  field_vector.push_back(std::make_shared<Field>("time", int64()));
-  field_vector.push_back(std::make_shared<Field>("id", int32()));
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
 
-  int num_rows = 0;
-  std::vector<int64_t> time_column;
-  std::vector<int32_t> id_column;
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
   for (int time = properties.start; time <= properties.end;
-         time += properties.time_frequency) {
+       time += properties.time_frequency) {
     for (int id = 0; id < properties.num_ids; id++) {
-      time_column.push_back(time);
-      id_column.push_back(id);
-      num_rows += 1;
+      time_column_builder.Append(time);
+      id_column_builder.Append(id);
     }
   }
-  std::shared_ptr<Array> time_array;
-  ArrayFromVector<Int64Type, int64_t>(int64(), time_column, &time_array);
-  columns.push_back(time_array);
-  std::shared_ptr<Array> id_array;
-  ArrayFromVector<Int32Type, int32_t>(int32(), id_column, &id_array);
-  columns.push_back(id_array);
+
+  int num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());

Review Comment:
   If you change the function to return `Result<std::shared_ptr<Table>>` (which you should) then you can use `ARROW_ASSIGN_OR_RAISE`.  `CHECK_...` and `ASSERT_...` should only be used in the test/benchmark files themselves.  In helper functions (e.g. `test_util.cc`) you should return a `Status` or a `Result<T>`



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), nullptr);
+
+  left_table_properties.column_prefix = "lt";
+  left_table_properties.seed = 0;
+  TableStats left_table_stats = MakeTable(left_table_properties);
+
+  size_t right_hand_rows = 0;
+  size_t right_hand_bytes = 0;
+  std::vector<TableStats> right_input_tables;
+  right_input_tables.reserve(num_right_tables);
+
+  for (int i = 0; i < num_right_tables; i++) {
+    right_table_properties.column_prefix = "rt" + std::to_string(i);
+    right_table_properties.seed = i + 1;
+    TableStats right_table_stats = MakeTable(right_table_properties);
+    right_hand_rows += right_table_stats.total_rows;
+    right_hand_bytes += right_table_stats.total_bytes;
+    right_input_tables.push_back(right_table_stats);
+  }
+
+  for (auto _ : state) {
+    state.PauseTiming();
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::vector<ExecNode*> input_nodes = {
+        MakeTableSourceNode(plan, left_table_stats.table, left_table_batch_size)};
+    input_nodes.reserve(right_input_tables.size() + 1);
+    for (TableStats table_stats : right_input_tables) {
+      input_nodes.push_back(
+          MakeTableSourceNode(plan, table_stats.table, right_table_batch_size));
+    }
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * join_node,
+                         MakeExecNode(factory_name, plan.get(), input_nodes, options));
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+  }
+
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_rows + right_hand_rows)),
+      benchmark::Counter::kIsRate);

Review Comment:
   When I see `total_rows_per_second` my brain tends to think "output rows" while this is "input rows".  Do we want to split this into `output_rows_per_second` and `input_rows_per_second`?  Also, why "total"?



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,41 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector;
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
+
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
+  for (int64_t time = properties.start; time <= properties.end;
+       time += properties.time_frequency) {
+    for (int32_t id = 0; id < properties.num_ids; id++) {
+      Status time_col_append_status = time_column_builder.Append(time);
+      Status id_col_append_status = id_column_builder.Append(id);
+    }
+  }
+
+  int64_t num_rows = time_column_builder.length();
+  columns.push_back(time_column_builder.Finish().ValueOrDie());
+  columns.push_back(id_column_builder.Finish().ValueOrDie());
+
+  for (int i = 0; i < properties.num_columns; i++) {
+    field_vector.push_back(
+        field(properties.column_prefix + std::to_string(i), float64()));
+    random::RandomArrayGenerator rand = random::RandomArrayGenerator(properties.seed + i);

Review Comment:
   ```suggestion
       random::RandomArrayGenerator rand(properties.seed + i);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,

Review Comment:
   It doesn't appear `left_table_batch_size` and `right_table_batch_size` are ever different values.  Could you simplify this down to just `batch_size`?



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), nullptr);
+
+  left_table_properties.column_prefix = "lt";
+  left_table_properties.seed = 0;
+  TableStats left_table_stats = MakeTable(left_table_properties);
+
+  size_t right_hand_rows = 0;
+  size_t right_hand_bytes = 0;
+  std::vector<TableStats> right_input_tables;
+  right_input_tables.reserve(num_right_tables);
+
+  for (int i = 0; i < num_right_tables; i++) {
+    right_table_properties.column_prefix = "rt" + std::to_string(i);
+    right_table_properties.seed = i + 1;
+    TableStats right_table_stats = MakeTable(right_table_properties);
+    right_hand_rows += right_table_stats.total_rows;
+    right_hand_bytes += right_table_stats.total_bytes;
+    right_input_tables.push_back(right_table_stats);
+  }
+
+  for (auto _ : state) {
+    state.PauseTiming();
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::vector<ExecNode*> input_nodes = {
+        MakeTableSourceNode(plan, left_table_stats.table, left_table_batch_size)};
+    input_nodes.reserve(right_input_tables.size() + 1);
+    for (TableStats table_stats : right_input_tables) {
+      input_nodes.push_back(
+          MakeTableSourceNode(plan, table_stats.table, right_table_batch_size));
+    }
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * join_node,
+                         MakeExecNode(factory_name, plan.get(), input_nodes, options));
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+  }
+
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_rows + right_hand_rows)),
+      benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() *
+                          (left_table_stats.total_bytes + right_hand_bytes)),
+      benchmark::Counter::kIsRate);
+
+  state.counters["maximum_peak_memory"] =
+      benchmark::Counter(static_cast<double>(ctx.memory_pool()->max_memory()));
+}
+
+static void AsOfJoinOverhead(benchmark::State& state) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(kTimeCol, kKeyCol, tolerance);
+  TableJoinOverhead(
+      state,
+      TableGenerationProperties{int(state.range(0)), int(state.range(1)),
+                                int(state.range(2)), "", kDefaultMinColumnVal,
+                                kDefaultMaxColumnVal, 0, kDefaultStart, kDefaultEnd},
+      int(state.range(3)),
+      TableGenerationProperties{int(state.range(5)), int(state.range(6)),
+                                int(state.range(7)), "", kDefaultMinColumnVal,
+                                kDefaultMaxColumnVal, 0, kDefaultStart, kDefaultEnd},
+      int(state.range(8)), int(state.range(4)), "asofjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) {
+  bench
+      ->ArgNames({"left_freq", "left_cols", "left_ids", "left_batch_size",
+                  "num_right_tables", "right_freq", "right_cols", "right_ids",
+                  "right_batch_size"})
+      ->UseRealTime();
+
+  int default_freq = 5;
+  int default_cols = 20;
+  int default_ids = 500;
+  int default_num_tables = 1;
+  int default_batch_size = 100;
+
+  for (int freq : {1, 5, 10}) {
+    bench->Args({freq, default_cols, default_ids, default_batch_size, default_num_tables,
+                 freq, default_cols, default_ids, default_batch_size});
+  }
+  for (int cols : {10, 20, 100}) {
+    bench->Args({default_freq, cols, default_ids, default_batch_size, default_num_tables,
+                 default_freq, cols, default_ids, default_batch_size});
+  }
+  for (int ids : {100, 500, 1000}) {
+    bench->Args({default_freq, default_cols, ids, default_batch_size, default_num_tables,
+                 default_freq, default_cols, ids, default_batch_size});
+  }
+  for (int num_tables : {1, 10, 50}) {
+    bench->Args({default_freq, default_cols, default_ids, default_batch_size, num_tables,
+                 default_freq, default_cols, default_ids, default_batch_size});
+  }
+  for (int batch_size : {1, 500, 1000}) {

Review Comment:
   These are extremely small values of `batch_size`.  Currently, when scanning from files, we use `32Ki`.  `1000` is probably a lower bound.  Can you use `1Ki`, `4Ki`, and `32Ki` instead?



##########
cpp/src/arrow/compute/exec/test_util.cc:
##########
@@ -459,5 +460,41 @@ void PrintTo(const Declaration& decl, std::ostream* os) {
   *os << "}";
 }
 
+std::shared_ptr<Table> MakeRandomTimeSeriesTable(
+    const TableGenerationProperties& properties) {
+  int total_columns = properties.num_columns + 2;
+  std::vector<std::shared_ptr<Array>> columns;
+  columns.reserve(total_columns);
+  arrow::FieldVector field_vector;
+  field_vector.reserve(total_columns);
+
+  field_vector.push_back(field("time", int64()));
+  field_vector.push_back(field("id", int32()));
+
+  Int64Builder time_column_builder;
+  Int32Builder id_column_builder;
+  for (int64_t time = properties.start; time <= properties.end;
+       time += properties.time_frequency) {
+    for (int32_t id = 0; id < properties.num_ids; id++) {
+      Status time_col_append_status = time_column_builder.Append(time);
+      Status id_col_append_status = id_column_builder.Append(id);

Review Comment:
   ```suggestion
         ARROW_RETURN_NOT_OK(time_column_builder.Append(time));
         ARROW_RETURN_NOT_OK(id_column_builder.Append(id));
   ```



##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/table.h"
+#include "arrow/testing/future_util.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* kTimeCol = "time";
+static const char* kKeyCol = "id";
+const int kDefaultStart = 0;
+const int kDefaultEnd = 500;
+const int kDefaultMinColumnVal = -10000;
+const int kDefaultMaxColumnVal = 10000;
+
+struct TableStats {
+  std::shared_ptr<Table> table;
+  size_t total_rows;
+  size_t total_bytes;
+};
+
+static TableStats MakeTable(const TableGenerationProperties& properties) {
+  std::shared_ptr<Table> table = MakeRandomTimeSeriesTable(properties);
+  size_t row_size = sizeof(double) * (table.get()->schema()->num_fields() - 2) +
+                    sizeof(int64_t) + sizeof(int32_t);
+  size_t rows = table.get()->num_rows();
+  return {table, rows, rows * row_size};
+}
+
+static ExecNode* MakeTableSourceNode(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                                     std::shared_ptr<Table> table, int batch_size) {
+  return *arrow::compute::MakeExecNode(
+      "table_source", plan.get(), {},
+      arrow::compute::TableSourceNodeOptions(table, batch_size));
+}
+
+static void TableJoinOverhead(benchmark::State& state,
+                              TableGenerationProperties left_table_properties,
+                              int left_table_batch_size,
+                              TableGenerationProperties right_table_properties,
+                              int right_table_batch_size, int num_right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  ExecContext ctx(default_memory_pool(), nullptr);
+
+  left_table_properties.column_prefix = "lt";
+  left_table_properties.seed = 0;
+  TableStats left_table_stats = MakeTable(left_table_properties);
+
+  size_t right_hand_rows = 0;
+  size_t right_hand_bytes = 0;
+  std::vector<TableStats> right_input_tables;
+  right_input_tables.reserve(num_right_tables);
+
+  for (int i = 0; i < num_right_tables; i++) {
+    right_table_properties.column_prefix = "rt" + std::to_string(i);
+    right_table_properties.seed = i + 1;
+    TableStats right_table_stats = MakeTable(right_table_properties);
+    right_hand_rows += right_table_stats.total_rows;
+    right_hand_bytes += right_table_stats.total_bytes;
+    right_input_tables.push_back(right_table_stats);

Review Comment:
   ```suggestion
       right_input_tables.push_back(std::move(right_table_stats));
   ```



##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,39 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/// Specify properties of a table to be generated.
+struct TableGenerationProperties {
+  /// Indicates the amount of time between data points that lie between
+  /// the start and end parameters.
+  int time_frequency;
+  /// The number of additional random columns in the table.
+  int num_columns;
+  /// The number of unique keys in the table.
+  int num_ids;
+  /// Specifies the prefix of each randomly generated column.
+  std::string column_prefix;
+  /// Specifies the minimum value in the randomly generated column(s).
+  int min_column_value;
+  /// Specifies the maximum value in the randomly generated column(s).
+  int max_column_value;
+  /// The random seed the random array generator is given to generate the additional
+  /// columns.
+  int seed;
+  /// Specifies the beginning of 'time' recorded in the table, inclusive.
+  int start;
+  /// Specifies the end of 'time' recorded in the table, inclusive.
+  int end;
+};
+
+/// The table generated in accordance to the TableGenerationProperties has the following
+/// schema: time (int64) id (int32) [properties.column_prefix]0 (float64)

Review Comment:
   What's the `0` in `[properties.column_prefix]0`



-- 
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 #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   This error from the Windows CI is probably legitimate:
   
   ```
   unity_0_cxx.obj : error LNK2019: unresolved external symbol "class std::shared_ptr<class arrow::Table> __cdecl arrow::compute::MakeRandomTimeSeriesTable(struct arrow::compute::TableGenerationProperties const &)" (?MakeRandomTimeSeriesTable@compute@arrow@@YA?AV?$shared_ptr@VTable@arrow@@@std@@AEBUTableGenerationProperties@12@@Z) referenced in function "struct arrow::compute::TableStats __cdecl arrow::compute::MakeTable(struct arrow::compute::TableGenerationProperties const &)" (?MakeTable@compute@arrow@@YA?AUTableStats@12@AEBUTableGenerationProperties@12@@Z) [D:\a\arrow\arrow\build\cpp\src\arrow\compute\exec\arrow-compute-asof-join-benchmark.vcxproj]
   ```
   
   To fix this add `ARROW_TESTING_EXPORT` to `TableGenerationProperties` and `MakeRandomTimeSeriesTable` (Windows requires you to specifically label which functions are "external" and can be called outside of a shared object.


-- 
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] iChauster commented on a diff in pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/test_util.h:
##########
@@ -145,5 +145,39 @@ class Random64Bit {
   std::uniform_int_distribution<uint64_t> dist_;
 };
 
+/// Specify properties of a table to be generated.
+struct TableGenerationProperties {
+  /// Indicates the amount of time between data points that lie between
+  /// the start and end parameters.
+  int time_frequency;
+  /// The number of additional random columns in the table.
+  int num_columns;
+  /// The number of unique keys in the table.
+  int num_ids;
+  /// Specifies the prefix of each randomly generated column.
+  std::string column_prefix;
+  /// Specifies the minimum value in the randomly generated column(s).
+  int min_column_value;
+  /// Specifies the maximum value in the randomly generated column(s).
+  int max_column_value;
+  /// The random seed the random array generator is given to generate the additional
+  /// columns.
+  int seed;
+  /// Specifies the beginning of 'time' recorded in the table, inclusive.
+  int start;
+  /// Specifies the end of 'time' recorded in the table, inclusive.
+  int end;
+};
+
+/// The table generated in accordance to the TableGenerationProperties has the following
+/// schema: time (int64) id (int32) [properties.column_prefix]0 (float64)

Review Comment:
   Hmm, I think this one got caught in the linting / formatter and made it a bit unclear, but each column is numbered from 0 to n - 1 inclusive, so each column name is something like [properties.column_prefix][i] where i = {0...n-1}. Is there a way I can make this clearer through the comments?



-- 
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] iChauster commented on pull request #13426: ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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

   @westonpace updated!


-- 
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 #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   I agree it is initially interesting.  Long term I think the main advantage for an asof join over a hash join is going to be memory usage though.  Asof join can operate in a streaming fashion while hash join will require eventually spilling to disk as the input size increases.  So a performance benchmark between the two probably isn't needed long term.



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {

Review Comment:
   I see. One alternative I can think of is reading in the files in memory through a table object, and using a `TableSourceNode` in the preparation stage. I think this would assure everything is in-memory during `StartAndCollect`. I'm assuming that would be less I/O intensive?



-- 
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] iChauster commented on a diff in pull request #13426: [DRAFT] ARROW-16894: [C++] Add Benchmarks for Asof Join Node

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


##########
cpp/src/arrow/compute/exec/asof_join_benchmark.cc:
##########
@@ -0,0 +1,1023 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <condition_variable>
+#include <mutex>
+#include <string>
+
+#include "benchmark/benchmark.h"
+
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/task_util.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/dataset/partition.h"
+#include "arrow/filesystem/api.h"
+#include "arrow/ipc/api.h"
+#include "arrow/testing/future_util.h"
+#include "arrow/testing/generator.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace compute {
+
+static const char* time_col = "time";
+static const char* key_col = "id";
+static bool createdBenchmarkFiles = false;
+
+// requires export PYTHONPATH=/path/to/bamboo-streaming
+// calls generate_benchmark_files to create tables (feather files) varying in frequency,
+// width, key density for benchmarks. places generated files in benchmark/data. This
+// operation runs once at the beginning of benchmarking.
+static void DoSetup() {
+  if (!createdBenchmarkFiles) {
+    system(
+        "mkdir benchmark_data/ && python3 "
+        "~/summer/bamboo-streaming/bamboo/generate_benchmark_files.py");
+    createdBenchmarkFiles = true;
+  }
+}
+
+static void DoTeardown() { system("rm -rf benchmark_data/"); }
+
+static std::vector<std::string> generateRightHandTables(std::string freq, int width_index,
+                                                        int num_tables,
+                                                        int num_ids_index) {
+  auto const generate_file_name = [](std::string freq, std::string is_wide,
+                                     std::string num_ids, std::string num) {
+    return freq + "_" + is_wide + "_" + num_ids + num + ".feather";
+  };
+
+  std::string width_table[] = {"1_cols",  "10_cols", "20_cols",
+                               "40_cols", "80_cols", "100_cols"};   // 0 - 5
+  std::string num_ids_table[] = {"100_ids", "2000_ids", "5k_ids"};  // 0 - 2
+
+  std::string wide_string = width_table[width_index];
+  std::string ids = num_ids_table[num_ids_index];
+
+  std::vector<std::string> right_hand_tables;
+  for (int j = 1; j <= num_tables; j++) {
+    right_hand_tables.push_back(
+        generate_file_name(freq, wide_string, ids, std::to_string(j)));
+  }
+  return right_hand_tables;
+}
+
+// Wrapper to enable the use of RecordBatchFileReaders as RecordBatchReaders
+class RecordBatchFileReaderWrapper : public arrow::ipc::RecordBatchReader {
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> _reader;
+  int _next;
+
+ public:
+  virtual ~RecordBatchFileReaderWrapper() {}
+  explicit RecordBatchFileReaderWrapper(
+      std::shared_ptr<arrow::ipc::RecordBatchFileReader> reader)
+      : _reader(reader), _next(0) {}
+
+  virtual arrow::Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) {
+    // cout << "ReadNext _next=" << _next << "\n";
+    if (_next < _reader->num_record_batches()) {
+      ARROW_ASSIGN_OR_RAISE(*batch, _reader->ReadRecordBatch(_next++));
+      // cout << "\t --> " << (*batch)->num_rows() << "\n";
+    } else {
+      batch->reset();
+      // cout << "\t --> EOF\n";
+    }
+
+    return arrow::Status::OK();
+  }
+
+  virtual std::shared_ptr<arrow::Schema> schema() const { return _reader->schema(); }
+};
+
+static std::tuple<arrow::compute::ExecNode*, int64_t, int, size_t>
+make_arrow_ipc_reader_node(std::shared_ptr<arrow::compute::ExecPlan>& plan,
+                           std::shared_ptr<arrow::fs::FileSystem>& fs,
+                           const std::string& filename) {
+  // TODO: error checking
+  std::shared_ptr<arrow::io::RandomAccessFile> input = *fs->OpenInputFile(filename);
+  std::shared_ptr<arrow::ipc::RecordBatchFileReader> in_reader =
+      *arrow::ipc::RecordBatchFileReader::Open(input);
+  std::shared_ptr<RecordBatchFileReaderWrapper> reader(
+      new RecordBatchFileReaderWrapper(in_reader));
+
+  auto schema = reader->schema();
+  // we assume there is a time field represented in uint64, a key field of int32, and the
+  // remaining fields are float64.
+  size_t row_size =
+      sizeof(_Float64) * (schema->num_fields() - 2) + sizeof(int64_t) + sizeof(int32_t);
+  auto batch_gen = *arrow::compute::MakeReaderGenerator(
+      std::move(reader), arrow::internal::GetCpuThreadPool());
+  int64_t rows = in_reader->CountRows().ValueOrDie();
+  // cout << "create source("<<filename<<")\n";
+  return {*arrow::compute::MakeExecNode(
+              "source",    // registered type
+              plan.get(),  // execution plan
+              {},          // inputs
+              arrow::compute::SourceNodeOptions(
+                  std::make_shared<arrow::Schema>(*schema),  // options, )
+                  batch_gen)),
+          rows, in_reader->num_record_batches(), row_size * rows};
+}
+
+static void TableJoinOverhead(benchmark::State& state, std::string left_table,
+                              std::vector<std::string> right_tables,
+                              std::string factory_name, ExecNodeOptions& options) {
+  const std::string data_directory = "./benchmark_data/";
+  DoSetup();
+  ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool());
+  // std::cout << "beginning test for " << left_table << " and " << right_tables[0] << " "
+  // << factory_name << std::endl; std::cout << "starting with " <<
+  // ctx.memory_pool()->bytes_allocated() << std::endl;
+  int64_t rows;
+  int64_t bytes;
+  for (auto _ : state) {
+    state.PauseTiming();
+
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<arrow::compute::ExecPlan> plan,
+                         ExecPlan::Make(&ctx));
+    std::shared_ptr<arrow::fs::FileSystem> fs =
+        std::make_shared<arrow::fs::LocalFileSystem>();
+    arrow::compute::ExecNode* left_table_source;
+    int64_t left_table_rows;
+    int left_table_batches;
+    size_t left_table_bytes;
+    tie(left_table_source, left_table_rows, left_table_batches, left_table_bytes) =
+        make_arrow_ipc_reader_node(plan, fs, data_directory + left_table);
+    std::vector<ExecNode*> inputs = {left_table_source};
+    int right_hand_rows = 0;
+    int64_t right_hand_bytes = 0;
+    for (std::string right_table : right_tables) {
+      arrow::compute::ExecNode* right_table_source;
+      int64_t right_table_rows;
+      int right_table_batches;
+      size_t right_table_bytes;
+      tie(right_table_source, right_table_rows, right_table_batches, right_table_bytes) =
+          make_arrow_ipc_reader_node(plan, fs, data_directory + right_table);
+      inputs.push_back(right_table_source);
+      right_hand_rows += right_table_rows;
+      right_hand_bytes += right_table_bytes;
+    }
+    rows = left_table_rows + right_hand_rows;
+    bytes = left_table_bytes + right_hand_bytes;
+    ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * asof_join_node,
+                         MakeExecNode(factory_name, plan.get(), inputs, options));
+
+    AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+    MakeExecNode("sink", plan.get(), {asof_join_node}, SinkNodeOptions{&sink_gen});
+    state.ResumeTiming();
+    // std::cout << "starting and collecting with " <<
+    // ctx.memory_pool()->bytes_allocated() << std::endl;
+    ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen));
+    // std::cout << "finishing with " << ctx.memory_pool()->bytes_allocated() <<
+    // std::endl;
+  }
+  // std::cout << "reporting with " << ctx.memory_pool()->bytes_allocated() << std::endl;
+  state.counters["total_rows_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * rows), benchmark::Counter::kIsRate);
+
+  state.counters["total_bytes_per_second"] = benchmark::Counter(
+      static_cast<double>(state.iterations() * bytes), benchmark::Counter::kIsRate);
+}
+
+static void AsOfJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  int64_t tolerance = 0;
+  AsofJoinNodeOptions options = AsofJoinNodeOptions(time_col, key_col, tolerance);
+  TableJoinOverhead(state, left_table, right_tables, "asofjoin", options);
+}
+
+static void HashJoinOverhead(benchmark::State& state, std::string left_table,
+                             std::vector<std::string> right_tables) {
+  HashJoinNodeOptions options =
+      HashJoinNodeOptions({time_col, key_col}, {time_col, key_col});
+  TableJoinOverhead(state, left_table, right_tables, "hashjoin", options);
+}
+
+// this generates the set of right hand tables to test on.
+void SetArgs(benchmark::internal::Benchmark* bench) { bench->UseRealTime(); }
+
+BENCHMARK_CAPTURE(AsOfJoinOverhead,

Review Comment:
   Yep, agree. In its current state, (still containing both hashjoin / asofjoin), it runs locally in about ~6 minutes on my laptop.



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